You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2022/02/02 17:01:28 UTC
[flink] 01/02: [FLINK-25387][FLINK-25388][FLINK-25389][table-planner] Introduce ExecNodeMetadata
This is an automated email from the ASF dual-hosted git repository.
twalthr pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git
commit 05eecb0d4dcb783f54a49b7386b22932989a8d0c
Author: Marios Trivyzas <ma...@gmail.com>
AuthorDate: Fri Jan 28 15:24:55 2022 +0200
[FLINK-25387][FLINK-25388][FLINK-25389][table-planner] Introduce ExecNodeMetadata
- Introduce a new annotation `ExecNodeMetadata` on `ExecNode`s which is used to
improve the serialization/deserialization to/from JSON plan of `ExecNode`s and
facilitate the upgrade of the pipeline, since every ExecNode has now also a version
attached.
- List all the JSON plan eligible `ExecNode`s in `ExecNodeMetadataUtil` and use a
static list to register them in Jackson.
- Annotate all those eligible `ExecNode`s with the new annotation and provide a
name constructed by the class name using `-` separators. All versions are set now
to 1.
- Use an `ExecNodeContext` POJO which uses the uniqueId, name and version to
serialize/deserialize them in a JSON plan in the form of `<id>_<exec-node-name>_<version>`.
- Fix issues with `@JsonIgnoreProperties` and `@JsonIgnore`, and opt for the class
based annotation instead of the per field annotation of `@JsonIgnore`.
- Update the test plans with the new JSON scheme derived by the changes.
This closes #18479.
---
.../table/planner/plan/nodes/exec/ExecNode.java | 15 +-
.../planner/plan/nodes/exec/ExecNodeBase.java | 52 ++--
.../planner/plan/nodes/exec/ExecNodeContext.java | 163 ++++++++++++
.../planner/plan/nodes/exec/ExecNodeMetadata.java | 132 ++++++++++
.../plan/nodes/exec/ExecNodeTypeIdResolver.java | 67 +++++
.../planner/plan/nodes/exec/InputProperty.java | 4 -
...xecUnion.java => MultipleExecNodeMetadata.java} | 30 +--
.../exec/batch/BatchExecBoundedStreamScan.java | 8 +-
.../plan/nodes/exec/batch/BatchExecCalc.java | 4 +-
.../plan/nodes/exec/batch/BatchExecCorrelate.java | 4 +-
.../plan/nodes/exec/batch/BatchExecExchange.java | 8 +-
.../plan/nodes/exec/batch/BatchExecExpand.java | 4 +-
.../nodes/exec/batch/BatchExecHashAggregate.java | 8 +-
.../plan/nodes/exec/batch/BatchExecHashJoin.java | 8 +-
.../exec/batch/BatchExecHashWindowAggregate.java | 8 +-
.../plan/nodes/exec/batch/BatchExecLegacySink.java | 3 +
.../exec/batch/BatchExecLegacyTableSourceScan.java | 9 +-
.../plan/nodes/exec/batch/BatchExecLimit.java | 8 +-
.../plan/nodes/exec/batch/BatchExecLookupJoin.java | 4 +-
.../nodes/exec/batch/BatchExecMultipleInput.java | 8 +-
.../nodes/exec/batch/BatchExecNestedLoopJoin.java | 8 +-
.../nodes/exec/batch/BatchExecOverAggregate.java | 9 +-
.../exec/batch/BatchExecOverAggregateBase.java | 5 +-
.../plan/nodes/exec/batch/BatchExecPythonCalc.java | 21 +-
.../nodes/exec/batch/BatchExecPythonCorrelate.java | 4 +-
.../exec/batch/BatchExecPythonGroupAggregate.java | 8 +-
.../batch/BatchExecPythonGroupWindowAggregate.java | 8 +-
.../exec/batch/BatchExecPythonOverAggregate.java | 9 +-
.../plan/nodes/exec/batch/BatchExecRank.java | 8 +-
.../plan/nodes/exec/batch/BatchExecSink.java | 4 +-
.../plan/nodes/exec/batch/BatchExecSort.java | 8 +-
.../nodes/exec/batch/BatchExecSortAggregate.java | 8 +-
.../plan/nodes/exec/batch/BatchExecSortLimit.java | 8 +-
.../nodes/exec/batch/BatchExecSortMergeJoin.java | 8 +-
.../exec/batch/BatchExecSortWindowAggregate.java | 8 +-
.../nodes/exec/batch/BatchExecTableSourceScan.java | 8 +-
.../plan/nodes/exec/batch/BatchExecUnion.java | 8 +-
.../plan/nodes/exec/batch/BatchExecValues.java | 8 +-
.../exec/batch/BatchExecWindowTableFunction.java | 4 +-
.../plan/nodes/exec/common/CommonExecCalc.java | 13 +-
.../nodes/exec/common/CommonExecCorrelate.java | 13 +-
.../plan/nodes/exec/common/CommonExecExchange.java | 9 +-
.../plan/nodes/exec/common/CommonExecExpand.java | 9 +-
.../nodes/exec/common/CommonExecLegacySink.java | 5 +-
.../common/CommonExecLegacyTableSourceScan.java | 5 +-
.../nodes/exec/common/CommonExecLookupJoin.java | 14 +-
.../nodes/exec/common/CommonExecPythonCalc.java | 8 +-
.../exec/common/CommonExecPythonCorrelate.java | 8 +-
.../plan/nodes/exec/common/CommonExecSink.java | 11 +-
.../exec/common/CommonExecTableSourceScan.java | 6 +-
.../plan/nodes/exec/common/CommonExecUnion.java | 9 +-
.../plan/nodes/exec/common/CommonExecValues.java | 12 +-
.../exec/common/CommonExecWindowTableFunction.java | 8 +-
.../exec/serde/ExecNodeGraphJsonPlanGenerator.java | 15 +-
.../plan/nodes/exec/serde/JsonSerdeUtil.java | 6 +-
.../nodes/exec/stream/StreamExecAggregateBase.java | 4 +-
.../plan/nodes/exec/stream/StreamExecCalc.java | 19 +-
.../exec/stream/StreamExecChangelogNormalize.java | 18 +-
.../nodes/exec/stream/StreamExecCorrelate.java | 19 +-
.../exec/stream/StreamExecDataStreamScan.java | 8 +-
.../nodes/exec/stream/StreamExecDeduplicate.java | 18 +-
.../exec/stream/StreamExecDropUpdateBefore.java | 20 +-
.../plan/nodes/exec/stream/StreamExecExchange.java | 20 +-
.../plan/nodes/exec/stream/StreamExecExpand.java | 19 +-
.../stream/StreamExecGlobalGroupAggregate.java | 16 +-
.../stream/StreamExecGlobalWindowAggregate.java | 16 +-
.../exec/stream/StreamExecGroupAggregate.java | 18 +-
.../exec/stream/StreamExecGroupTableAggregate.java | 8 +-
.../stream/StreamExecGroupWindowAggregate.java | 18 +-
.../StreamExecIncrementalGroupAggregate.java | 16 +-
.../nodes/exec/stream/StreamExecIntervalJoin.java | 18 +-
.../plan/nodes/exec/stream/StreamExecJoin.java | 18 +-
.../nodes/exec/stream/StreamExecLegacySink.java | 3 +
.../stream/StreamExecLegacyTableSourceScan.java | 9 +-
.../plan/nodes/exec/stream/StreamExecLimit.java | 22 +-
.../exec/stream/StreamExecLocalGroupAggregate.java | 16 +-
.../stream/StreamExecLocalWindowAggregate.java | 16 +-
.../nodes/exec/stream/StreamExecLookupJoin.java | 19 +-
.../plan/nodes/exec/stream/StreamExecMatch.java | 18 +-
.../exec/stream/StreamExecMiniBatchAssigner.java | 16 +-
.../nodes/exec/stream/StreamExecMultipleInput.java | 8 +-
.../nodes/exec/stream/StreamExecOverAggregate.java | 18 +-
.../nodes/exec/stream/StreamExecPythonCalc.java | 18 +-
.../exec/stream/StreamExecPythonCorrelate.java | 26 +-
.../stream/StreamExecPythonGroupAggregate.java | 16 +-
.../StreamExecPythonGroupTableAggregate.java | 8 +-
.../StreamExecPythonGroupWindowAggregate.java | 18 +-
.../exec/stream/StreamExecPythonOverAggregate.java | 16 +-
.../plan/nodes/exec/stream/StreamExecRank.java | 18 +-
.../plan/nodes/exec/stream/StreamExecSink.java | 25 +-
.../plan/nodes/exec/stream/StreamExecSort.java | 8 +-
.../nodes/exec/stream/StreamExecSortLimit.java | 22 +-
.../exec/stream/StreamExecTableSourceScan.java | 22 +-
.../nodes/exec/stream/StreamExecTemporalJoin.java | 18 +-
.../nodes/exec/stream/StreamExecTemporalSort.java | 16 +-
.../plan/nodes/exec/stream/StreamExecUnion.java | 20 +-
.../plan/nodes/exec/stream/StreamExecValues.java | 22 +-
.../exec/stream/StreamExecWatermarkAssigner.java | 16 +-
.../exec/stream/StreamExecWindowAggregate.java | 16 +-
.../exec/stream/StreamExecWindowAggregateBase.java | 4 +-
.../exec/stream/StreamExecWindowDeduplicate.java | 18 +-
.../nodes/exec/stream/StreamExecWindowJoin.java | 18 +-
.../nodes/exec/stream/StreamExecWindowRank.java | 18 +-
.../exec/stream/StreamExecWindowTableFunction.java | 18 +-
.../planner/plan/utils/ExecNodeMetadataUtil.java | 290 +++++++++++++++++++++
.../plan/utils/ExecNodeMetadataUtilTest.java | 269 +++++++++++++++++++
.../test/resources/jsonplan/testGetJsonPlan.out | 10 +-
.../CalcJsonPlanTest_jsonplan/testComplexCalc.out | 12 +-
.../CalcJsonPlanTest_jsonplan/testSimpleFilter.out | 12 +-
.../testSimpleProject.out | 8 +-
.../testChangelogSource.out | 16 +-
.../testUpsertSource.out | 14 +-
.../testCrossJoin.out | 16 +-
.../testCrossJoinOverrideParameters.out | 16 +-
.../testJoinWithFilter.out | 16 +-
.../testLeftOuterJoinWithLiteralTrue.out | 16 +-
.../testDeduplication.out | 22 +-
.../ExpandJsonPlanTest_jsonplan/testExpand.out | 32 +--
...tDistinctAggCalls[isMiniBatchEnabled=false].out | 22 +-
...stDistinctAggCalls[isMiniBatchEnabled=true].out | 30 +--
...gCallsWithGroupBy[isMiniBatchEnabled=false].out | 22 +-
...ggCallsWithGroupBy[isMiniBatchEnabled=true].out | 30 +--
...AggWithoutGroupBy[isMiniBatchEnabled=false].out | 22 +-
...eAggWithoutGroupBy[isMiniBatchEnabled=true].out | 30 +--
...erDefinedAggCalls[isMiniBatchEnabled=false].out | 22 +-
...serDefinedAggCalls[isMiniBatchEnabled=true].out | 26 +-
.../testEventTimeHopWindow.out | 22 +-
.../testEventTimeSessionWindow.out | 22 +-
.../testEventTimeTumbleWindow.out | 26 +-
.../testProcTimeHopWindow.out | 26 +-
.../testProcTimeSessionWindow.out | 26 +-
.../testProcTimeTumbleWindow.out | 30 +--
.../testIncrementalAggregate.out | 32 +--
...lAggregateWithSumCountDistinctAndRetraction.out | 42 +--
.../testProcessingTimeInnerJoinWithOnClause.out | 48 ++--
.../testRowTimeInnerJoinWithOnClause.out | 40 +--
.../JoinJsonPlanTest_jsonplan/testInnerJoin.out | 20 +-
.../testInnerJoinWithEqualPk.out | 32 +--
.../testInnerJoinWithPk.out | 44 ++--
.../testLeftJoinNonEqui.out | 24 +-
.../LimitJsonPlanTest_jsonplan/testLimit.out | 18 +-
.../testJoinTemporalTable.out | 24 +-
...testJoinTemporalTableWithProjectionPushDown.out | 24 +-
.../testMatch.out | 20 +-
.../testProcTimeBoundedNonPartitionedRangeOver.out | 30 +--
.../testProcTimeBoundedPartitionedRangeOver.out | 30 +--
...undedPartitionedRowsOverWithBuiltinProctime.out | 26 +-
.../testProcTimeUnboundedPartitionedRangeOver.out | 30 +--
...stProctimeBoundedDistinctPartitionedRowOver.out | 30 +--
...edDistinctWithNonDistinctPartitionedRowOver.out | 30 +--
.../testRowTimeBoundedPartitionedRowsOver.out | 22 +-
.../testPythonCalc.out | 12 +-
.../testPythonFunctionInWhereClause.out | 20 +-
.../testJoinWithFilter.out | 20 +-
.../testPythonTableFunction.out | 20 +-
.../tesPythonAggCallsWithGroupBy.out | 22 +-
.../testEventTimeHopWindow.out | 26 +-
.../testEventTimeSessionWindow.out | 26 +-
.../testEventTimeTumbleWindow.out | 30 +--
.../testProcTimeHopWindow.out | 26 +-
.../testProcTimeSessionWindow.out | 26 +-
.../testProcTimeTumbleWindow.out | 30 +--
.../testProcTimeBoundedNonPartitionedRangeOver.out | 30 +--
.../testProcTimeBoundedPartitionedRangeOver.out | 30 +--
...undedPartitionedRowsOverWithBuiltinProctime.out | 26 +-
.../testProcTimeUnboundedPartitionedRangeOver.out | 30 +--
.../testRowTimeBoundedPartitionedRowsOver.out | 26 +-
.../stream/RankJsonPlanTest_jsonplan/testRank.out | 22 +-
.../testSortLimit.out | 18 +-
.../testOverwrite.out | 8 +-
.../testPartitioning.out | 12 +-
.../testWritingMetadata.out | 8 +-
.../testFilterPushDown.out | 8 +-
.../testLimitPushDown.out | 14 +-
.../testPartitionPushDown.out | 12 +-
.../testProjectPushDown.out | 8 +-
.../testReadingMetadata.out | 8 +-
.../testWatermarkPushDown.out | 8 +-
.../testJoinTemporalFunction.out | 32 +--
.../testTemporalTableJoin.out | 32 +--
.../testSortProcessingTime.out | 26 +-
.../testSortRowTime.out | 26 +-
.../UnionJsonPlanTest_jsonplan/testUnion.out | 14 +-
.../ValuesJsonPlanTest_jsonplan/testValues.out | 12 +-
.../testWatermarkAssigner.out | 12 +-
.../testDistinctSplitEnabled.out | 48 ++--
.../testEventTimeCumulateWindow.out | 34 +--
.../testEventTimeCumulateWindowWithOffset.out | 34 +--
.../testEventTimeHopWindow.out | 34 +--
.../testEventTimeHopWindowWithOffset.out | 34 +--
.../testEventTimeTumbleWindow.out | 34 +--
.../testEventTimeTumbleWindowWithOffset.out | 34 +--
.../testProcTimeCumulateWindow.out | 30 +--
.../testProcTimeHopWindow.out | 30 +--
.../testProcTimeTumbleWindow.out | 30 +--
.../testEventTimeTumbleWindow.out | 68 ++---
.../testFollowedByWindowDeduplicate.out | 34 +--
.../testFollowedByWindowJoin.out | 56 ++--
.../testFollowedByWindowRank.out | 34 +--
.../testIndividualWindowTVF.out | 24 +-
.../testIndividualWindowTVFProcessingTime.out | 28 +-
.../flink/table/planner/utils/TableTestBase.scala | 6 +-
202 files changed, 3042 insertions(+), 1550 deletions(-)
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java
index f5a129e..229305f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNode.java
@@ -32,25 +32,34 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonPro
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonTypeIdResolver;
import java.util.List;
+import static org.apache.flink.table.planner.plan.nodes.exec.ExecNode.FIELD_NAME_TYPE;
+
/**
* The representation of execution information for a {@link FlinkPhysicalRel}.
*
* @param <T> The type of the elements that result from this node.
*/
-@JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, include = JsonTypeInfo.As.PROPERTY, property = "class")
+@JsonTypeInfo(
+ use = JsonTypeInfo.Id.NAME,
+ include = JsonTypeInfo.As.EXISTING_PROPERTY,
+ property = FIELD_NAME_TYPE,
+ visible = true)
+@JsonTypeIdResolver(ExecNodeTypeIdResolver.class)
@Internal
public interface ExecNode<T> extends ExecNodeTranslator<T> {
String FIELD_NAME_ID = "id";
+ String FIELD_NAME_TYPE = "type";
String FIELD_NAME_DESCRIPTION = "description";
String FIELD_NAME_INPUT_PROPERTIES = "inputProperties";
String FIELD_NAME_OUTPUT_TYPE = "outputType";
- /** Gets the ID of this node. */
- @JsonProperty(value = FIELD_NAME_ID)
+ /** The unique ID of the node. */
+ @JsonProperty(value = FIELD_NAME_ID, index = 0)
int getId();
/** Returns a string which describes this node. */
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
index 9781ab4..e403b04 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeBase.java
@@ -18,7 +18,6 @@
package org.apache.flink.table.planner.plan.nodes.exec;
-import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.table.api.TableConfig;
@@ -31,6 +30,7 @@ import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.ArrayList;
import java.util.List;
@@ -46,54 +46,44 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
@JsonIgnoreProperties(ignoreUnknown = true)
public abstract class ExecNodeBase<T> implements ExecNode<T> {
- /** The unique identifier for each ExecNode in the json plan. */
- @JsonIgnore private final int id;
+ private final String description;
- @JsonIgnore private final String description;
+ private final LogicalType outputType;
- @JsonIgnore private final LogicalType outputType;
+ private final List<InputProperty> inputProperties;
- @JsonIgnore private final List<InputProperty> inputProperties;
+ private List<ExecEdge> inputEdges;
- @JsonIgnore private List<ExecEdge> inputEdges;
+ private transient Transformation<T> transformation;
- @JsonIgnore private transient Transformation<T> transformation;
+ /** Holds the context information (id, name, version) as deserialized from a JSON plan. */
+ @JsonProperty(value = FIELD_NAME_TYPE, access = JsonProperty.Access.WRITE_ONLY)
+ private final ExecNodeContext context;
- /** This is used to assign a unique ID to every ExecNode. */
- private static Integer idCounter = 0;
-
- /** Generate an unique ID for ExecNode. */
- public static int getNewNodeId() {
- idCounter++;
- return idCounter;
- }
-
- /** Reset the id counter to 0. */
- @VisibleForTesting
- public static void resetIdCounter() {
- idCounter = 0;
+ /**
+ * Retrieves the default context from the {@link ExecNodeMetadata} annotation to be serialized
+ * into the JSON plan.
+ */
+ @JsonProperty(value = FIELD_NAME_TYPE, access = JsonProperty.Access.READ_ONLY, index = 1)
+ protected final ExecNodeContext getContextFromAnnotation() {
+ return ExecNodeContext.newContext(this.getClass()).withId(getId());
}
- // used for json creator
protected ExecNodeBase(
int id,
+ ExecNodeContext context,
List<InputProperty> inputProperties,
LogicalType outputType,
String description) {
- this.id = id;
+ this.context = checkNotNull(context).withId(id);
this.inputProperties = checkNotNull(inputProperties);
this.outputType = checkNotNull(outputType);
this.description = checkNotNull(description);
}
- protected ExecNodeBase(
- List<InputProperty> inputProperties, LogicalType outputType, String description) {
- this(getNewNodeId(), inputProperties, outputType, description);
- }
-
@Override
public final int getId() {
- return id;
+ return context.getId();
}
@Override
@@ -191,7 +181,7 @@ public abstract class ExecNodeBase<T> implements ExecNode<T> {
protected String getFormattedOperatorDescription(String description, Configuration config) {
if (config.getBoolean(
OptimizerConfigOptions.TABLE_OPTIMIZER_SIMPLIFY_OPERATOR_NAME_ENABLED)) {
- return String.format("[%d]:%s", id, description);
+ return String.format("[%d]:%s", getId(), description);
}
return description;
}
@@ -200,7 +190,7 @@ public abstract class ExecNodeBase<T> implements ExecNode<T> {
String detailName, String simplifiedName, Configuration config) {
if (config.getBoolean(
OptimizerConfigOptions.TABLE_OPTIMIZER_SIMPLIFY_OPERATOR_NAME_ENABLED)) {
- return String.format("%s[%d]", simplifiedName, id);
+ return String.format("%s[%d]", simplifiedName, getId());
}
return detailName;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java
new file mode 100644
index 0000000..df63655
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeContext.java
@@ -0,0 +1,163 @@
+/*
+ * 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://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;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonValue;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DatabindContext;
+
+import javax.annotation.Nullable;
+
+import java.util.List;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * Helper class that holds the necessary identifier fields that are used for JSON plan serialization
+ * and deserialization. It is instantiated using {@link ExecNodeContext#newContext(Class)} when
+ * creating a new instance of an {@link ExecNode}, so that is contains the info from the {@link
+ * ExecNodeMetadata} annotation of the class with the latest {@link ExecNodeMetadata#version()}. It
+ * can also be instantiated with {@link ExecNodeContext#ExecNodeContext(String)} automatically when
+ * the {@link ExecNode} is deserialized from a JSON Plan, and in this case the {@link
+ * ExecNodeContext} contains the version that is read from the JSON Plan and not the latest one. The
+ * serialization format is {@code <name>_<version>}, see {@link ExecNodeContext#getTypeAsString()}.
+ */
+@Internal
+public final class ExecNodeContext {
+
+ /** This is used to assign a unique ID to every ExecNode. */
+ private static final AtomicInteger idCounter = new AtomicInteger(0);
+
+ /** Generate an unique ID for ExecNode. */
+ public static int newNodeId() {
+ return idCounter.incrementAndGet();
+ }
+
+ /** Reset the id counter to 0. */
+ @VisibleForTesting
+ public static void resetIdCounter() {
+ idCounter.set(0);
+ }
+
+ private final Integer id;
+ private final String name;
+ private final Integer version;
+
+ private ExecNodeContext() {
+ this(null, null, null);
+ }
+
+ private ExecNodeContext(String name, Integer version) {
+ this(null, name, version);
+ }
+
+ /**
+ * @param id The unique id of the {@link ExecNode}. See {@link ExecNode#getId()}. It can be null
+ * initially and then later set by using {@link #withId(int)} which creates a new instance
+ * of {@link ExecNodeContext} since it's immutable. This way we can satisfy both the {@link
+ * ExecNodeBase#ExecNodeBase(int, ExecNodeContext, List, LogicalType, String)} ctor, which
+ * is used for the {@link JsonCreator} ctors, where the {@code id} and the {@code context}
+ * are read separately, and the {@link ExecNodeBase#getContextFromAnnotation()} which
+ * creates a new context with a new id provided by: {@link #newNodeId()}.
+ * @param name The name of the {@link ExecNode}. See {@link ExecNodeMetadata#name()}.
+ * @param version The version of the {@link ExecNode}. See {@link ExecNodeMetadata#version()}.
+ */
+ private ExecNodeContext(@Nullable Integer id, String name, Integer version) {
+ this.id = id;
+ this.name = name;
+ this.version = version;
+ }
+
+ @JsonCreator
+ public ExecNodeContext(String value) {
+ this.id = null;
+ String[] split = value.split("_");
+ this.name = split[0];
+ this.version = Integer.valueOf(split[1]);
+ }
+
+ /** The unique identifier for each ExecNode in the JSON plan. */
+ int getId() {
+ return checkNotNull(id);
+ }
+
+ /** The type identifying an ExecNode in the JSON plan. See {@link ExecNodeMetadata#name()}. */
+ public String getName() {
+ return name;
+ }
+
+ /** The version of the ExecNode in the JSON plan. See {@link ExecNodeMetadata#version()}. */
+ public Integer getVersion() {
+ return version;
+ }
+
+ /**
+ * Set the unique ID of the node, so that the {@link ExecNodeContext}, together with the type
+ * related {@link #name} and {@link #version}, stores all the necessary info to uniquely
+ * reconstruct the {@link ExecNode}, and avoid storing the {@link #id} independently as a field
+ * in {@link ExecNodeBase}.
+ */
+ public ExecNodeContext withId(int id) {
+ return new ExecNodeContext(id, this.name, this.version);
+ }
+
+ /**
+ * Returns the {@link #name} and {@link #version}, to be serialized into the JSON plan as one
+ * string, which in turn will be parsed by {@link ExecNodeContext#ExecNodeContext(String)} when
+ * deserialized from a JSON plan or when needed by {@link
+ * ExecNodeTypeIdResolver#typeFromId(DatabindContext, String)}.
+ */
+ @JsonValue
+ public String getTypeAsString() {
+ return name + "_" + version;
+ }
+
+ @Override
+ public String toString() {
+ return getId() + "_" + getTypeAsString();
+ }
+
+ public static <T extends ExecNode<?>> ExecNodeContext newContext(Class<T> execNodeClass) {
+ ExecNodeMetadata metadata = ExecNodeMetadataUtil.latestAnnotation(execNodeClass);
+ if (metadata == null) {
+ if (!ExecNodeMetadataUtil.isUnsupported(execNodeClass)) {
+ throw new IllegalStateException(
+ String.format(
+ "ExecNode: %s is not listed in the unsupported classes since it is not annotated with: %s.",
+ execNodeClass.getCanonicalName(),
+ ExecNodeMetadata.class.getSimpleName()));
+ }
+ return new ExecNodeContext();
+ }
+ if (!ExecNodeMetadataUtil.execNodes().contains(execNodeClass)) {
+ throw new IllegalStateException(
+ String.format(
+ "ExecNode: %s is not listed in the supported classes and yet is annotated with: %s.",
+ execNodeClass.getCanonicalName(),
+ ExecNodeMetadata.class.getSimpleName()));
+ }
+ return new ExecNodeContext(metadata.name(), metadata.version());
+ }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeMetadata.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeMetadata.java
new file mode 100644
index 0000000..1276cd9
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeMetadata.java
@@ -0,0 +1,132 @@
+/*
+ * 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://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;
+
+import org.apache.flink.FlinkVersion;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.table.api.config.ExecutionConfigOptions;
+
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Repeatable;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * Annotation to be used for {@link ExecNode}s to keep necessary metadata when
+ * serializing/deserializing them in a plan. It's used for internal bookkeeping across Flink
+ * versions, and to provide necessary information to the testing infrastructure.
+ *
+ * <p>Each {@link ExecNode} needs to be annotated and provide the necessary metadata info so that it
+ * can be correctly serialized and later on instantiated from a string (JSON) plan.
+ *
+ * <p>It's possible for one {@link ExecNode} class to use multiple annotations to denote ability to
+ * upgrade to more versions. an {@link ExecNode} class can be annotated directly with multiple
+ * {@link ExecNodeMetadata} annotations, or with a single {@link MultipleExecNodeMetadata}
+ * annotation where the {@link MultipleExecNodeMetadata#value()} is an array of {@link
+ * ExecNodeMetadata} annotations.
+ */
+@Documented
+@Target(ElementType.TYPE)
+@Retention(RetentionPolicy.RUNTIME)
+@Repeatable(value = MultipleExecNodeMetadata.class)
+@Internal
+public @interface ExecNodeMetadata {
+ // main information
+
+ /**
+ * Unique name of the {@link ExecNode} for serialization/deserialization and uid() generation.
+ * Together with version, uniquely identifies the {@link ExecNode} class.
+ */
+ String name();
+
+ /**
+ * A positive integer denoting the evolving version of an {@link ExecNode}, used for
+ * serialization/deserialization and uid() generation. Together with {@link #name()}, uniquely
+ * identifies the {@link ExecNode} class.
+ */
+ int version();
+
+ // maintenance information for internal/community/test usage
+
+ /**
+ * Hard coded list of {@link ExecutionConfigOptions} keys of in the Flink version when the
+ * ExecNode was added. Does not reference instances in the {@link ExecutionConfigOptions} class
+ * in case those get refactored.
+ *
+ * <p>Completeness tests can verify that every option is set once in restore and change
+ * detection tests.
+ *
+ * <p>Completeness tests can verify that the {@link ExecutionConfigOptions} class still contains
+ * an option (via key or fallback key) for the given key.
+ *
+ * <p>Restore can verify whether the restored ExecNode config map contains only options of the
+ * given keys.
+ */
+ String[] consumedOptions() default {};
+
+ /**
+ * Set of operator names that can be part of the resulting {@link Transformation}s.
+ *
+ * <p>Restore and completeness tests can verify there exists at least one test that adds each
+ * operator and that the created {@link Transformation}s contain only operators with {@link
+ * Transformation#getUid()} containing the given operator names.
+ *
+ * <p>The concrete combinations or existence of these operators in the final pipeline depends on
+ * various parameters (both configuration and ExecNode-specific arguments such as interval size
+ * etc.).
+ */
+ String[] producedOperators() default {};
+
+ /**
+ * Used for plan validation and potentially plan migration.
+ *
+ * <p>Needs to be updated when the JSON for the {@link ExecNode} changes: e.g. after adding an
+ * attribute to the JSON spec of the ExecNode.
+ *
+ * <p>The annotation does not need to be updated for every Flink version. As the name suggests
+ * it is about the "minimum" version for a restore. If the minimum version is higher than the
+ * current Flink version, plan migration is necessary.
+ *
+ * <p>Changing this version will always result in a new {@link #version()} for the {@link
+ * ExecNode}.
+ *
+ * <p>Plan migration tests can use this information.
+ *
+ * <p>Completeness tests can verify that restore tests exist for all JSON plan variations.
+ */
+ FlinkVersion minPlanVersion();
+
+ /**
+ * Used for operator and potentially savepoint migration.
+ *
+ * <p>Needs to be updated whenever the state layout of an ExecNode changes. In some cases, the
+ * operator can implement and perform state migration. If the minimum version is higher than the
+ * current Flink version, savepoint migration is necessary.
+ *
+ * <p>Changing this version will always result in a new ExecNode {@link #version()}.
+ *
+ * <p>Restore tests can verify that operator migration works for all Flink state versions.
+ *
+ * <p>Completeness tests can verify that restore tests exist for all state variations.
+ */
+ FlinkVersion minStateVersion();
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeTypeIdResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeTypeIdResolver.java
new file mode 100644
index 0000000..dd24556
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeTypeIdResolver.java
@@ -0,0 +1,67 @@
+/*
+ * 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://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;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeInfo.Id;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DatabindContext;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.JavaType;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.jsontype.impl.TypeIdResolverBase;
+
+/**
+ * Helper class to implement the Jackson subtype serialization/deserialization. Instead of using the
+ * class name use the {@link ExecNodeMetadata#name()} and {@link ExecNodeMetadata#version()} to
+ * perform a lookup in a static map residing in {@link ExecNodeMetadataUtil}.
+ */
+@Internal
+class ExecNodeTypeIdResolver extends TypeIdResolverBase {
+
+ private JavaType superType;
+
+ @Override
+ public void init(JavaType baseType) {
+ superType = baseType;
+ }
+
+ @Override
+ public Id getMechanism() {
+ return Id.NAME;
+ }
+
+ @Override
+ public String idFromValue(Object obj) {
+ return idFromValueAndType(obj, obj.getClass());
+ }
+
+ @Override
+ public String idFromValueAndType(Object obj, Class<?> subType) {
+ return ((ExecNodeBase<?>) obj).getContextFromAnnotation().toString();
+ }
+
+ @Override
+ public JavaType typeFromId(DatabindContext context, String id) {
+ ExecNodeContext execNodeContext = new ExecNodeContext(id);
+ return context.constructSpecializedType(
+ superType,
+ ExecNodeMetadataUtil.retrieveExecNode(
+ execNodeContext.getName(), execNodeContext.getVersion()));
+ }
+}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/InputProperty.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/InputProperty.java
index b736926..cced3f3 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/InputProperty.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/InputProperty.java
@@ -24,7 +24,6 @@ import org.apache.flink.table.planner.plan.nodes.exec.serde.RequiredDistribution
import org.apache.flink.table.planner.plan.nodes.exec.serde.RequiredDistributionJsonSerializer;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
@@ -111,17 +110,14 @@ public class InputProperty {
this.priority = priority;
}
- @JsonIgnore
public RequiredDistribution getRequiredDistribution() {
return requiredDistribution;
}
- @JsonIgnore
public DamBehavior getDamBehavior() {
return damBehavior;
}
- @JsonIgnore
public int getPriority() {
return priority;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/MultipleExecNodeMetadata.java
similarity index 51%
copy from flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java
copy to flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/MultipleExecNodeMetadata.java
index cc3f46e..c4d24e2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/MultipleExecNodeMetadata.java
@@ -16,24 +16,24 @@
* limitations under the License.
*/
-package org.apache.flink.table.planner.plan.nodes.exec.batch;
+package org.apache.flink.table.planner.plan.nodes.exec;
-import org.apache.flink.table.data.RowData;
-import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
-import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
-import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecUnion;
-import org.apache.flink.table.types.logical.RowType;
+import org.apache.flink.annotation.Internal;
-import java.util.List;
+import java.lang.annotation.Documented;
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
/**
- * Batch {@link ExecNode} that is not a physical node and just union the inputs' records into one
- * node.
+ * Helper annotation to enable multiple {@link ExecNodeMetadata} annotations on an {@link ExecNode}
+ * class.
*/
-public class BatchExecUnion extends CommonExecUnion implements BatchExecNode<RowData> {
-
- public BatchExecUnion(
- List<InputProperty> inputProperties, RowType outputType, String description) {
- super(getNewNodeId(), inputProperties, outputType, description);
- }
+@Documented
+@Target(ElementType.TYPE)
+@Retention(RetentionPolicy.RUNTIME)
+@Internal
+public @interface MultipleExecNodeMetadata {
+ ExecNodeMetadata[] value();
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java
index 7e89e09..6bbfcb7 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecBoundedStreamScan.java
@@ -26,6 +26,7 @@ import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.utils.ScanUtil;
import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
@@ -53,7 +54,12 @@ public class BatchExecBoundedStreamScan extends ExecNodeBase<RowData>
List<String> qualifiedName,
RowType outputType,
String description) {
- super(Collections.emptyList(), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecBoundedStreamScan.class),
+ Collections.emptyList(),
+ outputType,
+ description);
this.dataStream = dataStream;
this.sourceType = sourceType;
this.fieldIndexes = fieldIndexes;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java
index d26f990..a7063af 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCalc.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc;
import org.apache.flink.table.runtime.operators.TableStreamOperator;
@@ -42,11 +43,12 @@ public class BatchExecCalc extends CommonExecCalc implements BatchExecNode<RowDa
RowType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecCalc.class),
projection,
condition,
TableStreamOperator.class,
false, // retainHeader
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java
index 6ee001f3..c8e538b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecCorrelate.java
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCorrelate;
import org.apache.flink.table.runtime.operators.TableStreamOperator;
@@ -43,12 +44,13 @@ public class BatchExecCorrelate extends CommonExecCorrelate implements BatchExec
RowType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecCorrelate.class),
joinType,
invocation,
condition,
TableStreamOperator.class,
false, // retainHeader
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java
index b311687..61f77c9 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExchange.java
@@ -33,6 +33,7 @@ import org.apache.flink.table.planner.codegen.HashCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty.HashDistribution;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty.RequiredDistribution;
@@ -60,7 +61,12 @@ public class BatchExecExchange extends CommonExecExchange implements BatchExecNo
@Nullable private StreamExchangeMode requiredExchangeMode;
public BatchExecExchange(InputProperty inputProperty, RowType outputType, String description) {
- super(getNewNodeId(), Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecExchange.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
}
public void setRequiredExchangeMode(@Nullable StreamExchangeMode requiredExchangeMode) {
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java
index c40ab1c..653f1d7 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecExpand;
import org.apache.flink.table.types.logical.RowType;
@@ -38,9 +39,10 @@ public class BatchExecExpand extends CommonExecExpand implements BatchExecNode<R
RowType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecExpand.class),
projects,
false, // retainHeader
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java
index 95c3b1f..f01f9c4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashAggregate.java
@@ -30,6 +30,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -67,7 +68,12 @@ public class BatchExecHashAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecHashAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.grouping = grouping;
this.auxGrouping = auxGrouping;
this.aggCalls = aggCalls;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java
index 36a7540..8371b16 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java
@@ -30,6 +30,7 @@ import org.apache.flink.table.planner.codegen.ProjectionCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
@@ -71,7 +72,12 @@ public class BatchExecHashJoin extends ExecNodeBase<RowData>
InputProperty rightInputProperty,
RowType outputType,
String description) {
- super(Arrays.asList(leftInputProperty, rightInputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecHashJoin.class),
+ Arrays.asList(leftInputProperty, rightInputProperty),
+ outputType,
+ description);
this.joinSpec = joinSpec;
this.leftIsBuild = leftIsBuild;
this.estimatedLeftAvgRowSize = estimatedLeftAvgRowSize;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java
index ed91994..a527127 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashWindowAggregate.java
@@ -32,6 +32,7 @@ import org.apache.flink.table.planner.plan.logical.LogicalWindow;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -80,7 +81,12 @@ public class BatchExecHashWindowAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecHashWindowAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.grouping = grouping;
this.auxGrouping = auxGrouping;
this.aggCalls = aggCalls;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java
index 049920c..4440d87 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacySink;
import org.apache.flink.table.runtime.types.ClassLogicalTypeConverter;
@@ -46,6 +47,8 @@ public class BatchExecLegacySink<T> extends CommonExecLegacySink<T> implements B
LogicalType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecLegacySink.class),
tableSink,
upsertKeys,
false, // needRetraction
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java
index 076382c..c973fa3 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacyTableSourceScan.java
@@ -29,6 +29,7 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacyTableSourceScan;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.planner.plan.utils.ScanUtil;
@@ -58,7 +59,13 @@ public class BatchExecLegacyTableSourceScan extends CommonExecLegacyTableSourceS
List<String> qualifiedName,
RowType outputType,
String description) {
- super(tableSource, qualifiedName, outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecLegacyTableSourceScan.class),
+ tableSource,
+ qualifiedName,
+ outputType,
+ description);
}
@Override
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java
index b1e3a43c..563dcca 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLimit.java
@@ -24,6 +24,7 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.runtime.operators.sort.LimitOperator;
@@ -45,7 +46,12 @@ public class BatchExecLimit extends ExecNodeBase<RowData> implements BatchExecNo
InputProperty inputProperty,
LogicalType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecLimit.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.isGlobal = isGlobal;
this.limitStart = limitStart;
this.limitEnd = limitEnd;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java
index 0620c01..f756c2e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLookupJoin;
import org.apache.flink.table.planner.plan.nodes.exec.spec.TemporalTableSourceSpec;
@@ -47,13 +48,14 @@ public class BatchExecLookupJoin extends CommonExecLookupJoin implements BatchEx
RowType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecLookupJoin.class),
joinType,
joinCondition,
temporalTableSourceSpec,
lookupKeys,
projectionOnTemporalTable,
filterOnTemporalTable,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java
index 2536889..24553d2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecMultipleInput.java
@@ -26,6 +26,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -74,7 +75,12 @@ public class BatchExecMultipleInput extends ExecNodeBase<RowData>
public BatchExecMultipleInput(
List<InputProperty> inputProperties, ExecNode<?> rootNode, String description) {
- super(inputProperties, rootNode.getOutputType(), description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecMultipleInput.class),
+ inputProperties,
+ rootNode.getOutputType(),
+ description);
this.rootNode = rootNode;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java
index 6595974..99c5c76 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecNestedLoopJoin.java
@@ -27,6 +27,7 @@ import org.apache.flink.table.planner.codegen.NestedLoopJoinCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
@@ -58,7 +59,12 @@ public class BatchExecNestedLoopJoin extends ExecNodeBase<RowData>
InputProperty rightInputProperty,
RowType outputType,
String description) {
- super(Arrays.asList(leftInputProperty, rightInputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecNestedLoopJoin.class),
+ Arrays.asList(leftInputProperty, rightInputProperty),
+ outputType,
+ description);
this.joinType = checkNotNull(joinType);
this.condition = checkNotNull(condition);
this.leftIsBuild = leftIsBuild;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java
index 74143fb..ab65ae4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregate.java
@@ -33,6 +33,7 @@ import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec.GroupSpec;
@@ -77,7 +78,13 @@ public class BatchExecOverAggregate extends BatchExecOverAggregateBase {
InputProperty inputProperty,
RowType outputType,
String description) {
- super(overSpec, inputProperty, outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecOverAggregate.class),
+ overSpec,
+ inputProperty,
+ outputType,
+ description);
}
@SuppressWarnings("unchecked")
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java
index afffa4f..a6919d4 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecOverAggregateBase.java
@@ -22,6 +22,7 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
@@ -44,11 +45,13 @@ public abstract class BatchExecOverAggregateBase extends ExecNodeBase<RowData>
protected final OverSpec overSpec;
public BatchExecOverAggregateBase(
+ int id,
+ ExecNodeContext context,
OverSpec overSpec,
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(id, context, Collections.singletonList(inputProperty), outputType, description);
this.overSpec = overSpec;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java
index b052668..9c6c598 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCalc.java
@@ -20,21 +20,17 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecPythonCalc;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
-
import org.apache.calcite.rex.RexNode;
import java.util.Collections;
import java.util.List;
/** Batch {@link ExecNode} for Python ScalarFunctions. */
-@JsonIgnoreProperties(ignoreUnknown = true)
public class BatchExecPythonCalc extends CommonExecPythonCalc implements BatchExecNode<RowData> {
public BatchExecPythonCalc(
@@ -42,21 +38,12 @@ public class BatchExecPythonCalc extends CommonExecPythonCalc implements BatchEx
InputProperty inputProperty,
RowType outputType,
String description) {
- this(
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecPythonCalc.class),
projection,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
}
-
- @JsonCreator
- public BatchExecPythonCalc(
- @JsonProperty(FIELD_NAME_PROJECTION) List<RexNode> projection,
- @JsonProperty(FIELD_NAME_ID) int id,
- @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
- @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
- @JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(projection, id, inputProperties, outputType, description);
- }
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java
index 4372ff9..8307f6f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonCorrelate.java
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecPythonCorrelate;
import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
@@ -39,9 +40,10 @@ public class BatchExecPythonCorrelate extends CommonExecPythonCorrelate
RowType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecPythonCorrelate.class),
joinType,
invocation,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java
index 139e850..bd7ce52 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupAggregate.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
@@ -68,7 +69,12 @@ public class BatchExecPythonGroupAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecPythonGroupAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.grouping = grouping;
this.auxGrouping = auxGrouping;
this.aggCalls = aggCalls;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java
index bb73f76..e7041a2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java
@@ -38,6 +38,7 @@ import org.apache.flink.table.planner.plan.logical.LogicalWindow;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
@@ -83,7 +84,12 @@ public class BatchExecPythonGroupWindowAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecPythonGroupWindowAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.grouping = grouping;
this.auxGrouping = auxGrouping;
this.aggCalls = aggCalls;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java
index ae622ec..02ad8d1 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java
@@ -34,6 +34,7 @@ import org.apache.flink.table.planner.codegen.ProjectionCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
@@ -72,7 +73,13 @@ public class BatchExecPythonOverAggregate extends BatchExecOverAggregateBase {
InputProperty inputProperty,
RowType outputType,
String description) {
- super(overSpec, inputProperty, outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecPythonOverAggregate.class),
+ overSpec,
+ inputProperty,
+ outputType,
+ description);
lowerBoundary = new ArrayList<>();
upperBoundary = new ArrayList<>();
aggCalls = new ArrayList<>();
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java
index bbe5006..c6c018e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecRank.java
@@ -26,6 +26,7 @@ import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.planner.plan.utils.SortUtil;
@@ -57,7 +58,12 @@ public class BatchExecRank extends ExecNodeBase<RowData> implements BatchExecNod
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecRank.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.partitionFields = partitionFields;
this.sortFields = sortFields;
this.rankStart = rankStart;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java
index 64a1c0c..9c8fa35 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java
@@ -24,6 +24,7 @@ import org.apache.flink.table.connector.sink.DynamicTableSink;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecSink;
import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec;
@@ -42,10 +43,11 @@ public class BatchExecSink extends CommonExecSink implements BatchExecNode<Objec
LogicalType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecSink.class),
tableSinkSpec,
ChangelogMode.insertOnly(),
true, // isBounded
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java
index c793a38..ca4128da 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSort.java
@@ -27,6 +27,7 @@ import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -50,7 +51,12 @@ public class BatchExecSort extends ExecNodeBase<RowData> implements BatchExecNod
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecSort.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.sortSpec = sortSpec;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java
index 6d47224..6c33975 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java
@@ -28,6 +28,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -65,7 +66,12 @@ public class BatchExecSortAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecSortAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.grouping = grouping;
this.auxGrouping = auxGrouping;
this.aggCalls = aggCalls;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java
index 56cb80c..4940cd1 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java
@@ -26,6 +26,7 @@ import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
@@ -58,7 +59,12 @@ public class BatchExecSortLimit extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecSortLimit.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.sortSpec = sortSpec;
this.limitStart = limitStart;
this.limitEnd = limitEnd;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
index 4bc8262..07ded9d 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java
@@ -29,6 +29,7 @@ import org.apache.flink.table.planner.codegen.sort.SortCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
@@ -74,7 +75,12 @@ public class BatchExecSortMergeJoin extends ExecNodeBase<RowData>
InputProperty rightInputProperty,
RowType outputType,
String description) {
- super(Arrays.asList(leftInputProperty, rightInputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecSortMergeJoin.class),
+ Arrays.asList(leftInputProperty, rightInputProperty),
+ outputType,
+ description);
this.joinType = checkNotNull(joinType);
this.leftKeys = checkNotNull(leftKeys);
this.rightKeys = checkNotNull(rightKeys);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java
index 4c44cac..530865e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortWindowAggregate.java
@@ -32,6 +32,7 @@ import org.apache.flink.table.planner.plan.logical.LogicalWindow;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -80,7 +81,12 @@ public class BatchExecSortWindowAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecSortWindowAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.grouping = grouping;
this.auxGrouping = auxGrouping;
this.aggCalls = aggCalls;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java
index eab5b1a..39c02fb 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java
@@ -26,6 +26,7 @@ import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan;
import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -41,7 +42,12 @@ public class BatchExecTableSourceScan extends CommonExecTableSourceScan
public BatchExecTableSourceScan(
DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) {
- super(tableSourceSpec, getNewNodeId(), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecTableSourceScan.class),
+ tableSourceSpec,
+ outputType,
+ description);
}
@Override
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java
index cc3f46e..506243b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecUnion.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecUnion;
import org.apache.flink.table.types.logical.RowType;
@@ -34,6 +35,11 @@ public class BatchExecUnion extends CommonExecUnion implements BatchExecNode<Row
public BatchExecUnion(
List<InputProperty> inputProperties, RowType outputType, String description) {
- super(getNewNodeId(), inputProperties, outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecUnion.class),
+ inputProperties,
+ outputType,
+ description);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java
index f3391bc..1c6b52f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecValues.java
@@ -22,6 +22,7 @@ import org.apache.flink.api.dag.Transformation;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecValues;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.types.logical.RowType;
@@ -34,7 +35,12 @@ import java.util.List;
public class BatchExecValues extends CommonExecValues implements BatchExecNode<RowData> {
public BatchExecValues(List<List<RexLiteral>> tuples, RowType outputType, String description) {
- super(tuples, getNewNodeId(), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecValues.class),
+ tuples,
+ outputType,
+ description);
}
@Override
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java
index 59676b1..0a7f652 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecWindowTableFunction.java
@@ -24,6 +24,7 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecWindowTableFunction;
import org.apache.flink.table.types.logical.RowType;
@@ -40,8 +41,9 @@ public class BatchExecWindowTableFunction extends CommonExecWindowTableFunction
RowType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(BatchExecWindowTableFunction.class),
windowingStrategy,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java
index 73ba163..776475e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCalc.java
@@ -25,6 +25,7 @@ import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -33,8 +34,6 @@ import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexNode;
@@ -48,7 +47,6 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Base class for exec Calc. */
-@JsonIgnoreProperties(ignoreUnknown = true)
public abstract class CommonExecCalc extends ExecNodeBase<RowData>
implements SingleTransformationTranslator<RowData> {
public static final String FIELD_NAME_PROJECTION = "projection";
@@ -60,19 +58,20 @@ public abstract class CommonExecCalc extends ExecNodeBase<RowData>
@JsonProperty(FIELD_NAME_CONDITION)
private final @Nullable RexNode condition;
- @JsonIgnore private final Class<?> operatorBaseClass;
- @JsonIgnore private final boolean retainHeader;
+ private final Class<?> operatorBaseClass;
+ private final boolean retainHeader;
protected CommonExecCalc(
+ int id,
+ ExecNodeContext context,
List<RexNode> projection,
@Nullable RexNode condition,
Class<?> operatorBaseClass,
boolean retainHeader,
- int id,
List<InputProperty> inputProperties,
RowType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.projection = checkNotNull(projection);
this.condition = condition;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java
index 69dcb45..9fcd5d6 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecCorrelate.java
@@ -26,14 +26,13 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexCall;
@@ -48,7 +47,6 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Base {@link ExecNode} which matches along with join a Java/Scala user defined table function. */
-@JsonIgnoreProperties(ignoreUnknown = true)
public abstract class CommonExecCorrelate extends ExecNodeBase<RowData>
implements SingleTransformationTranslator<RowData> {
@@ -65,20 +63,21 @@ public abstract class CommonExecCorrelate extends ExecNodeBase<RowData>
@JsonProperty(FIELD_NAME_CONDITION)
private final @Nullable RexNode condition;
- @JsonIgnore private final Class<?> operatorBaseClass;
- @JsonIgnore private final boolean retainHeader;
+ private final Class<?> operatorBaseClass;
+ private final boolean retainHeader;
public CommonExecCorrelate(
+ int id,
+ ExecNodeContext context,
FlinkJoinType joinType,
RexCall invocation,
@Nullable RexNode condition,
Class<?> operatorBaseClass,
boolean retainHeader,
- int id,
List<InputProperty> inputProperties,
RowType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.joinType = checkNotNull(joinType);
this.invocation = checkNotNull(invocation);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java
index fd91bf6..f19498a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExchange.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.types.logical.RowType;
@@ -35,7 +36,11 @@ public abstract class CommonExecExchange extends ExecNodeBase<RowData>
implements SingleTransformationTranslator<RowData> {
public CommonExecExchange(
- int id, List<InputProperty> inputProperties, RowType outputType, String description) {
- super(id, inputProperties, outputType, description);
+ int id,
+ ExecNodeContext context,
+ List<InputProperty> inputProperties,
+ RowType outputType,
+ String description) {
+ super(id, context, inputProperties, outputType, description);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java
index de28b37..da5eae0 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecExpand.java
@@ -26,6 +26,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -33,7 +34,6 @@ import org.apache.flink.table.runtime.operators.CodeGenOperatorFactory;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexNode;
@@ -52,16 +52,17 @@ public abstract class CommonExecExpand extends ExecNodeBase<RowData>
@JsonProperty(FIELD_NAME_PROJECTS)
private final List<List<RexNode>> projects;
- @JsonIgnore private final boolean retainHeader;
+ private final boolean retainHeader;
public CommonExecExpand(
+ int id,
+ ExecNodeContext context,
List<List<RexNode>> projects,
boolean retainHeader,
- int id,
List<InputProperty> inputProperties,
RowType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.projects = checkNotNull(projects);
checkArgument(
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
index af89f0a..3671fa1 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacySink.java
@@ -32,6 +32,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -67,6 +68,8 @@ public abstract class CommonExecLegacySink<T> extends ExecNodeBase<T>
protected final boolean isStreaming;
public CommonExecLegacySink(
+ int id,
+ ExecNodeContext context,
TableSink<T> tableSink,
@Nullable String[] upsertKeys,
boolean needRetraction,
@@ -74,7 +77,7 @@ public abstract class CommonExecLegacySink<T> extends ExecNodeBase<T>
InputProperty inputProperty,
LogicalType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(id, context, Collections.singletonList(inputProperty), outputType, description);
this.tableSink = tableSink;
this.upsertKeys = upsertKeys;
this.needRetraction = needRetraction;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java
index 9994d02..d7d92b9 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLegacyTableSourceScan.java
@@ -30,6 +30,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.utils.ScanUtil;
import org.apache.flink.table.planner.sources.TableSourceUtil;
@@ -67,11 +68,13 @@ public abstract class CommonExecLegacyTableSourceScan extends ExecNodeBase<RowDa
protected final List<String> qualifiedName;
public CommonExecLegacyTableSourceScan(
+ int id,
+ ExecNodeContext context,
TableSource<?> tableSource,
List<String> qualifiedName,
RowType outputType,
String description) {
- super(Collections.emptyList(), outputType, description);
+ super(id, context, Collections.emptyList(), outputType, description);
this.tableSource = tableSource;
this.qualifiedName = qualifiedName;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java
index ea00dc6..99c9ed8 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecLookupJoin.java
@@ -48,6 +48,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.TemporalTableSourceSpec;
@@ -76,8 +77,6 @@ import org.apache.flink.table.sources.TableSource;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.plan.RelOptTable;
@@ -134,7 +133,6 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* 3) join left input record and lookup-ed records <br>
* 4) only outputs the rows which match to the condition <br>
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData>
implements SingleTransformationTranslator<RowData> {
@@ -169,11 +167,13 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData>
@JsonProperty(FIELD_NAME_JOIN_CONDITION)
private final @Nullable RexNode joinCondition;
- @JsonIgnore private final boolean existCalcOnTemporalTable;
+ private final boolean existCalcOnTemporalTable;
- @JsonIgnore private final @Nullable RelDataType temporalTableOutputType;
+ private final @Nullable RelDataType temporalTableOutputType;
protected CommonExecLookupJoin(
+ int id,
+ ExecNodeContext context,
FlinkJoinType joinType,
@Nullable RexNode joinCondition,
// TODO: refactor this into TableSourceTable, once legacy TableSource is removed
@@ -181,11 +181,10 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData>
Map<Integer, LookupJoinUtil.LookupKey> lookupKeys,
@Nullable List<RexNode> projectionOnTemporalTable,
@Nullable RexNode filterOnTemporalTable,
- int id,
List<InputProperty> inputProperties,
RowType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.joinType = checkNotNull(joinType);
this.joinCondition = joinCondition;
@@ -204,7 +203,6 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase<RowData>
}
}
- @JsonIgnore
public TemporalTableSourceSpec getTemporalTableSourceSpec() {
return temporalTableSourceSpec;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java
index db17df4..dc0b670 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCalc.java
@@ -36,6 +36,7 @@ import org.apache.flink.table.planner.codegen.ProjectionCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
@@ -46,7 +47,6 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexCall;
@@ -64,7 +64,6 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Base class for exec Python Calc. */
-@JsonIgnoreProperties(ignoreUnknown = true)
public abstract class CommonExecPythonCalc extends ExecNodeBase<RowData>
implements SingleTransformationTranslator<RowData> {
@@ -86,12 +85,13 @@ public abstract class CommonExecPythonCalc extends ExecNodeBase<RowData>
private final List<RexNode> projection;
public CommonExecPythonCalc(
- List<RexNode> projection,
int id,
+ ExecNodeContext context,
+ List<RexNode> projection,
List<InputProperty> inputProperties,
RowType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.projection = checkNotNull(projection);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java
index 30eb6e3..9bb417c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecPythonCorrelate.java
@@ -35,6 +35,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
@@ -44,7 +45,6 @@ import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexCall;
@@ -58,7 +58,6 @@ import java.util.List;
import static org.apache.flink.util.Preconditions.checkArgument;
/** Base {@link ExecNode} which matches along with join a Python user defined table function. */
-@JsonIgnoreProperties(ignoreUnknown = true)
public abstract class CommonExecPythonCorrelate extends ExecNodeBase<RowData>
implements SingleTransformationTranslator<RowData> {
@@ -75,13 +74,14 @@ public abstract class CommonExecPythonCorrelate extends ExecNodeBase<RowData>
private final RexCall invocation;
public CommonExecPythonCorrelate(
+ int id,
+ ExecNodeContext context,
FlinkJoinType joinType,
RexCall invocation,
- int id,
List<InputProperty> inputProperties,
RowType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.joinType = joinType;
this.invocation = invocation;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java
index bca1997..c4edb08 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecSink.java
@@ -51,6 +51,7 @@ import org.apache.flink.table.planner.connectors.TransformationSinkProvider;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec;
@@ -74,7 +75,6 @@ import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.table.types.logical.utils.LogicalTypeChecks;
import org.apache.flink.types.RowKind;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.ArrayList;
@@ -94,18 +94,19 @@ public abstract class CommonExecSink extends ExecNodeBase<Object>
@JsonProperty(FIELD_NAME_DYNAMIC_TABLE_SINK)
protected final DynamicTableSinkSpec tableSinkSpec;
- @JsonIgnore private final ChangelogMode inputChangelogMode;
- @JsonIgnore private final boolean isBounded;
+ private final ChangelogMode inputChangelogMode;
+ private final boolean isBounded;
protected CommonExecSink(
+ int id,
+ ExecNodeContext context,
DynamicTableSinkSpec tableSinkSpec,
ChangelogMode inputChangelogMode,
boolean isBounded,
- int id,
List<InputProperty> inputProperties,
LogicalType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.tableSinkSpec = tableSinkSpec;
this.inputChangelogMode = inputChangelogMode;
this.isBounded = isBounded;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java
index f1dd3c2..0f50eb5 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecTableSourceScan.java
@@ -39,6 +39,7 @@ import org.apache.flink.table.planner.connectors.TransformationScanProvider;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec;
import org.apache.flink.table.runtime.connector.source.ScanRuntimeProviderContext;
@@ -61,11 +62,12 @@ public abstract class CommonExecTableSourceScan extends ExecNodeBase<RowData>
private final DynamicTableSourceSpec tableSourceSpec;
protected CommonExecTableSourceScan(
- DynamicTableSourceSpec tableSourceSpec,
int id,
+ ExecNodeContext context,
+ DynamicTableSourceSpec tableSourceSpec,
LogicalType outputType,
String description) {
- super(id, Collections.emptyList(), outputType, description);
+ super(id, context, Collections.emptyList(), outputType, description);
this.tableSourceSpec = tableSourceSpec;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java
index 02b3315..05591f2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecUnion.java
@@ -25,6 +25,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.types.logical.RowType;
@@ -40,8 +41,12 @@ public abstract class CommonExecUnion extends ExecNodeBase<RowData>
implements SingleTransformationTranslator<RowData> {
public CommonExecUnion(
- int id, List<InputProperty> inputProperties, RowType outputType, String description) {
- super(id, inputProperties, outputType, description);
+ int id,
+ ExecNodeContext context,
+ List<InputProperty> inputProperties,
+ RowType outputType,
+ String description) {
+ super(id, context, inputProperties, outputType, description);
}
@SuppressWarnings({"unchecked", "rawtypes"})
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java
index 65f6869..6712302 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecValues.java
@@ -24,12 +24,12 @@ import org.apache.flink.table.planner.codegen.ValuesCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.serde.RexNodeJsonSerializer;
import org.apache.flink.table.runtime.operators.values.ValuesInputFormat;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexLiteral;
@@ -44,11 +44,15 @@ public abstract class CommonExecValues extends ExecNodeBase<RowData>
public static final String FIELD_NAME_TUPLES = "tuples";
- @JsonIgnore private final List<List<RexLiteral>> tuples;
+ private final List<List<RexLiteral>> tuples;
public CommonExecValues(
- List<List<RexLiteral>> tuples, int id, RowType outputType, String description) {
- super(id, Collections.emptyList(), outputType, description);
+ int id,
+ ExecNodeContext context,
+ List<List<RexLiteral>> tuples,
+ RowType outputType,
+ String description) {
+ super(id, context, Collections.emptyList(), outputType, description);
this.tuples = tuples;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
index b951e07..d0e96c0 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/common/CommonExecWindowTableFunction.java
@@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrateg
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNode;
@@ -36,7 +37,6 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.runtime.util.TimeWindowUtil;
import org.apache.flink.table.types.logical.RowType;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.time.ZoneId;
@@ -47,7 +47,6 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Base {@link ExecNode} for window table-valued function. */
-@JsonIgnoreProperties(ignoreUnknown = true)
public abstract class CommonExecWindowTableFunction extends ExecNodeBase<RowData>
implements BatchExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -57,12 +56,13 @@ public abstract class CommonExecWindowTableFunction extends ExecNodeBase<RowData
protected final TimeAttributeWindowingStrategy windowingStrategy;
protected CommonExecWindowTableFunction(
- TimeAttributeWindowingStrategy windowingStrategy,
int id,
+ ExecNodeContext context,
+ TimeAttributeWindowingStrategy windowingStrategy,
List<InputProperty> inputProperties,
RowType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.windowingStrategy = checkNotNull(windowingStrategy);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeGraphJsonPlanGenerator.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeGraphJsonPlanGenerator.java
index a94fa7f..12bb2fe 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeGraphJsonPlanGenerator.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ExecNodeGraphJsonPlanGenerator.java
@@ -27,10 +27,11 @@ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLookupJoi
import org.apache.flink.table.planner.plan.nodes.exec.visitor.AbstractExecNodeExactlyOnceVisitor;
import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitor;
import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitorImpl;
+import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil;
import org.apache.flink.shaded.guava30.com.google.common.collect.Sets;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ObjectWriter;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
@@ -75,11 +76,12 @@ public class ExecNodeGraphJsonPlanGenerator {
new AbstractExecNodeExactlyOnceVisitor() {
@Override
protected void visitNode(ExecNode<?> node) {
- if (!JsonSerdeUtil.hasJsonCreatorAnnotation(node.getClass())) {
+ if (!ExecNodeMetadataUtil.execNodes().contains(node.getClass())) {
throw new TableException(
String.format(
- "%s does not implement @JsonCreator annotation on constructor.",
- node.getClass().getCanonicalName()));
+ "%s is not contained in %s.execNodes().",
+ node.getClass().getCanonicalName(),
+ ExecNodeMetadataUtil.class.getCanonicalName()));
}
if (node instanceof StreamExecLookupJoin) {
StreamExecLookupJoin streamExecLookupJoin = (StreamExecLookupJoin) node;
@@ -235,6 +237,7 @@ public class ExecNodeGraphJsonPlanGenerator {
* <p>Different from {@link ExecEdge}, {@link JsonPlanEdge} only stores the {@link ExecNode}'s
* id instead of instance.
*/
+ @JsonIgnoreProperties(ignoreUnknown = true)
public static class JsonPlanEdge {
public static final String FIELD_NAME_SOURCE = "source";
public static final String FIELD_NAME_TARGET = "target";
@@ -268,22 +271,18 @@ public class ExecNodeGraphJsonPlanGenerator {
this.exchangeMode = exchangeMode;
}
- @JsonIgnore
public int getSourceId() {
return sourceId;
}
- @JsonIgnore
public int getTargetId() {
return targetId;
}
- @JsonIgnore
public ExecEdge.Shuffle getShuffle() {
return shuffle;
}
- @JsonIgnore
public StreamExchangeMode getExchangeMode() {
return exchangeMode;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java
index e69dc79..5e83e95 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/JsonSerdeUtil.java
@@ -28,8 +28,7 @@ import org.apache.flink.table.catalog.WatermarkSpec;
import org.apache.flink.table.connector.ChangelogMode;
import org.apache.flink.table.expressions.ResolvedExpression;
import org.apache.flink.table.planner.plan.logical.LogicalWindow;
-import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
-import org.apache.flink.table.planner.plan.utils.ReflectionsUtil;
+import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil;
import org.apache.flink.table.runtime.groupwindow.WindowReference;
import org.apache.flink.table.types.DataType;
import org.apache.flink.table.types.logical.LogicalType;
@@ -116,8 +115,7 @@ public class JsonSerdeUtil {
private static Module createFlinkTableJacksonModule() {
final SimpleModule module = new SimpleModule("Flink table module");
- ReflectionsUtil.scanSubClasses(
- "org.apache.flink.table.planner.plan.nodes.exec", ExecNode.class)
+ ExecNodeMetadataUtil.execNodes().stream()
.forEach(c -> module.registerSubtypes(new NamedType(c, c.getName())));
registerSerializers(module);
registerDeserializers(module);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java
index 935f261..fde9fda 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecAggregateBase.java
@@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.types.logical.LogicalType;
@@ -41,10 +42,11 @@ public abstract class StreamExecAggregateBase extends ExecNodeBase<RowData>
protected StreamExecAggregateBase(
int id,
+ ExecNodeContext context,
List<InputProperty> inputProperties,
LogicalType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
index 998dea0..2fd339a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCalc.java
@@ -18,15 +18,17 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCalc;
import org.apache.flink.table.runtime.operators.TableStreamOperator;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexNode;
@@ -37,7 +39,11 @@ import java.util.Collections;
import java.util.List;
/** Stream {@link ExecNode} for Calc. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-calc",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecCalc extends CommonExecCalc implements StreamExecNode<RowData> {
public StreamExecCalc(
@@ -47,9 +53,10 @@ public class StreamExecCalc extends CommonExecCalc implements StreamExecNode<Row
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecCalc.class),
projection,
condition,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -57,18 +64,20 @@ public class StreamExecCalc extends CommonExecCalc implements StreamExecNode<Row
@JsonCreator
public StreamExecCalc(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_PROJECTION) List<RexNode> projection,
@JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(
+ id,
+ context,
projection,
condition,
TableStreamOperator.class,
true, // retainHeader
- id,
inputProperties,
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java
index 2fd7a47..43cf78c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecChangelogNormalize.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
@@ -31,6 +32,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -46,7 +49,6 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
@@ -58,7 +60,11 @@ import java.util.List;
* changelog stream that contains INSERT/UPDATE_BEFORE/UPDATE_AFTER/DELETE records without
* duplication.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-changelog-normalize",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecChangelogNormalize extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -78,9 +84,10 @@ public class StreamExecChangelogNormalize extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecChangelogNormalize.class),
uniqueKeys,
generateUpdateBefore,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -88,13 +95,14 @@ public class StreamExecChangelogNormalize extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecChangelogNormalize(
+ @JsonProperty(FIELD_NAME_ID) Integer id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.uniqueKeys = uniqueKeys;
this.generateUpdateBefore = generateUpdateBefore;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java
index b3456ed..d01f2ce 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecCorrelate.java
@@ -18,8 +18,11 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecCorrelate;
import org.apache.flink.table.runtime.operators.TableStreamOperator;
@@ -27,7 +30,6 @@ import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexCall;
@@ -41,7 +43,11 @@ import java.util.List;
/**
* Stream {@link ExecNode} which matches along with join a Java/Scala user defined table function.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-correlate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecCorrelate extends CommonExecCorrelate implements StreamExecNode<RowData> {
public StreamExecCorrelate(
@@ -52,10 +58,11 @@ public class StreamExecCorrelate extends CommonExecCorrelate implements StreamEx
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecCorrelate.class),
joinType,
invocation,
condition,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -63,20 +70,22 @@ public class StreamExecCorrelate extends CommonExecCorrelate implements StreamEx
@JsonCreator
public StreamExecCorrelate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType,
@JsonProperty(FIELD_NAME_FUNCTION_CALL) RexNode invocation,
@JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(
+ id,
+ context,
joinType,
(RexCall) invocation,
condition,
TableStreamOperator.class,
true, // retainHeader
- id,
inputProperties,
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java
index 3d4fbef..bf4662b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDataStreamScan.java
@@ -29,6 +29,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.functions.sql.FlinkSqlOperatorTable;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.utils.ScanUtil;
import org.apache.flink.table.planner.utils.JavaScalaConversionUtil;
@@ -68,7 +69,12 @@ public class StreamExecDataStreamScan extends ExecNodeBase<RowData>
List<String> qualifiedName,
RowType outputType,
String description) {
- super(Collections.emptyList(), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecDataStreamScan.class),
+ Collections.emptyList(),
+ outputType,
+ description);
this.dataStream = dataStream;
this.sourceType = sourceType;
this.fieldIndexes = fieldIndexes;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
index dbf9767..e90496b6 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeduplicate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.dag.Transformation;
@@ -34,6 +35,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -55,7 +58,6 @@ import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
@@ -69,7 +71,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* is an optimization of {@link StreamExecRank} for some special cases. Compared to {@link
* StreamExecRank}, this node could use mini-batch and access less state.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-deduplicate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecDeduplicate extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -124,11 +130,12 @@ public class StreamExecDeduplicate extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecDeduplicate.class),
uniqueKeys,
isRowtime,
keepLastRow,
generateUpdateBefore,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -136,15 +143,16 @@ public class StreamExecDeduplicate extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecDeduplicate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys,
@JsonProperty(FIELD_NAME_IS_ROWTIME) boolean isRowtime,
@JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.uniqueKeys = checkNotNull(uniqueKeys);
this.isRowtime = isRowtime;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java
index 2f26635..421b14f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDropUpdateBefore.java
@@ -18,12 +18,15 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.StreamFilter;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -31,7 +34,6 @@ import org.apache.flink.table.runtime.operators.misc.DropUpdateBeforeFunction;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
@@ -42,22 +44,32 @@ import java.util.List;
* optimization for the downstream operators that doesn't need the UPDATE_BEFORE messages, but the
* upstream operator can't drop it by itself (e.g. the source).
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-drop-update-before",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecDropUpdateBefore extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
public StreamExecDropUpdateBefore(
InputProperty inputProperty, RowType outputType, String description) {
- this(getNewNodeId(), Collections.singletonList(inputProperty), outputType, description);
+ this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecDropUpdateBefore.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
}
@JsonCreator
public StreamExecDropUpdateBefore(
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
}
@SuppressWarnings("unchecked")
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java
index 22ad137..645b979 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExchange.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.common.ExecutionConfig;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.transformations.PartitionTransformation;
@@ -28,6 +29,8 @@ import org.apache.flink.table.api.TableException;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty.HashDistribution;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecExchange;
@@ -37,7 +40,6 @@ import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
@@ -51,20 +53,30 @@ import static org.apache.flink.util.Preconditions.checkArgument;
*
* <p>TODO Remove this class once FLINK-21224 is finished.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-exchange",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecExchange extends CommonExecExchange implements StreamExecNode<RowData> {
public StreamExecExchange(InputProperty inputProperty, RowType outputType, String description) {
- this(getNewNodeId(), Collections.singletonList(inputProperty), outputType, description);
+ this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecExchange.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
}
@JsonCreator
public StreamExecExchange(
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java
index f10b965..ea17b4c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecExpand.java
@@ -18,14 +18,16 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecExpand;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexNode;
@@ -34,7 +36,11 @@ import java.util.Collections;
import java.util.List;
/** Stream {@link ExecNode} that can expand one row to multiple rows based on given projects. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-expand",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecExpand extends CommonExecExpand implements StreamExecNode<RowData> {
public StreamExecExpand(
@@ -43,8 +49,9 @@ public class StreamExecExpand extends CommonExecExpand implements StreamExecNode
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecExpand.class),
projects,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -52,15 +59,17 @@ public class StreamExecExpand extends CommonExecExpand implements StreamExecNode
@JsonCreator
public StreamExecExpand(
- @JsonProperty(FIELD_NAME_PROJECTS) List<List<RexNode>> projects,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_PROJECTS) List<List<RexNode>> projects,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(
+ id,
+ context,
projects,
true, // retainHeader
- id,
inputProperties,
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java
index 1e26443..f92e6de 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalGroupAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
@@ -31,6 +32,8 @@ import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonDeserializer;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonSerializer;
@@ -72,6 +75,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for unbounded global group aggregate. */
+@ExecNodeMetadata(
+ name = "stream-exec-global-group-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase {
private static final Logger LOG = LoggerFactory.getLogger(StreamExecGlobalGroupAggregate.class);
@@ -119,6 +127,8 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecGlobalGroupAggregate.class),
grouping,
aggCalls,
aggCallNeedRetractions,
@@ -126,7 +136,6 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase {
generateUpdateBefore,
needRetraction,
indexOfCountStar,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -134,6 +143,8 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase {
@JsonCreator
public StreamExecGlobalGroupAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions,
@@ -141,11 +152,10 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase {
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
@JsonProperty(FIELD_NAME_NEED_RETRACTION) boolean needRetraction,
@JsonProperty(FIELD_NAME_INDEX_OF_COUNT_STAR) @Nullable Integer indexOfCountStar,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
index ec81b0f..0b63663 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGlobalWindowAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
@@ -30,6 +31,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonDeserializer;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonSerializer;
@@ -68,6 +71,11 @@ import java.util.List;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for window table-valued based global aggregate. */
+@ExecNodeMetadata(
+ name = "stream-exec-global-window-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBase {
public static final String FIELD_NAME_LOCAL_AGG_INPUT_ROW_TYPE = "localAggInputRowType";
@@ -100,11 +108,12 @@ public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBa
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecGlobalWindowAggregate.class),
grouping,
aggCalls,
windowing,
namedWindowProperties,
- getNewNodeId(),
Collections.singletonList(inputProperty),
localAggInputRowType,
outputType,
@@ -113,17 +122,18 @@ public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBa
@JsonCreator
public StreamExecGlobalWindowAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing,
@JsonProperty(FIELD_NAME_NAMED_WINDOW_PROPERTIES)
NamedWindowProperty[] namedWindowProperties,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_LOCAL_AGG_INPUT_ROW_TYPE) RowType localAggInputRowType,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.windowing = checkNotNull(windowing);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java
index cac5b7d..ab2c810 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
@@ -31,6 +32,8 @@ import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
@@ -49,7 +52,6 @@ import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rel.core.AggregateCall;
@@ -68,7 +70,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
*
* <p>This node does support un-splittable aggregate function (e.g. STDDEV_POP).
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-group-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecGroupAggregate extends StreamExecAggregateBase {
private static final Logger LOG = LoggerFactory.getLogger(StreamExecGroupAggregate.class);
@@ -100,12 +106,13 @@ public class StreamExecGroupAggregate extends StreamExecAggregateBase {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecGroupAggregate.class),
grouping,
aggCalls,
aggCallNeedRetractions,
generateUpdateBefore,
needRetraction,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -113,16 +120,17 @@ public class StreamExecGroupAggregate extends StreamExecAggregateBase {
@JsonCreator
public StreamExecGroupAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
@JsonProperty(FIELD_NAME_NEED_RETRACTION) boolean needRetraction,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java
index fce5dae..a6aa856 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupTableAggregate.java
@@ -30,6 +30,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -76,7 +77,12 @@ public class StreamExecGroupTableAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecGroupTableAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
Preconditions.checkArgument(aggCalls.length == aggCallNeedRetractions.length);
this.grouping = grouping;
this.aggCalls = aggCalls;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
index d303585..639afb8 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecGroupWindowAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.table.api.TableConfig;
@@ -36,6 +37,8 @@ import org.apache.flink.table.planner.plan.logical.SlidingGroupWindow;
import org.apache.flink.table.planner.plan.logical.TumblingGroupWindow;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonDeserializer;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonSerializer;
@@ -63,7 +66,6 @@ import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize;
@@ -100,7 +102,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* * other is from the legacy GROUP WINDOW FUNCTION syntax. In the long future, {@link
* StreamExecGroupWindowAggregate} will be dropped.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-group-window-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase {
public static final String FIELD_NAME_WINDOW = "window";
@@ -136,12 +142,13 @@ public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecGroupWindowAggregate.class),
grouping,
aggCalls,
window,
namedWindowProperties,
needRetraction,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -149,17 +156,18 @@ public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase {
@JsonCreator
public StreamExecGroupWindowAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_WINDOW) LogicalWindow window,
@JsonProperty(FIELD_NAME_NAMED_WINDOW_PROPERTIES)
NamedWindowProperty[] namedWindowProperties,
@JsonProperty(FIELD_NAME_NEED_RETRACTION) boolean needRetraction,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java
index 76fcc33..541e39a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIncrementalGroupAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
@@ -28,6 +29,8 @@ import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonDeserializer;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalTypeJsonSerializer;
@@ -60,6 +63,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for unbounded incremental group aggregate. */
+@ExecNodeMetadata(
+ name = "stream-exec-incremental-group-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase {
public static final String FIELD_NAME_PARTIAL_AGG_GROUPING = "partialAggGrouping";
@@ -108,13 +116,14 @@ public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecIncrementalGroupAggregate.class),
partialAggGrouping,
finalAggGrouping,
partialOriginalAggCalls,
partialAggCallNeedRetractions,
partialLocalAggInputType,
partialAggNeedRetraction,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -122,6 +131,8 @@ public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase
@JsonCreator
public StreamExecIncrementalGroupAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_PARTIAL_AGG_GROUPING) int[] partialAggGrouping,
@JsonProperty(FIELD_NAME_FINAL_AGG_GROUPING) int[] finalAggGrouping,
@JsonProperty(FIELD_NAME_PARTIAL_ORIGINAL_AGG_CALLS)
@@ -130,11 +141,10 @@ public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase
boolean[] partialAggCallNeedRetractions,
@JsonProperty(FIELD_NAME_PARTIAL_LOCAL_AGG_INPUT_TYPE) RowType partialLocalAggInputType,
@JsonProperty(FIELD_NAME_PARTIAL_AGG_NEED_RETRACTION) boolean partialAggNeedRetraction,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.partialAggGrouping = checkNotNull(partialAggGrouping);
this.finalAggGrouping = checkNotNull(finalAggGrouping);
this.partialOriginalAggCalls = checkNotNull(partialOriginalAggCalls);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java
index 036d71e..00165d9 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecIntervalJoin.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.configuration.Configuration;
import org.apache.flink.streaming.api.operators.StreamFlatMap;
@@ -32,6 +33,8 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.IntervalJoinSpec;
@@ -55,7 +58,6 @@ import org.apache.flink.util.Preconditions;
import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.slf4j.Logger;
@@ -64,7 +66,11 @@ import org.slf4j.LoggerFactory;
import java.util.List;
/** {@link StreamExecNode} for a time interval stream join. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-interval-join",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, MultipleTransformationTranslator<RowData> {
@@ -81,8 +87,9 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecIntervalJoin.class),
intervalJoinSpec,
- getNewNodeId(),
Lists.newArrayList(leftInputProperty, rightInputProperty),
outputType,
description);
@@ -90,12 +97,13 @@ public class StreamExecIntervalJoin extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecIntervalJoin(
- @JsonProperty(FIELD_NAME_INTERVAL_JOIN_SPEC) IntervalJoinSpec intervalJoinSpec,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_INTERVAL_JOIN_SPEC) IntervalJoinSpec intervalJoinSpec,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
Preconditions.checkArgument(inputProperties.size() == 2);
this.intervalJoinSpec = Preconditions.checkNotNull(intervalJoinSpec);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java
index c6461ae..6035814 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecJoin.java
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
import org.apache.flink.table.api.TableConfig;
@@ -26,6 +27,8 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
@@ -44,7 +47,6 @@ import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
@@ -58,7 +60,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* <p>Regular joins are the most generic type of join in which any new records or changes to either
* side of the join input are visible and are affecting the whole join result.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-join",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecJoin extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
public static final String FIELD_NAME_JOIN_SPEC = "joinSpec";
@@ -83,10 +89,11 @@ public class StreamExecJoin extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecJoin.class),
joinSpec,
leftUniqueKeys,
rightUniqueKeys,
- getNewNodeId(),
Lists.newArrayList(leftInputProperty, rightInputProperty),
outputType,
description);
@@ -94,14 +101,15 @@ public class StreamExecJoin extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecJoin(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec,
@JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) List<int[]> leftUniqueKeys,
@JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) List<int[]> rightUniqueKeys,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 2);
this.joinSpec = checkNotNull(joinSpec);
this.leftUniqueKeys = leftUniqueKeys;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java
index b1e4ee6..eeb6840 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacySink.java
@@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacySink;
import org.apache.flink.table.runtime.typeutils.TypeCheckUtils;
@@ -50,6 +51,8 @@ public class StreamExecLegacySink<T> extends CommonExecLegacySink<T> implements
LogicalType outputType,
String description) {
super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecLegacySink.class),
tableSink,
upsertKeys,
needRetraction,
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java
index e8b182d..f8fe5a3 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLegacyTableSourceScan.java
@@ -31,6 +31,7 @@ import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
import org.apache.flink.table.planner.codegen.OperatorCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLegacyTableSourceScan;
import org.apache.flink.table.planner.plan.utils.ScanUtil;
import org.apache.flink.table.planner.sources.TableSourceUtil;
@@ -67,7 +68,13 @@ public class StreamExecLegacyTableSourceScan extends CommonExecLegacyTableSource
List<String> qualifiedName,
RowType outputType,
String description) {
- super(tableSource, qualifiedName, outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecLegacyTableSourceScan.class),
+ tableSource,
+ qualifiedName,
+ outputType,
+ description);
}
@SuppressWarnings("unchecked")
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java
index b1d7210..33b04a2 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLimit.java
@@ -18,11 +18,14 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
@@ -32,18 +35,20 @@ import org.apache.flink.table.runtime.operators.rank.RankType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
import java.util.List;
/** Stream {@link ExecNode} for Limit. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-limit",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecLimit extends StreamExecRank {
- @JsonIgnore private final long limitEnd;
+ private final long limitEnd;
public StreamExecLimit(
long limitStart,
@@ -54,10 +59,11 @@ public class StreamExecLimit extends StreamExecRank {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecLimit.class),
new ConstantRankRange(limitStart + 1, limitEnd),
getRankStrategy(needRetraction),
generateUpdateBefore,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -65,14 +71,17 @@ public class StreamExecLimit extends StreamExecRank {
@JsonCreator
public StreamExecLimit(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange,
@JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(
+ id,
+ context,
RankType.ROW_NUMBER,
PartitionSpec.ALL_IN_ONE,
SortSpec.ANY,
@@ -80,7 +89,6 @@ public class StreamExecLimit extends StreamExecRank {
rankStrategy,
false, // outputRankNumber
generateUpdateBefore,
- id,
inputProperties,
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java
index 26ddb89..b476f0f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalGroupAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.codegen.CodeGeneratorContext;
@@ -25,6 +26,8 @@ import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
@@ -51,6 +54,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for unbounded local group aggregate. */
+@ExecNodeMetadata(
+ name = "stream-exec-local-group-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecLocalGroupAggregate extends StreamExecAggregateBase {
@JsonProperty(FIELD_NAME_GROUPING)
@@ -76,11 +84,12 @@ public class StreamExecLocalGroupAggregate extends StreamExecAggregateBase {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecLocalGroupAggregate.class),
grouping,
aggCalls,
aggCallNeedRetractions,
needRetraction,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -88,15 +97,16 @@ public class StreamExecLocalGroupAggregate extends StreamExecAggregateBase {
@JsonCreator
public StreamExecLocalGroupAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions,
@JsonProperty(FIELD_NAME_NEED_RETRACTION) boolean needRetraction,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
index 67b8803..93ec6cd 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLocalWindowAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
@@ -29,6 +30,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
@@ -64,6 +67,11 @@ import java.util.List;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for window table-valued based local aggregate. */
+@ExecNodeMetadata(
+ name = "stream-exec-local-window-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBase {
private static final long WINDOW_AGG_MEMORY_RATIO = 100;
@@ -87,10 +95,11 @@ public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBas
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecLocalWindowAggregate.class),
grouping,
aggCalls,
windowing,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -98,14 +107,15 @@ public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBas
@JsonCreator
public StreamExecLocalWindowAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.windowing = checkNotNull(windowing);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java
index 845d188..9ca03e5 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecLookupJoin.java
@@ -18,7 +18,10 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecLookupJoin;
import org.apache.flink.table.planner.plan.nodes.exec.spec.TemporalTableSourceSpec;
@@ -27,7 +30,6 @@ import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexNode;
@@ -39,7 +41,11 @@ import java.util.List;
import java.util.Map;
/** {@link StreamExecNode} for temporal table join that implemented by lookup. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-lookup-join",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecLookupJoin extends CommonExecLookupJoin implements StreamExecNode<RowData> {
public StreamExecLookupJoin(
FlinkJoinType joinType,
@@ -52,13 +58,14 @@ public class StreamExecLookupJoin extends CommonExecLookupJoin implements Stream
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecLookupJoin.class),
joinType,
joinCondition,
temporalTableSourceSpec,
lookupKeys,
projectionOnTemporalTable,
filterOnTemporalTable,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -66,6 +73,8 @@ public class StreamExecLookupJoin extends CommonExecLookupJoin implements Stream
@JsonCreator
public StreamExecLookupJoin(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType,
@JsonProperty(FIELD_NAME_JOIN_CONDITION) @Nullable RexNode joinCondition,
@JsonProperty(FIELD_NAME_TEMPORAL_TABLE)
@@ -75,18 +84,18 @@ public class StreamExecLookupJoin extends CommonExecLookupJoin implements Stream
List<RexNode> projectionOnTemporalTable,
@JsonProperty(FIELD_NAME_FILTER_ON_TEMPORAL_TABLE) @Nullable
RexNode filterOnTemporalTable,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(
+ id,
+ context,
joinType,
joinCondition,
temporalTableSourceSpec,
lookupKeys,
projectionOnTemporalTable,
filterOnTemporalTable,
- id,
inputProperties,
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
index 0e529dc..23affc0 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMatch.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.api.common.typeutils.TypeSerializer;
import org.apache.flink.api.dag.Transformation;
@@ -43,6 +44,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.MatchSpec;
@@ -63,7 +66,6 @@ import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.MathUtils;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexCall;
@@ -86,7 +88,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} which matches along with MATCH_RECOGNIZE. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-match",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecMatch extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, MultipleTransformationTranslator<RowData> {
@@ -101,8 +107,9 @@ public class StreamExecMatch extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecMatch.class),
matchSpec,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -110,12 +117,13 @@ public class StreamExecMatch extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecMatch(
- @JsonProperty(FIELD_NAME_MATCH_SPEC) MatchSpec matchSpec,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_MATCH_SPEC) MatchSpec matchSpec,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.matchSpec = checkNotNull(matchSpec);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java
index cbaf8cf..02f08b8 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMiniBatchAssigner.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.table.api.TableException;
@@ -25,6 +26,8 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -52,6 +55,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* because runtime doesn't support customized events and the watermark mechanism fully meets
* mini-batch needs.
*/
+@ExecNodeMetadata(
+ name = "stream-exec-mini-batch-assigner",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecMiniBatchAssigner extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -66,8 +74,9 @@ public class StreamExecMiniBatchAssigner extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecMiniBatchAssigner.class),
miniBatchInterval,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -75,12 +84,13 @@ public class StreamExecMiniBatchAssigner extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecMiniBatchAssigner(
- @JsonProperty(FIELD_NAME_MINI_BATCH_INTERVAL) MiniBatchInterval miniBatchInterval,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_MINI_BATCH_INTERVAL) MiniBatchInterval miniBatchInterval,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.miniBatchInterval = checkNotNull(miniBatchInterval);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java
index 24d2998..4493c8a 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecMultipleInput.java
@@ -23,6 +23,7 @@ import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import java.util.List;
@@ -61,7 +62,12 @@ public class StreamExecMultipleInput extends ExecNodeBase<RowData>
public StreamExecMultipleInput(
List<InputProperty> inputProperties, ExecNode<?> rootNode, String description) {
- super(inputProperties, rootNode.getOutputType(), description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecMultipleInput.class),
+ inputProperties,
+ rootNode.getOutputType(),
+ description);
this.rootNode = rootNode;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
index 5d984fb..9151244 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecOverAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.functions.KeyedProcessFunction;
import org.apache.flink.streaming.api.operators.KeyedProcessOperator;
@@ -32,6 +33,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
@@ -56,7 +59,6 @@ import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rel.core.AggregateCall;
@@ -78,7 +80,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for time-based over operator. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-over-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecOverAggregate extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
private static final Logger LOG = LoggerFactory.getLogger(StreamExecOverAggregate.class);
@@ -94,8 +100,9 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecOverAggregate.class),
overSpec,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -103,12 +110,13 @@ public class StreamExecOverAggregate extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecOverAggregate(
- @JsonProperty(FIELD_NAME_OVER_SPEC) OverSpec overSpec,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_OVER_SPEC) OverSpec overSpec,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.overSpec = checkNotNull(overSpec);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java
index 2fc7fb9..d85b437 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCalc.java
@@ -18,14 +18,16 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecPythonCalc;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexNode;
@@ -34,7 +36,11 @@ import java.util.Collections;
import java.util.List;
/** Stream {@link ExecNode} for Python ScalarFunctions. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-python-calc",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecPythonCalc extends CommonExecPythonCalc implements StreamExecNode<RowData> {
public StreamExecPythonCalc(
@@ -43,8 +49,9 @@ public class StreamExecPythonCalc extends CommonExecPythonCalc implements Stream
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecPythonCalc.class),
projection,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -52,11 +59,12 @@ public class StreamExecPythonCalc extends CommonExecPythonCalc implements Stream
@JsonCreator
public StreamExecPythonCalc(
- @JsonProperty(FIELD_NAME_PROJECTION) List<RexNode> projection,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_PROJECTION) List<RexNode> projection,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(projection, id, inputProperties, outputType, description);
+ super(id, context, projection, inputProperties, outputType, description);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java
index 91585e9e..eabcedc 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonCorrelate.java
@@ -18,14 +18,16 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecPythonCorrelate;
import org.apache.flink.table.runtime.operators.join.FlinkJoinType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexCall;
@@ -35,9 +37,14 @@ import java.util.Collections;
import java.util.List;
/** Stream exec node which matches along with join a Python user defined table function. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-python-correlate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecPythonCorrelate extends CommonExecPythonCorrelate
implements StreamExecNode<RowData> {
+
public StreamExecPythonCorrelate(
FlinkJoinType joinType,
RexCall invocation,
@@ -45,9 +52,10 @@ public class StreamExecPythonCorrelate extends CommonExecPythonCorrelate
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecPythonCorrelate.class),
joinType,
invocation,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -55,12 +63,20 @@ public class StreamExecPythonCorrelate extends CommonExecPythonCorrelate
@JsonCreator
public StreamExecPythonCorrelate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType,
@JsonProperty(FIELD_NAME_FUNCTION_CALL) RexNode invocation,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(joinType, (RexCall) invocation, id, inputProperties, outputType, description);
+ super(
+ id,
+ context,
+ joinType,
+ (RexCall) invocation,
+ inputProperties,
+ outputType,
+ description);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
index 7491278..4e08596 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
@@ -31,6 +32,8 @@ import org.apache.flink.table.functions.python.PythonAggregateFunctionInfo;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -59,6 +62,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for Python unbounded group aggregate. */
+@ExecNodeMetadata(
+ name = "stream-exec-python-group-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase {
private static final Logger LOG = LoggerFactory.getLogger(StreamExecPythonGroupAggregate.class);
@@ -90,12 +98,13 @@ public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecPythonGroupAggregate.class),
grouping,
aggCalls,
aggCallNeedRetractions,
generateUpdateBefore,
needRetraction,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -103,16 +112,17 @@ public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase {
@JsonCreator
public StreamExecPythonGroupAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
@JsonProperty(FIELD_NAME_NEED_RETRACTION) boolean needRetraction,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.aggCallNeedRetractions = checkNotNull(aggCallNeedRetractions);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
index ef68aab..ee57f86 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupTableAggregate.java
@@ -32,6 +32,7 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.CommonPythonUtil;
@@ -78,7 +79,12 @@ public class StreamExecPythonGroupTableAggregate extends ExecNodeBase<RowData>
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecPythonGroupTableAggregate.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.grouping = grouping;
this.aggCalls = aggCalls;
this.aggCallNeedRetractions = aggCallNeedRetractions;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
index facfa77..c5dfc5f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonGroupWindowAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
@@ -43,6 +44,8 @@ import org.apache.flink.table.planner.plan.logical.SlidingGroupWindow;
import org.apache.flink.table.planner.plan.logical.TumblingGroupWindow;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonDeserializer;
import org.apache.flink.table.planner.plan.nodes.exec.serde.LogicalWindowJsonSerializer;
@@ -73,7 +76,6 @@ import org.apache.flink.table.runtime.util.TimeWindowUtil;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonSerialize;
@@ -102,7 +104,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for group widow aggregate (Python user defined aggregate function). */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-python-group-window-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBase {
private static final Logger LOGGER =
LoggerFactory.getLogger(StreamExecPythonGroupWindowAggregate.class);
@@ -157,13 +163,14 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecPythonGroupWindowAggregate.class),
grouping,
aggCalls,
window,
namedWindowProperties,
generateUpdateBefore,
needRetraction,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -171,6 +178,8 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas
@JsonCreator
public StreamExecPythonGroupWindowAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_WINDOW) LogicalWindow window,
@@ -178,11 +187,10 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas
NamedWindowProperty[] namedWindowProperties,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
@JsonProperty(FIELD_NAME_NEED_RETRACTION) boolean needRetraction,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
index 6ada01d..4a8d5f0 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecPythonOverAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.api.java.tuple.Tuple2;
import org.apache.flink.configuration.Configuration;
@@ -35,6 +36,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.OverSpec;
@@ -67,6 +70,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for python time-based over operator. */
+@ExecNodeMetadata(
+ name = "stream-exec-python-over-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecPythonOverAggregate extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
private static final Logger LOG = LoggerFactory.getLogger(StreamExecPythonOverAggregate.class);
@@ -99,8 +107,9 @@ public class StreamExecPythonOverAggregate extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecPythonOverAggregate.class),
overSpec,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -108,12 +117,13 @@ public class StreamExecPythonOverAggregate extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecPythonOverAggregate(
- @JsonProperty(FIELD_NAME_OVER_SPEC) OverSpec overSpec,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_OVER_SPEC) OverSpec overSpec,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.overSpec = checkNotNull(overSpec);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java
index 40ad72b..d702d4d 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecRank.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.annotation.Experimental;
import org.apache.flink.api.common.state.StateTtlConfig;
import org.apache.flink.api.dag.Transformation;
@@ -34,6 +35,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
@@ -61,7 +64,6 @@ import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
@@ -72,7 +74,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for Rank. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-rank",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecRank extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -127,6 +133,8 @@ public class StreamExecRank extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecRank.class),
rankType,
partitionSpec,
sortSpec,
@@ -134,7 +142,6 @@ public class StreamExecRank extends ExecNodeBase<RowData>
rankStrategy,
outputRankNumber,
generateUpdateBefore,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -142,6 +149,8 @@ public class StreamExecRank extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecRank(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType,
@JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec,
@JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec,
@@ -149,11 +158,10 @@ public class StreamExecRank extends ExecNodeBase<RowData>
@JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy,
@JsonProperty(FIELD_NAME_OUTPUT_RANK_NUMBER) boolean outputRankNumber,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.rankType = checkNotNull(rankType);
this.rankRange = checkNotNull(rankRange);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java
index b9528a0..0f9d4e3 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSink.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.connector.ChangelogMode;
@@ -27,6 +28,8 @@ import org.apache.flink.table.planner.connectors.CollectDynamicSink;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecSink;
import org.apache.flink.table.planner.plan.nodes.exec.serde.ChangelogModeJsonDeserializer;
@@ -37,7 +40,6 @@ import org.apache.flink.table.types.logical.LogicalType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonInclude;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.annotation.JsonDeserialize;
@@ -52,7 +54,11 @@ import java.util.stream.Collectors;
* Stream {@link ExecNode} to to write data into an external sink defined by a {@link
* DynamicTableSink}.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-sink",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecSink extends CommonExecSink implements StreamExecNode<Object> {
public static final String FIELD_NAME_INPUT_CHANGELOG_MODE = "inputChangelogMode";
@@ -74,32 +80,33 @@ public class StreamExecSink extends CommonExecSink implements StreamExecNode<Obj
LogicalType outputType,
boolean upsertMaterialize,
String description) {
- super(
+ this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecSink.class),
tableSinkSpec,
inputChangelogMode,
- false, // isBounded
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
+ upsertMaterialize,
description);
- this.inputChangelogMode = inputChangelogMode;
- this.upsertMaterialize = upsertMaterialize;
}
@JsonCreator
public StreamExecSink(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_DYNAMIC_TABLE_SINK) DynamicTableSinkSpec tableSinkSpec,
@JsonProperty(FIELD_NAME_INPUT_CHANGELOG_MODE) ChangelogMode inputChangelogMode,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) LogicalType outputType,
@JsonProperty(FIELD_NAME_REQUIRE_UPSERT_MATERIALIZE) boolean upsertMaterialize,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(
+ id,
+ context,
tableSinkSpec,
inputChangelogMode,
false, // isBounded
- id,
inputProperties,
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java
index 3ff0a5b..f1d7edd 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSort.java
@@ -29,6 +29,7 @@ import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -63,7 +64,12 @@ public class StreamExecSort extends ExecNodeBase<RowData> implements StreamExecN
InputProperty inputProperty,
RowType outputType,
String description) {
- super(Collections.singletonList(inputProperty), outputType, description);
+ super(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecSort.class),
+ Collections.singletonList(inputProperty),
+ outputType,
+ description);
this.sortSpec = sortSpec;
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java
index ed27223..bb4b44c 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecSortLimit.java
@@ -18,10 +18,13 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.table.api.TableException;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
@@ -31,18 +34,20 @@ import org.apache.flink.table.runtime.operators.rank.RankType;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnore;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
import java.util.List;
/** {@link StreamExecNode} for Sort with limit. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-sort-limit",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecSortLimit extends StreamExecRank {
- @JsonIgnore private final long limitEnd;
+ private final long limitEnd;
public StreamExecSortLimit(
SortSpec sortSpec,
@@ -54,11 +59,12 @@ public class StreamExecSortLimit extends StreamExecRank {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecSortLimit.class),
sortSpec,
new ConstantRankRange(limitStart + 1, limitEnd),
rankStrategy,
generateUpdateBefore,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -66,16 +72,19 @@ public class StreamExecSortLimit extends StreamExecRank {
@JsonCreator
public StreamExecSortLimit(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec,
@JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange,
@JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy,
@JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
super(
+ id,
+ context,
RankType.ROW_NUMBER,
PartitionSpec.ALL_IN_ONE,
sortSpec,
@@ -83,7 +92,6 @@ public class StreamExecSortLimit extends StreamExecRank {
rankStrategy,
false, // outputRankNumber
generateUpdateBefore,
- id,
inputProperties,
outputType,
description);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java
index 5176302..3092d6e 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTableSourceScan.java
@@ -18,41 +18,53 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.common.io.InputFormat;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.connector.source.ScanTableSource;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecTableSourceScan;
import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec;
import org.apache.flink.table.runtime.typeutils.InternalTypeInfo;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
/**
* Stream {@link ExecNode} to read data from an external source defined by a {@link
* ScanTableSource}.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-table-source-scan",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecTableSourceScan extends CommonExecTableSourceScan
implements StreamExecNode<RowData> {
public StreamExecTableSourceScan(
DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) {
- super(tableSourceSpec, getNewNodeId(), outputType, description);
+ this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecTableSourceScan.class),
+ tableSourceSpec,
+ outputType,
+ description);
}
@JsonCreator
public StreamExecTableSourceScan(
- @JsonProperty(FIELD_NAME_SCAN_TABLE_SOURCE) DynamicTableSourceSpec tableSourceSpec,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_SCAN_TABLE_SOURCE) DynamicTableSourceSpec tableSourceSpec,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(tableSourceSpec, id, outputType, description);
+ super(id, context, tableSourceSpec, outputType, description);
}
@Override
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
index f244522..f31bd0d 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalJoin.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.TwoInputStreamOperator;
import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
@@ -33,6 +34,8 @@ import org.apache.flink.table.planner.codegen.GeneratedExpression;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
@@ -50,7 +53,6 @@ import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.util.Preconditions;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Arrays;
@@ -64,7 +66,11 @@ import java.util.Optional;
* <p>The legacy temporal table function join is the subset of temporal table join, the only
* difference is the validation, we reuse most same logic here.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-temporal-join",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecTemporalJoin extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -96,11 +102,12 @@ public class StreamExecTemporalJoin extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecTemporalJoin.class),
joinSpec,
isTemporalTableFunctionJoin,
leftTimeAttributeIndex,
rightTimeAttributeIndex,
- getNewNodeId(),
Arrays.asList(leftInputProperty, rightInputProperty),
outputType,
description);
@@ -108,15 +115,16 @@ public class StreamExecTemporalJoin extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecTemporalJoin(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec,
@JsonProperty(FIELD_NAME_IS_TEMPORAL_FUNCTION_JOIN) boolean isTemporalTableFunctionJoin,
@JsonProperty(FIELD_NAME_LEFT_TIME_ATTRIBUTE_INDEX) int leftTimeAttributeIndex,
@JsonProperty(FIELD_NAME_RIGHT_TIME_ATTRIBUTE_INDEX) int rightTimeAttributeIndex,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
Preconditions.checkArgument(inputProperties.size() == 2);
Preconditions.checkArgument(
rightTimeAttributeIndex == FIELD_INDEX_FOR_PROC_TIME_ATTRIBUTE
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java
index 37bde9c..093496f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecTemporalSort.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.transformations.OneInputTransformation;
import org.apache.flink.table.api.TableConfig;
@@ -27,6 +28,8 @@ import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator;
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.MultipleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.SortSpec;
@@ -51,6 +54,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** {@link StreamExecNode} for time-ascending-order Sort without `limit`. */
+@ExecNodeMetadata(
+ name = "stream-exec-temporal-sort",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecTemporalSort extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, MultipleTransformationTranslator<RowData> {
@@ -65,8 +73,9 @@ public class StreamExecTemporalSort extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecTemporalSort.class),
sortSpec,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -74,12 +83,13 @@ public class StreamExecTemporalSort extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecTemporalSort(
- @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.sortSpec = checkNotNull(sortSpec);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java
index e232968..68c9f9b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecUnion.java
@@ -18,14 +18,16 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecUnion;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.List;
@@ -34,20 +36,30 @@ import java.util.List;
* Stream {@link ExecNode} that is not a physical node and just union the inputs' records into one
* node.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-union",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecUnion extends CommonExecUnion implements StreamExecNode<RowData> {
public StreamExecUnion(
List<InputProperty> inputProperties, RowType outputType, String description) {
- this(getNewNodeId(), inputProperties, outputType, description);
+ this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecUnion.class),
+ inputProperties,
+ outputType,
+ description);
}
@JsonCreator
public StreamExecUnion(
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java
index d5c3858..ccc7384 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecValues.java
@@ -18,13 +18,15 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecValues;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.calcite.rex.RexLiteral;
@@ -32,19 +34,29 @@ import org.apache.calcite.rex.RexLiteral;
import java.util.List;
/** Stream {@link ExecNode} that read records from given values. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-values",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecValues extends CommonExecValues implements StreamExecNode<RowData> {
public StreamExecValues(List<List<RexLiteral>> tuples, RowType outputType, String description) {
- this(tuples, getNewNodeId(), outputType, description);
+ this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecValues.class),
+ tuples,
+ outputType,
+ description);
}
@JsonCreator
public StreamExecValues(
- @JsonProperty(FIELD_NAME_TUPLES) List<List<RexLiteral>> tuples,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_TUPLES) List<List<RexLiteral>> tuples,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(tuples, id, outputType, description);
+ super(id, context, tuples, outputType, description);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
index b70e071..b2f514f 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWatermarkAssigner.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.table.api.TableConfig;
import org.apache.flink.table.api.config.ExecutionConfigOptions;
@@ -27,6 +28,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -49,6 +52,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} which generates watermark based on the input elements. */
+@ExecNodeMetadata(
+ name = "stream-exec-watermark-assigner",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecWatermarkAssigner extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
public static final String FIELD_NAME_WATERMARK_EXPR = "watermarkExpr";
@@ -67,9 +75,10 @@ public class StreamExecWatermarkAssigner extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecWatermarkAssigner.class),
watermarkExpr,
rowtimeFieldIndex,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -77,13 +86,14 @@ public class StreamExecWatermarkAssigner extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecWatermarkAssigner(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_WATERMARK_EXPR) RexNode watermarkExpr,
@JsonProperty(FIELD_NAME_ROWTIME_FIELD_INDEX) int rowtimeFieldIndex,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.watermarkExpr = checkNotNull(watermarkExpr);
this.rowtimeFieldIndex = rowtimeFieldIndex;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
index 92efe00..ed09757 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
@@ -30,6 +31,8 @@ import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
import org.apache.flink.table.planner.plan.utils.AggregateInfoList;
@@ -71,6 +74,11 @@ import static org.apache.flink.util.Preconditions.checkNotNull;
* other is from the legacy GROUP WINDOW FUNCTION syntax. In the long future, {@link
* StreamExecGroupWindowAggregate} will be dropped.
*/
+@ExecNodeMetadata(
+ name = "stream-exec-window-aggregate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase {
private static final long WINDOW_AGG_MEMORY_RATIO = 100;
@@ -99,11 +107,12 @@ public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase {
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecWindowAggregate.class),
grouping,
aggCalls,
windowing,
namedWindowProperties,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -111,16 +120,17 @@ public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase {
@JsonCreator
public StreamExecWindowAggregate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_GROUPING) int[] grouping,
@JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls,
@JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing,
@JsonProperty(FIELD_NAME_NAMED_WINDOW_PROPERTIES)
NamedWindowProperty[] namedWindowProperties,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
this.grouping = checkNotNull(grouping);
this.aggCalls = checkNotNull(aggCalls);
this.windowing = checkNotNull(windowing);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java
index 055ead5..59adf55 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowAggregateBase.java
@@ -28,6 +28,7 @@ import org.apache.flink.table.planner.plan.logical.WindowAttachedWindowingStrate
import org.apache.flink.table.planner.plan.logical.WindowSpec;
import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigner;
import org.apache.flink.table.runtime.operators.window.slicing.SliceAssigners;
@@ -51,10 +52,11 @@ public abstract class StreamExecWindowAggregateBase extends StreamExecAggregateB
protected StreamExecWindowAggregateBase(
int id,
+ ExecNodeContext context,
List<InputProperty> inputProperties,
LogicalType outputType,
String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
index dd7b5a8..a234f5b 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowDeduplicate.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
@@ -31,6 +32,8 @@ import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil;
@@ -44,7 +47,6 @@ import org.apache.flink.table.runtime.util.TimeWindowUtil;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.time.ZoneId;
@@ -55,7 +57,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for Window Deduplicate. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-window-deduplicate",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecWindowDeduplicate extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -87,11 +93,12 @@ public class StreamExecWindowDeduplicate extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecWindowDeduplicate.class),
partitionKeys,
orderKey,
keepLastRow,
windowing,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -99,15 +106,16 @@ public class StreamExecWindowDeduplicate extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecWindowDeduplicate(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_PARTITION_KEYS) int[] partitionKeys,
@JsonProperty(FIELD_NAME_ORDER_KEY) int orderKey,
@JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow,
@JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.partitionKeys = checkNotNull(partitionKeys);
this.orderKey = orderKey;
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
index aee6206..3820806 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowJoin.java
@@ -19,6 +19,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.transformations.TwoInputTransformation;
import org.apache.flink.table.api.TableConfig;
@@ -29,6 +30,8 @@ import org.apache.flink.table.planner.plan.logical.WindowAttachedWindowingStrate
import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.JoinSpec;
@@ -45,7 +48,6 @@ import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.guava30.com.google.common.collect.Lists;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.time.ZoneId;
@@ -55,7 +57,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** {@link StreamExecNode} for WindowJoin. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-window-join",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecWindowJoin extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
public static final String FIELD_NAME_JOIN_SPEC = "joinSpec";
@@ -80,10 +86,11 @@ public class StreamExecWindowJoin extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecWindowJoin.class),
joinSpec,
leftWindowing,
rightWindowing,
- getNewNodeId(),
Lists.newArrayList(leftInputProperty, rightInputProperty),
outputType,
description);
@@ -91,14 +98,15 @@ public class StreamExecWindowJoin extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecWindowJoin(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec,
@JsonProperty(FIELD_NAME_LEFT_WINDOWING) WindowingStrategy leftWindowing,
@JsonProperty(FIELD_NAME_RIGHT_WINDOWING) WindowingStrategy rightWindowing,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 2);
this.joinSpec = checkNotNull(joinSpec);
validate(leftWindowing);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
index 150bc69..adb3894 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowRank.java
@@ -18,6 +18,7 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.api.dag.Transformation;
import org.apache.flink.streaming.api.operators.OneInputStreamOperator;
import org.apache.flink.streaming.api.operators.SimpleOperatorFactory;
@@ -32,6 +33,8 @@ import org.apache.flink.table.planner.plan.logical.WindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator;
import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec;
@@ -51,7 +54,6 @@ import org.apache.flink.table.runtime.util.TimeWindowUtil;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.time.ZoneId;
@@ -63,7 +65,11 @@ import static org.apache.flink.util.Preconditions.checkArgument;
import static org.apache.flink.util.Preconditions.checkNotNull;
/** Stream {@link ExecNode} for WindowRank. */
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-window-rank",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecWindowRank extends ExecNodeBase<RowData>
implements StreamExecNode<RowData>, SingleTransformationTranslator<RowData> {
@@ -105,13 +111,14 @@ public class StreamExecWindowRank extends ExecNodeBase<RowData>
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecWindowRank.class),
rankType,
partitionSpec,
sortSpec,
rankRange,
outputRankNumber,
windowing,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -119,17 +126,18 @@ public class StreamExecWindowRank extends ExecNodeBase<RowData>
@JsonCreator
public StreamExecWindowRank(
+ @JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
@JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType,
@JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec,
@JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec,
@JsonProperty(FIELD_NAME_RANK_RANG) RankRange rankRange,
@JsonProperty(FIELD_NAME_OUTPUT_RANK_NUMBER) boolean outputRankNumber,
@JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing,
- @JsonProperty(FIELD_NAME_ID) int id,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(id, inputProperties, outputType, description);
+ super(id, context, inputProperties, outputType, description);
checkArgument(inputProperties.size() == 1);
this.rankType = checkNotNull(rankType);
this.partitionSpec = checkNotNull(partitionSpec);
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java
index 23c5f45..71601d1 100644
--- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecWindowTableFunction.java
@@ -18,15 +18,17 @@
package org.apache.flink.table.planner.plan.nodes.exec.stream;
+import org.apache.flink.FlinkVersion;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy;
import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
import org.apache.flink.table.planner.plan.nodes.exec.common.CommonExecWindowTableFunction;
import org.apache.flink.table.types.logical.RowType;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
-import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonIgnoreProperties;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Collections;
@@ -38,7 +40,11 @@ import java.util.List;
* well additional 3 columns named {@code window_start}, {@code window_end}, {@code window_time} to
* indicate the assigned window.
*/
-@JsonIgnoreProperties(ignoreUnknown = true)
+@ExecNodeMetadata(
+ name = "stream-exec-window-table-function",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
public class StreamExecWindowTableFunction extends CommonExecWindowTableFunction
implements StreamExecNode<RowData> {
@@ -48,8 +54,9 @@ public class StreamExecWindowTableFunction extends CommonExecWindowTableFunction
RowType outputType,
String description) {
this(
+ ExecNodeContext.newNodeId(),
+ ExecNodeContext.newContext(StreamExecWindowTableFunction.class),
windowingStrategy,
- getNewNodeId(),
Collections.singletonList(inputProperty),
outputType,
description);
@@ -57,11 +64,12 @@ public class StreamExecWindowTableFunction extends CommonExecWindowTableFunction
@JsonCreator
public StreamExecWindowTableFunction(
- @JsonProperty(FIELD_NAME_WINDOWING) TimeAttributeWindowingStrategy windowingStrategy,
@JsonProperty(FIELD_NAME_ID) int id,
+ @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context,
+ @JsonProperty(FIELD_NAME_WINDOWING) TimeAttributeWindowingStrategy windowingStrategy,
@JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List<InputProperty> inputProperties,
@JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType,
@JsonProperty(FIELD_NAME_DESCRIPTION) String description) {
- super(windowingStrategy, id, inputProperties, outputType, description);
+ super(id, context, windowingStrategy, inputProperties, outputType, description);
}
}
diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java
new file mode 100644
index 0000000..7e3d43b
--- /dev/null
+++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtil.java
@@ -0,0 +1,290 @@
+/*
+ * 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://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.utils;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
+import org.apache.flink.table.planner.plan.nodes.exec.MultipleExecNodeMetadata;
+import org.apache.flink.table.planner.plan.nodes.exec.serde.JsonSerdeUtil;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDataStreamScan;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDropUpdateBefore;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExpand;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalWindowAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupTableAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIncrementalGroupAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIntervalJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacySink;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacyTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLimit;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalWindowAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLookupJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMatch;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMultipleInput;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecNode;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecOverAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCalc;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCorrelate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupTableAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupWindowAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonOverAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecRank;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSort;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSortLimit;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTemporalJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTemporalSort;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecUnion;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecValues;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowAggregate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowDeduplicate;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowJoin;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction;
+
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+
+/** Utility class for {@link ExecNodeMetadata} related functionality. */
+@Internal
+public final class ExecNodeMetadataUtil {
+
+ private ExecNodeMetadataUtil() {
+ // no instantiation
+ }
+
+ private static final Set<Class<? extends ExecNode<?>>> EXEC_NODES =
+ new HashSet<Class<? extends ExecNode<?>>>() {
+ {
+ add(StreamExecCalc.class);
+ add(StreamExecChangelogNormalize.class);
+ add(StreamExecCorrelate.class);
+ add(StreamExecDeduplicate.class);
+ add(StreamExecDropUpdateBefore.class);
+ add(StreamExecExchange.class);
+ add(StreamExecExpand.class);
+ add(StreamExecGlobalGroupAggregate.class);
+ add(StreamExecGlobalWindowAggregate.class);
+ add(StreamExecGroupAggregate.class);
+ add(StreamExecGroupWindowAggregate.class);
+ add(StreamExecIncrementalGroupAggregate.class);
+ add(StreamExecIntervalJoin.class);
+ add(StreamExecJoin.class);
+ add(StreamExecLimit.class);
+ add(StreamExecLocalGroupAggregate.class);
+ add(StreamExecLocalWindowAggregate.class);
+ add(StreamExecLookupJoin.class);
+ add(StreamExecMatch.class);
+ add(StreamExecMiniBatchAssigner.class);
+ add(StreamExecOverAggregate.class);
+ add(StreamExecPythonCalc.class);
+ add(StreamExecPythonCorrelate.class);
+ add(StreamExecPythonGroupAggregate.class);
+ add(StreamExecPythonGroupWindowAggregate.class);
+ add(StreamExecPythonOverAggregate.class);
+ add(StreamExecRank.class);
+ add(StreamExecSink.class);
+ add(StreamExecSortLimit.class);
+ add(StreamExecTableSourceScan.class);
+ add(StreamExecTemporalJoin.class);
+ add(StreamExecTemporalSort.class);
+ add(StreamExecUnion.class);
+ add(StreamExecValues.class);
+ add(StreamExecWatermarkAssigner.class);
+ add(StreamExecWindowAggregate.class);
+ add(StreamExecWindowDeduplicate.class);
+ add(StreamExecWindowJoin.class);
+ add(StreamExecWindowRank.class);
+ add(StreamExecWindowTableFunction.class);
+ }
+ };
+
+ private static final Map<ExecNodeNameVersion, Class<? extends ExecNode<?>>> LOOKUP_MAP =
+ new HashMap<>();
+
+ static {
+ for (Class<? extends ExecNode<?>> execNodeClass : EXEC_NODES) {
+ addToLookupMap(execNodeClass);
+ }
+ }
+
+ @SuppressWarnings("rawtypes")
+ static final Set<Class<? extends ExecNode>> UNSUPPORTED_JSON_SERDE_CLASSES =
+ new HashSet<Class<? extends ExecNode>>() {
+ {
+ add(StreamExecDataStreamScan.class);
+ add(StreamExecLegacyTableSourceScan.class);
+ add(StreamExecLegacySink.class);
+ add(StreamExecGroupTableAggregate.class);
+ add(StreamExecPythonGroupTableAggregate.class);
+ add(StreamExecSort.class);
+ add(StreamExecMultipleInput.class);
+ }
+ };
+
+ public static Set<Class<? extends ExecNode<?>>> execNodes() {
+ return EXEC_NODES;
+ }
+
+ public static Class<? extends ExecNode<?>> retrieveExecNode(String name, int version) {
+ return LOOKUP_MAP.get(new ExecNodeNameVersion(name, version));
+ }
+
+ public static <T extends ExecNode<?>> boolean isUnsupported(Class<T> execNode) {
+ return !StreamExecNode.class.isAssignableFrom(execNode)
+ || UNSUPPORTED_JSON_SERDE_CLASSES.contains(execNode);
+ }
+
+ @VisibleForTesting
+ static void addTestNode(Class<? extends ExecNode<?>> execNodeClass) {
+ addToLookupMap(execNodeClass);
+ }
+
+ private static <T extends ExecNode<?>> List<ExecNodeMetadata> extractMetadataFromAnnotation(
+ Class<T> execNodeClass) {
+ List<ExecNodeMetadata> metadata = new ArrayList<>();
+ ExecNodeMetadata annotation = execNodeClass.getDeclaredAnnotation(ExecNodeMetadata.class);
+ if (annotation != null) {
+ metadata.add(annotation);
+ }
+
+ MultipleExecNodeMetadata annotations =
+ execNodeClass.getDeclaredAnnotation(MultipleExecNodeMetadata.class);
+ if (annotations != null) {
+ if (metadata.isEmpty()) {
+ for (ExecNodeMetadata annot : annotations.value()) {
+ if (annot != null) {
+ metadata.add(annot);
+ }
+ }
+ } else {
+ throw new IllegalStateException(
+ String.format(
+ "ExecNode: %s is annotated both with %s and %s. Please use only "
+ + "%s or multiple %s",
+ execNodeClass.getCanonicalName(),
+ ExecNodeMetadata.class,
+ MultipleExecNodeMetadata.class,
+ MultipleExecNodeMetadata.class,
+ ExecNodeMetadata.class));
+ }
+ }
+ return metadata;
+ }
+
+ private static void addToLookupMap(Class<? extends ExecNode<?>> execNodeClass) {
+ if (!JsonSerdeUtil.hasJsonCreatorAnnotation(execNodeClass)) {
+ throw new IllegalStateException(
+ String.format(
+ "ExecNode: %s does not implement @JsonCreator annotation on "
+ + "constructor.",
+ execNodeClass.getCanonicalName()));
+ }
+
+ List<ExecNodeMetadata> metadata = extractMetadataFromAnnotation(execNodeClass);
+ if (metadata.isEmpty()) {
+ throw new IllegalStateException(
+ String.format(
+ "ExecNode: %s is missing %s annotation.",
+ execNodeClass.getCanonicalName(),
+ ExecNodeMetadata.class.getSimpleName()));
+ }
+
+ for (ExecNodeMetadata meta : metadata) {
+ doAddToMap(new ExecNodeNameVersion(meta.name(), meta.version()), execNodeClass);
+ }
+ }
+
+ private static void doAddToMap(
+ ExecNodeNameVersion key, Class<? extends ExecNode<?>> execNodeClass) {
+ if (LOOKUP_MAP.containsKey(key)) {
+ throw new IllegalStateException(String.format("Found duplicate ExecNode: %s.", key));
+ }
+ LOOKUP_MAP.put(key, execNodeClass);
+ }
+
+ /**
+ * Returns the {@link ExecNodeMetadata} annotation of the class with the highest (most recent)
+ * {@link ExecNodeMetadata#version()}.
+ */
+ public static <T extends ExecNode<?>> ExecNodeMetadata latestAnnotation(
+ Class<T> execNodeClass) {
+ List<ExecNodeMetadata> sortedAnnotations = extractMetadataFromAnnotation(execNodeClass);
+ if (sortedAnnotations.isEmpty()) {
+ return null;
+ }
+ sortedAnnotations.sort(Comparator.comparingInt(ExecNodeMetadata::version));
+ return sortedAnnotations.get(sortedAnnotations.size() - 1);
+ }
+
+ /** Helper Pojo used as a tuple for the {@link #LOOKUP_MAP}. */
+ private static final class ExecNodeNameVersion {
+
+ private final String name;
+ private final int version;
+
+ private ExecNodeNameVersion(String name, int version) {
+ this.name = name;
+ this.version = version;
+ }
+
+ @Override
+ public String toString() {
+ return String.format("name: %s, version: %s", name, version);
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ ExecNodeNameVersion that = (ExecNodeNameVersion) o;
+ return version == that.version && Objects.equals(name, that.name);
+ }
+
+ @Override
+ public int hashCode() {
+ return Objects.hash(name, version);
+ }
+ }
+}
diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java
new file mode 100644
index 0000000..7fd6a00
--- /dev/null
+++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/utils/ExecNodeMetadataUtilTest.java
@@ -0,0 +1,269 @@
+/*
+ * 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://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.utils;
+
+import org.apache.flink.FlinkVersion;
+import org.apache.flink.api.dag.Transformation;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.planner.delegation.PlannerBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeContext;
+import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata;
+import org.apache.flink.table.planner.plan.nodes.exec.InputProperty;
+import org.apache.flink.table.planner.plan.nodes.exec.MultipleExecNodeMetadata;
+import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecNode;
+import org.apache.flink.table.types.logical.LogicalType;
+
+import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
+
+import org.assertj.core.api.Condition;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.assertj.core.api.Assertions.assertThat;
+import static org.assertj.core.api.Assertions.assertThatThrownBy;
+
+/** Test for {@link ExecNodeMetadataUtil}. */
+public class ExecNodeMetadataUtilTest {
+
+ @Test
+ public void testNoJsonCreator() {
+ assertThatThrownBy(() -> ExecNodeMetadataUtil.addTestNode(DummyNodeNoJsonCreator.class))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessage(
+ "ExecNode: org.apache.flink.table.planner.plan.utils."
+ + "ExecNodeMetadataUtilTest.DummyNodeNoJsonCreator does not "
+ + "implement @JsonCreator annotation on constructor.");
+ }
+
+ @Test
+ public void testNoAnnotation() {
+ assertThatThrownBy(() -> ExecNodeMetadataUtil.addTestNode(DummyNodeNoAnnotation.class))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessage(
+ "ExecNode: org.apache.flink.table.planner.plan.utils."
+ + "ExecNodeMetadataUtilTest.DummyNodeNoAnnotation is missing "
+ + "ExecNodeMetadata annotation. This is a bug, please contact "
+ + "developers.");
+ }
+
+ @Test
+ public void testBothAnnotations() {
+ assertThatThrownBy(() -> ExecNodeMetadataUtil.addTestNode(DummyNodeBothAnnotations.class))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessage(
+ "ExecNode: org.apache.flink.table.planner.plan.utils."
+ + "ExecNodeMetadataUtilTest.DummyNodeBothAnnotations is annotated "
+ + "both with interface org.apache.flink.table.planner.plan.nodes."
+ + "exec.ExecNodeMetadata and interface org.apache.flink.table."
+ + "planner.plan.nodes.exec.MultipleExecNodeMetadata. Please use "
+ + "only interface org.apache.flink.table.planner.plan.nodes.exec."
+ + "MultipleExecNodeMetadata or multiple interface org.apache.flink."
+ + "table.planner.plan.nodes.exec.ExecNodeMetadata");
+ }
+
+ @Test
+ public void testMultipleAnnotations() {
+ // Using MultipleExecNodeMetadata annotation
+ ExecNodeMetadataUtil.addTestNode(DummyNode.class);
+ assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node", 1))
+ .isSameAs(DummyNode.class);
+ assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node", 2))
+ .isSameAs(DummyNode.class);
+ assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node", 3))
+ .isSameAs(DummyNode.class);
+ assertThat(ExecNodeMetadataUtil.latestAnnotation(DummyNode.class))
+ .has(new Condition<>(m -> m.version() == 3, "version"))
+ .has(
+ new Condition<>(
+ m -> m.minPlanVersion() == FlinkVersion.v1_15, "minPlanVersion"))
+ .has(
+ new Condition<>(
+ m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion"));
+
+ // Using multiple individual ExecNodeMetadata annotations
+ ExecNodeMetadataUtil.addTestNode(DummyNodeMultipleAnnotations.class);
+ assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node-multiple-annotations", 1))
+ .isSameAs(DummyNodeMultipleAnnotations.class);
+ assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node-multiple-annotations", 2))
+ .isSameAs(DummyNodeMultipleAnnotations.class);
+ assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node-multiple-annotations", 3))
+ .isSameAs(DummyNodeMultipleAnnotations.class);
+ assertThat(ExecNodeMetadataUtil.latestAnnotation(DummyNodeMultipleAnnotations.class))
+ .has(new Condition<>(m -> m.version() == 3, "version"))
+ .has(
+ new Condition<>(
+ m -> m.minPlanVersion() == FlinkVersion.v1_15, "minPlanVersion"))
+ .has(
+ new Condition<>(
+ m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion"));
+ }
+
+ @Test
+ public void testNewContext() {
+ assertThatThrownBy(() -> ExecNodeContext.newContext(DummyNodeNoAnnotation.class))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessage(
+ "ExecNode: org.apache.flink.table.planner.plan.utils."
+ + "ExecNodeMetadataUtilTest.DummyNodeNoAnnotation is not listed in the "
+ + "unsupported classes since it is not annotated with: ExecNodeMetadata.");
+
+ assertThatThrownBy(() -> ExecNodeContext.newContext(DummyNode.class))
+ .isInstanceOf(IllegalStateException.class)
+ .hasMessage(
+ "ExecNode: org.apache.flink.table.planner.plan.utils."
+ + "ExecNodeMetadataUtilTest.DummyNode is not listed in the supported "
+ + "classes and yet is annotated with: ExecNodeMetadata.");
+ }
+
+ @MultipleExecNodeMetadata({
+ @ExecNodeMetadata(
+ name = "dummy-node",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_13,
+ minStateVersion = FlinkVersion.v1_13),
+ @ExecNodeMetadata(
+ name = "dummy-node",
+ version = 2,
+ minPlanVersion = FlinkVersion.v1_14,
+ minStateVersion = FlinkVersion.v1_14),
+ @ExecNodeMetadata(
+ name = "dummy-node",
+ version = 3,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
+ })
+ private static class DummyNode extends ExecNodeBase<RowData> {
+
+ @JsonCreator
+ protected DummyNode(
+ ExecNodeContext context,
+ List<InputProperty> properties,
+ LogicalType outputType,
+ String description) {
+ super(10, context, properties, outputType, description);
+ }
+
+ @Override
+ protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
+ return null;
+ }
+ }
+
+ @ExecNodeMetadata(
+ name = "dummy-node-multiple-annotations",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_13,
+ minStateVersion = FlinkVersion.v1_13)
+ @ExecNodeMetadata(
+ name = "dummy-node-multiple-annotations",
+ version = 2,
+ minPlanVersion = FlinkVersion.v1_14,
+ minStateVersion = FlinkVersion.v1_14)
+ @ExecNodeMetadata(
+ name = "dummy-node-multiple-annotations",
+ version = 3,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
+ private static class DummyNodeMultipleAnnotations extends ExecNodeBase<RowData> {
+
+ @JsonCreator
+ protected DummyNodeMultipleAnnotations(
+ ExecNodeContext context,
+ List<InputProperty> properties,
+ LogicalType outputType,
+ String description) {
+ super(10, context, properties, outputType, description);
+ }
+
+ @Override
+ protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
+ return null;
+ }
+ }
+
+ private static class DummyNodeNoJsonCreator extends ExecNodeBase<RowData> {
+
+ protected DummyNodeNoJsonCreator(
+ ExecNodeContext context,
+ List<InputProperty> properties,
+ LogicalType outputType,
+ String description) {
+ super(10, context, properties, outputType, description);
+ }
+
+ @Override
+ protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
+ return null;
+ }
+ }
+
+ private static class DummyNodeNoAnnotation extends ExecNodeBase<RowData>
+ implements StreamExecNode<RowData> {
+
+ @JsonCreator
+ protected DummyNodeNoAnnotation(
+ ExecNodeContext context,
+ List<InputProperty> properties,
+ LogicalType outputType,
+ String description) {
+ super(10, context, properties, outputType, description);
+ }
+
+ @Override
+ protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
+ return null;
+ }
+ }
+
+ @MultipleExecNodeMetadata({
+ @ExecNodeMetadata(
+ name = "dummy-node",
+ version = 1,
+ minPlanVersion = FlinkVersion.v1_14,
+ minStateVersion = FlinkVersion.v1_14),
+ @ExecNodeMetadata(
+ name = "dummy-node",
+ version = 2,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
+ })
+ @ExecNodeMetadata(
+ name = "dummy-node",
+ version = 3,
+ minPlanVersion = FlinkVersion.v1_15,
+ minStateVersion = FlinkVersion.v1_15)
+ private static class DummyNodeBothAnnotations extends ExecNodeBase<RowData> {
+
+ @JsonCreator
+ protected DummyNodeBothAnnotations(
+ ExecNodeContext context,
+ List<InputProperty> properties,
+ LogicalType outputType,
+ String description) {
+ super(10, context, properties, outputType, description);
+ }
+
+ @Override
+ protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
+ return null;
+ }
+ }
+}
diff --git a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out
index a3646a4..1bc32ad 100644
--- a/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out
+++ b/flink-table/flink-table-planner/src/test/resources/jsonplan/testGetJsonPlan.out
@@ -2,7 +2,8 @@
"flinkVersion": "",
"nodes": [
{
- "class": "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id": 1,
+ "type": "stream-exec-table-source-scan_1",
"scanTableSource": {
"table": {
"identifier": "`default_catalog`.`default_database`.`MyTable`",
@@ -32,13 +33,13 @@
}
}
},
- "id": 1,
"outputType": "ROW<`a` BIGINT, `b` INT, `c` VARCHAR(2147483647)>",
"description": "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties": []
},
{
- "class": "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id": 2,
+ "type": "stream-exec-sink_1",
"dynamicTableSink": {
"table": {
"identifier": "`default_catalog`.`default_database`.`MySink`",
@@ -71,7 +72,6 @@
"inputChangelogMode": [
"INSERT"
],
- "id": 2,
"inputProperties": [
{
"requiredDistribution": {
@@ -95,4 +95,4 @@
"shuffleMode": "PIPELINED"
}
]
-}
\ No newline at end of file
+}
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out
index a8276e3..8c3775f 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testComplexCalc.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -34,12 +35,12 @@
"predicates" : [ ]
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, filter=[]]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -254,7 +255,6 @@
} ],
"type" : "BOOLEAN"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -265,7 +265,8 @@
"outputType" : "ROW<`a` BIGINT, `a1` VARCHAR(2147483647), `b` INT NOT NULL, `b1` VARCHAR(2147483647), `c1` VARCHAR(2147483647), `c2` VARCHAR(2147483647), `d1` TIMESTAMP(3)>",
"description" : "Calc(select=[a, CAST(a AS VARCHAR(2147483647)) AS a1, b, udf2(b, b, d) AS b1, udf3(c, b) AS c1, udf4(SUBSTRING(c, 1, 5)) AS c2, udf5(d, 1000) AS d1], where=[(((udf1(a) > 0) OR ((a * b) < 100)) AND (b > 10))])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 3,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -304,7 +305,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out
index ffe1a74..1f3fae1 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleFilter.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -34,12 +35,12 @@
"predicates" : [ ]
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, filter=[]]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -75,7 +76,6 @@
} ],
"type" : "BOOLEAN NOT NULL"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -86,7 +86,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>",
"description" : "Calc(select=[a, b, c, d], where=[(b > 0)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 3,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -116,7 +117,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out
index 039e893..2ae314d 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CalcJsonPlanTest_jsonplan/testSimpleProject.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -39,12 +40,12 @@
"producedType" : "ROW<`a` BIGINT, `b` INT NOT NULL> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 2,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -68,7 +69,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out
index bae06db..63449c0 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testChangelogSource.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -45,13 +46,12 @@
"producedType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDropUpdateBefore",
"id" : 2,
+ "type" : "stream-exec-drop-update-before_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -62,8 +62,8 @@
"outputType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL>",
"description" : "DropUpdateBefore"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 3,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -75,10 +75,10 @@
"outputType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL>",
"description" : "Exchange(distribution=[hash[a, b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize",
+ "id" : 4,
+ "type" : "stream-exec-changelog-normalize_1",
"uniqueKeys" : [ 0, 1 ],
"generateUpdateBefore" : true,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -89,7 +89,8 @@
"outputType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL>",
"description" : "ChangelogNormalize(key=[a, b])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 5,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -114,7 +115,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER", "DELETE" ],
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out
index 236531c..b89ceb2 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ChangelogSourceJsonPlanTest_jsonplan/testUpsertSource.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -45,13 +46,12 @@
"producedType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 2,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -63,10 +63,10 @@
"outputType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL>",
"description" : "Exchange(distribution=[hash[a, b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize",
+ "id" : 3,
+ "type" : "stream-exec-changelog-normalize_1",
"uniqueKeys" : [ 0, 1 ],
"generateUpdateBefore" : true,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -77,7 +77,8 @@
"outputType" : "ROW<`a` BIGINT NOT NULL, `b` INT NOT NULL>",
"description" : "ChangelogNormalize(key=[a, b])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 4,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -102,7 +103,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER", "DELETE" ],
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out
index 0a3d0f2..87de3ba 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoin.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,12 +31,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate",
+ "id" : 2,
+ "type" : "stream-exec-correlate_1",
"joinType" : "INNER",
"functionCall" : {
"kind" : "REX_CALL",
@@ -56,7 +57,6 @@
"type" : "ROW<`f0` VARCHAR(2147483647)> NOT NULL"
},
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -67,7 +67,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3), `f0` VARCHAR(2147483647)>",
"description" : "Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,f0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, TIMESTAMP(3) d, VARCHAR(2147483647) f0)], joinType=[INNER])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 2,
@@ -78,7 +79,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -89,7 +89,8 @@
"outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>",
"description" : "Calc(select=[c, f0 AS s])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 4,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -113,7 +114,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out
index a1e36d8..dc10074 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testCrossJoinOverrideParameters.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,12 +31,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate",
+ "id" : 2,
+ "type" : "stream-exec-correlate_1",
"joinType" : "INNER",
"functionCall" : {
"kind" : "REX_CALL",
@@ -60,7 +61,6 @@
"type" : "ROW<`f0` VARCHAR(2147483647)> NOT NULL"
},
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -71,7 +71,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3), `f0` VARCHAR(2147483647)>",
"description" : "Correlate(invocation=[func1($cor0.c, _UTF-16LE'$')], correlate=[table(func1($cor0.c,'$'))], select=[a,b,c,d,f0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, TIMESTAMP(3) d, VARCHAR(2147483647) f0)], joinType=[INNER])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 2,
@@ -82,7 +83,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -93,7 +93,8 @@
"outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>",
"description" : "Calc(select=[c, f0 AS s])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 4,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -117,7 +118,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out
index 0466543..4707a8c 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testJoinWithFilter.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,12 +31,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate",
+ "id" : 2,
+ "type" : "stream-exec-correlate_1",
"joinType" : "INNER",
"functionCall" : {
"kind" : "REX_CALL",
@@ -56,7 +57,6 @@
"type" : "ROW<`f0` VARCHAR(2147483647)> NOT NULL"
},
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -67,7 +67,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3), `f0` VARCHAR(2147483647)>",
"description" : "Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,f0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, TIMESTAMP(3) d, VARCHAR(2147483647) f0)], joinType=[INNER])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 2,
@@ -95,7 +96,6 @@
} ],
"type" : "BOOLEAN"
},
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -106,7 +106,8 @@
"outputType" : "ROW<`c` VARCHAR(2147483647), `f0` VARCHAR(2147483647)>",
"description" : "Calc(select=[c, f0], where=[(c = f0)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 4,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -130,7 +131,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out
index f4d582f..502cb30 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/CorrelateJsonPlanTest_jsonplan/testLeftOuterJoinWithLiteralTrue.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,12 +31,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate",
+ "id" : 2,
+ "type" : "stream-exec-correlate_1",
"joinType" : "LEFT",
"functionCall" : {
"kind" : "REX_CALL",
@@ -56,7 +57,6 @@
"type" : "ROW<`f0` VARCHAR(2147483647)> NOT NULL"
},
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -67,7 +67,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` TIMESTAMP(3), `f0` VARCHAR(2147483647)>",
"description" : "Correlate(invocation=[func1($cor0.c)], correlate=[table(func1($cor0.c))], select=[a,b,c,d,f0], rowType=[RecordType(BIGINT a, INTEGER b, VARCHAR(2147483647) c, TIMESTAMP(3) d, VARCHAR(2147483647) f0)], joinType=[LEFT])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 2,
@@ -78,7 +79,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -89,7 +89,8 @@
"outputType" : "ROW<`c` VARCHAR(2147483647), `s` VARCHAR(2147483647)>",
"description" : "Calc(select=[c, f0 AS s])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 4,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -113,7 +114,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out
index 248a7b3..563b5d8 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/DeduplicationJsonPlanTest_jsonplan/testDeduplication.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`srcValuesTable`",
@@ -52,12 +53,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`order_id` BIGINT, `user` VARCHAR(2147483647), `product` VARCHAR(2147483647), `order_time` TIMESTAMP(3)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, srcValuesTable]], fields=[order_id, user, product, order_time])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -90,7 +91,6 @@
}
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -124,8 +124,8 @@
},
"description" : "Calc(select=[order_id, user, product, order_time, PROCTIME() AS $4])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 3,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -160,12 +160,12 @@
},
"description" : "Exchange(distribution=[hash[product]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDeduplicate",
+ "id" : 4,
+ "type" : "stream-exec-deduplicate_1",
"uniqueKeys" : [ 2 ],
"isRowtime" : false,
"keepLastRow" : false,
"generateUpdateBefore" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -199,7 +199,8 @@
},
"description" : "Deduplicate(keep=[FirstRow], key=[product], order=[PROCTIME])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 5,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -218,7 +219,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -229,7 +229,8 @@
"outputType" : "ROW<`order_id` BIGINT, `user` VARCHAR(2147483647), `product` VARCHAR(2147483647), `order_time` TIMESTAMP(3)>",
"description" : "Calc(select=[order_id, user, product, order_time])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`sink`",
@@ -259,7 +260,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out
index f4765e1..cb50c43 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/ExpandJsonPlanTest_jsonplan/testExpand.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -27,12 +28,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -99,7 +100,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -110,7 +110,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` INT NOT NULL, `$f4` INT>",
"description" : "Calc(select=[a, b, c, MOD(HASH_CODE(b), 1024) AS $f3, MOD(HASH_CODE(c), 1024) AS $f4])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExpand",
+ "id" : 3,
+ "type" : "stream-exec-expand_1",
"projects" : [ [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -160,7 +161,6 @@
"value" : "2",
"type" : "BIGINT NOT NULL"
} ] ],
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -171,7 +171,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` INT, `$f4` INT, `$e` BIGINT NOT NULL>",
"description" : "Expand(projects=[{a, b, c, $f3, null AS $f4, 1 AS $e}, {a, b, c, null AS $f3, $f4, 2 AS $e}])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 4,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -228,7 +229,6 @@
"type" : "BOOLEAN NOT NULL"
} ],
"condition" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -239,8 +239,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` INT, `$f4` INT, `$g_1` BOOLEAN NOT NULL, `$g_2` BOOLEAN NOT NULL>",
"description" : "Calc(select=[a, b, c, $f3, $f4, ($e = 1) AS $g_1, ($e = 2) AS $g_2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -252,7 +252,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` INT, `$f4` INT, `$g_1` BOOLEAN NOT NULL, `$g_2` BOOLEAN NOT NULL>",
"description" : "Exchange(distribution=[hash[a, $f3, $f4]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 6,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0, 3, 4 ],
"aggCalls" : [ {
"name" : null,
@@ -284,7 +285,6 @@
"aggCallNeedRetractions" : [ false, false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -295,8 +295,8 @@
"outputType" : "ROW<`a` BIGINT, `$f3` INT, `$f4` INT, `$f3_0` BIGINT NOT NULL, `$f4_0` VARCHAR(2147483647)>",
"description" : "GroupAggregate(groupBy=[a, $f3, $f4], partialFinalType=[PARTIAL], select=[a, $f3, $f4, COUNT(DISTINCT b) FILTER $g_1 AS $f3_0, FIRST_VALUE(c) FILTER $g_2 AS $f4_0])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 7,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -308,7 +308,8 @@
"outputType" : "ROW<`a` BIGINT, `$f3` INT, `$f4` INT, `$f3_0` BIGINT NOT NULL, `$f4_0` VARCHAR(2147483647)>",
"description" : "Exchange(distribution=[hash[a]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 8,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : null,
@@ -340,7 +341,6 @@
"aggCallNeedRetractions" : [ true, true ],
"generateUpdateBefore" : true,
"needRetraction" : true,
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -351,7 +351,8 @@
"outputType" : "ROW<`a` BIGINT, `$f1` BIGINT NOT NULL, `$f2` VARCHAR(2147483647)>",
"description" : "GroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0_RETRACT($f3_0) AS $f1, FIRST_VALUE_RETRACT($f4_0) AS $f2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 9,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -379,7 +380,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER", "DELETE" ],
- "id" : 9,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out
index 6a70d4b..9c55127 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=false].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,12 +31,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 3,
@@ -71,7 +72,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -82,8 +82,8 @@
"outputType" : "ROW<`d` BIGINT, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "Calc(select=[d, a, (b > 10) AS $f2, b, c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 3,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -95,7 +95,8 @@
"outputType" : "ROW<`d` BIGINT, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "Exchange(distribution=[hash[d]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "cnt_a1",
@@ -179,7 +180,6 @@
"aggCallNeedRetractions" : [ false, false, false, false, false, false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -190,7 +190,8 @@
"outputType" : "ROW<`d` BIGINT, `cnt_a1` BIGINT NOT NULL, `cnt_a2` BIGINT NOT NULL, `sum_a` BIGINT, `sum_b` INT NOT NULL, `avg_b` INT NOT NULL, `cnt_d` BIGINT NOT NULL>",
"description" : "GroupAggregate(groupBy=[d], select=[d, COUNT(DISTINCT a) FILTER $f2 AS cnt_a1, COUNT(DISTINCT a) AS cnt_a2, SUM(DISTINCT a) AS sum_a, SUM(DISTINCT b) AS sum_b, AVG(b) AS avg_b, COUNT(DISTINCT c) AS cnt_d])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 5,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -266,7 +267,6 @@
"type" : "BIGINT"
} ],
"condition" : null,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -277,7 +277,8 @@
"outputType" : "ROW<`d` BIGINT, `cnt_a1` BIGINT, `cnt_a2` BIGINT, `sum_a` BIGINT, `sum_b` INT, `avg_b` DOUBLE, `cnt_c` BIGINT>",
"description" : "Calc(select=[d, CAST(cnt_a1 AS BIGINT) AS cnt_a1, CAST(cnt_a2 AS BIGINT) AS cnt_a2, sum_a, CAST(sum_b AS INTEGER) AS sum_b, CAST(avg_b AS DOUBLE) AS avg_b, CAST(cnt_d AS BIGINT) AS cnt_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -317,7 +318,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out
index 740fa1a..5d91478 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testDistinctAggCalls[isMiniBatchEnabled=true].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,17 +31,16 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner",
+ "id" : 2,
+ "type" : "stream-exec-mini-batch-assigner_1",
"miniBatchInterval" : {
"interval" : 10000,
"mode" : "ProcTime"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -51,7 +51,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` BIGINT>",
"description" : "MiniBatchAssigner(interval=[10000ms], mode=[ProcTime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 3,
@@ -87,7 +88,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -98,7 +98,8 @@
"outputType" : "ROW<`d` BIGINT, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "Calc(select=[d, a, (b > 10) AS $f2, b, c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-local-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "cnt_a1",
@@ -181,7 +182,6 @@
} ],
"aggCallNeedRetractions" : [ false, false, false, false, false, false ],
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -269,8 +269,8 @@
},
"description" : "LocalGroupAggregate(groupBy=[d], select=[d, COUNT(distinct$0 a) FILTER $f2 AS count$0, COUNT(distinct$0 a) AS count$1, SUM(distinct$0 a) AS sum$2, SUM(distinct$1 b) AS sum$3, AVG(b) AS (sum$4, count$5), COUNT(distinct$2 c) AS count$6, DISTINCT(a) AS distinct$0, DISTINCT(b) AS distinct$1, DISTINCT(c) AS distinct$2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -359,7 +359,8 @@
},
"description" : "Exchange(distribution=[hash[d]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate",
+ "id" : 6,
+ "type" : "stream-exec-global-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "cnt_a1",
@@ -444,7 +445,6 @@
"localAggInputRowType" : "ROW<`d` BIGINT, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -455,7 +455,8 @@
"outputType" : "ROW<`d` BIGINT, `cnt_a1` BIGINT NOT NULL, `cnt_a2` BIGINT NOT NULL, `sum_a` BIGINT, `sum_b` INT NOT NULL, `avg_b` INT NOT NULL, `cnt_d` BIGINT NOT NULL>",
"description" : "GlobalGroupAggregate(groupBy=[d], select=[d, COUNT(distinct$0 count$0) AS cnt_a1, COUNT(distinct$0 count$1) AS cnt_a2, SUM(distinct$0 sum$2) AS sum_a, SUM(distinct$1 sum$3) AS sum_b, AVG((sum$4, count$5)) AS avg_b, COUNT(distinct$2 count$6) AS cnt_d])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 7,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -531,7 +532,6 @@
"type" : "BIGINT"
} ],
"condition" : null,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -542,7 +542,8 @@
"outputType" : "ROW<`d` BIGINT, `cnt_a1` BIGINT, `cnt_a2` BIGINT, `sum_a` BIGINT, `sum_b` INT, `avg_b` DOUBLE, `cnt_c` BIGINT>",
"description" : "Calc(select=[d, CAST(cnt_a1 AS BIGINT) AS cnt_a1, CAST(cnt_a2 AS BIGINT) AS cnt_a2, sum_a, CAST(sum_b AS INTEGER) AS sum_b, CAST(avg_b AS DOUBLE) AS avg_b, CAST(cnt_d AS BIGINT) AS cnt_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 8,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -582,7 +583,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out
index d539982..289cd23 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=false].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -39,12 +40,12 @@
"producedType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `c` VARCHAR(2147483647)> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, a, c], metadata=[]]], fields=[b, a, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -76,7 +77,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -87,8 +87,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "Calc(select=[b, a, (b > 1) AS $f2, c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 3,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -100,7 +100,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "cnt_a",
@@ -145,7 +146,6 @@
"aggCallNeedRetractions" : [ false, false, false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -156,7 +156,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `cnt_a` BIGINT NOT NULL, `max_b` INT, `min_c` VARCHAR(2147483647)>",
"description" : "GroupAggregate(groupBy=[b], select=[b, COUNT(a) AS cnt_a, MAX(b) FILTER $f2 AS max_b, MIN(c) AS min_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 5,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "REX_CALL",
"operator" : {
@@ -202,7 +203,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -213,7 +213,8 @@
"outputType" : "ROW<`b` BIGINT, `cnt_a` BIGINT, `max_b` BIGINT, `min_c` VARCHAR(2147483647)>",
"description" : "Calc(select=[CAST(b AS BIGINT) AS b, CAST(cnt_a AS BIGINT) AS cnt_a, CAST(max_b AS BIGINT) AS max_b, min_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -244,7 +245,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out
index 9043eb2..05c77c2 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggCallsWithGroupBy[isMiniBatchEnabled=true].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -39,17 +40,16 @@
"producedType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `c` VARCHAR(2147483647)> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, a, c], metadata=[]]], fields=[b, a, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner",
+ "id" : 2,
+ "type" : "stream-exec-mini-batch-assigner_1",
"miniBatchInterval" : {
"interval" : 10000,
"mode" : "ProcTime"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -60,7 +60,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "MiniBatchAssigner(interval=[10000ms], mode=[ProcTime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -92,7 +93,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -103,7 +103,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "Calc(select=[b, a, (b > 1) AS $f2, c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-local-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "cnt_a",
@@ -147,7 +148,6 @@
} ],
"aggCallNeedRetractions" : [ false, false, false ],
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -158,8 +158,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `count$0` BIGINT, `max$1` INT, `min$2` VARCHAR(2147483647)>",
"description" : "LocalGroupAggregate(groupBy=[b], select=[b, COUNT(a) AS count$0, MAX(b) FILTER $f2 AS max$1, MIN(c) AS min$2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -171,7 +171,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `count$0` BIGINT, `max$1` INT, `min$2` VARCHAR(2147483647)>",
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate",
+ "id" : 6,
+ "type" : "stream-exec-global-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "cnt_a",
@@ -217,7 +218,6 @@
"localAggInputRowType" : "ROW<`b` INT NOT NULL, `a` BIGINT, `$f2` BOOLEAN NOT NULL, `c` VARCHAR(2147483647)>",
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -228,7 +228,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `cnt_a` BIGINT NOT NULL, `max_b` INT, `min_c` VARCHAR(2147483647)>",
"description" : "GlobalGroupAggregate(groupBy=[b], select=[b, COUNT(count$0) AS cnt_a, MAX(max$1) AS max_b, MIN(min$2) AS min_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 7,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "REX_CALL",
"operator" : {
@@ -274,7 +275,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -285,7 +285,8 @@
"outputType" : "ROW<`b` BIGINT, `cnt_a` BIGINT, `max_b` BIGINT, `min_c` VARCHAR(2147483647)>",
"description" : "Calc(select=[CAST(b AS BIGINT) AS b, CAST(cnt_a AS BIGINT) AS cnt_a, CAST(max_b AS BIGINT) AS max_b, min_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 8,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -316,7 +317,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out
index e3dd454..2ea8d75 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=false].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -39,12 +40,12 @@
"producedType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c], metadata=[]]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -85,7 +86,6 @@
"type" : "BOOLEAN NOT NULL"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -96,8 +96,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` BOOLEAN NOT NULL>",
"description" : "Calc(select=[a, b, c, (a > 1) IS TRUE AS $f3])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 3,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "SINGLETON"
@@ -108,7 +108,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` BOOLEAN NOT NULL>",
"description" : "Exchange(distribution=[single])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ ],
"aggCalls" : [ {
"name" : "avg_a",
@@ -166,7 +167,6 @@
"aggCallNeedRetractions" : [ false, false, false, false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -177,7 +177,8 @@
"outputType" : "ROW<`avg_a` BIGINT, `cnt` BIGINT NOT NULL, `min_b` INT, `max_c` VARCHAR(2147483647)>",
"description" : "GroupAggregate(select=[AVG(a) AS avg_a, COUNT(*) AS cnt, MIN(b) AS min_b, MAX(c) FILTER $f3 AS max_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 5,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "REX_CALL",
"operator" : {
@@ -236,7 +237,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -247,7 +247,8 @@
"outputType" : "ROW<`avg_a` DOUBLE, `cnt` BIGINT, `cnt_b` BIGINT, `min_b` BIGINT, `max_c` VARCHAR(2147483647)>",
"description" : "Calc(select=[CAST(avg_a AS DOUBLE) AS avg_a, CAST(cnt AS BIGINT) AS cnt, CAST(cnt AS BIGINT) AS cnt_b, CAST(min_b AS BIGINT) AS min_b, max_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -281,7 +282,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out
index ce2bf0c..c708b46 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testSimpleAggWithoutGroupBy[isMiniBatchEnabled=true].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -39,17 +40,16 @@
"producedType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b, c], metadata=[]]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner",
+ "id" : 2,
+ "type" : "stream-exec-mini-batch-assigner_1",
"miniBatchInterval" : {
"interval" : 10000,
"mode" : "ProcTime"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -60,7 +60,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647)>",
"description" : "MiniBatchAssigner(interval=[10000ms], mode=[ProcTime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -101,7 +102,6 @@
"type" : "BOOLEAN NOT NULL"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -112,7 +112,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` BOOLEAN NOT NULL>",
"description" : "Calc(select=[a, b, c, (a > 1) IS TRUE AS $f3])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-local-group-aggregate_1",
"grouping" : [ ],
"aggCalls" : [ {
"name" : "avg_a",
@@ -169,7 +170,6 @@
} ],
"aggCallNeedRetractions" : [ false, false, false, false ],
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -180,8 +180,8 @@
"outputType" : "ROW<`sum$0` BIGINT, `count$1` BIGINT, `count1$2` BIGINT, `min$3` INT, `max$4` VARCHAR(2147483647)>",
"description" : "LocalGroupAggregate(select=[AVG(a) AS (sum$0, count$1), COUNT(*) AS count1$2, MIN(b) AS min$3, MAX(c) FILTER $f3 AS max$4])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "SINGLETON"
@@ -192,7 +192,8 @@
"outputType" : "ROW<`sum$0` BIGINT, `count$1` BIGINT, `count1$2` BIGINT, `min$3` INT, `max$4` VARCHAR(2147483647)>",
"description" : "Exchange(distribution=[single])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate",
+ "id" : 6,
+ "type" : "stream-exec-global-group-aggregate_1",
"grouping" : [ ],
"aggCalls" : [ {
"name" : "avg_a",
@@ -251,7 +252,6 @@
"localAggInputRowType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `$f3` BOOLEAN NOT NULL>",
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -262,7 +262,8 @@
"outputType" : "ROW<`avg_a` BIGINT, `cnt` BIGINT NOT NULL, `min_b` INT, `max_c` VARCHAR(2147483647)>",
"description" : "GlobalGroupAggregate(select=[AVG((sum$0, count$1)) AS avg_a, COUNT(count1$2) AS cnt, MIN(min$3) AS min_b, MAX(max$4) AS max_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 7,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "REX_CALL",
"operator" : {
@@ -321,7 +322,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -332,7 +332,8 @@
"outputType" : "ROW<`avg_a` DOUBLE, `cnt` BIGINT, `cnt_b` BIGINT, `min_b` BIGINT, `max_c` VARCHAR(2147483647)>",
"description" : "Calc(select=[CAST(avg_a AS DOUBLE) AS avg_a, CAST(cnt AS BIGINT) AS cnt, CAST(cnt AS BIGINT) AS cnt_b, CAST(min_b AS BIGINT) AS min_b, max_c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 8,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -366,7 +367,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out
index 2a111db..11328fd 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=false].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,12 +31,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -62,7 +63,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -73,8 +73,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `$f1` INT NOT NULL, `$f2` INT NOT NULL, `d` BIGINT, `a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "Calc(select=[b, 10 AS $f1, 5 AS $f2, d, a, c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 3,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -86,7 +86,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `$f1` INT NOT NULL, `$f2` INT NOT NULL, `d` BIGINT, `a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "a1",
@@ -156,7 +157,6 @@
"aggCallNeedRetractions" : [ false, false, false, false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -167,7 +167,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `a1` BIGINT, `a2` BIGINT, `a3` BIGINT, `c1` BIGINT>",
"description" : "GroupAggregate(groupBy=[b], select=[b, my_sum1(b, $f1) AS a1, my_sum2($f2, b) AS a2, my_avg(d, a) AS a3, my_count(c) AS c1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 5,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "REX_CALL",
"operator" : {
@@ -199,7 +200,6 @@
"type" : "BIGINT"
} ],
"condition" : null,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -210,7 +210,8 @@
"outputType" : "ROW<`b` BIGINT, `a1` BIGINT, `a2` BIGINT, `a3` BIGINT, `c1` BIGINT>",
"description" : "Calc(select=[CAST(b AS BIGINT) AS b, a1, a2, a3, c1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -244,7 +245,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out
index 7b1b75d..eb8fa31 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupAggregateJsonPlanTest_jsonplan/testUserDefinedAggCalls[isMiniBatchEnabled=true].out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -30,17 +31,16 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c, d])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner",
+ "id" : 2,
+ "type" : "stream-exec-mini-batch-assigner_1",
"miniBatchInterval" : {
"interval" : 10000,
"mode" : "ProcTime"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -51,7 +51,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL, `c` VARCHAR(2147483647), `d` BIGINT>",
"description" : "MiniBatchAssigner(interval=[10000ms], mode=[ProcTime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -78,7 +79,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -89,8 +89,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `$f1` INT NOT NULL, `$f2` INT NOT NULL, `d` BIGINT, `a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "Calc(select=[b, 10 AS $f1, 5 AS $f2, d, a, c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -102,7 +102,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `$f1` INT NOT NULL, `$f2` INT NOT NULL, `d` BIGINT, `a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 5,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "a1",
@@ -172,7 +173,6 @@
"aggCallNeedRetractions" : [ false, false, false, false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -183,7 +183,8 @@
"outputType" : "ROW<`b` INT NOT NULL, `a1` BIGINT, `a2` BIGINT, `a3` BIGINT, `c1` BIGINT>",
"description" : "GroupAggregate(groupBy=[b], select=[b, my_sum1(b, $f1) AS a1, my_sum2($f2, b) AS a2, my_avg(d, a) AS a3, my_count(c) AS c1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 6,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "REX_CALL",
"operator" : {
@@ -215,7 +216,6 @@
"type" : "BIGINT"
} ],
"condition" : null,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -226,7 +226,8 @@
"outputType" : "ROW<`b` BIGINT, `a1` BIGINT, `a2` BIGINT, `a3` BIGINT, `c1` BIGINT>",
"description" : "Calc(select=[CAST(b AS BIGINT) AS b, a1, a2, a3, c1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 7,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -260,7 +261,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out
index 40b7077..671f459 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeHopWindow.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -90,12 +91,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -123,7 +124,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -134,7 +134,8 @@
"outputType" : "ROW<`b` BIGINT, `rowtime` TIMESTAMP(3), `c` VARCHAR(2147483647), `a` INT>",
"description" : "Calc(select=[b, TO_TIMESTAMP(c) AS rowtime, c, a])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -154,7 +155,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 1,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -184,8 +184,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -216,7 +216,8 @@
},
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate",
+ "id" : 5,
+ "type" : "stream-exec-group-window-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "EXPR$1",
@@ -271,7 +272,6 @@
},
"namedWindowProperties" : [ ],
"needRetraction" : false,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -282,7 +282,8 @@
"outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>",
"description" : "GroupWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w$, rowtime, 10000, 5000)], select=[b, COUNT(c) AS EXPR$1, SUM(a) AS EXPR$2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -308,7 +309,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out
index 468da78..d3fa051 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeSessionWindow.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -90,12 +91,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -123,7 +124,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -134,7 +134,8 @@
"outputType" : "ROW<`b` BIGINT, `rowtime` TIMESTAMP(3), `c` VARCHAR(2147483647), `a` INT>",
"description" : "Calc(select=[b, TO_TIMESTAMP(c) AS rowtime, c, a])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -154,7 +155,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 1,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -184,8 +184,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -216,7 +216,8 @@
},
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate",
+ "id" : 5,
+ "type" : "stream-exec-group-window-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "EXPR$1",
@@ -269,7 +270,6 @@
},
"namedWindowProperties" : [ ],
"needRetraction" : false,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -280,7 +280,8 @@
"outputType" : "ROW<`b` BIGINT, `EXPR$1` BIGINT NOT NULL, `EXPR$2` INT>",
"description" : "GroupWindowAggregate(groupBy=[b], window=[SessionGroupWindow('w$, rowtime, 10000)], select=[b, COUNT(c) AS EXPR$1, SUM(a) AS EXPR$2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -306,7 +307,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out
index 176553e..390a7bb 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testEventTimeTumbleWindow.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -90,12 +91,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -123,7 +124,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -134,7 +134,8 @@
"outputType" : "ROW<`b` BIGINT, `rowtime` TIMESTAMP(3), `a` INT, `c` VARCHAR(2147483647)>",
"description" : "Calc(select=[b, TO_TIMESTAMP(c) AS rowtime, a, c])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -154,7 +155,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 1,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -184,8 +184,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -216,7 +216,8 @@
},
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate",
+ "id" : 5,
+ "type" : "stream-exec-group-window-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "EXPR$3",
@@ -351,7 +352,6 @@
}
} ],
"needRetraction" : false,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -400,7 +400,8 @@
},
"description" : "GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, rowtime, 5000)], properties=[w$start, w$end, w$rowtime, w$proctime], select=[b, COUNT(*) AS EXPR$3, SUM(a) AS EXPR$4, COUNT(DISTINCT c) AS EXPR$5, concat_distinct_agg(c) AS EXPR$6, start('w$) AS w$start, end('w$) AS w$end, rowtime('w$) AS w$rowtime, proctime('w$) AS w$proctime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 6,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -431,7 +432,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -442,7 +442,8 @@
"outputType" : "ROW<`b` BIGINT, `window_start` TIMESTAMP(3) NOT NULL, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$3` BIGINT NOT NULL, `EXPR$4` INT, `EXPR$5` BIGINT NOT NULL, `EXPR$6` VARCHAR(2147483647)>",
"description" : "Calc(select=[b, w$start AS window_start, w$end AS window_end, EXPR$3, EXPR$4, EXPR$5, EXPR$6])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 7,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -480,7 +481,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out
index d9281c6..868a12e 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeHopWindow.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -90,12 +91,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -137,7 +138,6 @@
}
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -171,7 +171,8 @@
},
"description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime, PROCTIME() AS proctime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -191,7 +192,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 3,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -229,7 +229,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 4,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -249,7 +250,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -277,8 +277,8 @@
},
"description" : "Calc(select=[b, proctime, a])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -307,7 +307,8 @@
},
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate",
+ "id" : 6,
+ "type" : "stream-exec-group-window-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "EXPR$1",
@@ -351,7 +352,6 @@
},
"namedWindowProperties" : [ ],
"needRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -362,7 +362,8 @@
"outputType" : "ROW<`b` BIGINT, `EXPR$1` INT>",
"description" : "GroupWindowAggregate(groupBy=[b], window=[SlidingGroupWindow('w$, proctime, 600000, 300000)], select=[b, SUM(a) AS EXPR$1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 7,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -385,7 +386,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out
index da70507..6743ce1 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeSessionWindow.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -90,12 +91,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `b` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -137,7 +138,6 @@
}
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -171,7 +171,8 @@
},
"description" : "Calc(select=[a, b, c, TO_TIMESTAMP(c) AS rowtime, PROCTIME() AS proctime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -191,7 +192,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 3,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -229,7 +229,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 4,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -249,7 +250,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -277,8 +277,8 @@
},
"description" : "Calc(select=[b, proctime, a])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -307,7 +307,8 @@
},
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate",
+ "id" : 6,
+ "type" : "stream-exec-group-window-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "EXPR$1",
@@ -349,7 +350,6 @@
},
"namedWindowProperties" : [ ],
"needRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -360,7 +360,8 @@
"outputType" : "ROW<`b` BIGINT, `EXPR$1` INT>",
"description" : "GroupWindowAggregate(groupBy=[b], window=[SessionGroupWindow('w$, proctime, 600000)], select=[b, SUM(a) AS EXPR$1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 7,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -383,7 +384,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out
index 33b3e9f..5461115 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/GroupWindowAggregateJsonPlanTest_jsonplan/testProcTimeTumbleWindow.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -99,12 +100,12 @@
"producedType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`b` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[b, c], metadata=[]]], fields=[b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -138,7 +139,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -166,7 +166,8 @@
},
"description" : "Calc(select=[b, PROCTIME() AS proctime, TO_TIMESTAMP(c) AS rowtime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -186,7 +187,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 2,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -218,7 +218,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 4,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -234,7 +235,6 @@
}
} ],
"condition" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -259,8 +259,8 @@
},
"description" : "Calc(select=[b, proctime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -286,7 +286,8 @@
},
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate",
+ "id" : 6,
+ "type" : "stream-exec-group-window-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "EXPR$2",
@@ -371,7 +372,6 @@
}
} ],
"needRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -404,7 +404,8 @@
},
"description" : "GroupWindowAggregate(groupBy=[b], window=[TumblingGroupWindow('w$, proctime, 900000)], properties=[w$start, w$end, w$proctime], select=[b, COUNT(*) AS EXPR$2, start('w$) AS w$start, end('w$) AS w$end, proctime('w$) AS w$proctime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 7,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -419,7 +420,6 @@
"type" : "BIGINT NOT NULL"
} ],
"condition" : null,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -430,7 +430,8 @@
"outputType" : "ROW<`b` BIGINT, `window_end` TIMESTAMP(3) NOT NULL, `EXPR$2` BIGINT NOT NULL>",
"description" : "Calc(select=[b, w$end AS window_end, EXPR$2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 8,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -456,7 +457,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out
index 1c75755..1dcb517 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregate.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -39,17 +40,16 @@
"producedType" : "ROW<`a` BIGINT, `c` VARCHAR(2147483647)> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, c], metadata=[]]], fields=[a, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner",
+ "id" : 2,
+ "type" : "stream-exec-mini-batch-assigner_1",
"miniBatchInterval" : {
"interval" : 10000,
"mode" : "ProcTime"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -60,7 +60,8 @@
"outputType" : "ROW<`a` BIGINT, `c` VARCHAR(2147483647)>",
"description" : "MiniBatchAssigner(interval=[10000ms], mode=[ProcTime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 3,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -97,7 +98,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -108,7 +108,8 @@
"outputType" : "ROW<`a` BIGINT, `c` VARCHAR(2147483647), `$f2` INT>",
"description" : "Calc(select=[a, c, MOD(HASH_CODE(c), 1024) AS $f2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate",
+ "id" : 4,
+ "type" : "stream-exec-local-group-aggregate_1",
"grouping" : [ 0, 2 ],
"aggCalls" : [ {
"name" : null,
@@ -126,7 +127,6 @@
} ],
"aggCallNeedRetractions" : [ false ],
"needRetraction" : false,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -171,8 +171,8 @@
},
"description" : "LocalGroupAggregate(groupBy=[a, $f2], partialFinalType=[PARTIAL], select=[a, $f2, COUNT(distinct$0 c) AS count$0, DISTINCT(c) AS distinct$0])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -218,7 +218,8 @@
},
"description" : "Exchange(distribution=[hash[a, $f2]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIncrementalGroupAggregate",
+ "id" : 6,
+ "type" : "stream-exec-incremental-group-aggregate_1",
"partialAggGrouping" : [ 0, 1 ],
"finalAggGrouping" : [ 0 ],
"partialOriginalAggCalls" : [ {
@@ -238,7 +239,6 @@
"partialAggCallNeedRetractions" : [ false ],
"partialLocalAggInputRowType" : "ROW<`a` BIGINT, `c` VARCHAR(2147483647), `$f2` INT>",
"partialAggNeedRetraction" : false,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -249,8 +249,8 @@
"outputType" : "ROW<`a` BIGINT, `count$0` BIGINT>",
"description" : "IncrementalGroupAggregate(partialAggGrouping=[a, $f2], finalAggGrouping=[a], select=[a, COUNT(distinct$0 count$0) AS count$0])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 7,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -262,7 +262,8 @@
"outputType" : "ROW<`a` BIGINT, `count$0` BIGINT>",
"description" : "Exchange(distribution=[hash[a]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate",
+ "id" : 8,
+ "type" : "stream-exec-global-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : null,
@@ -282,7 +283,6 @@
"localAggInputRowType" : "ROW<`a` BIGINT, `$f2` INT, `$f2_0` BIGINT NOT NULL>",
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -293,7 +293,8 @@
"outputType" : "ROW<`a` BIGINT, `$f1` BIGINT NOT NULL>",
"description" : "GlobalGroupAggregate(groupBy=[a], partialFinalType=[FINAL], select=[a, $SUM0(count$0) AS $f1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 9,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -318,7 +319,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER", "DELETE" ],
- "id" : 9,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out
index 711db96..39b7461 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IncrementalAggregateJsonPlanTest_jsonplan/testIncrementalAggregateWithSumCountDistinctAndRetraction.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -39,17 +40,16 @@
"producedType" : "ROW<`a` BIGINT, `b` INT NOT NULL> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a, b], metadata=[]]], fields=[a, b])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner",
+ "id" : 2,
+ "type" : "stream-exec-mini-batch-assigner_1",
"miniBatchInterval" : {
"interval" : 10000,
"mode" : "ProcTime"
},
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -60,7 +60,8 @@
"outputType" : "ROW<`a` BIGINT, `b` INT NOT NULL>",
"description" : "MiniBatchAssigner(interval=[10000ms], mode=[ProcTime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate",
+ "id" : 3,
+ "type" : "stream-exec-local-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "b",
@@ -91,7 +92,6 @@
} ],
"aggCallNeedRetractions" : [ false, false ],
"needRetraction" : false,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -102,8 +102,8 @@
"outputType" : "ROW<`a` BIGINT, `count1$0` BIGINT, `max$1` INT>",
"description" : "LocalGroupAggregate(groupBy=[a], select=[a, COUNT(*) AS count1$0, MAX(b) AS max$1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -115,7 +115,8 @@
"outputType" : "ROW<`a` BIGINT, `count1$0` BIGINT, `max$1` INT>",
"description" : "Exchange(distribution=[hash[a]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate",
+ "id" : 5,
+ "type" : "stream-exec-global-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "b",
@@ -148,7 +149,6 @@
"localAggInputRowType" : "ROW<`a` BIGINT, `b` INT NOT NULL>",
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -159,7 +159,8 @@
"outputType" : "ROW<`a` BIGINT, `b` BIGINT NOT NULL, `b1` INT NOT NULL>",
"description" : "GlobalGroupAggregate(groupBy=[a], select=[a, COUNT(count1$0) AS b, MAX(max$1) AS b1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 6,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -196,7 +197,6 @@
"type" : "INT NOT NULL"
} ],
"condition" : null,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -207,7 +207,8 @@
"outputType" : "ROW<`b` BIGINT NOT NULL, `b1` INT NOT NULL, `$f2` INT NOT NULL>",
"description" : "Calc(select=[b, b1, MOD(HASH_CODE(b1), 1024) AS $f2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate",
+ "id" : 7,
+ "type" : "stream-exec-local-group-aggregate_1",
"grouping" : [ 0, 2 ],
"aggCalls" : [ {
"name" : null,
@@ -251,7 +252,6 @@
} ],
"aggCallNeedRetractions" : [ true, true, true ],
"needRetraction" : true,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -297,8 +297,8 @@
},
"description" : "LocalGroupAggregate(groupBy=[b, $f2], partialFinalType=[PARTIAL], select=[b, $f2, SUM_RETRACT(b1) AS (sum$0, count$1), COUNT_RETRACT(distinct$0 b1) AS count$2, COUNT_RETRACT(*) AS count1$3, DISTINCT(b1) AS distinct$0])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 8,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -345,7 +345,8 @@
},
"description" : "Exchange(distribution=[hash[b, $f2]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIncrementalGroupAggregate",
+ "id" : 9,
+ "type" : "stream-exec-incremental-group-aggregate_1",
"partialAggGrouping" : [ 0, 1 ],
"finalAggGrouping" : [ 0 ],
"partialOriginalAggCalls" : [ {
@@ -391,7 +392,6 @@
"partialAggCallNeedRetractions" : [ true, true, true ],
"partialLocalAggInputRowType" : "ROW<`b` BIGINT NOT NULL, `b1` INT NOT NULL, `$f2` INT NOT NULL>",
"partialAggNeedRetraction" : true,
- "id" : 9,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -402,8 +402,8 @@
"outputType" : "ROW<`b` BIGINT NOT NULL, `sum$0` INT, `count$1` BIGINT, `count$2` BIGINT, `count1$3` BIGINT>",
"description" : "IncrementalGroupAggregate(partialAggGrouping=[b, $f2], finalAggGrouping=[b], select=[b, SUM_RETRACT((sum$0, count$1)) AS (sum$0, count$1), COUNT_RETRACT(distinct$0 count$2) AS count$2, COUNT_RETRACT(count1$3) AS count1$3])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 10,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -415,7 +415,8 @@
"outputType" : "ROW<`b` BIGINT NOT NULL, `sum$0` INT, `count$1` BIGINT, `count$2` BIGINT, `count1$3` BIGINT>",
"description" : "Exchange(distribution=[hash[b]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate",
+ "id" : 11,
+ "type" : "stream-exec-global-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : null,
@@ -462,7 +463,6 @@
"generateUpdateBefore" : true,
"needRetraction" : true,
"indexOfCountStar" : 2,
- "id" : 11,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -473,7 +473,8 @@
"outputType" : "ROW<`b` BIGINT NOT NULL, `$f1` INT NOT NULL, `$f2` BIGINT NOT NULL, `$f3` BIGINT NOT NULL>",
"description" : "GlobalGroupAggregate(groupBy=[b], partialFinalType=[FINAL], select=[b, SUM_RETRACT((sum$0, count$1)) AS $f1, $SUM0_RETRACT(count$2) AS $f2, $SUM0_RETRACT(count1$3) AS $f3], indexOfCountStar=[2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 12,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -504,7 +505,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER", "DELETE" ],
- "id" : 12,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out
index 9d71e6c..f03de17 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testProcessingTimeInnerJoinWithOnClause.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`A`",
@@ -109,12 +110,12 @@
"producedType" : "ROW<`a` INT, `c` BIGINT> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `c` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a, c], metadata=[]]], fields=[a, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -157,7 +158,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -185,7 +185,8 @@
},
"description" : "Calc(select=[a, PROCTIME() AS proctime, TO_TIMESTAMP(FROM_UNIXTIME(c)) AS rowtime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -205,7 +206,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 2,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -237,7 +237,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 4,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -253,7 +254,6 @@
}
} ],
"condition" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -278,8 +278,8 @@
},
"description" : "Calc(select=[a, proctime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -305,7 +305,8 @@
},
"description" : "Exchange(distribution=[hash[a]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 6,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`B`",
@@ -404,12 +405,12 @@
}
}
},
- "id" : 6,
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 7,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -460,7 +461,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -494,7 +494,8 @@
},
"description" : "Calc(select=[a, b, c, PROCTIME() AS proctime, TO_TIMESTAMP(FROM_UNIXTIME(c)) AS rowtime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 8,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -514,7 +515,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 4,
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -552,7 +552,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 9,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -572,7 +573,6 @@
}
} ],
"condition" : null,
- "id" : 9,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -600,8 +600,8 @@
},
"description" : "Calc(select=[a, b, proctime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 10,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -630,7 +630,8 @@
},
"description" : "Exchange(distribution=[hash[a]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIntervalJoin",
+ "id" : 11,
+ "type" : "stream-exec-interval-join_1",
"intervalJoinSpec" : {
"joinSpec" : {
"joinType" : "INNER",
@@ -647,7 +648,6 @@
"rightTimeIndex" : 2
}
},
- "id" : 11,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -692,7 +692,8 @@
},
"description" : "IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=false, leftLowerBound=-3600000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (proctime >= (proctime0 - 3600000:INTERVAL HOUR)) AND (proctime <= (proctime0 + 3600000:INTERVAL HOUR)))], select=[a, proctime, a0, b, proctime0])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 12,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -703,7 +704,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 12,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -714,7 +714,8 @@
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647)>",
"description" : "Calc(select=[a, b])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 13,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -738,7 +739,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 13,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out
index 4d330cf..956a6d4 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/IntervalJoinJsonPlanTest_jsonplan/testRowTimeInnerJoinWithOnClause.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`A`",
@@ -109,12 +110,12 @@
"producedType" : "ROW<`a` INT, `c` BIGINT> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `c` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a, c], metadata=[]]], fields=[a, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -143,7 +144,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -154,7 +154,8 @@
"outputType" : "ROW<`a` INT, `rowtime` TIMESTAMP(3)>",
"description" : "Calc(select=[a, TO_TIMESTAMP(FROM_UNIXTIME(c)) AS rowtime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -174,7 +175,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 1,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -198,8 +198,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -224,7 +224,8 @@
},
"description" : "Exchange(distribution=[hash[a]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 5,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`B`",
@@ -323,12 +324,12 @@
}
}
},
- "id" : 5,
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, B]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 6,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -361,7 +362,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -372,7 +372,8 @@
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `rowtime` TIMESTAMP(3)>",
"description" : "Calc(select=[a, b, TO_TIMESTAMP(FROM_UNIXTIME(c)) AS rowtime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 7,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -392,7 +393,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 2,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -419,8 +419,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 8,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -448,7 +448,8 @@
},
"description" : "Exchange(distribution=[hash[a]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIntervalJoin",
+ "id" : 9,
+ "type" : "stream-exec-interval-join_1",
"intervalJoinSpec" : {
"joinSpec" : {
"joinType" : "INNER",
@@ -465,7 +466,6 @@
"rightTimeIndex" : 2
}
},
- "id" : 9,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -508,7 +508,8 @@
},
"description" : "IntervalJoin(joinType=[InnerJoin], windowBounds=[isRowTime=true, leftLowerBound=-10000, leftUpperBound=3600000, leftTimeIndex=1, rightTimeIndex=2], where=[((a = a0) AND (rowtime >= (rowtime0 - 10000:INTERVAL SECOND)) AND (rowtime <= (rowtime0 + 3600000:INTERVAL HOUR)))], select=[a, rowtime, a0, b, rowtime0])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 10,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -519,7 +520,6 @@
"type" : "VARCHAR(2147483647)"
} ],
"condition" : null,
- "id" : 10,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -530,7 +530,8 @@
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647)>",
"description" : "Calc(select=[a, b])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 11,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -554,7 +555,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 11,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out
index cd54c63..fae9d80 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoin.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`A`",
@@ -36,13 +37,12 @@
"producedType" : "ROW<`a1` INT> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a1` INT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a1], metadata=[]]], fields=[a1])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 2,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -54,7 +54,8 @@
"outputType" : "ROW<`a1` INT>",
"description" : "Exchange(distribution=[hash[a1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 3,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`B`",
@@ -89,13 +90,12 @@
"producedType" : "ROW<`b1` INT> NOT NULL"
} ]
},
- "id" : 3,
"outputType" : "ROW<`b1` INT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, B, project=[b1], metadata=[]]], fields=[b1])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -107,7 +107,8 @@
"outputType" : "ROW<`b1` INT>",
"description" : "Exchange(distribution=[hash[b1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin",
+ "id" : 5,
+ "type" : "stream-exec-join_1",
"joinSpec" : {
"joinType" : "INNER",
"leftKeys" : [ 0 ],
@@ -117,7 +118,6 @@
},
"leftUniqueKeys" : [ ],
"rightUniqueKeys" : [ ],
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -134,7 +134,8 @@
"outputType" : "ROW<`a1` INT, `b1` INT>",
"description" : "Join(joinType=[InnerJoin], where=[(a1 = b1)], select=[a1, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -158,7 +159,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out
index fb9444d..a75de5c 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithEqualPk.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`A`",
@@ -36,13 +37,12 @@
"producedType" : "ROW<`a1` INT, `a2` BIGINT> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a1` INT, `a2` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a1, a2], metadata=[]]], fields=[a1, a2])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 2,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -54,13 +54,13 @@
"outputType" : "ROW<`a1` INT, `a2` BIGINT>",
"description" : "Exchange(distribution=[hash[a1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 3,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ ],
"aggCallNeedRetractions" : [ ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -71,8 +71,8 @@
"outputType" : "ROW<`a1` INT>",
"description" : "GroupAggregate(groupBy=[a1], select=[a1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -84,7 +84,8 @@
"outputType" : "ROW<`a1` INT>",
"description" : "Exchange(distribution=[hash[a1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 5,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`B`",
@@ -119,13 +120,12 @@
"producedType" : "ROW<`b1` INT, `b2` BIGINT> NOT NULL"
} ]
},
- "id" : 5,
"outputType" : "ROW<`b1` INT, `b2` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, B, project=[b1, b2], metadata=[]]], fields=[b1, b2])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 6,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -137,13 +137,13 @@
"outputType" : "ROW<`b1` INT, `b2` BIGINT>",
"description" : "Exchange(distribution=[hash[b1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 7,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ ],
"aggCallNeedRetractions" : [ ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -154,8 +154,8 @@
"outputType" : "ROW<`b1` INT>",
"description" : "GroupAggregate(groupBy=[b1], select=[b1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 8,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -167,7 +167,8 @@
"outputType" : "ROW<`b1` INT>",
"description" : "Exchange(distribution=[hash[b1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin",
+ "id" : 9,
+ "type" : "stream-exec-join_1",
"joinSpec" : {
"joinType" : "INNER",
"leftKeys" : [ 0 ],
@@ -177,7 +178,6 @@
},
"leftUniqueKeys" : [ [ 0 ] ],
"rightUniqueKeys" : [ [ 0 ] ],
- "id" : 9,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -194,7 +194,8 @@
"outputType" : "ROW<`a1` INT, `b1` INT>",
"description" : "Join(joinType=[InnerJoin], where=[(a1 = b1)], select=[a1, b1], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[JoinKeyContainsUniqueKey])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 10,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -219,7 +220,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 10,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out
index 671252e..5326ed8 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testInnerJoinWithPk.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`A`",
@@ -36,13 +37,12 @@
"producedType" : "ROW<`a1` INT, `a2` BIGINT> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a1` INT, `a2` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a1, a2], metadata=[]]], fields=[a1, a2])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 2,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -54,7 +54,8 @@
"outputType" : "ROW<`a1` INT, `a2` BIGINT>",
"description" : "Exchange(distribution=[hash[a1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 3,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "a2",
@@ -73,7 +74,6 @@
"aggCallNeedRetractions" : [ false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -84,7 +84,8 @@
"outputType" : "ROW<`a1` INT, `a2` BIGINT>",
"description" : "GroupAggregate(groupBy=[a1], select=[a1, SUM(a2) AS a2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 4,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -95,7 +96,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -106,8 +106,8 @@
"outputType" : "ROW<`a2` BIGINT, `a1` INT>",
"description" : "Calc(select=[a2, a1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 5,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -119,7 +119,8 @@
"outputType" : "ROW<`a2` BIGINT, `a1` INT>",
"description" : "Exchange(distribution=[hash[a2]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 6,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`B`",
@@ -154,13 +155,12 @@
"producedType" : "ROW<`b1` INT, `b2` BIGINT> NOT NULL"
} ]
},
- "id" : 6,
"outputType" : "ROW<`b1` INT, `b2` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, B, project=[b1, b2], metadata=[]]], fields=[b1, b2])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 7,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -172,7 +172,8 @@
"outputType" : "ROW<`b1` INT, `b2` BIGINT>",
"description" : "Exchange(distribution=[hash[b1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate",
+ "id" : 8,
+ "type" : "stream-exec-group-aggregate_1",
"grouping" : [ 0 ],
"aggCalls" : [ {
"name" : "b2",
@@ -191,7 +192,6 @@
"aggCallNeedRetractions" : [ false ],
"generateUpdateBefore" : true,
"needRetraction" : false,
- "id" : 8,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -202,7 +202,8 @@
"outputType" : "ROW<`b1` INT, `b2` BIGINT>",
"description" : "GroupAggregate(groupBy=[b1], select=[b1, SUM(b2) AS b2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 9,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -213,7 +214,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 9,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -224,8 +224,8 @@
"outputType" : "ROW<`b2` BIGINT, `b1` INT>",
"description" : "Calc(select=[b2, b1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 10,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -237,7 +237,8 @@
"outputType" : "ROW<`b2` BIGINT, `b1` INT>",
"description" : "Exchange(distribution=[hash[b2]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin",
+ "id" : 11,
+ "type" : "stream-exec-join_1",
"joinSpec" : {
"joinType" : "INNER",
"leftKeys" : [ 0 ],
@@ -247,7 +248,6 @@
},
"leftUniqueKeys" : [ [ 1 ] ],
"rightUniqueKeys" : [ [ 1 ] ],
- "id" : 11,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -264,7 +264,8 @@
"outputType" : "ROW<`a2` BIGINT, `a1` INT, `b2` BIGINT, `b1` INT>",
"description" : "Join(joinType=[InnerJoin], where=[(a2 = b2)], select=[a2, a1, b2, b1], leftInputSpec=[HasUniqueKey], rightInputSpec=[HasUniqueKey])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 12,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 1,
@@ -283,7 +284,6 @@
"type" : "BIGINT"
} ],
"condition" : null,
- "id" : 12,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -294,7 +294,8 @@
"outputType" : "ROW<`a1` INT, `a2` BIGINT, `b1` INT, `b2` BIGINT>",
"description" : "Calc(select=[a1, a2, b1, b2])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 13,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -325,7 +326,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER" ],
- "id" : 13,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out
index 97a962e..7cd757b 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/JoinJsonPlanTest_jsonplan/testLeftJoinNonEqui.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`A`",
@@ -36,13 +37,12 @@
"producedType" : "ROW<`a1` INT, `a2` BIGINT> NOT NULL"
} ]
},
- "id" : 1,
"outputType" : "ROW<`a1` INT, `a2` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, A, project=[a1, a2], metadata=[]]], fields=[a1, a2])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 2,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -54,7 +54,8 @@
"outputType" : "ROW<`a1` INT, `a2` BIGINT>",
"description" : "Exchange(distribution=[hash[a1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 3,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`B`",
@@ -89,13 +90,12 @@
"producedType" : "ROW<`b1` INT, `b2` BIGINT> NOT NULL"
} ]
},
- "id" : 3,
"outputType" : "ROW<`b1` INT, `b2` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, B, project=[b1, b2], metadata=[]]], fields=[b1, b2])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 4,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "HASH",
@@ -107,7 +107,8 @@
"outputType" : "ROW<`b1` INT, `b2` BIGINT>",
"description" : "Exchange(distribution=[hash[b1]])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin",
+ "id" : 5,
+ "type" : "stream-exec-join_1",
"joinSpec" : {
"joinType" : "LEFT",
"leftKeys" : [ 0 ],
@@ -134,7 +135,6 @@
},
"leftUniqueKeys" : [ ],
"rightUniqueKeys" : [ ],
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -151,7 +151,8 @@
"outputType" : "ROW<`a1` INT, `a2` BIGINT, `b1` INT, `b2` BIGINT>",
"description" : "Join(joinType=[LeftOuterJoin], where=[((a1 = b1) AND (a2 > b2))], select=[a1, a2, b1, b2], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 6,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -162,7 +163,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -173,7 +173,8 @@
"outputType" : "ROW<`a1` INT, `b1` INT>",
"description" : "Calc(select=[a1, b1])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 7,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -198,7 +199,6 @@
}
},
"inputChangelogMode" : [ "INSERT", "UPDATE_BEFORE", "UPDATE_AFTER", "DELETE" ],
- "id" : 7,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out
index b78b730..aa1c8c2 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LimitJsonPlanTest_jsonplan/testLimit.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -42,13 +43,12 @@
"limit" : 10
} ]
},
- "id" : 1,
"outputType" : "ROW<`a` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable, project=[a], metadata=[], limit=[10]]], fields=[a])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange",
"id" : 2,
+ "type" : "stream-exec-exchange_1",
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "SINGLETON"
@@ -59,7 +59,8 @@
"outputType" : "ROW<`a` BIGINT>",
"description" : "Exchange(distribution=[single])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLimit",
+ "id" : 3,
+ "type" : "stream-exec-limit_1",
"rankRange" : {
"type" : "Constant",
"start" : 1,
@@ -69,7 +70,6 @@
"type" : "AppendFast"
},
"generateUpdateBefore" : false,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -88,7 +88,8 @@
},
"outputRowNumber" : false
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 4,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -99,7 +100,6 @@
"type" : "BIGINT"
} ],
"condition" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -110,7 +110,8 @@
"outputType" : "ROW<`a` BIGINT, `a0` BIGINT>",
"description" : "Calc(select=[a, a AS a0])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 5,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -135,7 +136,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out
index b780ab4..811a30c 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTable.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -100,12 +101,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -156,7 +157,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -190,7 +190,8 @@
},
"description" : "Calc(select=[a, b, c, PROCTIME() AS proctime, TO_TIMESTAMP(FROM_UNIXTIME(c)) AS rowtime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -210,7 +211,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 4,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -248,7 +248,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLookupJoin",
+ "id" : 4,
+ "type" : "stream-exec-lookup-join_1",
"joinType" : "INNER",
"joinCondition" : null,
"temporalTable" : {
@@ -287,7 +288,6 @@
},
"projectionOnTemporalTable" : null,
"filterOnTemporalTable" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -334,7 +334,8 @@
},
"description" : "LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=a], select=[a, b, c, proctime, rowtime, id, name, age])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 5,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -405,7 +406,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -416,7 +416,8 @@
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT, `proctime` TIMESTAMP(3), `rowtime` TIMESTAMP(3), `id` INT, `name` VARCHAR(2147483647), `age` INT>",
"description" : "Calc(select=[a, b, c, CAST(PROCTIME_MATERIALIZE(proctime) AS TIMESTAMP(3)) AS proctime, CAST(rowtime AS TIMESTAMP(3)) AS rowtime, id, name, age])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -458,7 +459,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out
index f4a6eeb..e78bfbe 100644
--- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out
+++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/nodes/exec/stream/LookupJoinJsonPlanTest_jsonplan/testJoinTemporalTableWithProjectionPushDown.out
@@ -1,7 +1,8 @@
{
"flinkVersion" : "",
"nodes" : [ {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan",
+ "id" : 1,
+ "type" : "stream-exec-table-source-scan_1",
"scanTableSource" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MyTable`",
@@ -100,12 +101,12 @@
}
}
},
- "id" : 1,
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT>",
"description" : "TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c])",
"inputProperties" : [ ]
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 2,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -156,7 +157,6 @@
"type" : "TIMESTAMP(3)"
} ],
"condition" : null,
- "id" : 2,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -190,7 +190,8 @@
},
"description" : "Calc(select=[a, b, c, PROCTIME() AS proctime, TO_TIMESTAMP(FROM_UNIXTIME(c)) AS rowtime])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner",
+ "id" : 3,
+ "type" : "stream-exec-watermark-assigner_1",
"watermarkExpr" : {
"kind" : "REX_CALL",
"operator" : {
@@ -210,7 +211,6 @@
"type" : "TIMESTAMP(3)"
},
"rowtimeFieldIndex" : 4,
- "id" : 3,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -248,7 +248,8 @@
},
"description" : "WatermarkAssigner(rowtime=[rowtime], watermark=[(rowtime - 1000:INTERVAL SECOND)])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLookupJoin",
+ "id" : 4,
+ "type" : "stream-exec-lookup-join_1",
"joinType" : "INNER",
"joinCondition" : null,
"temporalTable" : {
@@ -291,7 +292,6 @@
"type" : "INT"
} ],
"filterOnTemporalTable" : null,
- "id" : 4,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -332,7 +332,8 @@
},
"description" : "LookupJoin(table=[default_catalog.default_database.LookupTable], joinType=[InnerJoin], async=[false], lookup=[id=a], select=[a, b, c, proctime, rowtime, id])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc",
+ "id" : 5,
+ "type" : "stream-exec-calc_1",
"projection" : [ {
"kind" : "INPUT_REF",
"inputIndex" : 0,
@@ -395,7 +396,6 @@
"type" : "INT"
} ],
"condition" : null,
- "id" : 5,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
@@ -406,7 +406,8 @@
"outputType" : "ROW<`a` INT, `b` VARCHAR(2147483647), `c` BIGINT, `proctime` TIMESTAMP(3), `rowtime` TIMESTAMP(3), `id` INT>",
"description" : "Calc(select=[a, b, c, CAST(PROCTIME_MATERIALIZE(proctime) AS TIMESTAMP(3)) AS proctime, CAST(rowtime AS TIMESTAMP(3)) AS rowtime, id])"
}, {
- "class" : "org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink",
+ "id" : 6,
+ "type" : "stream-exec-sink_1",
"dynamicTableSink" : {
"table" : {
"identifier" : "`default_catalog`.`default_database`.`MySink`",
@@ -442,7 +443,6 @@
}
},
"inputChangelogMode" : [ "INSERT" ],
- "id" : 6,
"inputProperties" : [ {
"requiredDistribution" : {
"type" : "UNKNOWN"
... 6784 lines suppressed ...