Skip to content

Commit

Permalink
[FLINK-26055][table] Use FlinkVersion rather than version string from…
Browse files Browse the repository at this point in the history
… core in persisted plan

This closes apache#18755.
  • Loading branch information
slinkydeveloper authored and twalthr committed Feb 15, 2022
1 parent 54cedcb commit 009d255
Show file tree
Hide file tree
Showing 10 changed files with 217 additions and 12 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.api;

import org.apache.flink.FlinkVersion;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.table.api.config.TableConfigOptions;

Expand Down Expand Up @@ -109,7 +110,7 @@ default void writeToFile(File file) {
// --- Accessors

/** Returns the Flink version used to compile the plan. */
String getFlinkVersion();
FlinkVersion getFlinkVersion();

/**
* Returns the AST of the specified statement and the execution plan to compute the result of
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.planner.plan;

import org.apache.flink.FlinkVersion;
import org.apache.flink.annotation.Internal;
import org.apache.flink.table.api.CompiledPlan;
import org.apache.flink.table.api.ExplainDetail;
Expand Down Expand Up @@ -90,7 +91,7 @@ public void writeToFile(File file, boolean ignoreIfExists) {
}

@Override
public String getFlinkVersion() {
public FlinkVersion getFlinkVersion() {
return this.execNodeGraph.getFlinkVersion();
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@

package org.apache.flink.table.planner.plan.nodes.exec;

import org.apache.flink.runtime.util.EnvironmentInformation;
import org.apache.flink.FlinkVersion;

import java.util.List;

Expand All @@ -27,14 +27,14 @@

/** The {@link ExecNodeGraph} representing the {@link ExecNode} topology. */
public class ExecNodeGraph {
private final String flinkVersion;
private final FlinkVersion flinkVersion;
private final List<ExecNode<?>> rootNodes;

public ExecNodeGraph(List<ExecNode<?>> rootNodes) {
this(EnvironmentInformation.getVersion(), rootNodes);
this(FlinkVersion.current(), rootNodes);
}

public ExecNodeGraph(String flinkVersion, List<ExecNode<?>> rootNodes) {
public ExecNodeGraph(FlinkVersion flinkVersion, List<ExecNode<?>> rootNodes) {
this.flinkVersion = checkNotNull(flinkVersion, "The flinkVersion should not be null.");
this.rootNodes = checkNotNull(rootNodes, "The rootNodes should not be null.");
checkArgument(!rootNodes.isEmpty(), "The rootNodes should not be empty.");
Expand All @@ -44,7 +44,7 @@ public List<ExecNode<?>> getRootNodes() {
return rootNodes;
}

public String getFlinkVersion() {
public FlinkVersion getFlinkVersion() {
return flinkVersion;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.nodes.exec.serde;

import org.apache.flink.FlinkVersion;
import org.apache.flink.table.api.ValidationException;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer;

import java.io.IOException;

/** JSON deserializer for {@link FlinkVersion}. */
public class FlinkVersionJsonDeserializer extends StdDeserializer<FlinkVersion> {
private static final long serialVersionUID = 1L;

public FlinkVersionJsonDeserializer() {
super(FlinkVersion.class);
}

@Override
public FlinkVersion deserialize(JsonParser jsonParser, DeserializationContext ctx)
throws IOException {
String codeString = jsonParser.getValueAsString();
return FlinkVersion.byCode(codeString)
.orElseThrow(
() ->
new ValidationException(
"Unknown Flink version '" + codeString + "'"));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,45 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.nodes.exec.serde;

import org.apache.flink.FlinkVersion;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer;

import java.io.IOException;

/** JSON serializer for {@link FlinkVersion}. */
public class FlinkVersionJsonSerializer extends StdSerializer<FlinkVersion> {
private static final long serialVersionUID = 1L;

public FlinkVersionJsonSerializer() {
super(FlinkVersion.class);
}

@Override
public void serialize(
FlinkVersion flinkVersion,
JsonGenerator jsonGenerator,
SerializerProvider serializerProvider)
throws IOException {
jsonGenerator.writeString(flinkVersion.toString());
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.planner.plan.nodes.exec.serde;

import org.apache.flink.FlinkVersion;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
Expand Down Expand Up @@ -53,7 +54,7 @@ class JsonPlanGraph {
public static final String FIELD_NAME_EDGES = "edges";

@JsonProperty(FIELD_NAME_FLINK_VERSION)
private final String flinkVersion;
private final FlinkVersion flinkVersion;

@JsonProperty(FIELD_NAME_NODES)
private final List<ExecNode<?>> nodes;
Expand All @@ -63,7 +64,7 @@ class JsonPlanGraph {

@JsonCreator
public JsonPlanGraph(
@JsonProperty(FIELD_NAME_FLINK_VERSION) String flinkVersion,
@JsonProperty(FIELD_NAME_FLINK_VERSION) FlinkVersion flinkVersion,
@JsonProperty(FIELD_NAME_NODES) List<ExecNode<?>> nodes,
@JsonProperty(FIELD_NAME_EDGES) List<JsonPlanEdge> edges) {
this.flinkVersion = flinkVersion;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.planner.plan.nodes.exec.serde;

import org.apache.flink.FlinkVersion;
import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.catalog.ContextResolvedTable;
import org.apache.flink.table.catalog.ObjectIdentifier;
Expand Down Expand Up @@ -132,6 +133,7 @@ private static Module createFlinkTableJacksonModule() {

private static void registerSerializers(SimpleModule module) {
module.addSerializer(new ExecNodeGraphJsonSerializer());
module.addSerializer(new FlinkVersionJsonSerializer());
// ObjectIdentifierJsonSerializer is needed for LogicalType serialization
module.addSerializer(new ObjectIdentifierJsonSerializer());
// LogicalTypeJsonSerializer is needed for RelDataType serialization
Expand All @@ -157,6 +159,7 @@ private static void registerSerializers(SimpleModule module) {
@SuppressWarnings({"unchecked", "rawtypes"})
private static void registerDeserializers(SimpleModule module) {
module.addDeserializer(ExecNodeGraph.class, new ExecNodeGraphJsonDeserializer());
module.addDeserializer(FlinkVersion.class, new FlinkVersionJsonDeserializer());
// ObjectIdentifierJsonDeserializer is needed for LogicalType deserialization
module.addDeserializer(ObjectIdentifier.class, new ObjectIdentifierJsonDeserializer());
// LogicalTypeJsonSerializer is needed for RelDataType serialization
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,11 @@

package org.apache.flink.table.api;

import org.apache.flink.FlinkVersion;
import org.apache.flink.table.api.config.TableConfigOptions;
import org.apache.flink.table.api.internal.TableResultInternal;
import org.apache.flink.table.planner.utils.JsonPlanTestBase;
import org.apache.flink.table.planner.utils.JsonTestUtils;
import org.apache.flink.table.planner.utils.TableTestUtil;

import org.apache.commons.io.FileUtils;
Expand Down Expand Up @@ -289,11 +291,16 @@ public void testCompileAndExecutePlanWithStatementSet() throws Exception {
}

@Test
public void testExplainPlan() {
public void testExplainPlan() throws IOException {
String planFromResources =
JsonTestUtils.setFlinkVersion(
JsonTestUtils.readFromResource("/jsonplan/testGetJsonPlan.out"),
FlinkVersion.current())
.toString();

String actual =
tableEnv.explainPlan(
tableEnv.loadPlan(
PlanReference.fromResource("/jsonplan/testGetJsonPlan.out")),
tableEnv.loadPlan(PlanReference.fromJsonString(planFromResources)),
ExplainDetail.JSON_EXECUTION_PLAN);
String expected = TableTestUtil.readFromResource("/explain/testExplainJsonPlan.out");
assertThat(TableTestUtil.replaceNodeIdInOperator(TableTestUtil.replaceStreamNodeId(actual)))
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,53 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.plan.nodes.exec.serde;

import org.apache.flink.FlinkVersion;

import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.parallel.Execution;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;

import java.io.IOException;

import static org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeTestUtil.configuredSerdeContext;
import static org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeTestUtil.testJsonRoundTrip;
import static org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeTestUtil.toJson;
import static org.assertj.core.api.Assertions.assertThat;
import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT;

/** Tests for {@link FlinkVersion} serialization and deserialization. */
@Execution(CONCURRENT)
final class FlinkVersionJsonSerdeTest {

@ParameterizedTest
@EnumSource(FlinkVersion.class)
void testFlinkVersions(FlinkVersion flinkVersion) throws IOException {
testJsonRoundTrip(flinkVersion, FlinkVersion.class);
}

@Test
void testManualString() throws IOException {
final String flinkVersion = "1.15";

assertThat(toJson(configuredSerdeContext(), FlinkVersion.v1_15))
.isEqualTo(JsonSerdeUtil.getObjectMapper().writeValueAsString(flinkVersion));
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,46 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http:https://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.planner.utils;

import org.apache.flink.FlinkVersion;
import org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JsonNode;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.node.ObjectNode;

import java.io.IOException;

/** This class contains a collection of generic utilities to deal with JSON in tests. */
public final class JsonTestUtils {

private JsonTestUtils() {}

public static JsonNode readFromResource(String path) throws IOException {
return JsonSerdeUtil.getObjectMapper().readTree(JsonTestUtils.class.getResource(path));
}

public static JsonNode setFlinkVersion(JsonNode target, FlinkVersion flinkVersion) {
return ((ObjectNode) target)
.set("flinkVersion", JsonSerdeUtil.getObjectMapper().valueToTree(flinkVersion));
}

public static JsonNode clearFlinkVersion(JsonNode target) {
return ((ObjectNode) target).remove("flinkVersion");
}
}

0 comments on commit 009d255

Please sign in to comment.