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 81d07b8d51001..d2b06fd534248 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 @@ -50,6 +50,7 @@ public interface ExecNode extends ExecNodeTranslator { String FIELD_NAME_ID = "id"; String FIELD_NAME_TYPE = "type"; + String FIELD_NAME_CONFIGURATION = "configuration"; String FIELD_NAME_DESCRIPTION = "description"; String FIELD_NAME_INPUT_PROPERTIES = "inputProperties"; String FIELD_NAME_OUTPUT_TYPE = "outputType"; 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 dbb8810fcc7a7..06b88507b3fca 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 @@ -24,6 +24,7 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.exec.serde.ConfigurationJsonSerializerFilter; import org.apache.flink.table.planner.plan.nodes.exec.utils.TransformationMetadata; import org.apache.flink.table.planner.plan.nodes.exec.visitor.ExecNodeVisitor; import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil; @@ -31,6 +32,7 @@ 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.JsonInclude; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import java.util.ArrayList; @@ -70,13 +72,30 @@ protected final ExecNodeContext getContextFromAnnotation() { return ExecNodeContext.newContext(this.getClass()).withId(getId()); } + @JsonProperty(value = FIELD_NAME_CONFIGURATION, access = JsonProperty.Access.WRITE_ONLY) + private final ReadableConfig persistedConfig; + + @JsonProperty( + value = FIELD_NAME_CONFIGURATION, + access = JsonProperty.Access.READ_ONLY, + index = 2) + // Custom filter to exclude node configuration if no consumed options are used + @JsonInclude( + value = JsonInclude.Include.CUSTOM, + valueFilter = ConfigurationJsonSerializerFilter.class) + public ReadableConfig getPersistedConfig() { + return persistedConfig; + } + protected ExecNodeBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { this.context = checkNotNull(context).withId(id); + this.persistedConfig = persistedConfig == null ? new Configuration() : persistedConfig; this.inputProperties = checkNotNull(inputProperties); this.outputType = checkNotNull(outputType); this.description = checkNotNull(description); @@ -129,7 +148,7 @@ public final Transformation translateToPlan(Planner planner) { translateToPlanInternal( (PlannerBase) planner, new ExecNodeConfig( - ((PlannerBase) planner).getTableConfig(), new Configuration())); + ((PlannerBase) planner).getTableConfig(), persistedConfig)); if (this instanceof SingleTransformationTranslator) { if (inputsContainSingleton()) { transformation.setParallelism(1); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java index 2a3e0a092dc9b..22b0ef24cd428 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/ExecNodeConfig.java @@ -31,16 +31,16 @@ /** * Configuration view which is used combine the {@link PlannerBase#getTableConfig()} with the {@link - * ExecNodeBase#getNodeConfig()} configuration. The persisted configuration of the {@link ExecNode} - * which is deserialized from the JSON plan has precedence over the {@link + * ExecNodeBase#getPersistedConfig()} configuration. The persisted configuration of the {@link + * ExecNode} which is deserialized from the JSON plan has precedence over the {@link * PlannerBase#getTableConfig()}. */ @Internal public final class ExecNodeConfig implements ReadableConfig { // See https://issues.apache.org/jira/browse/FLINK-26190 - // Used only for the deprecated getMaxIdleStateRetentionTime to also satisfy tests which - // manipulate maxIdleStateRetentionTime, like OverAggregateHarnessTest. + // Used only by CommonPythonUtil#getMergedConfig(StreamExecutionEnvironment, TableConfig)} + // otherwise it can be changed to ReadableConfig. private final TableConfig tableConfig; private final ReadableConfig nodeConfig; 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 index d00d681ac4c15..0abb680b13655 100644 --- 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 @@ -20,6 +20,8 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil; import org.apache.flink.table.types.logical.LogicalType; @@ -33,6 +35,7 @@ import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.regex.Pattern; +import java.util.stream.Stream; import static org.apache.flink.util.Preconditions.checkNotNull; @@ -81,10 +84,11 @@ private ExecNodeContext(String name, Integer 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()}. + * ExecNodeBase#ExecNodeBase(int, ExecNodeContext, ReadableConfig, 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()}. */ @@ -176,4 +180,22 @@ public static > ExecNodeContext newContext(Class execNo } return new ExecNodeContext(metadata.name(), metadata.version()); } + + /** + * Create a configuration for the {@link ExecNode}, ready to be persisted to a JSON plan. + * + * @param execNodeClass The {@link ExecNode} class. + * @param tableConfig The planner configuration (include the {@link TableConfig}). + * @return The {@link ExecNode} configuration, which contains the consumed options for the node, + * defined by {@link ExecNodeMetadata#consumedOptions()}, along with their values. + */ + public static > ReadableConfig newPersistedConfig( + Class execNodeClass, ReadableConfig tableConfig) { + return ExecNodeMetadataUtil.newPersistedConfig( + execNodeClass, + tableConfig, + Stream.concat( + ExecNodeMetadataUtil.TABLE_CONFIG_OPTIONS.stream(), + ExecNodeMetadataUtil.EXECUTION_CONFIG_OPTIONS.stream())); + } } 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 7696bf6ca070a..a5891462196c1 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -48,6 +49,7 @@ public class BatchExecBoundedStreamScan extends ExecNodeBase private final List qualifiedName; public BatchExecBoundedStreamScan( + ReadableConfig tableConfig, DataStream dataStream, DataType sourceType, int[] fieldIndexes, @@ -57,6 +59,7 @@ public BatchExecBoundedStreamScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecBoundedStreamScan.class), + ExecNodeContext.newPersistedConfig(BatchExecBoundedStreamScan.class, tableConfig), Collections.emptyList(), outputType, description); 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 a7063afcd9769..f2f3f5b6b3676 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -37,6 +38,7 @@ public class BatchExecCalc extends CommonExecCalc implements BatchExecNode { public BatchExecCalc( + ReadableConfig tableConfig, List projection, @Nullable RexNode condition, InputProperty inputProperty, @@ -45,6 +47,7 @@ public BatchExecCalc( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecCalc.class), + ExecNodeContext.newPersistedConfig(BatchExecCalc.class, tableConfig), projection, condition, TableStreamOperator.class, 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 c8e538b8ddd40..3b5450c094103 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -37,6 +38,7 @@ public class BatchExecCorrelate extends CommonExecCorrelate implements BatchExecNode { public BatchExecCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -46,6 +48,7 @@ public BatchExecCorrelate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecCorrelate.class), + ExecNodeContext.newPersistedConfig(BatchExecCorrelate.class, tableConfig), joinType, invocation, condition, 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 1f7f0221c37f8..dd4d7c737b08f 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 @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.api.transformations.StreamExchangeMode; import org.apache.flink.streaming.runtime.partitioner.BroadcastPartitioner; @@ -66,10 +67,15 @@ public class BatchExecExchange extends CommonExecExchange implements BatchExecNo // if it's None, use value from configuration @Nullable private StreamExchangeMode requiredExchangeMode; - public BatchExecExchange(InputProperty inputProperty, RowType outputType, String description) { + public BatchExecExchange( + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecExchange.class), + ExecNodeContext.newPersistedConfig(BatchExecExchange.class, tableConfig), 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/BatchExecExpand.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecExpand.java index 653f1d7e19fcd..c91e21c1da042 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -34,6 +35,7 @@ public class BatchExecExpand extends CommonExecExpand implements BatchExecNode { public BatchExecExpand( + ReadableConfig tableConfig, List> projects, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public BatchExecExpand( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecExpand.class), + ExecNodeContext.newPersistedConfig(BatchExecExpand.class, tableConfig), projects, false, // retainHeader Collections.singletonList(inputProperty), 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 4f9d01bdfa934..4945d4251e875 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -59,6 +60,7 @@ public class BatchExecHashAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecHashAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -71,6 +73,7 @@ public BatchExecHashAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecHashAggregate.class, tableConfig), 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/BatchExecHashJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecHashJoin.java index 13ac38a5ec67e..79fbfe1a0d5b1 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -61,6 +62,7 @@ public class BatchExecHashJoin extends ExecNodeBase private final boolean tryDistinctBuildRow; public BatchExecHashJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, int estimatedLeftAvgRowSize, int estimatedRightAvgRowSize, @@ -75,6 +77,7 @@ public BatchExecHashJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecHashJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); 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 10bd976a6aaf0..7afb9e1956fb5 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 @@ -20,6 +20,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -67,6 +68,7 @@ public class BatchExecHashWindowAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecHashWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -84,6 +86,7 @@ public BatchExecHashWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecHashWindowAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecHashWindowAggregate.class, tableConfig), 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/BatchExecLegacySink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLegacySink.java index 4440d8725c279..c478366028051 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -41,6 +42,7 @@ public class BatchExecLegacySink extends CommonExecLegacySink implements BatchExecNode { public BatchExecLegacySink( + ReadableConfig tableConfig, TableSink tableSink, @Nullable String[] upsertKeys, InputProperty inputProperty, @@ -49,6 +51,7 @@ public BatchExecLegacySink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLegacySink.class), + ExecNodeContext.newPersistedConfig(BatchExecLegacySink.class, tableConfig), 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 3f15872ef1cff..f07e36d493e76 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; @@ -55,6 +56,7 @@ public class BatchExecLegacyTableSourceScan extends CommonExecLegacyTableSourceS implements BatchExecNode { public BatchExecLegacyTableSourceScan( + ReadableConfig tableConfig, TableSource tableSource, List qualifiedName, RowType outputType, @@ -62,6 +64,8 @@ public BatchExecLegacyTableSourceScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLegacyTableSourceScan.class), + ExecNodeContext.newPersistedConfig( + BatchExecLegacyTableSourceScan.class, tableConfig), tableSource, qualifiedName, outputType, 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 001001c150ae4..17844d6617b29 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -41,6 +42,7 @@ public class BatchExecLimit extends ExecNodeBase implements BatchExecNo private final boolean isGlobal; public BatchExecLimit( + ReadableConfig tableConfig, long limitStart, long limitEnd, boolean isGlobal, @@ -50,6 +52,7 @@ public BatchExecLimit( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLimit.class), + ExecNodeContext.newPersistedConfig(BatchExecLimit.class, tableConfig), 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/BatchExecLookupJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecLookupJoin.java index f756c2ea269ad..c9f9180941fc6 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -38,6 +39,7 @@ /** {@link BatchExecNode} for temporal table join that implemented by lookup. */ public class BatchExecLookupJoin extends CommonExecLookupJoin implements BatchExecNode { public BatchExecLookupJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, TemporalTableSourceSpec temporalTableSourceSpec, @@ -50,6 +52,7 @@ public BatchExecLookupJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecLookupJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecLookupJoin.class, tableConfig), joinType, joinCondition, temporalTableSourceSpec, 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 15fd0498d7686..1420610190f15 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.ChainingStrategy; import org.apache.flink.streaming.api.transformations.MultipleInputTransformation; import org.apache.flink.table.data.RowData; @@ -78,6 +79,7 @@ public class BatchExecMultipleInput extends ExecNodeBase private final List originalEdges; public BatchExecMultipleInput( + ReadableConfig tableConfig, List inputProperties, ExecNode rootNode, List originalEdges, @@ -85,6 +87,7 @@ public BatchExecMultipleInput( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecMultipleInput.class), + ExecNodeContext.newPersistedConfig(BatchExecMultipleInput.class, tableConfig), inputProperties, rootNode.getOutputType(), description); 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 dce2f3c2aeb9d..282754ee2510f 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -51,6 +52,7 @@ public class BatchExecNestedLoopJoin extends ExecNodeBase private final boolean singleRowJoin; public BatchExecNestedLoopJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, RexNode condition, boolean leftIsBuild, @@ -62,6 +64,7 @@ public BatchExecNestedLoopJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecNestedLoopJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecNestedLoopJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); 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 f26099f2a4bca..1ac66a3e0236c 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.config.ExecutionConfigOptions; @@ -75,6 +76,7 @@ public class BatchExecOverAggregate extends BatchExecOverAggregateBase { public BatchExecOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -82,6 +84,7 @@ public BatchExecOverAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecOverAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecOverAggregate.class, tableConfig), overSpec, inputProperty, outputType, 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 3121d9d2c2132..860d40b34e14a 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -47,11 +48,18 @@ public abstract class BatchExecOverAggregateBase extends ExecNodeBase public BatchExecOverAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, String description) { - super(id, context, Collections.singletonList(inputProperty), outputType, description); + super( + id, + context, + persistedConfig, + 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 9c6c59876ae00..7ecd93ac0df21 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -34,6 +35,7 @@ public class BatchExecPythonCalc extends CommonExecPythonCalc implements BatchExecNode { public BatchExecPythonCalc( + ReadableConfig tableConfig, List projection, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public BatchExecPythonCalc( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonCalc.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonCalc.class, tableConfig), projection, Collections.singletonList(inputProperty), outputType, 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 8307f6f27444e..0d58251a80092 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -34,6 +35,7 @@ public class BatchExecPythonCorrelate extends CommonExecPythonCorrelate implements BatchExecNode { public BatchExecPythonCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, InputProperty inputProperty, @@ -42,6 +44,7 @@ public BatchExecPythonCorrelate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonCorrelate.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonCorrelate.class, tableConfig), joinType, invocation, Collections.singletonList(inputProperty), 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 ca95b34332ba1..1a4c0ed395ecc 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -63,6 +64,7 @@ public class BatchExecPythonGroupAggregate extends ExecNodeBase private final AggregateCall[] aggCalls; public BatchExecPythonGroupAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -72,6 +74,8 @@ public BatchExecPythonGroupAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + BatchExecPythonGroupAggregate.class, tableConfig), 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/BatchExecPythonGroupWindowAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonGroupWindowAggregate.java index d1fbbcc8ef520..70c5be91b1d32 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -75,6 +76,7 @@ public class BatchExecPythonGroupWindowAggregate extends ExecNodeBase private final NamedWindowProperty[] namedWindowProperties; public BatchExecPythonGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -87,6 +89,8 @@ public BatchExecPythonGroupWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + BatchExecPythonGroupWindowAggregate.class, tableConfig), 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/BatchExecPythonOverAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecPythonOverAggregate.java index 0d65b07d7cddd..c27f084bbe247 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -69,6 +70,7 @@ public class BatchExecPythonOverAggregate extends BatchExecOverAggregateBase { private final List aggWindowIndex; public BatchExecPythonOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -76,6 +78,7 @@ public BatchExecPythonOverAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecPythonOverAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecPythonOverAggregate.class, tableConfig), overSpec, inputProperty, outputType, 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 26f2a70078fd7..872f932eb183e 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 @@ -20,6 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator; @@ -51,6 +52,7 @@ public class BatchExecRank extends ExecNodeBase implements InputSortedE private final boolean outputRankNumber; public BatchExecRank( + ReadableConfig tableConfig, int[] partitionFields, int[] sortFields, long rankStart, @@ -62,6 +64,7 @@ public BatchExecRank( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecRank.class), + ExecNodeContext.newPersistedConfig(BatchExecRank.class, tableConfig), 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/BatchExecSink.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSink.java index d9c5b35d04fa4..da1637f962096 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; import org.apache.flink.table.data.RowData; @@ -39,6 +40,7 @@ */ public class BatchExecSink extends CommonExecSink implements BatchExecNode { public BatchExecSink( + ReadableConfig tableConfig, DynamicTableSinkSpec tableSinkSpec, InputProperty inputProperty, LogicalType outputType, @@ -46,6 +48,7 @@ public BatchExecSink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSink.class), + ExecNodeContext.newPersistedConfig(BatchExecSink.class, tableConfig), tableSinkSpec, ChangelogMode.insertOnly(), true, // isBounded 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 9bfbc9aa609cc..b964d6b56f8d0 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -47,6 +48,7 @@ public class BatchExecSort extends ExecNodeBase implements BatchExecNod private final SortSpec sortSpec; public BatchExecSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -54,6 +56,7 @@ public BatchExecSort( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSort.class), + ExecNodeContext.newPersistedConfig(BatchExecSort.class, tableConfig), 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/BatchExecSortAggregate.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortAggregate.java index cffa32bdf3ead..8709b622bf4d4 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -58,6 +59,7 @@ public class BatchExecSortAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecSortAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -70,6 +72,7 @@ public BatchExecSortAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecSortAggregate.class, tableConfig), 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/BatchExecSortLimit.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortLimit.java index 78bba8bc3655c..556a01ad942ed 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -53,6 +54,7 @@ public class BatchExecSortLimit extends ExecNodeBase private final boolean isGlobal; public BatchExecSortLimit( + ReadableConfig tableConfig, SortSpec sortSpec, long limitStart, long limitEnd, @@ -63,6 +65,7 @@ public BatchExecSortLimit( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortLimit.class), + ExecNodeContext.newPersistedConfig(BatchExecSortLimit.class, tableConfig), 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/BatchExecSortMergeJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecSortMergeJoin.java index 7c89f5c9d7487..aa39534e1bd30 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -65,6 +66,7 @@ public class BatchExecSortMergeJoin extends ExecNodeBase private final boolean leftIsSmaller; public BatchExecSortMergeJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, int[] leftKeys, int[] rightKeys, @@ -78,6 +80,7 @@ public BatchExecSortMergeJoin( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortMergeJoin.class), + ExecNodeContext.newPersistedConfig(BatchExecSortMergeJoin.class, tableConfig), Arrays.asList(leftInputProperty, rightInputProperty), outputType, description); 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 6b3b02a7c438c..941d2d7de782c 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 @@ -20,6 +20,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; @@ -67,6 +68,7 @@ public class BatchExecSortWindowAggregate extends ExecNodeBase private final boolean isFinal; public BatchExecSortWindowAggregate( + ReadableConfig tableConfig, int[] grouping, int[] auxGrouping, AggregateCall[] aggCalls, @@ -84,6 +86,7 @@ public BatchExecSortWindowAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecSortWindowAggregate.class), + ExecNodeContext.newPersistedConfig(BatchExecSortWindowAggregate.class, tableConfig), 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/BatchExecTableSourceScan.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/batch/BatchExecTableSourceScan.java index bfb58159d6337..95e98273fb9ce 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 @@ -20,6 +20,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.InputFormatSourceFunction; import org.apache.flink.table.connector.source.ScanTableSource; @@ -42,10 +43,14 @@ public class BatchExecTableSourceScan extends CommonExecTableSourceScan implements BatchExecNode { public BatchExecTableSourceScan( - DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) { + ReadableConfig tableConfig, + DynamicTableSourceSpec tableSourceSpec, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecTableSourceScan.class), + ExecNodeContext.newPersistedConfig(BatchExecTableSourceScan.class, tableConfig), tableSourceSpec, outputType, description); 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 506243b060496..d365206b356f0 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -34,10 +35,14 @@ public class BatchExecUnion extends CommonExecUnion implements BatchExecNode { public BatchExecUnion( - List inputProperties, RowType outputType, String description) { + ReadableConfig tableConfig, + List inputProperties, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecUnion.class), + ExecNodeContext.newPersistedConfig(BatchExecUnion.class, tableConfig), 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 ab8700f44534b..b1be0f8395488 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -35,10 +36,15 @@ /** Batch {@link ExecNode} that read records from given values. */ public class BatchExecValues extends CommonExecValues implements BatchExecNode { - public BatchExecValues(List> tuples, RowType outputType, String description) { + public BatchExecValues( + ReadableConfig tableConfig, + List> tuples, + RowType outputType, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecValues.class), + ExecNodeContext.newPersistedConfig(BatchExecValues.class, tableConfig), tuples, outputType, description); 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 1e36ed1778f94..b00ce302c1940 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.batch; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -37,6 +38,7 @@ public class BatchExecWindowTableFunction extends CommonExecWindowTableFunction implements BatchExecNode { public BatchExecWindowTableFunction( + ReadableConfig tableConfig, TimeAttributeWindowingStrategy windowingStrategy, InputProperty inputProperty, RowType outputType, @@ -44,6 +46,7 @@ public BatchExecWindowTableFunction( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(BatchExecWindowTableFunction.class), + ExecNodeContext.newPersistedConfig(BatchExecWindowTableFunction.class, tableConfig), windowingStrategy, Collections.singletonList(inputProperty), outputType, 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 9a7b9e77f41bb..850023d47e078 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CalcCodeGenerator; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; @@ -68,6 +69,7 @@ public abstract class CommonExecCalc extends ExecNodeBase protected CommonExecCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, @Nullable RexNode condition, Class operatorBaseClass, @@ -75,7 +77,7 @@ protected CommonExecCalc( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 9cb5a031ae155..6f0317ea12c9e 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.CorrelateCodeGenerator; @@ -72,6 +73,7 @@ public abstract class CommonExecCorrelate extends ExecNodeBase public CommonExecCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -80,7 +82,7 @@ public CommonExecCorrelate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 f19498acecd74..809bcf94f27ae 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -38,9 +39,10 @@ public abstract class CommonExecExchange extends ExecNodeBase public CommonExecExchange( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 361c9fcede69c..174692cd2b863 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.ExpandCodeGenerator; @@ -60,12 +61,13 @@ public abstract class CommonExecExpand extends ExecNodeBase public CommonExecExpand( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List> projects, boolean retainHeader, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 65dadfa9a2bd8..5b17a3aea597c 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 @@ -20,6 +20,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.datastream.DataStreamSink; import org.apache.flink.table.api.TableException; @@ -70,6 +71,7 @@ public abstract class CommonExecLegacySink extends ExecNodeBase public CommonExecLegacySink( int id, ExecNodeContext context, + ReadableConfig persistedConfig, TableSink tableSink, @Nullable String[] upsertKeys, boolean needRetraction, @@ -77,7 +79,13 @@ public CommonExecLegacySink( InputProperty inputProperty, LogicalType outputType, String description) { - super(id, context, Collections.singletonList(inputProperty), outputType, description); + super( + id, + context, + persistedConfig, + 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 0c57259acce21..5769d8ad86a56 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableException; @@ -71,11 +72,12 @@ public abstract class CommonExecLegacyTableSourceScan extends ExecNodeBase tableSource, List qualifiedName, RowType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, 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 af5bccc206408..d8354f445f105 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 @@ -22,6 +22,7 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.AsyncDataStream; import org.apache.flink.streaming.api.functions.ProcessFunction; import org.apache.flink.streaming.api.functions.async.AsyncFunction; @@ -176,6 +177,7 @@ public abstract class CommonExecLookupJoin extends ExecNodeBase protected CommonExecLookupJoin( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, // TODO: refactor this into TableSourceTable, once legacy TableSource is removed @@ -186,7 +188,7 @@ protected CommonExecLookupJoin( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, inputProperties, outputType, description); checkArgument(inputProperties.size() == 1); this.joinType = checkNotNull(joinType); this.joinCondition = joinCondition; 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 30a39fb46664e..736a0ef982232 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -84,11 +85,12 @@ public abstract class CommonExecPythonCalc extends ExecNodeBase public CommonExecPythonCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 927e361254598..8607148dda889 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -69,12 +70,13 @@ public abstract class CommonExecPythonCorrelate extends ExecNodeBase public CommonExecPythonCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexCall invocation, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 a5cf73879b445..00cf4daea7368 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 @@ -110,13 +110,14 @@ public abstract class CommonExecSink extends ExecNodeBase protected CommonExecSink( int id, ExecNodeContext context, + ReadableConfig persistedConfig, DynamicTableSinkSpec tableSinkSpec, ChangelogMode inputChangelogMode, boolean isBounded, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 8d335407c3a7e..07e193b81e5bb 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 @@ -74,10 +74,11 @@ public abstract class CommonExecTableSourceScan extends ExecNodeBase protected CommonExecTableSourceScan( int id, ExecNodeContext context, + ReadableConfig persistedConfig, DynamicTableSourceSpec tableSourceSpec, LogicalType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, 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 329475906b435..dddf6c6da8944 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.UnionTransformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -44,10 +45,11 @@ public abstract class CommonExecUnion extends ExecNodeBase public CommonExecUnion( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 e9d5afded6044..7636d7f38422b 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.ValuesCodeGenerator; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -50,10 +51,11 @@ public abstract class CommonExecValues extends ExecNodeBase public CommonExecValues( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List> tuples, RowType outputType, String description) { - super(id, context, Collections.emptyList(), outputType, description); + super(id, context, persistedConfig, 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 998ab8e7db16a..c39189e339bca 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.common; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy; @@ -62,11 +63,12 @@ public abstract class CommonExecWindowTableFunction extends ExecNodeBase inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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/processor/ForwardHashExchangeProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java index eae6279b901e3..d8238605ec084 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/ForwardHashExchangeProcessor.java @@ -17,6 +17,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.processor; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; import org.apache.flink.table.planner.plan.nodes.exec.ExecNode; @@ -69,6 +70,7 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) if (!context.getPlanner().getExecEnv().getConfig().isDynamicGraph()) { return execGraph; } + ReadableConfig tableConfig = context.getPlanner().getTableConfig(); ExecNodeVisitor visitor = new AbstractExecNodeExactlyOnceVisitor() { @Override @@ -91,7 +93,8 @@ protected void visitNode(ExecNode node) { // sort node and its output can also be connected by // ForwardPartitioner ExecEdge newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true); newEdges.set(i, newEdge); changed = true; } @@ -111,6 +114,7 @@ protected void visitNode(ExecNode node) { ExecNode sort = edge.getSource(); ExecEdge newEdgeOfSort = addExchangeAndReconnectEdge( + tableConfig, sort.getInputEdges().get(0), inputProperty, false); @@ -122,12 +126,14 @@ protected void visitNode(ExecNode node) { // sort node and its output can also be connected by // ForwardPartitioner newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true); } else { // add Exchange with keep_input_as_is distribution as the input // of the node newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, false); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, false); updateOriginalEdgeInMultipleInput( node, i, (BatchExecExchange) newEdge.getSource()); } @@ -138,7 +144,8 @@ protected void visitNode(ExecNode node) { // if operation chaining is disabled, this could mark sure the sort // node and its output can also be connected by ForwardPartitioner ExecEdge newEdge = - addExchangeAndReconnectEdge(edge, inputProperty, true); + addExchangeAndReconnectEdge( + tableConfig, edge, inputProperty, true); newEdges.set(i, newEdge); changed = true; } @@ -154,7 +161,10 @@ protected void visitNode(ExecNode node) { // TODO This implementation should be updated once FLINK-21224 is finished. private ExecEdge addExchangeAndReconnectEdge( - ExecEdge edge, InputProperty inputProperty, boolean strict) { + ReadableConfig tableConfig, + ExecEdge edge, + InputProperty inputProperty, + boolean strict) { ExecNode target = edge.getTarget(); ExecNode source = edge.getSource(); if (source instanceof CommonExecExchange) { @@ -168,13 +178,13 @@ private ExecEdge addExchangeAndReconnectEdge( || source instanceof BatchExecPythonCorrelate) { ExecEdge newEdge = addExchangeAndReconnectEdge( - source.getInputEdges().get(0), inputProperty, strict); + tableConfig, source.getInputEdges().get(0), inputProperty, strict); source.setInputEdges(Collections.singletonList(newEdge)); } BatchExecExchange exchange = createExchangeWithKeepInputAsIsDistribution( - inputProperty, strict, (RowType) edge.getOutputType()); + tableConfig, inputProperty, strict, (RowType) edge.getOutputType()); ExecEdge newEdge = new ExecEdge(source, exchange, edge.getShuffle(), edge.getExchangeMode()); exchange.setInputEdges(Collections.singletonList(newEdge)); @@ -182,7 +192,10 @@ private ExecEdge addExchangeAndReconnectEdge( } private BatchExecExchange createExchangeWithKeepInputAsIsDistribution( - InputProperty inputProperty, boolean strict, RowType outputRowType) { + ReadableConfig tableConfig, + InputProperty inputProperty, + boolean strict, + RowType outputRowType) { InputProperty newInputProperty = InputProperty.builder() .requiredDistribution( @@ -191,7 +204,8 @@ private BatchExecExchange createExchangeWithKeepInputAsIsDistribution( .damBehavior(inputProperty.getDamBehavior()) .priority(inputProperty.getPriority()) .build(); - return new BatchExecExchange(newInputProperty, outputRowType, newInputProperty.toString()); + return new BatchExecExchange( + tableConfig, newInputProperty, outputRowType, newInputProperty.toString()); } private boolean hasExchangeInput(ExecEdge edge) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java index fc245d4f4690d..6495b2a2c906c 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/MultipleInputNodeCreationProcessor.java @@ -21,6 +21,7 @@ import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple3; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.SourceTransformation; import org.apache.flink.streaming.api.transformations.StreamExchangeMode; import org.apache.flink.table.planner.plan.nodes.exec.ExecEdge; @@ -92,7 +93,8 @@ public ExecNodeGraph process(ExecNodeGraph execGraph, ProcessorContext context) optimizeMultipleInputGroups(orderedWrappers, context); // create the real multiple input nodes - List> newRootNodes = createMultipleInputNodes(rootWrappers); + List> newRootNodes = + createMultipleInputNodes(context.getPlanner().getTableConfig(), rootWrappers); return new ExecNodeGraph(newRootNodes); } @@ -451,23 +453,27 @@ static boolean isChainableSource(ExecNode node, ProcessorContext context) { // Multiple Input Nodes Creating // -------------------------------------------------------------------------------- - private List> createMultipleInputNodes(List rootWrappers) { + private List> createMultipleInputNodes( + ReadableConfig tableConfig, List rootWrappers) { List> result = new ArrayList<>(); Map> visitedMap = new HashMap<>(); for (ExecNodeWrapper rootWrapper : rootWrappers) { - result.add(getMultipleInputNode(rootWrapper, visitedMap)); + result.add(getMultipleInputNode(tableConfig, rootWrapper, visitedMap)); } return result; } private ExecNode getMultipleInputNode( - ExecNodeWrapper wrapper, Map> visitedMap) { + ReadableConfig tableConfig, + ExecNodeWrapper wrapper, + Map> visitedMap) { if (visitedMap.containsKey(wrapper)) { return visitedMap.get(wrapper); } for (int i = 0; i < wrapper.inputs.size(); i++) { - ExecNode multipleInputNode = getMultipleInputNode(wrapper.inputs.get(i), visitedMap); + ExecNode multipleInputNode = + getMultipleInputNode(tableConfig, wrapper.inputs.get(i), visitedMap); ExecEdge execEdge = ExecEdge.builder().source(multipleInputNode).target(wrapper.execNode).build(); wrapper.execNode.replaceInputEdge(i, execEdge); @@ -475,7 +481,7 @@ private ExecNode getMultipleInputNode( ExecNode ret; if (wrapper.group != null && wrapper == wrapper.group.root) { - ret = createMultipleInputNode(wrapper.group, visitedMap); + ret = createMultipleInputNode(tableConfig, wrapper.group, visitedMap); } else { ret = wrapper.execNode; } @@ -484,7 +490,9 @@ private ExecNode getMultipleInputNode( } private ExecNode createMultipleInputNode( - MultipleInputGroup group, Map> visitedMap) { + ReadableConfig tableConfig, + MultipleInputGroup group, + Map> visitedMap) { // calculate the inputs of the multiple input node List, InputProperty, ExecEdge>> inputs = new ArrayList<>(); for (ExecNodeWrapper member : group.members) { @@ -505,14 +513,16 @@ private ExecNode createMultipleInputNode( } if (isStreaming) { - return createStreamMultipleInputNode(group, inputs); + return createStreamMultipleInputNode(tableConfig, group, inputs); } else { - return createBatchMultipleInputNode(group, inputs); + return createBatchMultipleInputNode(tableConfig, group, inputs); } } private StreamExecMultipleInput createStreamMultipleInputNode( - MultipleInputGroup group, List, InputProperty, ExecEdge>> inputs) { + ReadableConfig tableConfig, + MultipleInputGroup group, + List, InputProperty, ExecEdge>> inputs) { ExecNode rootNode = group.root.execNode; List> inputNodes = new ArrayList<>(); for (Tuple3, InputProperty, ExecEdge> tuple3 : inputs) { @@ -523,6 +533,7 @@ private StreamExecMultipleInput createStreamMultipleInputNode( ExecNodeUtil.getMultipleInputDescription(rootNode, inputNodes, new ArrayList<>()); StreamExecMultipleInput multipleInput = new StreamExecMultipleInput( + tableConfig, inputNodes.stream() .map(i -> InputProperty.DEFAULT) .collect(Collectors.toList()), @@ -538,7 +549,9 @@ private StreamExecMultipleInput createStreamMultipleInputNode( } private BatchExecMultipleInput createBatchMultipleInputNode( - MultipleInputGroup group, List, InputProperty, ExecEdge>> inputs) { + ReadableConfig tableConfig, + MultipleInputGroup group, + List, InputProperty, ExecEdge>> inputs) { // first calculate the input orders using InputPriorityConflictResolver Set> inputSet = new HashSet<>(); for (Tuple3, InputProperty, ExecEdge> tuple3 : inputs) { @@ -571,7 +584,8 @@ private BatchExecMultipleInput createBatchMultipleInputNode( String description = ExecNodeUtil.getMultipleInputDescription(rootNode, inputNodes, inputProperties); BatchExecMultipleInput multipleInput = - new BatchExecMultipleInput(inputProperties, rootNode, originalEdges, description); + new BatchExecMultipleInput( + tableConfig, inputProperties, rootNode, originalEdges, description); List inputEdges = new ArrayList<>(inputNodes.size()); for (ExecNode inputNode : inputNodes) { diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java index 655ca5bdc7a92..a6588df84d9d8 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolver.java @@ -43,7 +43,7 @@ public class InputPriorityConflictResolver extends InputPriorityGraphGenerator { private final StreamExchangeMode exchangeMode; - private final ReadableConfig configuration; + private final ReadableConfig tableConfig; /** * Create a {@link InputPriorityConflictResolver} for the given {@link ExecNode} graph. @@ -58,10 +58,10 @@ public InputPriorityConflictResolver( List> roots, InputProperty.DamBehavior safeDamBehavior, StreamExchangeMode exchangeMode, - ReadableConfig configuration) { + ReadableConfig tableConfig) { super(roots, Collections.emptySet(), safeDamBehavior); this.exchangeMode = exchangeMode; - this.configuration = configuration; + this.tableConfig = tableConfig; } public void detectAndResolve() { @@ -87,7 +87,10 @@ protected void resolveInputPriorityConflict(ExecNode node, int higherInput, i // we should split it into two nodes BatchExecExchange newExchange = new BatchExecExchange( - inputProperty, (RowType) exchange.getOutputType(), "Exchange"); + tableConfig, + inputProperty, + (RowType) exchange.getOutputType(), + "Exchange"); newExchange.setRequiredExchangeMode(exchangeMode); newExchange.setInputEdges(exchange.getInputEdges()); newNode = newExchange; @@ -95,6 +98,7 @@ protected void resolveInputPriorityConflict(ExecNode node, int higherInput, i // create new BatchExecExchange with new inputProperty BatchExecExchange newExchange = new BatchExecExchange( + tableConfig, inputProperty, (RowType) exchange.getOutputType(), exchange.getDescription()); @@ -138,7 +142,10 @@ private BatchExecExchange createExchange(ExecNode node, int idx) { .build(); BatchExecExchange exchange = new BatchExecExchange( - newInputProperty, (RowType) inputNode.getOutputType(), "Exchange"); + tableConfig, + newInputProperty, + (RowType) inputNode.getOutputType(), + "Exchange"); exchange.setRequiredExchangeMode(exchangeMode); ExecEdge execEdge = ExecEdge.builder().source(inputNode).target(exchange).build(); exchange.setInputEdges(Collections.singletonList(execEdge)); @@ -170,7 +177,7 @@ protected void visitNode(ExecNode node) { } private InputProperty.DamBehavior getDamBehavior() { - if (getBatchStreamExchangeMode(configuration, exchangeMode) == StreamExchangeMode.BATCH) { + if (getBatchStreamExchangeMode(tableConfig, exchangeMode) == StreamExchangeMode.BATCH) { return InputProperty.DamBehavior.BLOCKING; } else { return InputProperty.DamBehavior.PIPELINED; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java new file mode 100644 index 0000000000000..d5cf4321e68e7 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonDeserializer.java @@ -0,0 +1,52 @@ +/* + * 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.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonParser; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.type.TypeReference; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.DeserializationContext; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.deser.std.StdDeserializer; + +import java.io.IOException; +import java.util.Map; + +/** + * Custom deserializer for {@link Configuration} used for {@link ExecNodeBase#getPersistedConfig}. + */ +@Internal +class ConfigurationJsonDeserializer extends StdDeserializer { + + private static final TypeReference MAP_STRING_STRING_TYPE_REF = + new TypeReference>() {}; + + ConfigurationJsonDeserializer() { + super(Configuration.class); + } + + @Override + public Configuration deserialize( + JsonParser jsonParser, DeserializationContext deserializationContext) + throws IOException { + return Configuration.fromMap(jsonParser.readValueAs(MAP_STRING_STRING_TYPE_REF)); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java new file mode 100644 index 0000000000000..0a1c0afba3168 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializer.java @@ -0,0 +1,52 @@ +/* + * 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.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.core.JsonGenerator; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.SerializerProvider; +import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.databind.ser.std.StdSerializer; + +import java.io.IOException; +import java.util.TreeMap; + +/** + * Custom serializer for {@link Configuration} used for {@link ExecNodeBase#getPersistedConfig()}. + */ +@Internal +class ConfigurationJsonSerializer extends StdSerializer { + + ConfigurationJsonSerializer() { + super(Configuration.class); + } + + @Override + public void serialize( + Configuration configuration, + JsonGenerator jsonGenerator, + SerializerProvider serializerProvider) + throws IOException { + // Use TreeMap to have a nice -sorted by key- output of the config options. + serializerProvider.defaultSerializeValue( + new TreeMap<>(configuration.toMap()), jsonGenerator); + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java new file mode 100644 index 0000000000000..1d1a4a5e82d0c --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/serde/ConfigurationJsonSerializerFilter.java @@ -0,0 +1,43 @@ +/* + * 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.serde; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeBase; + +/** + * Custom filtering for {@link Configuration} used by {@link ExecNodeBase#getPersistedConfig()} to + * avoid serializing null or empty configurations. + */ +@Internal +public class ConfigurationJsonSerializerFilter { + + @Override + public boolean equals(Object obj) { + if (obj == null) { + return true; + } + if (obj instanceof Configuration) { + Configuration other = (Configuration) obj; + return other.toMap().isEmpty(); + } + return true; + } +} 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 f044d0d33cd0a..8a601601d3f99 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.catalog.Column; import org.apache.flink.table.catalog.ContextResolvedTable; @@ -139,6 +140,7 @@ private static Module createFlinkTableJacksonModule() { private static void registerSerializers(SimpleModule module) { module.addSerializer(new ExecNodeGraphJsonSerializer()); module.addSerializer(new FlinkVersionJsonSerializer()); + module.addSerializer(new ConfigurationJsonSerializer()); module.addSerializer(new ObjectIdentifierJsonSerializer()); module.addSerializer(new LogicalTypeJsonSerializer()); module.addSerializer(new DataTypeJsonSerializer()); @@ -161,6 +163,7 @@ private static void registerSerializers(SimpleModule module) { private static void registerDeserializers(SimpleModule module) { module.addDeserializer(ExecNodeGraph.class, new ExecNodeGraphJsonDeserializer()); module.addDeserializer(FlinkVersion.class, new FlinkVersionJsonDeserializer()); + module.addDeserializer(ReadableConfig.class, new ConfigurationJsonDeserializer()); module.addDeserializer(ObjectIdentifier.class, new ObjectIdentifierJsonDeserializer()); module.addDeserializer(LogicalType.class, new LogicalTypeJsonDeserializer()); module.addDeserializer(RowType.class, (StdDeserializer) new LogicalTypeJsonDeserializer()); 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 fde9fda4bc4c6..ee33e0a580c34 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -43,10 +44,11 @@ public abstract class StreamExecAggregateBase extends ExecNodeBase protected StreamExecAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 90112df84c2fe..72b4be1a87160 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -48,6 +49,7 @@ public class StreamExecCalc extends CommonExecCalc implements StreamExecNode { public StreamExecCalc( + ReadableConfig tableConfig, List projection, @Nullable RexNode condition, InputProperty inputProperty, @@ -56,6 +58,7 @@ public StreamExecCalc( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecCalc.class), + ExecNodeContext.newPersistedConfig(StreamExecCalc.class, tableConfig), projection, condition, Collections.singletonList(inputProperty), @@ -67,6 +70,7 @@ public StreamExecCalc( public StreamExecCalc( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PROJECTION) List projection, @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @@ -75,6 +79,7 @@ public StreamExecCalc( super( id, context, + persistedConfig, projection, condition, TableStreamOperator.class, 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 5c82e812698cd..077c950847687 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 @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -85,6 +86,7 @@ public class StreamExecChangelogNormalize extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecChangelogNormalize( + ReadableConfig tableConfig, int[] uniqueKeys, boolean generateUpdateBefore, InputProperty inputProperty, @@ -93,6 +95,7 @@ public StreamExecChangelogNormalize( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecChangelogNormalize.class), + ExecNodeContext.newPersistedConfig(StreamExecChangelogNormalize.class, tableConfig), uniqueKeys, generateUpdateBefore, Collections.singletonList(inputProperty), @@ -104,12 +107,13 @@ public StreamExecChangelogNormalize( public StreamExecChangelogNormalize( @JsonProperty(FIELD_NAME_ID) Integer id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys, @JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 c843ecfbe8afc..eb493898fab4b 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -52,6 +53,7 @@ public class StreamExecCorrelate extends CommonExecCorrelate implements StreamExecNode { public StreamExecCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, @Nullable RexNode condition, @@ -61,6 +63,7 @@ public StreamExecCorrelate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecCorrelate.class), + ExecNodeContext.newPersistedConfig(StreamExecCorrelate.class, tableConfig), joinType, invocation, condition, @@ -73,6 +76,7 @@ public StreamExecCorrelate( public StreamExecCorrelate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType, @JsonProperty(FIELD_NAME_FUNCTION_CALL) RexNode invocation, @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition, @@ -82,6 +86,7 @@ public StreamExecCorrelate( super( id, context, + persistedConfig, joinType, (RexCall) invocation, condition, 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 d6d353a7dac86..e298ede69a63b 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.calcite.FlinkRelBuilder; @@ -63,6 +64,7 @@ public class StreamExecDataStreamScan extends ExecNodeBase private final List qualifiedName; public StreamExecDataStreamScan( + ReadableConfig tableConfig, DataStream dataStream, DataType sourceType, int[] fieldIndexes, @@ -73,6 +75,7 @@ public StreamExecDataStreamScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDataStreamScan.class), + ExecNodeContext.newPersistedConfig(StreamExecDataStreamScan.class, tableConfig), Collections.emptyList(), outputType, description); 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 c5bf069867c55..93425a9e35e0e 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 @@ -105,6 +105,7 @@ public class StreamExecDeduplicate extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecDeduplicate( + ReadableConfig tableConfig, int[] uniqueKeys, boolean isRowtime, boolean keepLastRow, @@ -115,6 +116,7 @@ public StreamExecDeduplicate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDeduplicate.class), + ExecNodeContext.newPersistedConfig(StreamExecDeduplicate.class, tableConfig), uniqueKeys, isRowtime, keepLastRow, @@ -128,6 +130,7 @@ public StreamExecDeduplicate( public StreamExecDeduplicate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_UNIQUE_KEYS) int[] uniqueKeys, @JsonProperty(FIELD_NAME_IS_ROWTIME) boolean isRowtime, @JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow, @@ -135,7 +138,7 @@ public StreamExecDeduplicate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 eb8cdcb54a199..cc2ef75f809fe 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.StreamFilter; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -57,10 +58,14 @@ public class StreamExecDropUpdateBefore extends ExecNodeBase public static final String DROP_UPDATE_BEFORE_TRANSFORMATION = "drop-update-before"; public StreamExecDropUpdateBefore( - InputProperty inputProperty, RowType outputType, String description) { + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecDropUpdateBefore.class), + ExecNodeContext.newPersistedConfig(StreamExecDropUpdateBefore.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -70,10 +75,11 @@ public StreamExecDropUpdateBefore( public StreamExecDropUpdateBefore( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 f7b5d524aad98..c8d37f23c7162 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 @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.PartitionTransformation; import org.apache.flink.streaming.runtime.partitioner.GlobalPartitioner; import org.apache.flink.streaming.runtime.partitioner.KeyGroupStreamPartitioner; @@ -64,10 +65,15 @@ public class StreamExecExchange extends CommonExecExchange implements StreamExec public static final String EXCHANGE_TRANSFORMATION = "exchange"; - public StreamExecExchange(InputProperty inputProperty, RowType outputType, String description) { + public StreamExecExchange( + ReadableConfig tableConfig, + InputProperty inputProperty, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecExchange.class), + ExecNodeContext.newPersistedConfig(StreamExecExchange.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); @@ -77,10 +83,11 @@ public StreamExecExchange(InputProperty inputProperty, RowType outputType, Strin public StreamExecExchange( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 d46c30f87f2ce..e1ec85678fc0d 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -45,6 +46,7 @@ public class StreamExecExpand extends CommonExecExpand implements StreamExecNode { public StreamExecExpand( + ReadableConfig tableConfig, List> projects, InputProperty inputProperty, RowType outputType, @@ -52,6 +54,7 @@ public StreamExecExpand( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecExpand.class), + ExecNodeContext.newPersistedConfig(StreamExecExpand.class, tableConfig), projects, Collections.singletonList(inputProperty), outputType, @@ -62,6 +65,7 @@ public StreamExecExpand( public StreamExecExpand( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PROJECTS) List> projects, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @@ -69,6 +73,7 @@ public StreamExecExpand( super( id, context, + persistedConfig, projects, true, // retainHeader inputProperties, 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 b546b46e8f44f..768f33eed0f1c 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; @@ -116,6 +117,7 @@ public class StreamExecGlobalGroupAggregate extends StreamExecAggregateBase { protected final Integer indexOfCountStar; public StreamExecGlobalGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -129,6 +131,8 @@ public StreamExecGlobalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGlobalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGlobalGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -145,6 +149,7 @@ public StreamExecGlobalGroupAggregate( public StreamExecGlobalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -155,7 +160,7 @@ public StreamExecGlobalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 1c123bfc8fede..c0a3fb5f0c107 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -99,6 +100,7 @@ public class StreamExecGlobalWindowAggregate extends StreamExecWindowAggregateBa private final RowType localAggInputRowType; public StreamExecGlobalWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -110,6 +112,8 @@ public StreamExecGlobalWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGlobalWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGlobalWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -124,6 +128,7 @@ public StreamExecGlobalWindowAggregate( public StreamExecGlobalWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @@ -133,7 +138,7 @@ public StreamExecGlobalWindowAggregate( @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, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 4f5988d9c33bf..9eb6b72ac45ff 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -102,6 +103,7 @@ public class StreamExecGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -113,6 +115,7 @@ public StreamExecGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -127,6 +130,7 @@ public StreamExecGroupAggregate( public StreamExecGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -135,7 +139,7 @@ public StreamExecGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 95671ce8e6aea..ee378956aedd1 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -72,6 +73,7 @@ public class StreamExecGroupTableAggregate extends ExecNodeBase private final boolean needRetraction; public StreamExecGroupTableAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -83,6 +85,8 @@ public StreamExecGroupTableAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupTableAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGroupTableAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); 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 53afa080f9138..66d22c82f0b59 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 @@ -138,6 +138,7 @@ public class StreamExecGroupWindowAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -149,6 +150,8 @@ public StreamExecGroupWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecGroupWindowAggregate.class, tableConfig), grouping, aggCalls, window, @@ -163,6 +166,7 @@ public StreamExecGroupWindowAggregate( public StreamExecGroupWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOW) LogicalWindow window, @@ -172,7 +176,7 @@ public StreamExecGroupWindowAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 adf4ae9160fff..9eddf592a3055 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.data.RowData; @@ -106,6 +107,7 @@ public class StreamExecIncrementalGroupAggregate extends StreamExecAggregateBase private final boolean partialAggNeedRetraction; public StreamExecIncrementalGroupAggregate( + ReadableConfig tableConfig, int[] partialAggGrouping, int[] finalAggGrouping, AggregateCall[] partialOriginalAggCalls, @@ -118,6 +120,8 @@ public StreamExecIncrementalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecIncrementalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecIncrementalGroupAggregate.class, tableConfig), partialAggGrouping, finalAggGrouping, partialOriginalAggCalls, @@ -133,6 +137,7 @@ public StreamExecIncrementalGroupAggregate( public StreamExecIncrementalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PARTIAL_AGG_GROUPING) int[] partialAggGrouping, @JsonProperty(FIELD_NAME_FINAL_AGG_GROUPING) int[] finalAggGrouping, @JsonProperty(FIELD_NAME_PARTIAL_ORIGINAL_AGG_CALLS) @@ -144,7 +149,7 @@ public StreamExecIncrementalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 62f5b9ce34e3b..a1cc6e9c180f9 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 @@ -96,6 +96,7 @@ public class StreamExecIntervalJoin extends ExecNodeBase private final IntervalJoinSpec intervalJoinSpec; public StreamExecIntervalJoin( + ReadableConfig tableConfig, IntervalJoinSpec intervalJoinSpec, InputProperty leftInputProperty, InputProperty rightInputProperty, @@ -104,6 +105,7 @@ public StreamExecIntervalJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecIntervalJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecIntervalJoin.class, tableConfig), intervalJoinSpec, Lists.newArrayList(leftInputProperty, rightInputProperty), outputType, @@ -114,11 +116,12 @@ public StreamExecIntervalJoin( public StreamExecIntervalJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INTERVAL_JOIN_SPEC) IntervalJoinSpec intervalJoinSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 060f06bb28d35..260094ba45a78 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 @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -85,6 +86,7 @@ public class StreamExecJoin extends ExecNodeBase private final List rightUniqueKeys; public StreamExecJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, List leftUniqueKeys, List rightUniqueKeys, @@ -95,6 +97,7 @@ public StreamExecJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecJoin.class, tableConfig), joinSpec, leftUniqueKeys, rightUniqueKeys, @@ -107,13 +110,14 @@ public StreamExecJoin( public StreamExecJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_LEFT_UNIQUE_KEYS) List leftUniqueKeys, @JsonProperty(FIELD_NAME_RIGHT_UNIQUE_KEYS) List rightUniqueKeys, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 eeb68403c59b2..04fdde6c11b83 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -44,6 +45,7 @@ public class StreamExecLegacySink extends CommonExecLegacySink implements StreamExecNode { public StreamExecLegacySink( + ReadableConfig tableConfig, TableSink tableSink, @Nullable String[] upsertKeys, boolean needRetraction, @@ -53,6 +55,7 @@ public StreamExecLegacySink( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLegacySink.class), + ExecNodeContext.newPersistedConfig(StreamExecLegacySink.class, tableConfig), 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 78bd2ac083982..6e9703a995425 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.InputSplit; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -63,6 +64,7 @@ public class StreamExecLegacyTableSourceScan extends CommonExecLegacyTableSource implements StreamExecNode { public StreamExecLegacyTableSourceScan( + ReadableConfig tableConfig, TableSource tableSource, List qualifiedName, RowType outputType, @@ -70,6 +72,8 @@ public StreamExecLegacyTableSourceScan( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLegacyTableSourceScan.class), + ExecNodeContext.newPersistedConfig( + StreamExecLegacyTableSourceScan.class, tableConfig), tableSource, qualifiedName, outputType, 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 3e4bb09ba6565..2f673a04aedd7 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -54,6 +55,7 @@ public class StreamExecLimit extends StreamExecRank { private final long limitEnd; public StreamExecLimit( + ReadableConfig tableConfig, long limitStart, long limitEnd, boolean generateUpdateBefore, @@ -64,6 +66,7 @@ public StreamExecLimit( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLimit.class), + ExecNodeContext.newPersistedConfig(StreamExecLimit.class, tableConfig), new ConstantRankRange(limitStart + 1, limitEnd), getRankStrategy(needRetraction), generateUpdateBefore, @@ -76,6 +79,7 @@ public StreamExecLimit( public StreamExecLimit( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange, @JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy, @JsonProperty(FIELD_NAME_GENERATE_UPDATE_BEFORE) boolean generateUpdateBefore, @@ -85,6 +89,7 @@ public StreamExecLimit( super( id, context, + persistedConfig, RankType.ROW_NUMBER, PartitionSpec.ALL_IN_ONE, SortSpec.ANY, 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 cbb2ad75147d2..c012c751925f6 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.CodeGeneratorContext; import org.apache.flink.table.planner.codegen.agg.AggsHandlerCodeGenerator; @@ -85,6 +86,7 @@ public class StreamExecLocalGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecLocalGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -95,6 +97,8 @@ public StreamExecLocalGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLocalGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecLocalGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -108,6 +112,7 @@ public StreamExecLocalGroupAggregate( public StreamExecLocalGroupAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_AGG_CALL_NEED_RETRACTIONS) boolean[] aggCallNeedRetractions, @@ -115,7 +120,7 @@ public StreamExecLocalGroupAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 77ddf7b397a59..e69271242abe5 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.table.data.RowData; @@ -94,6 +95,7 @@ public class StreamExecLocalWindowAggregate extends StreamExecWindowAggregateBas private final WindowingStrategy windowing; public StreamExecLocalWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -103,6 +105,8 @@ public StreamExecLocalWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLocalWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecLocalWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -115,13 +119,14 @@ public StreamExecLocalWindowAggregate( public StreamExecLocalWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 fdc80776201ca..8954958882726 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -49,6 +50,7 @@ minStateVersion = FlinkVersion.v1_15) public class StreamExecLookupJoin extends CommonExecLookupJoin implements StreamExecNode { public StreamExecLookupJoin( + ReadableConfig tableConfig, FlinkJoinType joinType, @Nullable RexNode joinCondition, TemporalTableSourceSpec temporalTableSourceSpec, @@ -61,6 +63,7 @@ public StreamExecLookupJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecLookupJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecLookupJoin.class, tableConfig), joinType, joinCondition, temporalTableSourceSpec, @@ -76,6 +79,7 @@ public StreamExecLookupJoin( public StreamExecLookupJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType joinType, @JsonProperty(FIELD_NAME_JOIN_CONDITION) @Nullable RexNode joinCondition, @JsonProperty(FIELD_NAME_TEMPORAL_TABLE) @@ -91,6 +95,7 @@ public StreamExecLookupJoin( super( id, context, + persistedConfig, joinType, joinCondition, temporalTableSourceSpec, 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 974cdddbeb16a..0863ca7ccad8c 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 @@ -110,6 +110,7 @@ public class StreamExecMatch extends ExecNodeBase private final MatchSpec matchSpec; public StreamExecMatch( + ReadableConfig tableConfig, MatchSpec matchSpec, InputProperty inputProperty, RowType outputType, @@ -117,6 +118,7 @@ public StreamExecMatch( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMatch.class), + ExecNodeContext.newPersistedConfig(StreamExecMatch.class, tableConfig), matchSpec, Collections.singletonList(inputProperty), outputType, @@ -127,11 +129,12 @@ public StreamExecMatch( public StreamExecMatch( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_MATCH_SPEC) MatchSpec matchSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 2bbc423214d59..e2a0da4e9c197 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -73,6 +74,7 @@ public class StreamExecMiniBatchAssigner extends ExecNodeBase private final MiniBatchInterval miniBatchInterval; public StreamExecMiniBatchAssigner( + ReadableConfig tableConfig, MiniBatchInterval miniBatchInterval, InputProperty inputProperty, RowType outputType, @@ -80,6 +82,7 @@ public StreamExecMiniBatchAssigner( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMiniBatchAssigner.class), + ExecNodeContext.newPersistedConfig(StreamExecMiniBatchAssigner.class, tableConfig), miniBatchInterval, Collections.singletonList(inputProperty), outputType, @@ -90,11 +93,12 @@ public StreamExecMiniBatchAssigner( public StreamExecMiniBatchAssigner( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_MINI_BATCH_INTERVAL) MiniBatchInterval miniBatchInterval, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 b230f3cde2111..278feee81bd44 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -62,10 +63,14 @@ public class StreamExecMultipleInput extends ExecNodeBase private final ExecNode rootNode; public StreamExecMultipleInput( - List inputProperties, ExecNode rootNode, String description) { + ReadableConfig tableConfig, + List inputProperties, + ExecNode rootNode, + String description) { super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecMultipleInput.class), + ExecNodeContext.newPersistedConfig(StreamExecMultipleInput.class, tableConfig), inputProperties, rootNode.getOutputType(), description); 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 b43cc441b3290..ae7a9b20ad2ba 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.functions.KeyedProcessFunction; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -100,6 +101,7 @@ public class StreamExecOverAggregate extends ExecNodeBase private final OverSpec overSpec; public StreamExecOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -107,6 +109,7 @@ public StreamExecOverAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecOverAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecOverAggregate.class, tableConfig), overSpec, Collections.singletonList(inputProperty), outputType, @@ -117,11 +120,12 @@ public StreamExecOverAggregate( public StreamExecOverAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_OVER_SPEC) OverSpec overSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 211221dd356b6..496ca96c1eb9a 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,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -34,6 +35,7 @@ public class StreamExecPythonCalc extends CommonExecPythonCalc implements StreamExecNode { public StreamExecPythonCalc( + ReadableConfig tableConfig, List projection, InputProperty inputProperty, RowType outputType, @@ -41,6 +43,7 @@ public StreamExecPythonCalc( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonCalc.class), + ExecNodeContext.newPersistedConfig(StreamExecPythonCalc.class, tableConfig), projection, Collections.singletonList(inputProperty), outputType, @@ -50,10 +53,11 @@ public StreamExecPythonCalc( public StreamExecPythonCalc( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List projection, List inputProperties, RowType outputType, String description) { - super(id, context, projection, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 992445cf9de05..0ce687a804c47 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,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -36,6 +37,7 @@ public class StreamExecPythonCorrelate extends CommonExecPythonCorrelate implements StreamExecNode { public StreamExecPythonCorrelate( + ReadableConfig tableConfig, FlinkJoinType joinType, RexCall invocation, InputProperty inputProperty, @@ -44,6 +46,7 @@ public StreamExecPythonCorrelate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonCorrelate.class), + ExecNodeContext.newPersistedConfig(StreamExecPythonCorrelate.class, tableConfig), joinType, invocation, Collections.singletonList(inputProperty), @@ -54,6 +57,7 @@ public StreamExecPythonCorrelate( public StreamExecPythonCorrelate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, FlinkJoinType joinType, RexNode invocation, List inputProperties, @@ -62,6 +66,7 @@ public StreamExecPythonCorrelate( super( id, context, + persistedConfig, joinType, (RexCall) invocation, inputProperties, 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 0a977a60aa556..d55ac1b7a23ca 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -78,6 +79,7 @@ public class StreamExecPythonGroupAggregate extends StreamExecAggregateBase { private final boolean needRetraction; public StreamExecPythonGroupAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -89,6 +91,8 @@ public StreamExecPythonGroupAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupAggregate.class, tableConfig), grouping, aggCalls, aggCallNeedRetractions, @@ -103,6 +107,7 @@ public StreamExecPythonGroupAggregate( public StreamExecPythonGroupAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -111,7 +116,7 @@ public StreamExecPythonGroupAggregate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 583fafca69258..2303f03345c6a 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -74,6 +75,7 @@ public class StreamExecPythonGroupTableAggregate extends ExecNodeBase private final boolean needRetraction; public StreamExecPythonGroupTableAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, boolean[] aggCallNeedRetractions, @@ -85,6 +87,8 @@ public StreamExecPythonGroupTableAggregate( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupTableAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupTableAggregate.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); 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 3856ac9858672..88331c116a3cc 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -72,8 +73,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.calcite.rel.core.AggregateCall; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -132,6 +131,7 @@ public class StreamExecPythonGroupWindowAggregate extends StreamExecAggregateBas private final boolean generateUpdateBefore; public StreamExecPythonGroupWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -144,6 +144,8 @@ public StreamExecPythonGroupWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonGroupWindowAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonGroupWindowAggregate.class, tableConfig), grouping, aggCalls, window, @@ -155,10 +157,10 @@ public StreamExecPythonGroupWindowAggregate( description); } - @JsonCreator public StreamExecPythonGroupWindowAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, int[] grouping, AggregateCall[] aggCalls, LogicalWindow window, @@ -168,7 +170,7 @@ public StreamExecPythonGroupWindowAggregate( List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 68a751baf59bd..1071a80e81714 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 @@ -21,6 +21,7 @@ import org.apache.flink.api.dag.Transformation; import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.memory.ManagedMemoryUseCase; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -50,8 +51,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.calcite.rel.core.AggregateCall; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,6 +92,7 @@ public class StreamExecPythonOverAggregate extends ExecNodeBase private final OverSpec overSpec; public StreamExecPythonOverAggregate( + ReadableConfig tableConfig, OverSpec overSpec, InputProperty inputProperty, RowType outputType, @@ -100,21 +100,23 @@ public StreamExecPythonOverAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecPythonOverAggregate.class), + ExecNodeContext.newPersistedConfig( + StreamExecPythonOverAggregate.class, tableConfig), overSpec, Collections.singletonList(inputProperty), outputType, description); } - @JsonCreator public StreamExecPythonOverAggregate( int id, ExecNodeContext context, + ReadableConfig persistedConfig, OverSpec overSpec, List inputProperties, RowType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 2d01c1896301c..62b36f4a150a3 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 @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.state.StateTtlConfig; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.KeyedProcessOperator; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; @@ -114,6 +115,7 @@ public class StreamExecRank extends ExecNodeBase private final boolean generateUpdateBefore; public StreamExecRank( + ReadableConfig tableConfig, RankType rankType, PartitionSpec partitionSpec, SortSpec sortSpec, @@ -127,6 +129,7 @@ public StreamExecRank( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecRank.class), + ExecNodeContext.newPersistedConfig(StreamExecRank.class, tableConfig), rankType, partitionSpec, sortSpec, @@ -143,6 +146,7 @@ public StreamExecRank( public StreamExecRank( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType, @JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @@ -153,7 +157,7 @@ public StreamExecRank( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 1d19b456eb94f..40acb2f964222 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -82,6 +83,7 @@ public class StreamExecSink extends CommonExecSink implements StreamExecNode inputProperties, @@ -112,6 +116,7 @@ public StreamExecSink( super( id, context, + persistedConfig, tableSinkSpec, inputChangelogMode, false, // isBounded 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 53e2b8f9b2d14..0315fc0f32bfd 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.sort.ComparatorCodeGenerator; @@ -51,6 +52,7 @@ public class StreamExecSort extends ExecNodeBase implements StreamExecN private final SortSpec sortSpec; public StreamExecSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -58,6 +60,7 @@ public StreamExecSort( super( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecSort.class), + ExecNodeContext.newPersistedConfig(StreamExecSort.class, tableConfig), Collections.singletonList(inputProperty), outputType, description); 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 ab3f5b7bcdba4..fd36729592264 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -53,6 +54,7 @@ public class StreamExecSortLimit extends StreamExecRank { private final long limitEnd; public StreamExecSortLimit( + ReadableConfig tableConfig, SortSpec sortSpec, long limitStart, long limitEnd, @@ -64,6 +66,7 @@ public StreamExecSortLimit( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecSortLimit.class), + ExecNodeContext.newPersistedConfig(StreamExecSortLimit.class, tableConfig), sortSpec, new ConstantRankRange(limitStart + 1, limitEnd), rankStrategy, @@ -77,6 +80,7 @@ public StreamExecSortLimit( public StreamExecSortLimit( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @JsonProperty(FIELD_NAME_RANK_RANG) ConstantRankRange rankRange, @JsonProperty(FIELD_NAME_RANK_STRATEGY) RankProcessStrategy rankStrategy, @@ -88,6 +92,7 @@ public StreamExecSortLimit( super( id, context, + persistedConfig, RankType.ROW_NUMBER, PartitionSpec.ALL_IN_ONE, sortSpec, 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 d7f642baf65c7..9fb292bd96030 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 @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.connector.source.ScanTableSource; import org.apache.flink.table.data.RowData; @@ -49,10 +50,14 @@ public class StreamExecTableSourceScan extends CommonExecTableSourceScan implements StreamExecNode { public StreamExecTableSourceScan( - DynamicTableSourceSpec tableSourceSpec, RowType outputType, String description) { + ReadableConfig tableConfig, + DynamicTableSourceSpec tableSourceSpec, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTableSourceScan.class), + ExecNodeContext.newPersistedConfig(StreamExecTableSourceScan.class, tableConfig), tableSourceSpec, outputType, description); @@ -62,10 +67,11 @@ public StreamExecTableSourceScan( public StreamExecTableSourceScan( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SCAN_TABLE_SOURCE) DynamicTableSourceSpec tableSourceSpec, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, tableSourceSpec, outputType, description); + super(id, context, persistedConfig, 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 3a1326d776df9..16a803f71198a 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.api.TableException; @@ -97,6 +98,7 @@ public class StreamExecTemporalJoin extends ExecNodeBase private final int rightTimeAttributeIndex; public StreamExecTemporalJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, boolean isTemporalTableFunctionJoin, int leftTimeAttributeIndex, @@ -108,6 +110,7 @@ public StreamExecTemporalJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTemporalJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecTemporalJoin.class, tableConfig), joinSpec, isTemporalTableFunctionJoin, leftTimeAttributeIndex, @@ -121,6 +124,7 @@ public StreamExecTemporalJoin( public StreamExecTemporalJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @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, @@ -128,7 +132,7 @@ public StreamExecTemporalJoin( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 29d76cf3bcf07..629977da2aeec 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.OneInputTransformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -71,6 +72,7 @@ public class StreamExecTemporalSort extends ExecNodeBase private final SortSpec sortSpec; public StreamExecTemporalSort( + ReadableConfig tableConfig, SortSpec sortSpec, InputProperty inputProperty, RowType outputType, @@ -78,6 +80,7 @@ public StreamExecTemporalSort( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecTemporalSort.class), + ExecNodeContext.newPersistedConfig(StreamExecTemporalSort.class, tableConfig), sortSpec, Collections.singletonList(inputProperty), outputType, @@ -88,11 +91,12 @@ public StreamExecTemporalSort( public StreamExecTemporalSort( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 68c9f9b4aa20b..0cfbde1fb938f 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -44,10 +45,14 @@ public class StreamExecUnion extends CommonExecUnion implements StreamExecNode { public StreamExecUnion( - List inputProperties, RowType outputType, String description) { + ReadableConfig tableConfig, + List inputProperties, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecUnion.class), + ExecNodeContext.newPersistedConfig(StreamExecUnion.class, tableConfig), inputProperties, outputType, description); @@ -57,9 +62,10 @@ public StreamExecUnion( public StreamExecUnion( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 3b6c984d51614..9ace41eaf896c 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -42,10 +43,15 @@ minStateVersion = FlinkVersion.v1_15) public class StreamExecValues extends CommonExecValues implements StreamExecNode { - public StreamExecValues(List> tuples, RowType outputType, String description) { + public StreamExecValues( + ReadableConfig tableConfig, + List> tuples, + RowType outputType, + String description) { this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecValues.class), + ExecNodeContext.newPersistedConfig(StreamExecValues.class, tableConfig), tuples, outputType, description); @@ -55,9 +61,10 @@ public StreamExecValues(List> tuples, RowType outputType, Strin public StreamExecValues( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_TUPLES) List> tuples, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, tuples, outputType, description); + super(id, context, persistedConfig, 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 6bcbde6efb263..6d71a3fdbc206 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.codegen.WatermarkGeneratorCodeGenerator; @@ -73,6 +74,7 @@ public class StreamExecWatermarkAssigner extends ExecNodeBase private final int rowtimeFieldIndex; public StreamExecWatermarkAssigner( + ReadableConfig tableConfig, RexNode watermarkExpr, int rowtimeFieldIndex, InputProperty inputProperty, @@ -81,6 +83,7 @@ public StreamExecWatermarkAssigner( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWatermarkAssigner.class), + ExecNodeContext.newPersistedConfig(StreamExecWatermarkAssigner.class, tableConfig), watermarkExpr, rowtimeFieldIndex, Collections.singletonList(inputProperty), @@ -92,12 +95,13 @@ public StreamExecWatermarkAssigner( public StreamExecWatermarkAssigner( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_WATERMARK_EXPR) RexNode watermarkExpr, @JsonProperty(FIELD_NAME_ROWTIME_FIELD_INDEX) int rowtimeFieldIndex, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 96796e5dd82fc..e515b1db49962 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -104,6 +105,7 @@ public class StreamExecWindowAggregate extends StreamExecWindowAggregateBase { private final NamedWindowProperty[] namedWindowProperties; public StreamExecWindowAggregate( + ReadableConfig tableConfig, int[] grouping, AggregateCall[] aggCalls, WindowingStrategy windowing, @@ -114,6 +116,7 @@ public StreamExecWindowAggregate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowAggregate.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowAggregate.class, tableConfig), grouping, aggCalls, windowing, @@ -127,6 +130,7 @@ public StreamExecWindowAggregate( public StreamExecWindowAggregate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_GROUPING) int[] grouping, @JsonProperty(FIELD_NAME_AGG_CALLS) AggregateCall[] aggCalls, @JsonProperty(FIELD_NAME_WINDOWING) WindowingStrategy windowing, @@ -135,7 +139,7 @@ public StreamExecWindowAggregate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 59adf5531b484..b9a01a041f269 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 @@ -18,6 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.planner.plan.logical.CumulativeWindowSpec; import org.apache.flink.table.planner.plan.logical.HoppingWindowSpec; @@ -53,10 +54,11 @@ public abstract class StreamExecWindowAggregateBase extends StreamExecAggregateB protected StreamExecWindowAggregateBase( int id, ExecNodeContext context, + ReadableConfig persistedConfig, List inputProperties, LogicalType outputType, String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 35091f74901b4..4f8dd6087e0ca 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -90,6 +91,7 @@ public class StreamExecWindowDeduplicate extends ExecNodeBase private final WindowingStrategy windowing; public StreamExecWindowDeduplicate( + ReadableConfig tableConfig, int[] partitionKeys, int orderKey, boolean keepLastRow, @@ -100,6 +102,7 @@ public StreamExecWindowDeduplicate( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowDeduplicate.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowDeduplicate.class, tableConfig), partitionKeys, orderKey, keepLastRow, @@ -113,6 +116,7 @@ public StreamExecWindowDeduplicate( public StreamExecWindowDeduplicate( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_PARTITION_KEYS) int[] partitionKeys, @JsonProperty(FIELD_NAME_ORDER_KEY) int orderKey, @JsonProperty(FIELD_NAME_KEEP_LAST_ROW) boolean keepLastRow, @@ -120,7 +124,7 @@ public StreamExecWindowDeduplicate( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 dcd2a42e9d90f..c15aac2128e36 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 @@ -21,6 +21,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.api.TableException; import org.apache.flink.table.data.RowData; @@ -84,6 +85,7 @@ public class StreamExecWindowJoin extends ExecNodeBase private final WindowingStrategy rightWindowing; public StreamExecWindowJoin( + ReadableConfig tableConfig, JoinSpec joinSpec, WindowingStrategy leftWindowing, WindowingStrategy rightWindowing, @@ -94,6 +96,7 @@ public StreamExecWindowJoin( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowJoin.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowJoin.class, tableConfig), joinSpec, leftWindowing, rightWindowing, @@ -106,13 +109,14 @@ public StreamExecWindowJoin( public StreamExecWindowJoin( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_JOIN_SPEC) JoinSpec joinSpec, @JsonProperty(FIELD_NAME_LEFT_WINDOWING) WindowingStrategy leftWindowing, @JsonProperty(FIELD_NAME_RIGHT_WINDOWING) WindowingStrategy rightWindowing, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 302e69358eb72..3ca0a08d84c8d 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 @@ -20,6 +20,7 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.operators.OneInputStreamOperator; import org.apache.flink.streaming.api.operators.SimpleOperatorFactory; import org.apache.flink.streaming.api.transformations.OneInputTransformation; @@ -106,6 +107,7 @@ public class StreamExecWindowRank extends ExecNodeBase private final WindowingStrategy windowing; public StreamExecWindowRank( + ReadableConfig tableConfig, RankType rankType, PartitionSpec partitionSpec, SortSpec sortSpec, @@ -118,6 +120,7 @@ public StreamExecWindowRank( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowRank.class), + ExecNodeContext.newPersistedConfig(StreamExecWindowRank.class, tableConfig), rankType, partitionSpec, sortSpec, @@ -133,6 +136,7 @@ public StreamExecWindowRank( public StreamExecWindowRank( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_RANK_TYPE) RankType rankType, @JsonProperty(FIELD_NAME_PARTITION_SPEC) PartitionSpec partitionSpec, @JsonProperty(FIELD_NAME_SORT_SPEC) SortSpec sortSpec, @@ -142,7 +146,7 @@ public StreamExecWindowRank( @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, inputProperties, outputType, description); + super(id, context, persistedConfig, 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 38ea6ee89a2a0..7728712a6af29 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 @@ -19,6 +19,7 @@ package org.apache.flink.table.planner.plan.nodes.exec.stream; import org.apache.flink.FlinkVersion; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -51,6 +52,7 @@ public class StreamExecWindowTableFunction extends CommonExecWindowTableFunction implements StreamExecNode { public StreamExecWindowTableFunction( + ReadableConfig tableConfig, TimeAttributeWindowingStrategy windowingStrategy, InputProperty inputProperty, RowType outputType, @@ -58,6 +60,8 @@ public StreamExecWindowTableFunction( this( ExecNodeContext.newNodeId(), ExecNodeContext.newContext(StreamExecWindowTableFunction.class), + ExecNodeContext.newPersistedConfig( + StreamExecWindowTableFunction.class, tableConfig), windowingStrategy, Collections.singletonList(inputProperty), outputType, @@ -68,10 +72,18 @@ public StreamExecWindowTableFunction( public StreamExecWindowTableFunction( @JsonProperty(FIELD_NAME_ID) int id, @JsonProperty(FIELD_NAME_TYPE) ExecNodeContext context, + @JsonProperty(FIELD_NAME_CONFIGURATION) ReadableConfig persistedConfig, @JsonProperty(FIELD_NAME_WINDOWING) TimeAttributeWindowingStrategy windowingStrategy, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description) { - super(id, context, windowingStrategy, inputProperties, outputType, description); + super( + id, + context, + persistedConfig, + 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 index c295e31482683..2da4fffb1c46f 100644 --- 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 @@ -20,6 +20,12 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.TableConfigOptions; 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; @@ -73,7 +79,10 @@ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank; import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction; +import javax.annotation.Nullable; + import java.util.ArrayList; +import java.util.Arrays; import java.util.Comparator; import java.util.HashMap; import java.util.HashSet; @@ -81,6 +90,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.stream.Stream; /** Utility class for {@link ExecNodeMetadata} related functionality. */ @Internal @@ -159,6 +169,18 @@ private ExecNodeMetadataUtil() { } }; + public static final Set> TABLE_CONFIG_OPTIONS; + + static { + TABLE_CONFIG_OPTIONS = ConfigUtils.getAllConfigOptions(TableConfigOptions.class); + } + + public static final Set> EXECUTION_CONFIG_OPTIONS; + + static { + EXECUTION_CONFIG_OPTIONS = ConfigUtils.getAllConfigOptions(ExecutionConfigOptions.class); + } + public static Set>> execNodes() { return EXEC_NODES; } @@ -254,6 +276,63 @@ public static > ExecNodeMetadata latestAnnotation( return sortedAnnotations.get(sortedAnnotations.size() - 1); } + @Nullable + public static > String[] consumedOptions(Class execNodeClass) { + ExecNodeMetadata metadata = latestAnnotation(execNodeClass); + if (metadata == null) { + return null; + } + return metadata.consumedOptions(); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + public static > ReadableConfig newPersistedConfig( + Class execNodeClass, + ReadableConfig tableConfig, + Stream> configOptions) { + final Map> availableConfigOptions = new HashMap<>(); + configOptions.forEach( + co -> { + availableConfigOptions.put(co.key(), co); + co.fallbackKeys().forEach(k -> availableConfigOptions.put(k.getKey(), co)); + }); + + final Configuration persistedConfig = new Configuration(); + final String[] consumedOptions = ExecNodeMetadataUtil.consumedOptions(execNodeClass); + if (consumedOptions == null) { + return persistedConfig; + } + + final Map nodeConfigOptions = new HashMap<>(); + for (final String consumedOption : consumedOptions) { + ConfigOption configOption = availableConfigOptions.get(consumedOption); + if (configOption == null) { + throw new IllegalStateException( + String.format( + "ExecNode: %s, consumedOption: %s not listed in [%s].", + execNodeClass.getCanonicalName(), + consumedOption, + String.join( + ", ", + Arrays.asList( + TableConfigOptions.class.getSimpleName(), + ExecutionConfigOptions.class.getSimpleName())))); + } + if (nodeConfigOptions.containsKey(configOption)) { + throw new IllegalStateException( + String.format( + "ExecNode: %s, consumedOption: %s is listed multiple times in " + + "consumedOptions, potentially also with " + + "fallback/deprecated key.", + execNodeClass.getCanonicalName(), consumedOption)); + } else { + nodeConfigOptions.put(configOption, tableConfig.get(configOption)); + } + } + nodeConfigOptions.forEach(persistedConfig::set); + return persistedConfig; + } + /** Helper Pojo used as a tuple for the {@link #LOOKUP_MAP}. */ private static final class ExecNodeNameVersion { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala index 12282936c2f0b..8122033837cac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalBoundedStreamScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecBoundedStreamScan import org.apache.flink.table.planner.plan.schema.DataStreamTable import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType @@ -71,6 +72,7 @@ class BatchPhysicalBoundedStreamScan( override def translateToExecNode(): ExecNode[_] = { new BatchExecBoundedStreamScan( + unwrapTableConfig(this), boundedStreamTable.dataStream, boundedStreamTable.dataType, boundedStreamTable.fieldIndexes, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala index 6fefa39f436e8..2c62493ce4dd7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCalc.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecCalc -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -54,6 +55,7 @@ class BatchPhysicalCalc( } new BatchExecCalc( + unwrapTableConfig(this), projection, condition, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala index f6dc6105e16f6..6ba5e703e1b61 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalCorrelate.scala @@ -19,9 +19,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -65,6 +66,7 @@ class BatchPhysicalCorrelate( override def translateToExecNode(): ExecNode[_] = { new BatchExecCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], condition.orNull, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala index e58063f96127c..cd28172d21621 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExchange.scala @@ -49,6 +49,7 @@ class BatchPhysicalExchange( override def translateToExecNode(): ExecNode[_] = { new BatchExecExchange( + unwrapTableConfig(this), getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala index 237929433b7cb..99905c0b8f99f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalExpand.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.calcite.Expand import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecExpand -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -53,10 +54,10 @@ class BatchPhysicalExpand( override def translateToExecNode(): ExecNode[_] = { new BatchExecExpand( + unwrapTableConfig(this), projects, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala index cff42f57a82b6..20743c9636e22 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashAggregate.scala @@ -149,6 +149,7 @@ class BatchPhysicalHashAggregate( InputProperty.hashDistribution(grouping) } new BatchExecHashAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -160,7 +161,6 @@ class BatchPhysicalHashAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala index 1606874fe7dde..708208128448c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.FlinkConventions import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashJoin import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, JoinUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.HashJoinType import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer @@ -166,18 +167,18 @@ class BatchPhysicalHashJoin( val rightRowCount = Util.first(mq.getRowCount(right), 200000).toLong val (leftEdge, rightEdge) = getInputProperties new BatchExecHashJoin( - joinSpec, - leftRowSize, - rightRowSize, - leftRowCount, - rightRowCount, - leftIsBuild, - tryDistinctBuildRow, - leftEdge, - rightEdge, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + unwrapTableConfig(this), + joinSpec, + leftRowSize, + rightRowSize, + leftRowCount, + rightRowCount, + leftIsBuild, + tryDistinctBuildRow, + leftEdge, + rightEdge, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } private def getInputProperties: (InputProperty, InputProperty) = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala index 5838941603f35..b47f36b2d2216 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalHashWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -89,6 +90,7 @@ class BatchPhysicalHashWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecHashWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -105,7 +107,6 @@ class BatchPhysicalHashWindowAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala index fcdb53a9591d3..28d1445493e5b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacySink.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.LegacySink import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLegacySink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.UpdatingPlanChecker +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.sinks.{TableSink, UpsertStreamTableSink} @@ -57,13 +58,13 @@ class BatchPhysicalLegacySink[T]( case _ => Option.empty[Array[String]] } new BatchExecLegacySink[T]( + unwrapTableConfig(this), sink, upsertKeys.orNull, // the input records will not trigger any output of a sink because it has no output, // so it's dam behavior is BLOCKING InputProperty.builder().damBehavior(InputProperty.DamBehavior.BLOCKING).build(), fromDataTypeToLogicalType(sink.getConsumedDataType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala index 8778505f8a1ab..be480d501148d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLegacyTableSourceScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLegacyTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLegacyTableSourceScan import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.sources.StreamTableSource import org.apache.calcite.plan._ @@ -61,6 +62,7 @@ class BatchPhysicalLegacyTableSourceScan( override def translateToExecNode(): ExecNode[_] = { new BatchExecLegacyTableSourceScan( + unwrapTableConfig(this), tableSource, getTable.getQualifiedName, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala index d3ee2fa4026d3..eee744c6d5ad1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLimit.scala @@ -22,9 +22,10 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.FlinkCost._ import org.apache.flink.table.planner.plan.cost.FlinkCostFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil.fetchToString import org.apache.flink.table.planner.plan.utils.SortUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel._ @@ -81,11 +82,12 @@ class BatchPhysicalLimit( override def translateToExecNode(): ExecNode[_] = { new BatchExecLimit( - limitStart, - limitEnd, - isGlobal, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + limitStart, + limitEnd, + isGlobal, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala index 98e3da6fed37f..0359472912356 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashAggregate.scala @@ -22,8 +22,9 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type @@ -121,6 +122,7 @@ class BatchPhysicalLocalHashAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecHashAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -129,8 +131,7 @@ class BatchPhysicalLocalHashAggregate( false, // isFinal is always false getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala index 7d81fe048f2a9..cf3a735ac17fd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalHashWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecHashWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -82,6 +83,7 @@ class BatchPhysicalLocalHashWindowAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecHashWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -95,7 +97,6 @@ class BatchPhysicalLocalHashWindowAggregate( false, // isFinal is always false InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala index 59faaf0483ca7..93cf6b8ada28f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortAggregate.scala @@ -22,8 +22,9 @@ import org.apache.flink.table.functions.UserDefinedFunction import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type @@ -129,6 +130,7 @@ class BatchPhysicalLocalSortAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -137,8 +139,7 @@ class BatchPhysicalLocalSortAggregate( false, // isFinal is always false getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala index 9a5fdca397f09..05a1902a1ccd7 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLocalSortWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -80,6 +81,7 @@ class BatchPhysicalLocalSortWindowAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -93,7 +95,6 @@ class BatchPhysicalLocalSortWindowAggregate( false, // isFinal is always false InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala index e66a32c45368d..900e19fd0e35a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalLookupJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLookupJoin import org.apache.flink.table.planner.plan.utils.{FlinkRexUtil, JoinTypeUtil} import org.apache.flink.table.planner.utils.JavaScalaConversionUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptTable, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -77,6 +78,7 @@ class BatchPhysicalLookupJoin( } new BatchExecLookupJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), remainingCondition.orNull, new TemporalTableSourceSpec(temporalTable), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala index 6faa2b92434a3..93119021dc464 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalNestedLoopJoin.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecNestedLoopJoin import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.typeutils.BinaryRowDataSerializer import org.apache.calcite.plan._ @@ -114,6 +115,7 @@ class BatchPhysicalNestedLoopJoin( override def translateToExecNode(): ExecNode[_] = { val (leftInputProperty, rightInputProperty) = getInputProperties new BatchExecNestedLoopJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), condition, leftIsBuild, @@ -121,8 +123,7 @@ class BatchPhysicalNestedLoopJoin( leftInputProperty, rightInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } def getInputProperties: (InputProperty, InputProperty) = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala index 5260dfe6e0d93..a45481462627f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalOverAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecOverAggrega import org.apache.flink.table.planner.plan.nodes.exec.spec.{OverSpec, PartitionSpec} import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -71,6 +72,7 @@ class BatchPhysicalOverAggregate( InputProperty.hashDistribution(partitionKeyIndices) } new BatchExecOverAggregate( + unwrapTableConfig(this), new OverSpec( new PartitionSpec(partitionKeyIndices), offsetAndInsensitiveSensitiveGroups.map(OverAggregateUtil.createGroupSpec(_, logicWindow)), @@ -78,7 +80,6 @@ class BatchPhysicalOverAggregate( OverAggregateUtil.calcOriginalInputFields(logicWindow)), InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala index a32457ba7da71..0fa260bf346ac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCalc.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonCalc import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,6 +59,7 @@ class BatchPhysicalPythonCalc( } new BatchExecPythonCalc( + unwrapTableConfig(this), projection, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala index 298476c0ce1df..3d59cea816f5d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonCorrelate.scala @@ -20,9 +20,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -74,11 +75,11 @@ class BatchPhysicalPythonCorrelate( } new BatchExecPythonCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala index 1aea359dc6571..136c7260cb2ea 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupAggregate.scala @@ -25,7 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonGroupAggregate import org.apache.flink.table.planner.plan.rules.physical.batch.BatchPhysicalJoinRuleBase import org.apache.flink.table.planner.plan.utils.{FlinkRelOptUtil, RelExplainUtil} -import org.apache.flink.table.planner.utils.ShortcutUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type.{HASH_DISTRIBUTED, SINGLETON} @@ -93,7 +93,7 @@ class BatchPhysicalPythonGroupAggregate( true } else { // If partialKey is enabled, try to use partial key to satisfy the required distribution - val tableConfig = ShortcutUtils.unwrapTableConfig(this) + val tableConfig = unwrapTableConfig(this) val partialKeyEnabled = tableConfig.get( BatchPhysicalJoinRuleBase.TABLE_OPTIMIZER_SHUFFLE_BY_PARTIAL_KEY_ENABLED) partialKeyEnabled && groupKeysList.containsAll(shuffleKeys) @@ -157,6 +157,7 @@ class BatchPhysicalPythonGroupAggregate( InputProperty.hashDistribution(grouping) } new BatchExecPythonGroupAggregate( + unwrapTableConfig(this), grouping, grouping ++ auxGrouping, aggCalls.toArray, @@ -165,8 +166,7 @@ class BatchPhysicalPythonGroupAggregate( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala index ae2c2f57b9373..09e9fcdf9b746 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonGroupWindowAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} @@ -104,6 +105,7 @@ class BatchPhysicalPythonGroupWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecPythonGroupWindowAggregate( + unwrapTableConfig(this), grouping, grouping ++ auxGrouping, aggCalls.toArray, @@ -112,7 +114,6 @@ class BatchPhysicalPythonGroupWindowAggregate( namedWindowProperties.toArray, InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala index 9f42ab41ee2ed..63535c045f0a9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalPythonOverAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecPythonOverA import org.apache.flink.table.planner.plan.nodes.exec.spec.{OverSpec, PartitionSpec} import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel._ @@ -70,6 +71,7 @@ class BatchPhysicalPythonOverAggregate( InputProperty.hashDistribution(partitionKeyIndices) } new BatchExecPythonOverAggregate( + unwrapTableConfig(this), new OverSpec( new PartitionSpec(partitionKeyIndices), offsetAndInsensitiveSensitiveGroups.map(OverAggregateUtil.createGroupSpec(_, logicWindow)), @@ -77,7 +79,6 @@ class BatchPhysicalPythonOverAggregate( OverAggregateUtil.calcOriginalInputFields(logicWindow)), InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala index a89612828e791..e34f131b389d2 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalRank.scala @@ -235,6 +235,7 @@ class BatchPhysicalRank( InputProperty.hashDistribution(partitionKey.toArray) } new BatchExecRank( + unwrapTableConfig(this), partitionKey.toArray, orderKey.getFieldCollations.map(_.getFieldIndex).toArray, rankStart, @@ -242,7 +243,6 @@ class BatchPhysicalRank( outputRankNumber, InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala index 1400b0990aad1..566fae152bc0f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSink.scala @@ -18,7 +18,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch -import org.apache.flink.table.catalog.{CatalogTable, ContextResolvedTable, ObjectIdentifier, ResolvedCatalogTable} +import org.apache.flink.table.catalog.ContextResolvedTable import org.apache.flink.table.connector.sink.DynamicTableSink import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.abilities.sink.SinkAbilitySpec @@ -26,7 +26,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.Sink import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSink import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} -import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -67,12 +67,12 @@ class BatchPhysicalSink( tableSinkSpec.setTableSink(tableSink) new BatchExecSink( + unwrapTableConfig(this), tableSinkSpec, // the input records will not trigger any output of a sink because it has no output, // so it's dam behavior is BLOCKING InputProperty.builder().damBehavior(InputProperty.DamBehavior.BLOCKING).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala index c1ac0b732eaa2..73236ee2d551d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSort.scala @@ -21,8 +21,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSort -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -79,10 +80,10 @@ class BatchPhysicalSort( @Override override def translateToExecNode(): ExecNode[_] = { new BatchExecSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.builder().damBehavior(InputProperty.DamBehavior.END_INPUT).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala index ebdc10728f753..280f9af6f2445 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortAggregate.scala @@ -156,6 +156,7 @@ class BatchPhysicalSortAggregate( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -164,8 +165,7 @@ class BatchPhysicalSortAggregate( true, // isFinal is always true getInputProperty, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def getInputProperty: InputProperty = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala index 16228f1a72987..861d7d9811bf0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortLimit.scala @@ -21,8 +21,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptCost, RelOptPlanner, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -100,13 +101,13 @@ class BatchPhysicalSortLimit( override def translateToExecNode(): ExecNode[_] = { new BatchExecSortLimit( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), limitStart, limitEnd, isGlobal, InputProperty.builder().damBehavior(InputProperty.DamBehavior.END_INPUT).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala index c2bfa50550f47..909489de3459b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortMergeJoin.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.cost.{FlinkCost, FlinkCostFactory} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortMergeJoin import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{FlinkRelMdUtil, FlinkRelOptUtil, JoinTypeUtil, JoinUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.FlinkJoinType import org.apache.calcite.plan._ @@ -173,6 +174,7 @@ class BatchPhysicalSortMergeJoin( FlinkTypeFactory.toLogicalRowType(right.getRowType)) new BatchExecSortMergeJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), joinSpec.getLeftKeys, joinSpec.getRightKeys, @@ -188,8 +190,7 @@ class BatchPhysicalSortMergeJoin( .damBehavior(InputProperty.DamBehavior.END_INPUT) .build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } private def estimateOutputSize(relNode: RelNode): Double = { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala index a013e283e6d4d..f48e51e20f354 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalSortWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecSortWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -85,6 +86,7 @@ class BatchPhysicalSortWindowAggregate( InputProperty.hashDistribution(grouping) } new BatchExecSortWindowAggregate( + unwrapTableConfig(this), grouping, auxGrouping, getAggCallList.toArray, @@ -98,7 +100,6 @@ class BatchPhysicalSortWindowAggregate( true, // isFinal is always true InputProperty.builder().requiredDistribution(requiredDistribution).build(), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala index e5f0ad2c7f602..7ab2bf7f7c945 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalTableSourceScan.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecTableSource import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpec import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalTableSourceScan import org.apache.flink.table.planner.plan.schema.TableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode @@ -72,6 +73,7 @@ class BatchPhysicalTableSourceScan( tableSourceSpec.setTableSource(tableSourceTable.tableSource) new BatchExecTableSourceScan( + unwrapTableConfig(this), tableSourceSpec, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala index 9d1d9986f0524..a440b01c71c66 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalUnion.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.{FlinkRelDistribution, FlinkRelDistributionTraitDef} import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecUnion -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptRule, RelTraitSet} import org.apache.calcite.rel.RelDistribution.Type._ @@ -95,9 +96,9 @@ class BatchPhysicalUnion( override def translateToExecNode(): ExecNode[_] = { new BatchExecUnion( + unwrapTableConfig(this), getInputs.map(_ => InputProperty.DEFAULT), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala index 23ff9dc89e12a..fbbcef7edc041 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalValues.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.batch 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.batch.BatchExecValues +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import com.google.common.collect.ImmutableList import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -57,9 +58,9 @@ class BatchPhysicalValues( override def translateToExecNode(): ExecNode[_] = { new BatchExecValues( + unwrapTableConfig(this), tuples.asList().map(_.asList()), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala index 0c888332bd951..1b75542827333 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/batch/BatchPhysicalWindowTableFunction.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrateg import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.batch.BatchExecWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,11 +59,11 @@ class BatchPhysicalWindowTableFunction( override def translateToExecNode(): ExecNode[_] = { new BatchExecWindowTableFunction( + unwrapTableConfig(this), windowing, // TODO set semantic window (such as session window) require other Dam Behavior InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala index 6603e1b515b3b..396765d8adc50 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCalc.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCalc import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -54,6 +55,7 @@ class StreamPhysicalCalc( } new StreamExecCalc( + unwrapTableConfig(this), projection, condition, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala index ce06e55379f65..7b5ff0e45981f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalChangelogNormalize.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecChangelogNormalize import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -67,6 +68,7 @@ class StreamPhysicalChangelogNormalize( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecChangelogNormalize( + unwrapTableConfig(this), uniqueKeys, generateUpdateBefore, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala index 710c4df7959fa..5fbdfc33b0307 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalCorrelate.scala @@ -19,9 +19,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -65,6 +66,7 @@ class StreamPhysicalCorrelate( override def translateToExecNode(): ExecNode[_] = { new StreamExecCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], condition.orNull, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala index c80cc0d40fbbb..d3b95bf5575b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDataStreamScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDataStreamScan import org.apache.flink.table.planner.plan.schema.DataStreamTable import org.apache.flink.table.planner.plan.utils.RelExplainUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.`type`.RelDataType @@ -73,6 +74,7 @@ class StreamPhysicalDataStreamScan( override def translateToExecNode(): ExecNode[_] = { new StreamExecDataStreamScan( + unwrapTableConfig(this), dataStreamTable.dataStream, dataStreamTable.dataType, dataStreamTable.fieldIndexes, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala index ddc772ca68848..ae1cabcfc97a0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeduplicate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDeduplicate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -73,13 +74,13 @@ class StreamPhysicalDeduplicate( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecDeduplicate( + unwrapTableConfig(this), uniqueKeys, isRowtime, keepLastRow, generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala index ba24f6e64cf83..3a9e75209e37e 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDropUpdateBefore.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDropUpdateBefore -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -60,9 +61,9 @@ class StreamPhysicalDropUpdateBefore( } new StreamExecDropUpdateBefore( + unwrapTableConfig(this), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala index e07caee902c9e..91c272cc8753b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExchange.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExchange -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalExchange +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelDistribution, RelNode} @@ -48,6 +49,7 @@ class StreamPhysicalExchange( override def translateToExecNode(): ExecNode[_] = { new StreamExecExchange( + unwrapTableConfig(this), InputProperty.builder.requiredDistribution(getRequiredDistribution).build, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala index 9e0c414c439a8..6cbfc613b1149 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalExpand.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.calcite.Expand import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecExpand -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -49,10 +50,10 @@ class StreamPhysicalExpand( override def translateToExecNode(): ExecNode[_] = { new StreamExecExpand( + unwrapTableConfig(this), projects, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala index 2e4dbb05f70fd..8f557037ddfc1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGlobalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -104,6 +105,7 @@ class StreamPhysicalGlobalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecGlobalGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -113,7 +115,6 @@ class StreamPhysicalGlobalGroupAggregate( indexOfCountStar.map(Integer.valueOf).orNull, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala index 00ae2fafaa468..31b1c6980188b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGlobalWindowAggregate.scala @@ -129,6 +129,7 @@ class StreamPhysicalGlobalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(unwrapTableConfig(this)) new StreamExecGlobalWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, @@ -136,7 +137,6 @@ class StreamPhysicalGlobalWindowAggregate( InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(inputRowTypeOfLocalAgg), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala index 07bca07984748..c80ea2efa7e22 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils, RelExplainUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -84,6 +85,7 @@ class StreamPhysicalGroupAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -91,7 +93,6 @@ class StreamPhysicalGroupAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala index 5097a404a42ca..a94868b54c97b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupTableAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupTableAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalGroupTableAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupTableAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -70,7 +72,6 @@ class StreamPhysicalGroupTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala index 679fc26fa273b..a24b8817ad757 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical._ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -75,6 +76,7 @@ class StreamPhysicalGroupWindowAggregate( override def translateToExecNode(): ExecNode[_] = { val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -82,7 +84,6 @@ class StreamPhysicalGroupWindowAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala index bdee5f655d06b..cc073cb2ebb5b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalGroupWindowTableAggregate.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.logical._ import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -75,6 +76,7 @@ class StreamPhysicalGroupWindowTableAggregate( override def translateToExecNode(): ExecNode[_] = { val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -82,7 +84,6 @@ class StreamPhysicalGroupWindowTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala index 48a2c999208a5..be37d1fef3931 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIncrementalGroupAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecIncrementalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -118,6 +119,7 @@ class StreamPhysicalIncrementalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecIncrementalGroupAggregate( + unwrapTableConfig(this), partialAggGrouping, finalAggGrouping, partialOriginalAggCalls, @@ -126,7 +128,6 @@ class StreamPhysicalIncrementalGroupAggregate( partialAggNeedRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala index 9b26cd447e65c..dd4640d597361 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalIntervalJoin.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.core.{Join, JoinRelType} @@ -92,10 +93,11 @@ class StreamPhysicalIntervalJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecIntervalJoin( - new IntervalJoinSpec(joinSpec, windowBounds), - InputProperty.DEFAULT, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + new IntervalJoinSpec(joinSpec, windowBounds), + InputProperty.DEFAULT, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala index f4b68a9fde8cf..0d33d411774c6 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalJoin.scala @@ -20,10 +20,11 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery -import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecJoin +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.JoinUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.typeutils.InternalTypeInfo import org.apache.calcite.plan._ @@ -126,12 +127,13 @@ class StreamPhysicalJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecJoin( - joinSpec, - getUniqueKeys(left, joinSpec.getLeftKeys), - getUniqueKeys(right, joinSpec.getRightKeys), - InputProperty.DEFAULT, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription) + unwrapTableConfig(this), + joinSpec, + getUniqueKeys(left, joinSpec.getLeftKeys), + getUniqueKeys(right, joinSpec.getRightKeys), + InputProperty.DEFAULT, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala index 4cb7bb7935d51..bdc2ff017b298 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacySink.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.LegacySink import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacySink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, UpdatingPlanChecker} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromDataTypeToLogicalType import org.apache.flink.table.sinks._ @@ -61,12 +62,12 @@ class StreamPhysicalLegacySink[T]( val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecLegacySink( + unwrapTableConfig(this), sink, upsertKeys.orNull, needRetraction, InputProperty.DEFAULT, fromDataTypeToLogicalType(sink.getConsumedDataType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala index 39ae30021508e..ae71c1f470c61 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLegacyTableSourceScan.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNode import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLegacyTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLegacyTableSourceScan import org.apache.flink.table.planner.plan.schema.LegacyTableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.sources.StreamTableSource import org.apache.calcite.plan._ @@ -57,6 +58,7 @@ class StreamPhysicalLegacyTableSourceScan( override def translateToExecNode(): ExecNode[_] = { new StreamExecLegacyTableSourceScan( + unwrapTableConfig(this), tableSource, getTable.getQualifiedName, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala index 0e2bd82015bc0..264aaaff5dd6a 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLimit.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -71,13 +72,13 @@ class StreamPhysicalLimit( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecLimit( - limitStart, - limitEnd, - generateUpdateBefore, - needRetraction, - InputProperty.DEFAULT, - FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + unwrapTableConfig(this), + limitStart, + limitEnd, + generateUpdateBefore, + needRetraction, + InputProperty.DEFAULT, + FlinkTypeFactory.toLogicalRowType(getRowType), + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala index be2d352d903b7..684274ea5aad5 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.PartialFinalType import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -91,13 +92,13 @@ class StreamPhysicalLocalGroupAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecLocalGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala index 61bfa8248ed09..9019b156a4842 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLocalWindowAggregate.scala @@ -126,12 +126,12 @@ class StreamPhysicalLocalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(unwrapTableConfig(this)) new StreamExecLocalWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala index f962e2909b0e8..d39c77795795c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalLookupJoin.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalLookupJoin import org.apache.flink.table.planner.plan.utils.{FlinkRexUtil, JoinTypeUtil} import org.apache.flink.table.planner.utils.JavaScalaConversionUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelOptTable, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -77,6 +78,7 @@ class StreamPhysicalLookupJoin( (null, null) } new StreamExecLookupJoin( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), remainingCondition.orNull, new TemporalTableSourceSpec(temporalTable), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala index 252af7c0a9254..c449762da57ac 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMatch.scala @@ -22,10 +22,11 @@ import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.logical.MatchRecognize import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMatch -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.MatchUtil import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -97,10 +98,10 @@ class StreamPhysicalMatch( override def translateToExecNode(): ExecNode[_] = { new StreamExecMatch( + unwrapTableConfig(this), MatchUtil.createMatchSpec(logicalMatch), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala index 524a0c59f41d1..d2e88acb38674 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalMiniBatchAssigner.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.`trait`.MiniBatchIntervalTraitDef import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecMiniBatchAssigner -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel} @@ -63,10 +64,10 @@ class StreamPhysicalMiniBatchAssigner( override def translateToExecNode(): ExecNode[_] = { val miniBatchInterval = traits.getTrait(MiniBatchIntervalTraitDef.INSTANCE).getMiniBatchInterval new StreamExecMiniBatchAssigner( + unwrapTableConfig(this), miniBatchInterval, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala index ff1ea7654d2cd..3409e3380127c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalOverAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecOverAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -57,10 +58,10 @@ class StreamPhysicalOverAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecOverAggregate( + unwrapTableConfig(this), OverAggregateUtil.createOverSpec(logicWindow), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala index b7eff1171a612..d946f708becc0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCalc.scala @@ -21,7 +21,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCalc -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -58,6 +59,7 @@ class StreamPhysicalPythonCalc( } new StreamExecPythonCalc( + unwrapTableConfig(this), projection, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala index 434f163a37b80..6c91710a95f3d 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonCorrelate.scala @@ -20,9 +20,10 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.api.TableException import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonCorrelate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.logical.FlinkLogicalTableFunctionScan import org.apache.flink.table.planner.plan.utils.JoinTypeUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -74,11 +75,11 @@ class StreamPhysicalPythonCorrelate( } new StreamExecPythonCorrelate( + unwrapTableConfig(this), JoinTypeUtil.getFlinkJoinType(joinType), scan.getCall.asInstanceOf[RexCall], InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala index 6099021572373..53fca11f65f27 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupAggregate.scala @@ -20,8 +20,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -79,6 +80,7 @@ class StreamPhysicalPythonGroupAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -86,7 +88,6 @@ class StreamPhysicalPythonGroupAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala index c77a5d766c963..bf6afa3c79676 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupTableAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupTableAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{AggregateUtil, ChangelogPlanUtils} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalPythonGroupTableAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupTableAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, aggCallNeedRetractions, @@ -70,7 +72,6 @@ class StreamPhysicalPythonGroupTableAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala index 3dc7043c7b1cd..ec623dee2f8bb 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonGroupWindowAggregate.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.LogicalWindow import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonGroupWindowAggregate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, WindowEmitStrategy} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.groupwindow.NamedWindowProperty import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -71,6 +72,7 @@ class StreamPhysicalPythonGroupWindowAggregate( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val needRetraction = !ChangelogPlanUtils.inputInsertOnly(this) new StreamExecPythonGroupWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, window, @@ -79,7 +81,6 @@ class StreamPhysicalPythonGroupWindowAggregate( needRetraction, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala index 677644b691a90..7f62f9f0bf90f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalPythonOverAggregate.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecPythonOverAggregate -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.OverAggregateUtil +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -56,11 +57,11 @@ class StreamPhysicalPythonOverAggregate( override def translateToExecNode(): ExecNode[_] = { new StreamExecPythonOverAggregate( + unwrapTableConfig(this), OverAggregateUtil.createOverSpec(logicWindow), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala index 61879f5bbe4ea..77e536bdf7409 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalRank.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecRank import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank._ import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -106,6 +107,7 @@ class StreamPhysicalRank( val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) val fieldCollations = orderKey.getFieldCollations new StreamExecRank( + unwrapTableConfig(this), rankType, new PartitionSpec(partitionKey.toArray), SortUtil.getSortSpec(fieldCollations), @@ -115,7 +117,6 @@ class StreamPhysicalRank( generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala index fcc391442b5e5..af63d5be3c4e0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSink.scala @@ -27,6 +27,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSinkSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSink import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{ChangelogPlanUtils, RelDescriptionWriterImpl} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -85,13 +86,13 @@ class StreamPhysicalSink( util.Arrays.asList(abilitySpecs: _*)) tableSinkSpec.setTableSink(tableSink) new StreamExecSink( + unwrapTableConfig(this), tableSinkSpec, inputChangelogMode, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), upsertMaterialize, - getDescriptionWithUpsert(upsertMaterialize) - ) + getDescriptionWithUpsert(upsertMaterialize)) } /** diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala index d0df328623d39..47331fa27cccd 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSort.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSort import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -71,11 +72,11 @@ class StreamPhysicalSort( override def translateToExecNode(): ExecNode[_] = { new StreamExecSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala index 346a77b5e72be..0e1d45ade0196 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalSortLimit.scala @@ -19,8 +19,9 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecSortLimit -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.core.Sort @@ -90,6 +91,7 @@ class StreamPhysicalSortLimit( override def translateToExecNode(): ExecNode[_] = { val generateUpdateBefore = ChangelogPlanUtils.generateUpdateBefore(this) new StreamExecSortLimit( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), limitStart, limitEnd, @@ -97,7 +99,6 @@ class StreamPhysicalSortLimit( generateUpdateBefore, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala index c334be817e4bc..a82160be5495c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTableSourceScan.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.DynamicTableSourceSpe import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTableSourceScan import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalTableSourceScan import org.apache.flink.table.planner.plan.schema.TableSourceTable +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan._ import org.apache.calcite.rel.RelNode @@ -63,6 +64,7 @@ class StreamPhysicalTableSourceScan( tableSourceSpec.setTableSource(tableSource) new StreamExecTableSourceScan( + unwrapTableConfig(this), tableSourceSpec, FlinkTypeFactory.toLogicalRowType(getRowType), getRelDetailedDescription) diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala index 4e1b359b4532d..933180bce029f 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalJoin.scala @@ -26,6 +26,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil import org.apache.flink.table.planner.plan.utils.TemporalJoinUtil.{TEMPORAL_JOIN_CONDITION, TEMPORAL_JOIN_CONDITION_PRIMARY_KEY} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.util.Preconditions.checkState import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -115,6 +116,7 @@ class StreamPhysicalTemporalJoin( } new StreamExecTemporalJoin( + unwrapTableConfig(this), temporalJoinSpec, isTemporalFunctionJoin, leftTimeAttributeInputRef, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala index 1e21455b41020..e1709d6aad1c0 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalTemporalSort.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecTemporalSort import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.{RelExplainUtil, SortUtil} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -62,10 +63,10 @@ class StreamPhysicalTemporalSort( override def translateToExecNode(): ExecNode[_] = { new StreamExecTemporalSort( + unwrapTableConfig(this), SortUtil.getSortSpec(sortCollation.getFieldCollations), InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala index f45873573f7ae..fbc4a9611b8d1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalUnion.scala @@ -20,7 +20,8 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream import org.apache.flink.table.planner.calcite.FlinkTypeFactory import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecUnion -import org.apache.flink.table.planner.plan.nodes.exec.{InputProperty, ExecNode} +import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.`type`.RelDataType @@ -59,9 +60,9 @@ class StreamPhysicalUnion( override def translateToExecNode(): ExecNode[_] = { new StreamExecUnion( + unwrapTableConfig(this), getInputs.map(_ => InputProperty.DEFAULT), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala index 9b511a84845aa..07ed842d36326 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalValues.scala @@ -21,6 +21,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream 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.stream.StreamExecValues +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import com.google.common.collect.ImmutableList import org.apache.calcite.plan._ @@ -52,9 +53,9 @@ class StreamPhysicalValues( override def translateToExecNode(): ExecNode[_] = { new StreamExecValues( + unwrapTableConfig(this), tuples.asList().map(_.asList()), FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala index 6ccbecb436b0f..46fe743103d9b 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWatermarkAssigner.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.nodes.calcite.WatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWatermarkAssigner import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.{RelNode, RelWriter} @@ -70,6 +71,7 @@ class StreamPhysicalWatermarkAssigner( override def translateToExecNode(): ExecNode[_] = { new StreamExecWatermarkAssigner( + unwrapTableConfig(this), watermarkExpr, rowtimeFieldIndex, InputProperty.DEFAULT, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala index 973049fa31ed3..8c5f6e6f9e6a9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowAggregate.scala @@ -104,13 +104,13 @@ class StreamPhysicalWindowAggregate( override def translateToExecNode(): ExecNode[_] = { checkEmitConfiguration(unwrapTableConfig(this)) new StreamExecWindowAggregate( + unwrapTableConfig(this), grouping, aggCalls.toArray, windowing, namedWindowProperties.toArray, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala index 1b77b2d8ad264..1a80564f76656 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowDeduplicate.scala @@ -22,6 +22,7 @@ import org.apache.flink.table.planner.plan.logical.WindowingStrategy import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowDeduplicate import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -79,13 +80,13 @@ class StreamPhysicalWindowDeduplicate( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowDeduplicate( + unwrapTableConfig(this), partitionKeys, orderKey, keepLastRow, windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala index cd3af33e5aad5..d24f806a05ac4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowJoin.scala @@ -25,6 +25,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.nodes.physical.common.CommonPhysicalJoin import org.apache.flink.table.planner.plan.utils.PythonUtil.containsPythonCall import org.apache.flink.table.planner.plan.utils.RelExplainUtil.preferExpressionFormat +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel._ @@ -110,6 +111,7 @@ class StreamPhysicalWindowJoin( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowJoin( + unwrapTableConfig(this), joinSpec, leftWindowing, rightWindowing, diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala index 16688e55f5e73..2aed05ad59eaa 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowRank.scala @@ -24,6 +24,7 @@ import org.apache.flink.table.planner.plan.nodes.exec.spec.PartitionSpec import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowRank import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} import org.apache.flink.table.planner.plan.utils._ +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.rank._ import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} @@ -94,6 +95,7 @@ class StreamPhysicalWindowRank( override def translateToExecNode(): ExecNode[_] = { val fieldCollations = orderKey.getFieldCollations new StreamExecWindowRank( + unwrapTableConfig(this), rankType, new PartitionSpec(partitionKey.toArray), SortUtil.getSortSpec(fieldCollations), @@ -102,7 +104,6 @@ class StreamPhysicalWindowRank( windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala index a2e2de5d6e927..4f438413fc1b9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalWindowTableFunction.scala @@ -23,6 +23,7 @@ import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrateg import org.apache.flink.table.planner.plan.nodes.common.CommonPhysicalWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowTableFunction import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty} +import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.calcite.plan.{RelOptCluster, RelTraitSet} import org.apache.calcite.rel.RelNode @@ -60,10 +61,10 @@ class StreamPhysicalWindowTableFunction( override def translateToExecNode(): ExecNode[_] = { new StreamExecWindowTableFunction( + unwrapTableConfig(this), windowing, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(getRowType), - getRelDetailedDescription - ) + getRelDetailedDescription) } } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java index 27997b6e96545..d2ae077e13283 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/api/CompiledPlanITCase.java @@ -19,6 +19,7 @@ package org.apache.flink.table.api; import org.apache.flink.FlinkVersion; +import org.apache.flink.table.api.config.ExecutionConfigOptions; import org.apache.flink.table.api.config.TableConfigOptions; import org.apache.flink.table.api.internal.TableResultInternal; import org.apache.flink.table.planner.utils.JsonPlanTestBase; @@ -36,6 +37,8 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.apache.flink.core.testutils.FlinkAssertions.anyCauseMatches; import static org.apache.flink.table.api.Expressions.$; @@ -74,7 +77,7 @@ public void setup() throws Exception { @Test public void testCompilePlanSql() throws IOException { CompiledPlan compiledPlan = - tableEnv.compilePlanSql("insert into MySink select * from MyTable"); + tableEnv.compilePlanSql("INSERT INTO MySink SELECT * FROM MyTable"); String expected = TableTestUtil.readFromResource("/jsonplan/testGetJsonPlan.out"); assertThat( TableTestUtil.replaceExecNodeId( @@ -91,7 +94,7 @@ public void testCompilePlanSql() throws IOException { public void testExecutePlanSql() throws Exception { File sinkPath = createSourceSinkTables(); - tableEnv.compilePlanSql("insert into sink select * from src").execute().await(); + tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src").execute().await(); assertResult(DATA, sinkPath); } @@ -112,7 +115,7 @@ public void testCompileWriteToFileAndThenExecuteSql() throws Exception { File sinkPath = createSourceSinkTables(); - CompiledPlan plan = tableEnv.compilePlanSql("insert into sink select * from src"); + CompiledPlan plan = tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src"); plan.writeToFile(planPath); tableEnv.executeSql(String.format("EXECUTE PLAN '%s'", planPath.toAbsolutePath())).await(); @@ -315,6 +318,46 @@ public void testExplainPlan() throws IOException { .isEqualTo(expected); } + @Test + public void testPersistedConfigOption() throws Exception { + Path planPath = Paths.get(URI.create(getTempDirPath("plan")).getPath(), "plan.json"); + FileUtils.createParentDirectories(planPath.toFile()); + + List data = + Stream.concat( + DATA.stream(), + Stream.of( + "4,2,This string is long", + "5,3,This is an even longer string")) + .collect(Collectors.toList()); + String[] sinkColumnDefinitions = new String[] {"a bigint", "b int", "c varchar(11)"}; + + createTestCsvSourceTable("src", data, COLUMNS_DEFINITION); + File sinkPath = createTestCsvSinkTable("sink", sinkColumnDefinitions); + + // Set config option to trim the strings, so it's persisted in the json plan + tableEnv.getConfig() + .getConfiguration() + .set( + ExecutionConfigOptions.TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER, + ExecutionConfigOptions.TypeLengthEnforcer.TRIM_PAD); + CompiledPlan plan = tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src"); + + // Set config option to trim the strings to IGNORE, to validate that the persisted config + // is overriding the environment setting. + tableEnv.getConfig() + .getConfiguration() + .set( + ExecutionConfigOptions.TABLE_EXEC_SINK_TYPE_LENGTH_ENFORCER, + ExecutionConfigOptions.TypeLengthEnforcer.IGNORE); + + plan.execute().await(); + List expected = + Stream.concat(DATA.stream(), Stream.of("4,2,This string", "5,3,This is an ")) + .collect(Collectors.toList()); + assertResult(expected, sinkPath); + } + @Test public void testBatchMode() { tableEnv = TableEnvironment.create(EnvironmentSettings.inBatchMode()); @@ -335,7 +378,7 @@ public void testBatchMode() { + " 'table-sink-class' = 'DEFAULT')"; tableEnv.executeSql(sinkTableDdl); - assertThatThrownBy(() -> tableEnv.compilePlanSql("insert into sink select * from src")) + assertThatThrownBy(() -> tableEnv.compilePlanSql("INSERT INTO sink SELECT * FROM src")) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("The compiled plan feature is not supported in batch mode."); } diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java index f4b6e9471e186..e79899d496ce7 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/processor/utils/InputPriorityConflictResolverTest.java @@ -111,6 +111,7 @@ public void testDeadlockCausedByExchange() { BatchExecExchange exchange = new BatchExecExchange( + new Configuration(), InputProperty.builder() .requiredDistribution(InputProperty.ANY_DISTRIBUTION) .build(), 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 index 0c59dbe02acc0..9301f79e72516 100644 --- 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 @@ -20,6 +20,9 @@ import org.apache.flink.FlinkVersion; import org.apache.flink.api.dag.Transformation; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; 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; @@ -42,7 +45,9 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.Stream; +import static org.apache.flink.configuration.ConfigOptions.key; import static org.apache.flink.table.planner.plan.utils.ExecNodeMetadataUtil.UNSUPPORTED_JSON_SERDE_CLASSES; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -104,6 +109,26 @@ public void testMultipleAnnotations() { new Condition<>( m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion")); + Configuration config = new Configuration(); + config.set(OPTION_1, 1); + config.set(OPTION_2, 2); + config.set(OPTION_3, 3); + config.set(OPTION_4, 4); + config.set(OPTION_5, 5); + config.set(OPTION_6, 6); + + ReadableConfig persistedConfig = + ExecNodeMetadataUtil.newPersistedConfig( + DummyNode.class, + config, + Stream.of(OPTION_1, OPTION_2, OPTION_3, OPTION_4, OPTION_5, OPTION_6)); + assertThat(persistedConfig.get(OPTION_1)).isEqualTo(1); + assertThat(persistedConfig.get(OPTION_2)).isEqualTo(OPTION_2.defaultValue()); + assertThat(persistedConfig.get(OPTION_3)).isEqualTo(3); + assertThat(persistedConfig.get(OPTION_4)).isEqualTo(4); + assertThat(persistedConfig.get(OPTION_5)).isEqualTo(5); + assertThat(persistedConfig.get(OPTION_6)).isEqualTo(OPTION_6.defaultValue()); + // Using multiple individual ExecNodeMetadata annotations ExecNodeMetadataUtil.addTestNode(DummyNodeMultipleAnnotations.class); assertThat(ExecNodeMetadataUtil.retrieveExecNode("dummy-node-multiple-annotations", 1)) @@ -120,6 +145,73 @@ public void testMultipleAnnotations() { .has( new Condition<>( m -> m.minPlanVersion() == FlinkVersion.v1_15, "minStateVersion")); + + assertThatThrownBy( + () -> + ExecNodeContext.newPersistedConfig( + DummyNodeMultipleAnnotations.class, config)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest.DummyNodeMultipleAnnotations, " + + "consumedOption: option111 not listed in [TableConfigOptions, " + + "ExecutionConfigOptions]."); + } + + @Test + public void testDuplicateConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_3))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateConsumedOptions, consumedOption: " + + "option2 is listed multiple times in consumedOptions, " + + "potentially also with fallback/deprecated key."); + } + + @Test + public void testDuplicateDeprecatedKeysConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateDeprecatedKeysConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateDeprecatedKeysConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_3))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateDeprecatedKeysConsumedOptions, " + + "consumedOption: option3-deprecated is listed multiple times in " + + "consumedOptions, potentially also with fallback/deprecated " + + "key."); + } + + @Test + public void testDuplicateFallbackKeysConsumedOptions() { + ExecNodeMetadataUtil.addTestNode(DummyNodeDuplicateFallbackKeysConsumedOptions.class); + assertThatThrownBy( + () -> + ExecNodeMetadataUtil.newPersistedConfig( + DummyNodeDuplicateFallbackKeysConsumedOptions.class, + new Configuration(), + Stream.of(OPTION_1, OPTION_2, OPTION_4))) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "ExecNode: org.apache.flink.table.planner.plan.utils." + + "ExecNodeMetadataUtilTest." + + "DummyNodeDuplicateFallbackKeysConsumedOptions, " + + "consumedOption: option4-fallback is listed multiple times in " + + "consumedOptions, potentially also with fallback/deprecated " + + "key."); } @Test @@ -128,15 +220,16 @@ public void testNewContext() { .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."); + + "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."); + + "ExecNodeMetadataUtilTest.DummyNode is not listed in the " + + "supported classes and yet is annotated with: ExecNodeMetadata."); } @Test @@ -178,103 +271,89 @@ public void testStreamExecNodeJsonSerdeCoverage() { @ExecNodeMetadata( name = "dummy-node", version = 1, + consumedOptions = {"option1", "option3-deprecated", "option5-deprecated"}, minPlanVersion = FlinkVersion.v1_13, minStateVersion = FlinkVersion.v1_13), @ExecNodeMetadata( name = "dummy-node", version = 2, + consumedOptions = {"option2", "option3-deprecated", "option5", "option6-fallback"}, minPlanVersion = FlinkVersion.v1_14, minStateVersion = FlinkVersion.v1_14), @ExecNodeMetadata( name = "dummy-node", version = 3, + consumedOptions = {"option1", "option3", "option4-fallback", "option5-deprecated"}, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) }) - private static class DummyNode extends ExecNodeBase { + private static class DummyNode extends AbstractDummyNode { @JsonCreator protected DummyNode( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 1, + consumedOptions = {"option1", "option2"}, minPlanVersion = FlinkVersion.v1_13, minStateVersion = FlinkVersion.v1_13) @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 2, + consumedOptions = {"option11", "option22"}, minPlanVersion = FlinkVersion.v1_14, minStateVersion = FlinkVersion.v1_14) @ExecNodeMetadata( name = "dummy-node-multiple-annotations", version = 3, + consumedOptions = {"option111", "option222"}, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) - private static class DummyNodeMultipleAnnotations extends ExecNodeBase { + private static class DummyNodeMultipleAnnotations extends AbstractDummyNode { @JsonCreator protected DummyNodeMultipleAnnotations( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } - private static class DummyNodeNoJsonCreator extends ExecNodeBase { + private static class DummyNodeNoJsonCreator extends AbstractDummyNode { protected DummyNodeNoJsonCreator( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } - private static class DummyNodeNoAnnotation extends ExecNodeBase + private static class DummyNodeNoAnnotation extends AbstractDummyNode implements StreamExecNode { @JsonCreator protected DummyNodeNoAnnotation( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); - } - - @Override - protected Transformation translateToPlanInternal( - PlannerBase planner, ExecNodeConfig config) { - return null; + super(context, persistedConfig, properties, outputType, description); } } @@ -295,15 +374,85 @@ protected Transformation translateToPlanInternal( version = 3, minPlanVersion = FlinkVersion.v1_15, minStateVersion = FlinkVersion.v1_15) - private static class DummyNodeBothAnnotations extends ExecNodeBase { + private static class DummyNodeBothAnnotations extends AbstractDummyNode { @JsonCreator protected DummyNodeBothAnnotations( ExecNodeContext context, + ReadableConfig persistedConfig, List properties, LogicalType outputType, String description) { - super(10, context, properties, outputType, description); + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option3", "option2"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-deprecated-keys-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option3", "option3-deprecated"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateDeprecatedKeysConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateDeprecatedKeysConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + @ExecNodeMetadata( + name = "dummy-node-duplicate-fallback-keys-consumedOptions", + version = 3, + consumedOptions = {"option1", "option2", "option4", "option4-fallback"}, + minPlanVersion = FlinkVersion.v1_15, + minStateVersion = FlinkVersion.v1_15) + private static class DummyNodeDuplicateFallbackKeysConsumedOptions extends AbstractDummyNode { + + @JsonCreator + protected DummyNodeDuplicateFallbackKeysConsumedOptions( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(context, persistedConfig, properties, outputType, description); + } + } + + private static class AbstractDummyNode extends ExecNodeBase { + + protected AbstractDummyNode( + ExecNodeContext context, + ReadableConfig persistedConfig, + List properties, + LogicalType outputType, + String description) { + super(10, context, persistedConfig, properties, outputType, description); } @Override @@ -312,4 +461,35 @@ protected Transformation translateToPlanInternal( return null; } } + + private static final ConfigOption OPTION_1 = + key("option1").intType().defaultValue(-1).withDescription("option1"); + private static final ConfigOption OPTION_2 = + key("option2").intType().defaultValue(-1).withDescription("option2"); + private static final ConfigOption OPTION_3 = + key("option3") + .intType() + .defaultValue(-1) + .withDeprecatedKeys("option3-deprecated") + .withDescription("option3"); + private static final ConfigOption OPTION_4 = + key("option4") + .intType() + .defaultValue(-1) + .withFallbackKeys("option4-fallback") + .withDescription("option4"); + private static final ConfigOption OPTION_5 = + key("option5") + .intType() + .defaultValue(-1) + .withFallbackKeys("option5-fallback") + .withDeprecatedKeys("option5-deprecated") + .withDescription("option5"); + private static final ConfigOption OPTION_6 = + key("option6") + .intType() + .defaultValue(-1) + .withDeprecatedKeys("option6-deprecated") + .withFallbackKeys("option6-fallback") + .withDescription("option6"); }