From 7c49fe54109e7e43cdd10713add2955dd7c50aa3 Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Wed, 3 Jun 2026 11:42:23 -0700 Subject: [PATCH 1/6] [FLINK-XXXXX][table] Extract changelog mode inference helpers to Java util - The following helper methods are pulled out into a new ChangelogModeInferenceUtils.java - getModifyKindSet - getDeleteKind - isNonUpsertKeyCondition - The Scala program now delegates these three methods to the Java util - This commit has no functional change and the ChangelogModeInferenceTest remains unchanged --- .../program/ChangelogModeInferenceUtils.java | 84 +++++++++++++++++++ .../FlinkChangelogModeInferenceProgram.scala | 36 ++------ 2 files changed, 91 insertions(+), 29 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java new file mode 100644 index 0000000000000..94b36f4d4fe4f --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java @@ -0,0 +1,84 @@ +/* + * 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.optimize.program; + +import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase; +import org.apache.flink.table.planner.plan.trait.DeleteKind; +import org.apache.flink.table.planner.plan.trait.DeleteKindTrait; +import org.apache.flink.table.planner.plan.trait.DeleteKindTraitDef; +import org.apache.flink.table.planner.plan.trait.ModifyKindSet; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTraitDef; +import org.apache.flink.table.planner.plan.utils.RexNodeExtractor; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.rex.RexProgram; +import org.apache.calcite.util.ImmutableBitSet; + +import java.util.Collections; +import java.util.Set; + +/** + * Shared helper methods for changelog mode inference, used by {@link + * FlinkChangelogModeInferenceProgram} and its trait visitors. + */ +final class ChangelogModeInferenceUtils { + + private ChangelogModeInferenceUtils() {} + + /** + * Whether the condition of the given calc only references non-upsert-key columns. If so, the + * calc can forward whatever changelog mode is required, because records are filtered based on + * columns that don't take part in the upsert key. + */ + static boolean isNonUpsertKeyCondition(StreamPhysicalCalcBase calc) { + RexProgram program = calc.getProgram(); + if (program.getCondition() == null) { + return false; + } + + RexNode condition = program.expandLocalRef(program.getCondition()); + FlinkRelMetadataQuery fmq = + FlinkRelMetadataQuery.reuseOrCreate(calc.getCluster().getMetadataQuery()); + Set upsertKeys = fmq.getUpsertKeys(calc.getInput()); + if (upsertKeys == null || upsertKeys.isEmpty()) { + // there are no upsert keys, so all columns are non-primary key columns + return true; + } + + int[] inputRefIndices = + RexNodeExtractor.extractRefInputFields(Collections.singletonList(condition)); + ImmutableBitSet inputRefSet = ImmutableBitSet.of(inputRefIndices); + return upsertKeys.stream().noneMatch(uk -> uk.contains(inputRefSet)); + } + + static ModifyKindSet getModifyKindSet(RelNode node) { + ModifyKindSetTrait modifyKindSetTrait = + node.getTraitSet().getTrait(ModifyKindSetTraitDef.INSTANCE()); + return modifyKindSetTrait.modifyKindSet(); + } + + static DeleteKind getDeleteKind(RelNode node) { + DeleteKindTrait deleteKindTrait = + node.getTraitSet().getTrait(DeleteKindTraitDef.INSTANCE()); + return deleteKindTrait.deleteKind(); + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala index fd646a34148b4..bc2320a3799e1 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala @@ -37,7 +37,7 @@ import org.apache.flink.table.planner.plan.schema.TableSourceTable import org.apache.flink.table.planner.plan.utils._ import org.apache.flink.table.planner.plan.utils.RankProcessStrategy.{AppendFastStrategy, RetractStrategy, UpdateFastStrategy} import org.apache.flink.table.planner.sinks.DataStreamTableSink -import org.apache.flink.table.planner.utils.{JavaScalaConversionUtil, ShortcutUtils} +import org.apache.flink.table.planner.utils.ShortcutUtils import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.FlinkJoinType import org.apache.flink.table.types.inference.{StaticArgument, StaticArgumentTrait} @@ -1578,36 +1578,14 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti } } - private def isNonUpsertKeyCondition(calc: StreamPhysicalCalcBase): Boolean = { - val program = calc.getProgram - if (program.getCondition == null) { - return false - } + private def isNonUpsertKeyCondition(calc: StreamPhysicalCalcBase): Boolean = + ChangelogModeInferenceUtils.isNonUpsertKeyCondition(calc) - val condition = program.expandLocalRef(calc.getProgram.getCondition) - val fmq = FlinkRelMetadataQuery.reuseOrCreate(calc.getCluster.getMetadataQuery) - val upsertKeys = fmq.getUpsertKeys(calc.getInput()) - if (upsertKeys == null || upsertKeys.isEmpty) { - // there are no upsert keys, so all columns are non-primary key columns - true - } else { - val inputRefIndices = - RexNodeExtractor - .extractRefInputFields(JavaScalaConversionUtil.toJava(Seq(condition))) - val inputRefSet = ImmutableBitSet.of(inputRefIndices: _*) - !upsertKeys.stream().anyMatch(uk => uk.contains(inputRefSet)) - } - } + private def getModifyKindSet(node: RelNode): ModifyKindSet = + ChangelogModeInferenceUtils.getModifyKindSet(node) - private def getModifyKindSet(node: RelNode): ModifyKindSet = { - val modifyKindSetTrait = node.getTraitSet.getTrait(ModifyKindSetTraitDef.INSTANCE) - modifyKindSetTrait.modifyKindSet - } - - private def getDeleteKind(node: RelNode): DeleteKind = { - val deleteKindTrait = node.getTraitSet.getTrait(DeleteKindTraitDef.INSTANCE) - deleteKindTrait.deleteKind - } + private def getDeleteKind(node: RelNode): DeleteKind = + ChangelogModeInferenceUtils.getDeleteKind(node) // ---------------------------------------------------------------------------------------------- // PTF helper methods From 0f2b11947245dbc6feadc2739ac8c7ff93319214 Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Wed, 3 Jun 2026 12:19:39 -0700 Subject: [PATCH 2/6] [FLINK-XXXXX][table] Move changelog inference PTF helpers to Java util - Move the following ptf changelog helpers methods to ChangelogModeInferenceUtils.java - toChangelogMode - ptfRequiresUpdateBefore - extractPtfTableArgComponents - toPtfChangelogContext - queryPtfChangelogMode - verifyPtfTableArgsForUpdates - The Scala program keeps thin forwarders until the migration is complete --- .../program/ChangelogModeInferenceUtils.java | 199 ++++++++++++++++++ .../FlinkChangelogModeInferenceProgram.scala | 150 ++----------- 2 files changed, 216 insertions(+), 133 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java index 94b36f4d4fe4f..c3c9dc6caa92a 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java @@ -18,23 +18,51 @@ package org.apache.flink.table.planner.plan.optimize.program; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.functions.BuiltInFunctionDefinition; +import org.apache.flink.table.functions.ChangelogFunction; +import org.apache.flink.table.functions.ChangelogFunction.ChangelogContext; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.flink.table.functions.TableSemantics; +import org.apache.flink.table.planner.calcite.RexTableArgCall; +import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction; import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery; import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalProcessTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel; import org.apache.flink.table.planner.plan.trait.DeleteKind; import org.apache.flink.table.planner.plan.trait.DeleteKindTrait; import org.apache.flink.table.planner.plan.trait.DeleteKindTraitDef; import org.apache.flink.table.planner.plan.trait.ModifyKindSet; import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait; import org.apache.flink.table.planner.plan.trait.ModifyKindSetTraitDef; +import org.apache.flink.table.planner.plan.trait.UpdateKindTrait; +import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils; import org.apache.flink.table.planner.plan.utils.RexNodeExtractor; +import org.apache.flink.table.planner.utils.ShortcutUtils; +import org.apache.flink.table.types.inference.CallContext; +import org.apache.flink.table.types.inference.StaticArgument; +import org.apache.flink.table.types.inference.StaticArgumentTrait; +import org.apache.flink.types.RowKind; +import org.apache.calcite.linq4j.Ord; import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rex.RexCall; import org.apache.calcite.rex.RexNode; import org.apache.calcite.rex.RexProgram; import org.apache.calcite.util.ImmutableBitSet; +import javax.annotation.Nullable; + import java.util.Collections; +import java.util.List; +import java.util.Optional; import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; + +import scala.Option; /** * Shared helper methods for changelog mode inference, used by {@link @@ -81,4 +109,175 @@ static DeleteKind getDeleteKind(RelNode node) { node.getTraitSet().getTrait(DeleteKindTraitDef.INSTANCE()); return deleteKindTrait.deleteKind(); } + + // ---------------------------------------------------------------------------------------------- + // PTF helper methods + // ---------------------------------------------------------------------------------------------- + + static ChangelogMode toChangelogMode( + StreamPhysicalRel node, + @Nullable UpdateKindTrait updateKindTrait, + @Nullable DeleteKindTrait deleteKindTrait) { + ChangelogMode.Builder modeBuilder = ChangelogMode.newBuilder(); + Option givenMode = ChangelogPlanUtils.getChangelogMode(node); + if (givenMode.isEmpty()) { + throw new IllegalStateException( + "Unable to derive changelog mode from node " + node + ". This is a bug."); + } + for (RowKind kind : givenMode.get().getContainedKinds()) { + modeBuilder.addContainedKind(kind); + } + if (updateKindTrait != null && UpdateKindTrait.BEFORE_AND_AFTER().equals(updateKindTrait)) { + modeBuilder.addContainedKind(RowKind.UPDATE_BEFORE); + } + if (deleteKindTrait != null && DeleteKindTrait.DELETE_BY_KEY().equals(deleteKindTrait)) { + modeBuilder.keyOnlyDeletes(true); + } + return modeBuilder.build(); + } + + /** + * Whether the PTF requires UPDATE_BEFORE from its input. Returns true unless partition keys + * cover the upsert keys (co-located) and the argument doesn't explicitly require UPDATE_BEFORE. + */ + static boolean ptfRequiresUpdateBefore( + StaticArgument tableArg, RexTableArgCall tableArgCall, StreamPhysicalRel input) { + ImmutableBitSet partitionKeys = ImmutableBitSet.of(tableArgCall.getPartitionKeys()); + FlinkRelMetadataQuery fmq = + FlinkRelMetadataQuery.reuseOrCreate(input.getCluster().getMetadataQuery()); + Set upsertKeys = fmq.getUpsertKeys(input); + return upsertKeys == null + || partitionKeys.isEmpty() + || !upsertKeys.contains(partitionKeys) + || tableArg.is(StaticArgumentTrait.REQUIRE_UPDATE_BEFORE); + } + + static PtfTableArgComponents extractPtfTableArgComponents( + StreamPhysicalProcessTableFunction process, + StreamPhysicalRel child, + Ord inputArg) { + StaticArgument tableArg = inputArg.e; + RexCall call = process.getCall(); + RexTableArgCall tableArgCall = (RexTableArgCall) call.getOperands().get(inputArg.i); + ModifyKindSet modifyKindSet = getModifyKindSet(child); + return new PtfTableArgComponents(tableArg, tableArgCall, modifyKindSet); + } + + private static ChangelogContext toPtfChangelogContext( + StreamPhysicalProcessTableFunction process, + List inputChangelogModes, + ChangelogMode outputChangelogMode) { + RexCall udfCall = StreamPhysicalProcessTableFunction.toUdfCall(process.getCall()); + List inputTimeColumns = + StreamPhysicalProcessTableFunction.toInputTimeColumns(process.getCall()); + BridgingSqlFunction function = (BridgingSqlFunction) udfCall.getOperator(); + CallContext callContext = + function.toCallContext( + udfCall, inputTimeColumns, inputChangelogModes, outputChangelogMode); + + // Expose a simplified context focused on changelog-relevant inputs: changelog modes, + // resolved literal arguments, and table semantics (e.g., partition-by columns). + return new ChangelogContext() { + @Override + public ChangelogMode getTableChangelogMode(int pos) { + TableSemantics tableSemantics = callContext.getTableSemantics(pos).orElse(null); + if (tableSemantics == null) { + return null; + } + return tableSemantics.changelogMode().orElse(null); + } + + @Override + public ChangelogMode getRequiredChangelogMode() { + return callContext.getOutputChangelogMode().orElse(null); + } + + @Override + public Optional getArgumentValue(int pos, Class clazz) { + return callContext.getArgumentValue(pos, clazz); + } + + @Override + public Optional getTableSemantics(int pos) { + return callContext.getTableSemantics(pos); + } + }; + } + + static T queryPtfChangelogMode( + StreamPhysicalProcessTableFunction process, + List children, + ChangelogMode requiredChangelogMode, + Function toTraitSet, + T defaultTraitSet) { + RexCall call = process.getCall(); + FunctionDefinition definition = ShortcutUtils.unwrapFunctionDefinition(call); + if (definition instanceof ChangelogFunction) { + ChangelogFunction changelogFunction = (ChangelogFunction) definition; + ChangelogContext changelogContext = + toPtfChangelogContext(process, toInputChangelogModes(children), requiredChangelogMode); + ChangelogMode changelogMode = changelogFunction.getChangelogMode(changelogContext); + verifyPtfTableArgsForUpdates(call, changelogMode); + return toTraitSet.apply(changelogMode); + } else if (definition instanceof BuiltInFunctionDefinition + && ((BuiltInFunctionDefinition) definition).getChangelogModeStrategy().isPresent()) { + BuiltInFunctionDefinition builtIn = (BuiltInFunctionDefinition) definition; + ChangelogContext changelogContext = + toPtfChangelogContext(process, toInputChangelogModes(children), requiredChangelogMode); + ChangelogMode changelogMode = + builtIn.getChangelogModeStrategy().get().inferChangelogMode(changelogContext); + verifyPtfTableArgsForUpdates(call, changelogMode); + return toTraitSet.apply(changelogMode); + } else { + return defaultTraitSet; + } + } + + private static List toInputChangelogModes(List children) { + return children.stream() + .map(child -> toChangelogMode(child, null, null)) + .collect(Collectors.toList()); + } + + /** + * Verifies that PTFs with upsert output (without UPDATE_BEFORE) use set semantics. + * + *

Retract mode (with UPDATE_BEFORE) is self-describing — each update carries either the old + * and new value, so downstream can process it without a key. Row semantics is safe. + * + *

Upsert mode (without UPDATE_BEFORE) requires a key to look up previous values, so set + * semantics with PARTITION BY is required. + */ + private static void verifyPtfTableArgsForUpdates(RexCall call, ChangelogMode changelogMode) { + if (changelogMode.containsOnly(RowKind.INSERT) + || changelogMode.contains(RowKind.UPDATE_BEFORE)) { + return; + } + for (Ord inputArg : StreamPhysicalProcessTableFunction.getProvidedInputArgs(call)) { + StaticArgument tableArg = inputArg.e; + if (tableArg.is(StaticArgumentTrait.ROW_SEMANTIC_TABLE)) { + throw new ValidationException( + "PTFs that take table arguments with row semantics don't support upsert " + + "output. Table argument '" + + tableArg.getName() + + "' of function '" + + call.getOperator().toString() + + "' must use set semantics."); + } + } + } + + /** Components of a PTF table argument, derived from a {@link RexTableArgCall} operand. */ + static final class PtfTableArgComponents { + final StaticArgument tableArg; + final RexTableArgCall tableArgCall; + final ModifyKindSet modifyKindSet; + + PtfTableArgComponents( + StaticArgument tableArg, RexTableArgCall tableArgCall, ModifyKindSet modifyKindSet) { + this.tableArg = tableArg; + this.tableArgCall = tableArgCall; + this.modifyKindSet = modifyKindSet; + } + } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala index bc2320a3799e1..95b8f26883cd4 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala @@ -23,10 +23,7 @@ import org.apache.flink.table.api.InsertConflictStrategy.ConflictBehavior import org.apache.flink.table.api.config.ExecutionConfigOptions import org.apache.flink.table.api.config.ExecutionConfigOptions.UpsertMaterialize import org.apache.flink.table.connector.ChangelogMode -import org.apache.flink.table.functions.{BuiltInFunctionDefinition, ChangelogFunction, TableSemantics} -import org.apache.flink.table.functions.ChangelogFunction.ChangelogContext import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexTableArgCall} -import org.apache.flink.table.planner.functions.bridging.BridgingSqlFunction import org.apache.flink.table.planner.plan.`trait`._ import org.apache.flink.table.planner.plan.`trait`.DeleteKindTrait.{deleteOnKeyOrNone, fullDeleteOrNone, DELETE_BY_KEY} import org.apache.flink.table.planner.plan.`trait`.UpdateKindTrait.{beforeAfterOrNone, onlyAfterOrNone, BEFORE_AND_AFTER, ONLY_UPDATE_AFTER} @@ -37,7 +34,6 @@ import org.apache.flink.table.planner.plan.schema.TableSourceTable import org.apache.flink.table.planner.plan.utils._ import org.apache.flink.table.planner.plan.utils.RankProcessStrategy.{AppendFastStrategy, RetractStrategy, UpdateFastStrategy} import org.apache.flink.table.planner.sinks.DataStreamTableSink -import org.apache.flink.table.planner.utils.ShortcutUtils import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.FlinkJoinType import org.apache.flink.table.types.inference.{StaticArgument, StaticArgumentTrait} @@ -46,7 +42,6 @@ import org.apache.flink.types.RowKind import org.apache.calcite.linq4j.Ord import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.JoinRelType -import org.apache.calcite.rex.RexCall import org.apache.calcite.util.ImmutableBitSet import scala.collection.JavaConversions._ @@ -1594,91 +1589,25 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti private def toChangelogMode( node: StreamPhysicalRel, updateKindTrait: Option[UpdateKindTrait], - deleteKindTrait: Option[DeleteKindTrait]): ChangelogMode = { - val modeBuilder = ChangelogMode.newBuilder() - val givenMode = ChangelogPlanUtils - .getChangelogMode(node) - .getOrElse( - throw new IllegalStateException( - s"Unable to derive changelog mode from node $node. This is a bug.")) - givenMode.getContainedKinds.foreach(modeBuilder.addContainedKind) - updateKindTrait match { - case None => - case Some(updateKindTrait: UpdateKindTrait) => - if (updateKindTrait == BEFORE_AND_AFTER) { - modeBuilder.addContainedKind(RowKind.UPDATE_BEFORE) - } - } - deleteKindTrait match { - case None => - case Some(deleteKindTrait: DeleteKindTrait) => - if (deleteKindTrait == DELETE_BY_KEY) { - modeBuilder.keyOnlyDeletes(true) - } - } - modeBuilder.build() - } + deleteKindTrait: Option[DeleteKindTrait]): ChangelogMode = + ChangelogModeInferenceUtils.toChangelogMode( + node, + updateKindTrait.orNull, + deleteKindTrait.orNull) - /** - * Whether the PTF requires UPDATE_BEFORE from its input. Returns true unless partition keys cover - * the upsert keys (co-located) and the argument doesn't explicitly require UPDATE_BEFORE. - */ private def ptfRequiresUpdateBefore( tableArg: StaticArgument, tableArgCall: RexTableArgCall, - input: StreamPhysicalRel): Boolean = { - val partitionKeys = ImmutableBitSet.of(tableArgCall.getPartitionKeys: _*) - val fmq = FlinkRelMetadataQuery.reuseOrCreate(input.getCluster.getMetadataQuery) - val upsertKeys = fmq.getUpsertKeys(input) - upsertKeys == null || partitionKeys.isEmpty || - !upsertKeys.contains(partitionKeys) || - tableArg.is(StaticArgumentTrait.REQUIRE_UPDATE_BEFORE) - } + input: StreamPhysicalRel): Boolean = + ChangelogModeInferenceUtils.ptfRequiresUpdateBefore(tableArg, tableArgCall, input) private def extractPtfTableArgComponents( process: StreamPhysicalProcessTableFunction, child: StreamPhysicalRel, inputArg: Ord[StaticArgument]): (StaticArgument, RexTableArgCall, ModifyKindSet) = { - val tableArg = inputArg.e - val call = process.getCall - val tableArgCall = call.operands.get(inputArg.i).asInstanceOf[RexTableArgCall] - val modifyKindSet = getModifyKindSet(child) - (tableArg, tableArgCall, modifyKindSet) - } - - private def toPtfChangelogContext( - process: StreamPhysicalProcessTableFunction, - inputChangelogModes: List[ChangelogMode], - outputChangelogMode: ChangelogMode): ChangelogContext = { - val udfCall = StreamPhysicalProcessTableFunction.toUdfCall(process.getCall) - val inputTimeColumns = StreamPhysicalProcessTableFunction.toInputTimeColumns(process.getCall) - val function = udfCall.getOperator.asInstanceOf[BridgingSqlFunction] - val callContext = - function.toCallContext(udfCall, inputTimeColumns, inputChangelogModes, outputChangelogMode) - - // Expose a simplified context focused on changelog-relevant inputs: changelog modes, - // resolved literal arguments, and table semantics (e.g., partition-by columns). - new ChangelogContext { - override def getTableChangelogMode(pos: Int): ChangelogMode = { - val tableSemantics = callContext.getTableSemantics(pos).orElse(null) - if (tableSemantics == null) { - return null - } - tableSemantics.changelogMode().orElse(null) - } - - override def getRequiredChangelogMode: ChangelogMode = { - callContext.getOutputChangelogMode.orElse(null) - } - - override def getArgumentValue[T](pos: Int, clazz: Class[T]): java.util.Optional[T] = { - callContext.getArgumentValue(pos, clazz) - } - - override def getTableSemantics(pos: Int): java.util.Optional[TableSemantics] = { - callContext.getTableSemantics(pos) - } - } + val components = + ChangelogModeInferenceUtils.extractPtfTableArgComponents(process, child, inputArg) + (components.tableArg, components.tableArgCall, components.modifyKindSet) } private def queryPtfChangelogMode[T]( @@ -1686,56 +1615,11 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti children: List[StreamPhysicalRel], requiredChangelogMode: ChangelogMode, toTraitSet: ChangelogMode => T, - defaultTraitSet: T): T = { - val call = process.getCall - val definition = ShortcutUtils.unwrapFunctionDefinition(call) - definition match { - case changelogFunction: ChangelogFunction => - val inputChangelogModes = children.map(toChangelogMode(_, None, None)) - val changelogContext = - toPtfChangelogContext(process, inputChangelogModes, requiredChangelogMode) - val changelogMode = changelogFunction.getChangelogMode(changelogContext) - verifyPtfTableArgsForUpdates(call, changelogMode) - toTraitSet(changelogMode) - case builtIn: BuiltInFunctionDefinition if builtIn.getChangelogModeStrategy.isPresent => - val inputChangelogModes = children.map(toChangelogMode(_, None, None)) - val changelogContext = - toPtfChangelogContext(process, inputChangelogModes, requiredChangelogMode) - val changelogMode = - builtIn.getChangelogModeStrategy.get().inferChangelogMode(changelogContext) - verifyPtfTableArgsForUpdates(call, changelogMode) - toTraitSet(changelogMode) - case _ => - defaultTraitSet - } - } - - /** - * Verifies that PTFs with upsert output (without UPDATE_BEFORE) use set semantics. - * - * Retract mode (with UPDATE_BEFORE) is self-describing — each update carries either the old and - * new value, so downstream can process it without a key. Row semantics is safe. - * - * Upsert mode (without UPDATE_BEFORE) requires a key to look up previous values, so set semantics - * with PARTITION BY is required. - */ - private def verifyPtfTableArgsForUpdates(call: RexCall, changelogMode: ChangelogMode): Unit = { - if ( - changelogMode.containsOnly(RowKind.INSERT) || changelogMode.contains(RowKind.UPDATE_BEFORE) - ) { - return - } - StreamPhysicalProcessTableFunction - .getProvidedInputArgs(call) - .map(_.e) - .foreach { - tableArg => - if (tableArg.is(StaticArgumentTrait.ROW_SEMANTIC_TABLE)) { - throw new ValidationException( - s"PTFs that take table arguments with row semantics don't support upsert output. " + - s"Table argument '${tableArg.getName}' of function '${call.getOperator.toString}' " + - s"must use set semantics.") - } - } - } + defaultTraitSet: T): T = + ChangelogModeInferenceUtils.queryPtfChangelogMode[T]( + process, + children, + requiredChangelogMode, + (mode: ChangelogMode) => toTraitSet(mode), + defaultTraitSet) } From 917790fa190defaa7ff7256bf4e19f8688816662 Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Wed, 3 Jun 2026 14:08:29 -0700 Subject: [PATCH 3/6] [FLINK-XXXXX][table] Move SatisfyDeleteKindTraitVisitor into a top level Java class --- .../SatisfyDeleteKindTraitVisitor.java | 568 ++++++++++++++++++ .../FlinkChangelogModeInferenceProgram.scala | 396 +----------- 2 files changed, 570 insertions(+), 394 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java new file mode 100644 index 0000000000000..f567e87b2b322 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java @@ -0,0 +1,568 @@ +/* + * 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.optimize.program; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupTableAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupWindowAggregateBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalIntermediateTableScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalIntervalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMLPredictTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalProcessTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonGroupAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonGroupTableAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonOverAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalVectorSearchTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction; +import org.apache.flink.table.planner.plan.optimize.ChangelogNormalizeRequirementResolver; +import org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.PtfTableArgComponents; +import org.apache.flink.table.planner.plan.trait.DeleteKind; +import org.apache.flink.table.planner.plan.trait.DeleteKindTrait; +import org.apache.flink.table.planner.plan.trait.DeleteKindTraitDef; +import org.apache.flink.table.planner.plan.trait.ModifyKind; +import org.apache.flink.table.planner.plan.trait.ModifyKindSet; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTraitDef; +import org.apache.flink.table.types.inference.StaticArgument; +import org.apache.flink.table.types.inference.StaticArgumentTrait; + +import org.apache.calcite.linq4j.Ord; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rex.RexCall; +import org.apache.calcite.util.ImmutableBitSet; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.Set; + +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.extractPtfTableArgComponents; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.getDeleteKind; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.getModifyKindSet; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.isNonUpsertKeyCondition; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.ptfRequiresUpdateBefore; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.queryPtfChangelogMode; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.toChangelogMode; + +/** + * A visitor which will try to satisfy the required {@link DeleteKindTrait} from root. + * + *

After traversed by this visitor, every node should have a correct {@link DeleteKindTrait} or + * returns {@link Optional#empty()} if the planner doesn't support to satisfy the required {@link + * DeleteKindTrait}. + */ +class SatisfyDeleteKindTraitVisitor { + + private final StreamOptimizeContext context; + + SatisfyDeleteKindTraitVisitor(StreamOptimizeContext context) { + this.context = context; + } + + /** + * Try to satisfy the required {@link DeleteKindTrait} from root. + * + *

Each node will first require a DeleteKindTrait to its children. The required + * DeleteKindTrait may come from the node's parent, or come from the node itself, depending on + * whether the node will destroy the trait provided by children or pass the trait from children. + * + *

If the node will pass the children's DeleteKindTrait without destroying it, then return a + * new node with new inputs and forwarded DeleteKindTrait. + * + *

If the node will destroy the children's UpdateKindTrait, then the node itself needs to be + * converted, or a new node should be generated to satisfy the required trait, such as marking + * itself not to generate UPDATE_BEFORE, or generating a new node to filter UPDATE_BEFORE. + * + * @param rel the node who should satisfy the requiredTrait + * @param requiredTrait the required DeleteKindTrait + * @return A converted node which satisfies required traits by input nodes of current node. Or + * {@link Optional#empty()} if required traits cannot be satisfied. + */ + Optional visit(StreamPhysicalRel rel, DeleteKindTrait requiredTrait) { + if (rel instanceof StreamPhysicalSink) { + StreamPhysicalSink sink = (StreamPhysicalSink) rel; + List sinkRequiredTraits = inferSinkRequiredTraits(sink); + return visitSink(sink, sinkRequiredTraits); + } else if (rel instanceof StreamPhysicalLegacySink) { + ModifyKindSet childModifyKindSet = getModifyKindSet(rel.getInput(0)); + DeleteKindTrait fullDelete = DeleteKindTrait.fullDeleteOrNone(childModifyKindSet); + return visitSink(rel, Collections.singletonList(fullDelete)); + } else if (requiresFullDeleteIfUpdates(rel)) { + // if not explicitly supported, all operators require full deletes if there are updates + List children = new ArrayList<>(); + for (RelNode child : rel.getInputs()) { + visit((StreamPhysicalRel) child, DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(child))) + .ifPresent(children::add); + } + return createNewNode( + rel, Optional.of(children), DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); + } else if (rel instanceof StreamPhysicalProcessTableFunction) { + // Required delete traits depend on the table argument declaration, + // input traits, partition keys, and upsert keys + StreamPhysicalProcessTableFunction process = (StreamPhysicalProcessTableFunction) rel; + RexCall call = process.getCall(); + List> inputArgs = + StreamPhysicalProcessTableFunction.getProvidedInputArgs(call); + List children = new ArrayList<>(); + List inputs = process.getInputs(); + for (int inputIndex = 0; inputIndex < inputs.size(); inputIndex++) { + final StreamPhysicalRel child = (StreamPhysicalRel) inputs.get(inputIndex); + final Optional visited; + // For PTF without table arguments (i.e. values child) + if (inputArgs.isEmpty()) { + visited = visit(child, DeleteKindTrait.NONE()); + } else { + // Derive the required delete trait for table arguments + Ord inputArg = inputArgs.get(inputIndex); + PtfTableArgComponents components = + extractPtfTableArgComponents(process, child, inputArg); + StaticArgument tableArg = components.tableArg; + final ModifyKindSet modifyKindSet = components.modifyKindSet; + if (tableArg.is(StaticArgumentTrait.SUPPORT_UPDATES) + && !ptfRequiresUpdateBefore(tableArg, components.tableArgCall, child) + && !tableArg.is(StaticArgumentTrait.REQUIRE_FULL_DELETE)) { + visited = + visit(child, DeleteKindTrait.deleteOnKeyOrNone(modifyKindSet)) + .or( + () -> + visit( + child, + DeleteKindTrait.fullDeleteOrNone( + modifyKindSet))); + } else { + visited = visit(child, DeleteKindTrait.fullDeleteOrNone(modifyKindSet)); + } + } + visited.ifPresent(children::add); + } + final ModifyKindSet modifyTrait = getModifyKindSet(rel); + // Query the PTF for full vs. partial deletes + DeleteKindTrait providedDeleteTrait = + queryPtfChangelogMode( + process, + children, + toChangelogMode(process, null, requiredTrait), + mode -> + mode.keyOnlyDeletes() + ? DeleteKindTrait.deleteOnKeyOrNone(modifyTrait) + : DeleteKindTrait.fullDeleteOrNone(modifyTrait), + DeleteKindTrait.fullDeleteOrNone(modifyTrait)); + return createNewNode(process, Optional.of(children), providedDeleteTrait); + } else if (rel instanceof StreamPhysicalJoin) { + StreamPhysicalJoin join = (StreamPhysicalJoin) rel; + List> children = new ArrayList<>(); + for (int childOrdinal = 0; childOrdinal < join.getInputs().size(); childOrdinal++) { + final StreamPhysicalRel physicalChild = + (StreamPhysicalRel) join.getInput(childOrdinal); + boolean supportsDeleteByKey = join.inputUniqueKeyContainsJoinKey(childOrdinal); + final ModifyKindSet inputModifyKindSet = getModifyKindSet(physicalChild); + if (supportsDeleteByKey && DeleteKindTrait.DELETE_BY_KEY().equals(requiredTrait)) { + children.add( + visit(physicalChild, DeleteKindTrait.deleteOnKeyOrNone(inputModifyKindSet)) + .or( + () -> + visit( + physicalChild, + DeleteKindTrait.fullDeleteOrNone( + inputModifyKindSet)))); + } else { + children.add( + visit(physicalChild, DeleteKindTrait.fullDeleteOrNone(inputModifyKindSet))); + } + } + if (children.stream().anyMatch(c -> !c.isPresent())) { + return Optional.empty(); + } + List childRels = present(children); + if (childRels.stream().anyMatch(r -> getDeleteKind(r) == DeleteKind.DELETE_BY_KEY)) { + return createNewNode( + join, Optional.of(childRels), DeleteKindTrait.deleteOnKeyOrNone(getModifyKindSet(rel))); + } else { + return createNewNode( + join, Optional.of(childRels), DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); + } + } else if (rel instanceof StreamPhysicalCalcBase) { + // if the condition is applied on the upsert key, we can emit whatever the requiredTrait + // is, because we will filter all records based on the condition that applies to that key + StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel; + if (DeleteKindTrait.DELETE_BY_KEY().equals(requiredTrait) + && isNonUpsertKeyCondition(calc)) { + return Optional.empty(); + } + // otherwise, forward DeleteKind requirement + Optional> children = visitChildren(rel, requiredTrait); + if (!children.isPresent()) { + return Optional.empty(); + } + DeleteKindTrait childTrait = + children.get().get(0).getTraitSet().getTrait(DeleteKindTraitDef.INSTANCE()); + return createNewNode(rel, children, childTrait); + } else if (rel instanceof StreamPhysicalExchange + || rel instanceof StreamPhysicalExpand + || rel instanceof StreamPhysicalMiniBatchAssigner + || rel instanceof StreamPhysicalDropUpdateBefore) { + // transparent forward requiredTrait to children + Optional> children = visitChildren(rel, requiredTrait); + if (!children.isPresent()) { + return Optional.empty(); + } + DeleteKindTrait childTrait = + children.get().get(0).getTraitSet().getTrait(DeleteKindTraitDef.INSTANCE()); + return createNewNode(rel, children, childTrait); + } else if (rel instanceof StreamPhysicalUnion) { + StreamPhysicalUnion union = (StreamPhysicalUnion) rel; + List> children = new ArrayList<>(); + for (RelNode childNode : union.getInputs()) { + StreamPhysicalRel child = (StreamPhysicalRel) childNode; + ModifyKindSet childModifyKindSet = getModifyKindSet(child); + DeleteKindTrait requiredChildTrait = + !childModifyKindSet.contains(ModifyKind.DELETE) + ? DeleteKindTrait.NONE() + : requiredTrait; + children.add(visit(child, requiredChildTrait)); + } + if (children.stream().anyMatch(c -> !c.isPresent())) { + return Optional.empty(); + } + List childRels = present(children); + List deleteKinds = new ArrayList<>(); + for (StreamPhysicalRel child : childRels) { + deleteKinds.add(child.getTraitSet().getTrait(DeleteKindTraitDef.INSTANCE())); + } + // union can just forward changes, can't actively satisfy to another changelog mode + final DeleteKindTrait providedTrait; + if (deleteKinds.stream().allMatch(k -> DeleteKindTrait.NONE().equals(k))) { + // if all the children is NONE, union is NONE + providedTrait = DeleteKindTrait.NONE(); + } else { + // otherwise, merge delete kinds. + DeleteKind merged = null; + for (DeleteKindTrait deleteKindTrait : deleteKinds) { + DeleteKind deleteKind = deleteKindTrait.deleteKind(); + merged = merged == null ? deleteKind : mergeDeleteKind(merged, deleteKind); + } + providedTrait = new DeleteKindTrait(merged); + } + return createNewNode(union, Optional.of(childRels), providedTrait); + } else if (rel instanceof StreamPhysicalChangelogNormalize) { + StreamPhysicalChangelogNormalize normalize = (StreamPhysicalChangelogNormalize) rel; + // if + // 1. we don't need to produce UPDATE_BEFORE, + // 2. children can satisfy the required delete trait, + // 3. the normalize doesn't have filter condition which we'd lose, + // 4. we don't use metadata columns + // we can skip ChangelogNormalize + if (!ChangelogNormalizeRequirementResolver.isRequired(normalize)) { + Optional> children = visitChildren(normalize, requiredTrait); + if (children.isPresent()) { + StreamPhysicalRel first = children.get().get(0); + RelNode input = + first instanceof StreamPhysicalExchange + ? ((StreamPhysicalExchange) first).getInput() + : normalize.getInput(); + return Optional.of((StreamPhysicalRel) input); + } + } + ModifyKindSet childModifyKindTrait = getModifyKindSet(rel.getInput(0)); + + // prefer delete by key, but accept both + Optional> children = + visitChildren(normalize, DeleteKindTrait.deleteOnKeyOrNone(childModifyKindTrait)) + .or( + () -> + visitChildren( + normalize, + DeleteKindTrait.fullDeleteOrNone( + childModifyKindTrait))); + + // changelog normalize produces full deletes + return createNewNode( + rel, children, DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); + } else if (rel instanceof StreamPhysicalTableSourceScan) { + // currently only support BEFORE_AND_AFTER if source produces updates + StreamPhysicalTableSourceScan ts = (StreamPhysicalTableSourceScan) rel; + DeleteKindTrait providedTrait = + DeleteKindTrait.fromChangelogMode(ts.tableSource().getChangelogMode()); + return createNewNode(rel, Optional.of(Collections.emptyList()), providedTrait); + } else if (rel instanceof StreamPhysicalDataStreamScan + || rel instanceof StreamPhysicalLegacyTableSourceScan + || rel instanceof StreamPhysicalValues) { + return createNewNode(rel, Optional.of(Collections.emptyList()), DeleteKindTrait.NONE()); + } else if (rel instanceof StreamPhysicalIntermediateTableScan) { + return createNewNode( + rel, + Optional.of(Collections.emptyList()), + DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); + } else if (rel instanceof StreamPhysicalMultiJoin) { + StreamPhysicalMultiJoin multiJoin = (StreamPhysicalMultiJoin) rel; + List> children = new ArrayList<>(); + for (int childOrdinal = 0; childOrdinal < multiJoin.getInputs().size(); childOrdinal++) { + final StreamPhysicalRel physicalChild = + (StreamPhysicalRel) multiJoin.getInput(childOrdinal); + boolean supportsDeleteByKey = + multiJoin.inputUniqueKeyContainsCommonJoinKey(childOrdinal); + final ModifyKindSet inputModifyKindSet = getModifyKindSet(physicalChild); + if (supportsDeleteByKey && DeleteKindTrait.DELETE_BY_KEY().equals(requiredTrait)) { + children.add( + visit(physicalChild, DeleteKindTrait.deleteOnKeyOrNone(inputModifyKindSet)) + .or( + () -> + visit( + physicalChild, + DeleteKindTrait.fullDeleteOrNone( + inputModifyKindSet)))); + } else { + children.add( + visit(physicalChild, DeleteKindTrait.fullDeleteOrNone(inputModifyKindSet))); + } + } + if (children.stream().anyMatch(c -> !c.isPresent())) { + return Optional.empty(); + } + List childRels = present(children); + if (childRels.stream().anyMatch(r -> getDeleteKind(r) == DeleteKind.DELETE_BY_KEY)) { + return createNewNode( + multiJoin, + Optional.of(childRels), + DeleteKindTrait.deleteOnKeyOrNone(getModifyKindSet(rel))); + } else { + return createNewNode( + multiJoin, + Optional.of(childRels), + DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); + } + } else { + throw new UnsupportedOperationException( + "Unsupported visit for " + rel.getClass().getSimpleName()); + } + } + + /** + * Operators that, if not explicitly supported, require full deletes when there are updates. The + * grouping mirrors the original Scala pattern match and must be checked before the more + * specific node types below. + */ + private static boolean requiresFullDeleteIfUpdates(StreamPhysicalRel rel) { + return rel instanceof StreamPhysicalGroupAggregate + || rel instanceof StreamPhysicalGroupTableAggregate + || rel instanceof StreamPhysicalLimit + || rel instanceof StreamPhysicalPythonGroupAggregate + || rel instanceof StreamPhysicalPythonGroupTableAggregate + || rel instanceof StreamPhysicalGroupWindowAggregateBase + || rel instanceof StreamPhysicalWindowAggregate + || rel instanceof StreamPhysicalSort + || rel instanceof StreamPhysicalRank + || rel instanceof StreamPhysicalSortLimit + || rel instanceof StreamPhysicalTemporalJoin + || rel instanceof StreamPhysicalCorrelateBase + || rel instanceof StreamPhysicalLookupJoin + || rel instanceof StreamPhysicalWatermarkAssigner + || rel instanceof StreamPhysicalWindowTableFunction + || rel instanceof StreamPhysicalWindowRank + || rel instanceof StreamPhysicalWindowDeduplicate + || rel instanceof StreamPhysicalTemporalSort + || rel instanceof StreamPhysicalMatch + || rel instanceof StreamPhysicalOverAggregate + || rel instanceof StreamPhysicalIntervalJoin + || rel instanceof StreamPhysicalPythonOverAggregate + || rel instanceof StreamPhysicalWindowJoin + || rel instanceof StreamPhysicalMLPredictTableFunction + || rel instanceof StreamPhysicalVectorSearchTableFunction; + } + + private static DeleteKind mergeDeleteKind(DeleteKind left, DeleteKind right) { + if (left == DeleteKind.NONE) { + return right; + } + if (right == DeleteKind.NONE) { + return left; + } + if (left == right) { + return left; + } + // if any of the union input produces DELETE_BY_KEY, the union produces delete by key + return DeleteKind.DELETE_BY_KEY; + } + + private Optional> visitChildren( + StreamPhysicalRel parent, DeleteKindTrait requiredChildrenTrait) { + List newChildren = new ArrayList<>(); + for (RelNode childNode : parent.getInputs()) { + Optional newChild = + visit((StreamPhysicalRel) childNode, requiredChildrenTrait); + if (!newChild.isPresent()) { + // return empty if one of the children can't satisfy + return Optional.empty(); + } + DeleteKindTrait providedTrait = + newChild.get().getTraitSet().getTrait(DeleteKindTraitDef.INSTANCE()); + if (!providedTrait.satisfies(requiredChildrenTrait)) { + // the provided trait can't satisfy required trait, thus we should return empty. + return Optional.empty(); + } + newChildren.add(newChild.get()); + } + return Optional.of(newChildren); + } + + private Optional createNewNode( + StreamPhysicalRel node, + Optional> childrenOption, + DeleteKindTrait providedDeleteTrait) { + if (!childrenOption.isPresent()) { + return Optional.empty(); + } + List children = childrenOption.get(); + ModifyKindSetTrait modifyKindSetTrait = + node.getTraitSet().getTrait(ModifyKindSetTraitDef.INSTANCE()); + String nodeDescription = node.getRelDetailedDescription(); + final boolean isDeleteKindValid; + switch (providedDeleteTrait.deleteKind()) { + case NONE: + isDeleteKindValid = !modifyKindSetTrait.modifyKindSet().contains(ModifyKind.DELETE); + break; + case DELETE_BY_KEY: + case FULL_DELETE: + isDeleteKindValid = modifyKindSetTrait.modifyKindSet().contains(ModifyKind.DELETE); + break; + default: + isDeleteKindValid = false; + } + if (!isDeleteKindValid) { + throw new TableException( + "DeleteKindTrait " + + providedDeleteTrait + + " conflicts with ModifyKindSetTrait " + + modifyKindSetTrait + + ". This is a bug in planner, please file an issue. \n" + + "Current node is " + + nodeDescription + + "."); + } + RelTraitSet newTraitSet = node.getTraitSet().plus(providedDeleteTrait); + return Optional.of((StreamPhysicalRel) node.copy(newTraitSet, new ArrayList(children))); + } + + private Optional visitSink( + StreamPhysicalRel sink, List sinkRequiredTraits) { + List> satisfiedChildren = new ArrayList<>(); + for (DeleteKindTrait requiredTrait : sinkRequiredTraits) { + visitChildren(sink, requiredTrait).ifPresent(satisfiedChildren::add); + } + if (satisfiedChildren.isEmpty()) { + return Optional.empty(); + } + RelTraitSet sinkTrait = sink.getTraitSet().plus(DeleteKindTrait.NONE()); + return Optional.of( + (StreamPhysicalRel) + sink.copy(sinkTrait, new ArrayList(satisfiedChildren.get(0)))); + } + + /** + * Infer sink required traits by the sink node and its input. Sink required traits is based on + * the sink node's changelog mode, the only exception is when sink's pk(s) not exactly the same + * as the changeLogUpsertKeys and sink's changelog mode is DELETE_BY_KEY. + */ + private List inferSinkRequiredTraits(StreamPhysicalSink sink) { + ModifyKindSet childModifyKindSet = getModifyKindSet(sink.getInput()); + ChangelogMode sinkChangelogMode = + sink.tableSink().getChangelogMode(childModifyKindSet.toDefaultChangelogMode()); + + DeleteKindTrait sinkDeleteTrait = DeleteKindTrait.fromChangelogMode(sinkChangelogMode); + + DeleteKindTrait fullDelete = DeleteKindTrait.fullDeleteOrNone(childModifyKindSet); + if (sinkDeleteTrait.equals(DeleteKindTrait.DELETE_BY_KEY())) { + if (areUpsertKeysDifferentFromPk(sink)) { + return Collections.singletonList(fullDelete); + } else { + return Arrays.asList(sinkDeleteTrait, fullDelete); + } + } else { + return Collections.singletonList(fullDelete); + } + } + + private boolean areUpsertKeysDifferentFromPk(StreamPhysicalSink sink) { + // if sink's pk(s) are not exactly match input changeLogUpsertKeys then it will fallback + // to beforeAndAfter mode for the correctness + boolean upsertKeyDifferentFromPk = false; + int[] sinkDefinedPks = sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes(); + + if (sinkDefinedPks.length > 0) { + ImmutableBitSet sinkPks = ImmutableBitSet.of(sinkDefinedPks); + FlinkRelMetadataQuery fmq = + FlinkRelMetadataQuery.reuseOrCreate(sink.getCluster().getMetadataQuery()); + Set changeLogUpsertKeys = fmq.getUpsertKeys(sink.getInput()); + // if input is UA only, primary key != upsert key (upsert key can be null) we should + // fallback to beforeAndAfter. + // Notice: even sink pk(s) contains input upsert key we cannot optimize to UA only, + // this differs from batch job's unique key inference + if (changeLogUpsertKeys == null + || changeLogUpsertKeys.stream().noneMatch(k -> k.equals(sinkPks))) { + upsertKeyDifferentFromPk = true; + } + } + return upsertKeyDifferentFromPk; + } + + private static List present(List> options) { + List result = new ArrayList<>(options.size()); + for (Optional option : options) { + result.add(option.get()); + } + return result; + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala index 95b8f26883cd4..d1fb4d17956fc 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala @@ -25,11 +25,9 @@ import org.apache.flink.table.api.config.ExecutionConfigOptions.UpsertMaterializ import org.apache.flink.table.connector.ChangelogMode import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexTableArgCall} import org.apache.flink.table.planner.plan.`trait`._ -import org.apache.flink.table.planner.plan.`trait`.DeleteKindTrait.{deleteOnKeyOrNone, fullDeleteOrNone, DELETE_BY_KEY} import org.apache.flink.table.planner.plan.`trait`.UpdateKindTrait.{beforeAfterOrNone, onlyAfterOrNone, BEFORE_AND_AFTER, ONLY_UPDATE_AFTER} import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery import org.apache.flink.table.planner.plan.nodes.physical.stream._ -import org.apache.flink.table.planner.plan.optimize.ChangelogNormalizeRequirementResolver import org.apache.flink.table.planner.plan.schema.TableSourceTable import org.apache.flink.table.planner.plan.utils._ import org.apache.flink.table.planner.plan.utils.RankProcessStrategy.{AppendFastStrategy, RetractStrategy, UpdateFastStrategy} @@ -103,7 +101,8 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti } else { requiredDeleteKindTraits.flatMap { requiredDeleteKindTrait => - deleteKindTraitVisitor.visit(updateRoot.head, requiredDeleteKindTrait) + val deleteRoot = deleteKindTraitVisitor.visit(updateRoot.head, requiredDeleteKindTrait) + if (deleteRoot.isPresent) Some(deleteRoot.get) else None } } @@ -1185,403 +1184,12 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti } } - /** - * A visitor which will try to satisfy the required [[DeleteKindTrait]] from root. - * - *

After traversed by this visitor, every node should have a correct [[DeleteKindTrait]] or - * returns None if the planner doesn't support to satisfy the required [[DeleteKindTrait]]. - */ - private class SatisfyDeleteKindTraitVisitor(private val context: StreamOptimizeContext) { - - /** - * Try to satisfy the required [[DeleteKindTrait]] from root. - * - *

Each node will first require a DeleteKindTrait to its children. The required - * DeleteKindTrait may come from the node's parent, or come from the node itself, depending on - * whether the node will destroy the trait provided by children or pass the trait from children. - * - *

If the node will pass the children's DeleteKindTrait without destroying it, then return a - * new node with new inputs and forwarded DeleteKindTrait. - * - *

If the node will destroy the children's UpdateKindTrait, then the node itself needs to be - * converted, or a new node should be generated to satisfy the required trait, such as marking - * itself not to generate UPDATE_BEFORE, or generating a new node to filter UPDATE_BEFORE. - * - * @param rel - * the node who should satisfy the requiredTrait - * @param requiredTrait - * the required DeleteKindTrait - * @return - * A converted node which satisfies required traits by input nodes of current node. Or None if - * required traits cannot be satisfied. - */ - def visit(rel: StreamPhysicalRel, requiredTrait: DeleteKindTrait): Option[StreamPhysicalRel] = - rel match { - case sink: StreamPhysicalSink => - val sinkRequiredTraits = inferSinkRequiredTraits(sink) - visitSink(sink, sinkRequiredTraits) - - case sink: StreamPhysicalLegacySink[_] => - val childModifyKindSet = getModifyKindSet(sink.getInput) - val fullDelete = fullDeleteOrNone(childModifyKindSet) - visitSink(sink, Seq(fullDelete)) - - case _: StreamPhysicalGroupAggregate | _: StreamPhysicalGroupTableAggregate | - _: StreamPhysicalLimit | _: StreamPhysicalPythonGroupAggregate | - _: StreamPhysicalPythonGroupTableAggregate | _: StreamPhysicalGroupWindowAggregateBase | - _: StreamPhysicalWindowAggregate | _: StreamPhysicalSort | _: StreamPhysicalRank | - _: StreamPhysicalSortLimit | _: StreamPhysicalTemporalJoin | - _: StreamPhysicalCorrelateBase | _: StreamPhysicalLookupJoin | - _: StreamPhysicalWatermarkAssigner | _: StreamPhysicalWindowTableFunction | - _: StreamPhysicalWindowRank | _: StreamPhysicalWindowDeduplicate | - _: StreamPhysicalTemporalSort | _: StreamPhysicalMatch | - _: StreamPhysicalOverAggregate | _: StreamPhysicalIntervalJoin | - _: StreamPhysicalPythonOverAggregate | _: StreamPhysicalWindowJoin | - _: StreamPhysicalMLPredictTableFunction | _: StreamPhysicalVectorSearchTableFunction => - // if not explicitly supported, all operators require full deletes if there are updates - val children = rel.getInputs.map { - case child: StreamPhysicalRel => - val childModifyKindSet = getModifyKindSet(child) - this.visit(child, fullDeleteOrNone(childModifyKindSet)) - }.toList - createNewNode(rel, Some(children.flatten), fullDeleteOrNone(getModifyKindSet(rel))) - - case process: StreamPhysicalProcessTableFunction => - // Required delete traits depend on the table argument declaration, - // input traits, partition keys, and upsert keys - val call = process.getCall - val inputArgs = StreamPhysicalProcessTableFunction - .getProvidedInputArgs(call) - val children = process.getInputs - .map(_.asInstanceOf[StreamPhysicalRel]) - .zipWithIndex - .map { - case (child, inputIndex) => - // For PTF without table arguments (i.e. values child) - if (inputArgs.isEmpty) { - this.visit(child, DeleteKindTrait.NONE) - } - // Derive the required delete trait for table arguments - else { - val inputArg = inputArgs.get(inputIndex) - val (tableArg, tableArgCall, modifyKindSet) = - extractPtfTableArgComponents(process, child, inputArg) - if ( - tableArg.is(StaticArgumentTrait.SUPPORT_UPDATES) - && !ptfRequiresUpdateBefore(tableArg, tableArgCall, child) - && !tableArg.is(StaticArgumentTrait.REQUIRE_FULL_DELETE) - ) { - this - .visit(child, deleteOnKeyOrNone(modifyKindSet)) - .orElse(this.visit(child, fullDeleteOrNone(modifyKindSet))) - } else { - this.visit(child, fullDeleteOrNone(modifyKindSet)) - } - } - } - .toList - .flatten - val modifyTrait = getModifyKindSet(rel) - // Query the PTF for full vs. partial deletes - val providedDeleteTrait = queryPtfChangelogMode( - process, - children, - toChangelogMode(process, None, Some(requiredTrait)), - mode => - if (mode.keyOnlyDeletes()) { - deleteOnKeyOrNone(modifyTrait) - } else { - fullDeleteOrNone(modifyTrait) - }, - fullDeleteOrNone(modifyTrait) - ) - createNewNode(process, Some(children), providedDeleteTrait) - - case join: StreamPhysicalJoin => - val children = join.getInputs.zipWithIndex.map { - case (child, childOrdinal) => - val physicalChild = child.asInstanceOf[StreamPhysicalRel] - val supportsDeleteByKey = join.inputUniqueKeyContainsJoinKey(childOrdinal) - val inputModifyKindSet = getModifyKindSet(physicalChild) - if (supportsDeleteByKey && requiredTrait == DELETE_BY_KEY) { - this - .visit(physicalChild, deleteOnKeyOrNone(inputModifyKindSet)) - .orElse(this.visit(physicalChild, fullDeleteOrNone(inputModifyKindSet))) - } else { - this.visit(physicalChild, fullDeleteOrNone(inputModifyKindSet)) - } - } - if (children.exists(_.isEmpty)) { - None - } else { - val childRels = children.flatten.toList - if (childRels.exists(r => getDeleteKind(r) == DeleteKind.DELETE_BY_KEY)) { - createNewNode(join, Some(childRels), deleteOnKeyOrNone(getModifyKindSet(rel))) - } else { - createNewNode(join, Some(childRels), fullDeleteOrNone(getModifyKindSet(rel))) - } - } - - // if the condition is applied on the upsert key, we can emit whatever the requiredTrait - // is, because we will filter all records based on the condition that applies to that key - case calc: StreamPhysicalCalcBase => - if ( - requiredTrait == DeleteKindTrait.DELETE_BY_KEY && - isNonUpsertKeyCondition(calc) - ) { - None - } else { - // otherwise, forward DeleteKind requirement - visitChildren(rel, requiredTrait) match { - case None => None - case Some(children) => - val childTrait = children.head.getTraitSet.getTrait(DeleteKindTraitDef.INSTANCE) - createNewNode(rel, Some(children), childTrait) - } - } - - case _: StreamPhysicalExchange | _: StreamPhysicalExpand | - _: StreamPhysicalMiniBatchAssigner | _: StreamPhysicalDropUpdateBefore => - // transparent forward requiredTrait to children - visitChildren(rel, requiredTrait) match { - case None => None - case Some(children) => - val childTrait = children.head.getTraitSet.getTrait(DeleteKindTraitDef.INSTANCE) - createNewNode(rel, Some(children), childTrait) - } - - case union: StreamPhysicalUnion => - val children = union.getInputs.map { - case child: StreamPhysicalRel => - val childModifyKindSet = getModifyKindSet(child) - val requiredChildTrait = if (!childModifyKindSet.contains(ModifyKind.DELETE)) { - DeleteKindTrait.NONE - } else { - requiredTrait - } - this.visit(child, requiredChildTrait) - }.toList - - if (children.exists(_.isEmpty)) { - None - } else { - val deleteKinds = children.flatten - .map(_.getTraitSet.getTrait(DeleteKindTraitDef.INSTANCE)) - // union can just forward changes, can't actively satisfy to another changelog mode - val providedTrait = if (deleteKinds.forall(k => DeleteKindTrait.NONE == k)) { - // if all the children is NONE, union is NONE - DeleteKindTrait.NONE - } else { - // otherwise, merge update kinds. - val merged = deleteKinds - .map(_.deleteKind) - .reduce { - (l, r) => - (l, r) match { - case (DeleteKind.NONE, r: DeleteKind) => r - case (l: DeleteKind, DeleteKind.NONE) => l - case (l: DeleteKind, r: DeleteKind) => - if (l == r) { - l - } else { - // if any of the union input produces DELETE_BY_KEY, the union produces - // delete by key - DeleteKind.DELETE_BY_KEY - } - } - } - new DeleteKindTrait(merged) - } - createNewNode(union, Some(children.flatten), providedTrait) - } - - case normalize: StreamPhysicalChangelogNormalize => - // if - // 1. we don't need to produce UPDATE_BEFORE, - // 2. children can satisfy the required delete trait, - // 3. the normalize doesn't have filter condition which we'd lose, - // 4. we don't use metadata columns - // we can skip ChangelogNormalize - if (!ChangelogNormalizeRequirementResolver.isRequired(normalize)) { - visitChildren(normalize, requiredTrait) match { - case Some(children) => - val input = children.head match { - case exchange: StreamPhysicalExchange => - exchange.getInput - case _ => - normalize.getInput - } - return Some(input.asInstanceOf[StreamPhysicalRel]) - case _ => - } - } - val childModifyKindTrait = getModifyKindSet(rel.getInput(0)) - - // prefer delete by key, but accept both - val children = visitChildren(normalize, deleteOnKeyOrNone(childModifyKindTrait)) - .orElse(visitChildren(normalize, fullDeleteOrNone(childModifyKindTrait))) - - // changelog normalize produces full deletes - createNewNode(rel, children, fullDeleteOrNone(getModifyKindSet(rel))) - - case ts: StreamPhysicalTableSourceScan => - // currently only support BEFORE_AND_AFTER if source produces updates - val providedTrait = DeleteKindTrait.fromChangelogMode(ts.tableSource.getChangelogMode) - createNewNode(rel, Some(List()), providedTrait) - - case _: StreamPhysicalDataStreamScan | _: StreamPhysicalLegacyTableSourceScan | - _: StreamPhysicalValues => - createNewNode(rel, Some(List()), DeleteKindTrait.NONE) - - case _: StreamPhysicalIntermediateTableScan => - createNewNode(rel, Some(List()), fullDeleteOrNone(getModifyKindSet(rel))) - - case multiJoin: StreamPhysicalMultiJoin => - val children = multiJoin.getInputs.zipWithIndex.map { - case (child, childOrdinal) => - val physicalChild = child.asInstanceOf[StreamPhysicalRel] - val supportsDeleteByKey = multiJoin.inputUniqueKeyContainsCommonJoinKey(childOrdinal) - val inputModifyKindSet = getModifyKindSet(physicalChild) - if (supportsDeleteByKey && requiredTrait == DELETE_BY_KEY) { - this - .visit(physicalChild, deleteOnKeyOrNone(inputModifyKindSet)) - .orElse(this.visit(physicalChild, fullDeleteOrNone(inputModifyKindSet))) - } else { - this.visit(physicalChild, fullDeleteOrNone(inputModifyKindSet)) - } - } - if (children.exists(_.isEmpty)) { - None - } else { - val childRels = children.flatten.toList - if (childRels.exists(r => getDeleteKind(r) == DeleteKind.DELETE_BY_KEY)) { - createNewNode(multiJoin, Some(childRels), deleteOnKeyOrNone(getModifyKindSet(rel))) - } else { - createNewNode(multiJoin, Some(childRels), fullDeleteOrNone(getModifyKindSet(rel))) - } - } - - case _ => - throw new UnsupportedOperationException( - s"Unsupported visit for ${rel.getClass.getSimpleName}") - - } - - private def visitChildren( - parent: StreamPhysicalRel, - requiredChildrenTrait: DeleteKindTrait): Option[List[StreamPhysicalRel]] = { - val newChildren = for (child <- parent.getInputs) yield { - this.visit(child.asInstanceOf[StreamPhysicalRel], requiredChildrenTrait) match { - case None => - // return None if one of the children can't satisfy - return None - case Some(newChild) => - val providedTrait = newChild.getTraitSet.getTrait(DeleteKindTraitDef.INSTANCE) - if (!providedTrait.satisfies(requiredChildrenTrait)) { - // the provided trait can't satisfy required trait, thus we should return None. - return None - } - newChild - } - } - Some(newChildren.toList) - } - - private def createNewNode( - node: StreamPhysicalRel, - childrenOption: Option[List[StreamPhysicalRel]], - providedDeleteTrait: DeleteKindTrait): Option[StreamPhysicalRel] = childrenOption match { - case None => - None - case Some(children) => - val modifyKindSetTrait = node.getTraitSet.getTrait(ModifyKindSetTraitDef.INSTANCE) - val nodeDescription = node.getRelDetailedDescription - val isDeleteKindValid = providedDeleteTrait.deleteKind match { - case DeleteKind.NONE => - !modifyKindSetTrait.modifyKindSet.contains(ModifyKind.DELETE) - case DeleteKind.DELETE_BY_KEY | DeleteKind.FULL_DELETE => - modifyKindSetTrait.modifyKindSet.contains(ModifyKind.DELETE) - } - if (!isDeleteKindValid) { - throw new TableException( - s"DeleteKindTrait $providedDeleteTrait conflicts with " + - s"ModifyKindSetTrait $modifyKindSetTrait. " + - s"This is a bug in planner, please file an issue. \n" + - s"Current node is $nodeDescription.") - } - val newTraitSet = node.getTraitSet.plus(providedDeleteTrait) - Some(node.copy(newTraitSet, children).asInstanceOf[StreamPhysicalRel]) - } - - private def visitSink( - sink: StreamPhysicalRel, - sinkRequiredTraits: Seq[DeleteKindTrait]): Option[StreamPhysicalRel] = { - val children = sinkRequiredTraits.flatMap(t => visitChildren(sink, t)) - if (children.isEmpty) { - None - } else { - val sinkTrait = sink.getTraitSet.plus(DeleteKindTrait.NONE) - Some(sink.copy(sinkTrait, children.head).asInstanceOf[StreamPhysicalRel]) - } - } - - /** - * Infer sink required traits by the sink node and its input. Sink required traits is based on - * the sink node's changelog mode, the only exception is when sink's pk(s) not exactly the same - * as the changeLogUpsertKeys and sink' changelog mode is DELETE_BY_KEY. - */ - private def inferSinkRequiredTraits(sink: StreamPhysicalSink): Seq[DeleteKindTrait] = { - val childModifyKindSet = getModifyKindSet(sink.getInput) - val sinkChangelogMode = - sink.tableSink.getChangelogMode(childModifyKindSet.toDefaultChangelogMode) - - val sinkDeleteTrait = DeleteKindTrait.fromChangelogMode(sinkChangelogMode) - - val fullDelete = fullDeleteOrNone(childModifyKindSet) - if (sinkDeleteTrait.equals(DeleteKindTrait.DELETE_BY_KEY)) { - if (areUpsertKeysDifferentFromPk(sink)) { - Seq(fullDelete) - } else { - Seq(sinkDeleteTrait, fullDelete) - } - } else { - Seq(fullDelete) - } - } - - // ------------------------------------------------------------------------------------------- - - private def areUpsertKeysDifferentFromPk(sink: StreamPhysicalSink) = { - // if sink's pk(s) are not exactly match input changeLogUpsertKeys then it will fallback - // to beforeAndAfter mode for the correctness - var upsertKeyDifferentFromPk: Boolean = false - val sinkDefinedPks = sink.contextResolvedTable.getResolvedSchema.getPrimaryKeyIndexes - - if (sinkDefinedPks.nonEmpty) { - val sinkPks = ImmutableBitSet.of(sinkDefinedPks: _*) - val fmq = FlinkRelMetadataQuery.reuseOrCreate(sink.getCluster.getMetadataQuery) - val changeLogUpsertKeys = fmq.getUpsertKeys(sink.getInput) - // if input is UA only, primary key != upsert key (upsert key can be null) we should - // fallback to beforeAndAfter. - // Notice: even sink pk(s) contains input upsert key we cannot optimize to UA only, - // this differs from batch job's unique key inference - if (changeLogUpsertKeys == null || !changeLogUpsertKeys.exists(_.equals(sinkPks))) { - upsertKeyDifferentFromPk = true - } - } - upsertKeyDifferentFromPk - } - } - private def isNonUpsertKeyCondition(calc: StreamPhysicalCalcBase): Boolean = ChangelogModeInferenceUtils.isNonUpsertKeyCondition(calc) private def getModifyKindSet(node: RelNode): ModifyKindSet = ChangelogModeInferenceUtils.getModifyKindSet(node) - private def getDeleteKind(node: RelNode): DeleteKind = - ChangelogModeInferenceUtils.getDeleteKind(node) - // ---------------------------------------------------------------------------------------------- // PTF helper methods // ---------------------------------------------------------------------------------------------- From ba59ea22987de12e4e649e0542b83f331b58359c Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Wed, 3 Jun 2026 14:50:21 -0700 Subject: [PATCH 4/6] [FLINK-XXXXX][table] Move SatisfyUpdateKindTraitVisitor into a top level Java class --- .../SatisfyUpdateKindTraitVisitor.java | 849 ++++++++++++++++++ .../FlinkChangelogModeInferenceProgram.scala | 654 +------------- 2 files changed, 852 insertions(+), 651 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java new file mode 100644 index 0000000000000..7a36e7b03622a --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java @@ -0,0 +1,849 @@ +/* + * 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.optimize.program; + +import org.apache.flink.legacy.table.sinks.AppendStreamTableSink; +import org.apache.flink.legacy.table.sinks.RetractStreamTableSink; +import org.apache.flink.legacy.table.sinks.StreamTableSink; +import org.apache.flink.legacy.table.sinks.UpsertStreamTableSink; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.InsertConflictStrategy.ConflictBehavior; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.ExecutionConfigOptions.UpsertMaterialize; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.legacy.sinks.TableSink; +import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDropUpdateBefore; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupTableAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupWindowAggregateBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalIntermediateTableScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalIntervalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMLPredictTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalProcessTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonGroupAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonGroupTableAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonOverAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalVectorSearchTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction; +import org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.PtfTableArgComponents; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; +import org.apache.flink.table.planner.plan.trait.ModifyKind; +import org.apache.flink.table.planner.plan.trait.ModifyKindSet; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTraitDef; +import org.apache.flink.table.planner.plan.trait.UpdateKind; +import org.apache.flink.table.planner.plan.trait.UpdateKindTrait; +import org.apache.flink.table.planner.plan.trait.UpdateKindTraitDef; +import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils; +import org.apache.flink.table.planner.plan.utils.RankProcessStrategy; +import org.apache.flink.table.planner.sinks.DataStreamTableSink; +import org.apache.flink.table.planner.utils.ShortcutUtils; +import org.apache.flink.table.types.inference.StaticArgument; +import org.apache.flink.table.types.inference.StaticArgumentTrait; +import org.apache.flink.types.RowKind; + +import org.apache.calcite.linq4j.Ord; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.util.ImmutableBitSet; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.function.Function; + +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.extractPtfTableArgComponents; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.getModifyKindSet; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.isNonUpsertKeyCondition; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.ptfRequiresUpdateBefore; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.queryPtfChangelogMode; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.toChangelogMode; + +/** + * A visitor which will try to satisfy the required {@link UpdateKindTrait} from root. + * + *

After traversed by this visitor, every node should have a correct {@link UpdateKindTrait} or + * returns {@link Optional#empty()} if the planner doesn't support to satisfy the required {@link + * UpdateKindTrait}. + */ +class SatisfyUpdateKindTraitVisitor { + + private final StreamOptimizeContext context; + + SatisfyUpdateKindTraitVisitor(StreamOptimizeContext context) { + this.context = context; + } + + /** + * Try to satisfy the required {@link UpdateKindTrait} from root. + * + *

Each node will first require a UpdateKindTrait to its children. The required UpdateKindTrait + * may come from the node's parent, or come from the node itself, depending on whether the node + * will destroy the trait provided by children or pass the trait from children. + * + *

If the node will pass the children's UpdateKindTrait without destroying it, then return a + * new node with new inputs and forwarded UpdateKindTrait. + * + *

If the node will destroy the children's UpdateKindTrait, then the node itself needs to be + * converted, or a new node should be generated to satisfy the required trait, such as marking + * itself not to generate UPDATE_BEFORE, or generating a new node to filter UPDATE_BEFORE. + * + * @param rel the node who should satisfy the requiredTrait + * @param requiredUpdateTrait the required UpdateKindTrait + * @return A converted node which satisfies required traits by input nodes of current node. Or + * {@link Optional#empty()} if required traits cannot be satisfied. + */ + Optional visit( + StreamPhysicalRel rel, UpdateKindTrait requiredUpdateTrait) { + if (rel instanceof StreamPhysicalSink) { + StreamPhysicalSink sink = (StreamPhysicalSink) rel; + List sinkRequiredTraits = inferSinkRequiredTraits(sink); + boolean upsertMaterialize = analyzeUpsertMaterializeStrategy(sink); + return visitSink(sink.copy(upsertMaterialize), sinkRequiredTraits); + } else if (rel instanceof StreamPhysicalLegacySink) { + StreamPhysicalLegacySink legacySink = (StreamPhysicalLegacySink) rel; + ModifyKindSet childModifyKindSet = getModifyKindSet(legacySink.getInput()); + UpdateKindTrait onlyAfter = UpdateKindTrait.onlyAfterOrNone(childModifyKindSet); + UpdateKindTrait beforeAndAfter = UpdateKindTrait.beforeAfterOrNone(childModifyKindSet); + TableSink tableSink = legacySink.sink(); + final List sinkRequiredTraits; + if (tableSink instanceof UpsertStreamTableSink) { + // support both ONLY_AFTER and BEFORE_AFTER, but prefer ONLY_AFTER + sinkRequiredTraits = Arrays.asList(onlyAfter, beforeAndAfter); + } else if (tableSink instanceof RetractStreamTableSink) { + sinkRequiredTraits = Collections.singletonList(beforeAndAfter); + } else if (tableSink instanceof AppendStreamTableSink + || tableSink instanceof StreamTableSink) { + sinkRequiredTraits = Collections.singletonList(UpdateKindTrait.NONE()); + } else if (tableSink instanceof DataStreamTableSink) { + DataStreamTableSink ds = (DataStreamTableSink) tableSink; + if (ds.withChangeFlag()) { + if (ds.needUpdateBefore()) { + sinkRequiredTraits = Collections.singletonList(beforeAndAfter); + } else { + // support both ONLY_AFTER and BEFORE_AFTER, but prefer ONLY_AFTER + sinkRequiredTraits = Arrays.asList(onlyAfter, beforeAndAfter); + } + } else { + sinkRequiredTraits = Collections.singletonList(UpdateKindTrait.NONE()); + } + } else { + throw new UnsupportedOperationException( + "Unsupported sink '" + tableSink.getClass().getSimpleName() + "'"); + } + return visitSink(legacySink, sinkRequiredTraits); + } else if (requiresUpdateBeforeFromChildren(rel)) { + // Aggregate, TableAggregate, OverAggregate, Limit, GroupWindowAggregate, WindowAggregate, + // and WindowTableAggregate requires update_before if there are updates + UpdateKindTrait requiredChildUpdateTrait = + UpdateKindTrait.beforeAfterOrNone(getModifyKindSet(rel.getInput(0))); + Optional> children = visitChildren(rel, requiredChildUpdateTrait); + // use requiredTrait as providedTrait, because they should support all kinds of UpdateKind + return createNewNode(rel, children, requiredUpdateTrait); + } else if (requiresNoUpdateFromChildren(rel)) { + // WindowRank, WindowDeduplicate, Deduplicate, TemporalSort, CEP, + // and IntervalJoin, WindowJoin require nothing about UpdateKind. + Optional> children = visitChildren(rel, UpdateKindTrait.NONE()); + return createNewNode(rel, children, requiredUpdateTrait); + } else if (rel instanceof StreamPhysicalRank) { + StreamPhysicalRank rank = (StreamPhysicalRank) rel; + List rankStrategies = + RankProcessStrategy.analyzeRankProcessStrategies( + rank, rank.partitionKey(), rank.orderKey()); + return visitRankStrategies( + rankStrategies, requiredUpdateTrait, rank::copy); + } else if (rel instanceof StreamPhysicalSortLimit) { + StreamPhysicalSortLimit sortLimit = (StreamPhysicalSortLimit) rel; + List rankStrategies = + RankProcessStrategy.analyzeRankProcessStrategies( + sortLimit, ImmutableBitSet.of(), sortLimit.getCollation()); + return visitRankStrategies( + rankStrategies, requiredUpdateTrait, sortLimit::copy); + } else if (rel instanceof StreamPhysicalSort) { + StreamPhysicalSort sort = (StreamPhysicalSort) rel; + UpdateKindTrait requiredChildTrait = + UpdateKindTrait.beforeAfterOrNone(getModifyKindSet(sort.getInput())); + Optional> children = visitChildren(sort, requiredChildTrait); + return createNewNode(sort, children, requiredUpdateTrait); + } else if (rel instanceof StreamPhysicalJoin) { + StreamPhysicalJoin join = (StreamPhysicalJoin) rel; + boolean onlyAfterByParent = + requiredUpdateTrait.updateKind() == UpdateKind.ONLY_UPDATE_AFTER; + List> children = new ArrayList<>(); + for (int childOrdinal = 0; childOrdinal < join.getInputs().size(); childOrdinal++) { + StreamPhysicalRel physicalChild = (StreamPhysicalRel) join.getInput(childOrdinal); + boolean supportOnlyAfter = join.inputUniqueKeyContainsJoinKey(childOrdinal); + ModifyKindSet inputModifyKindSet = getModifyKindSet(physicalChild); + if (onlyAfterByParent) { + if (inputModifyKindSet.contains(ModifyKind.UPDATE) && !supportOnlyAfter) { + // the parent requires only-after, however, the join doesn't support this + children.add(Optional.empty()); + } else { + children.add( + visit(physicalChild, UpdateKindTrait.onlyAfterOrNone(inputModifyKindSet))); + } + } else { + children.add( + visit(physicalChild, UpdateKindTrait.beforeAfterOrNone(inputModifyKindSet))); + } + } + if (children.stream().anyMatch(c -> !c.isPresent())) { + return Optional.empty(); + } + return createNewNode(join, Optional.of(present(children)), requiredUpdateTrait); + } else if (rel instanceof StreamPhysicalTemporalJoin) { + StreamPhysicalTemporalJoin temporalJoin = (StreamPhysicalTemporalJoin) rel; + StreamPhysicalRel left = (StreamPhysicalRel) temporalJoin.getLeft(); + StreamPhysicalRel right = (StreamPhysicalRel) temporalJoin.getRight(); + + // the left input required trait depends on it's parent in temporal join + // the left input will send message to parent + boolean requiredUpdateBeforeByParent = + requiredUpdateTrait.updateKind() == UpdateKind.BEFORE_AND_AFTER; + ModifyKindSet leftInputModifyKindSet = getModifyKindSet(left); + UpdateKindTrait leftRequiredTrait = + requiredUpdateBeforeByParent + ? UpdateKindTrait.beforeAfterOrNone(leftInputModifyKindSet) + : UpdateKindTrait.onlyAfterOrNone(leftInputModifyKindSet); + Optional newLeftOption = visit(left, leftRequiredTrait); + + ModifyKindSet rightInputModifyKindSet = getModifyKindSet(right); + // currently temporal join support changelog stream as the right side + // so it supports both ONLY_AFTER and BEFORE_AFTER, but prefer ONLY_AFTER + Optional newRightOption = + visit(right, UpdateKindTrait.onlyAfterOrNone(rightInputModifyKindSet)); + if (!newRightOption.isPresent()) { + newRightOption = visit(right, UpdateKindTrait.beforeAfterOrNone(rightInputModifyKindSet)); + } + + if (newLeftOption.isPresent() && newRightOption.isPresent()) { + StreamPhysicalRel newLeft = newLeftOption.get(); + StreamPhysicalRel newRight = newRightOption.get(); + UpdateKindTrait leftTrait = + newLeft.getTraitSet().getTrait(UpdateKindTraitDef.INSTANCE()); + return createNewNode( + temporalJoin, Optional.of(Arrays.asList(newLeft, newRight)), leftTrait); + } else { + return Optional.empty(); + } + } else if (rel instanceof StreamPhysicalCalcBase) { + // if the condition is applied on the upsert key, we can emit whatever the requiredTrait + // is, because we will filter all records based on the condition that applies to that key + StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel; + if (requiredUpdateTrait.equals(UpdateKindTrait.ONLY_UPDATE_AFTER()) + && isNonUpsertKeyCondition(calc)) { + // we don't expect filter to satisfy ONLY_UPDATE_AFTER update kind, + // to solve the bad case like a single 'cnt < 10' condition after aggregation. + // See FLINK-9528. + return Optional.empty(); + } + // otherwise, forward UpdateKind requirement + Optional> children = visitChildren(rel, requiredUpdateTrait); + if (!children.isPresent()) { + return Optional.empty(); + } + UpdateKindTrait childTrait = + children.get().get(0).getTraitSet().getTrait(UpdateKindTraitDef.INSTANCE()); + return createNewNode(rel, children, childTrait); + } else if (isTransparentForwardOperator(rel)) { + // transparent forward requiredTrait to children + Optional> children = visitChildren(rel, requiredUpdateTrait); + if (!children.isPresent()) { + return Optional.empty(); + } + UpdateKindTrait childTrait = + children.get().get(0).getTraitSet().getTrait(UpdateKindTraitDef.INSTANCE()); + return createNewNode(rel, children, childTrait); + } else if (rel instanceof StreamPhysicalUnion) { + StreamPhysicalUnion union = (StreamPhysicalUnion) rel; + List> children = new ArrayList<>(); + for (RelNode childNode : union.getInputs()) { + StreamPhysicalRel child = (StreamPhysicalRel) childNode; + ModifyKindSet childModifyKindSet = getModifyKindSet(child); + UpdateKindTrait requiredChildTrait = + childModifyKindSet.isInsertOnly() ? UpdateKindTrait.NONE() : requiredUpdateTrait; + children.add(visit(child, requiredChildTrait)); + } + if (children.stream().anyMatch(c -> !c.isPresent())) { + return Optional.empty(); + } + List childRels = present(children); + List updateKinds = new ArrayList<>(); + for (StreamPhysicalRel child : childRels) { + updateKinds.add(child.getTraitSet().getTrait(UpdateKindTraitDef.INSTANCE())); + } + // union can just forward changes, can't actively satisfy to another changelog mode + final UpdateKindTrait providedTrait; + if (updateKinds.stream().allMatch(k -> UpdateKindTrait.NONE().equals(k))) { + // if all the children is NO_UPDATE, union is NO_UPDATE + providedTrait = UpdateKindTrait.NONE(); + } else { + // otherwise, merge update kinds. + UpdateKind merged = null; + for (UpdateKindTrait updateKindTrait : updateKinds) { + UpdateKind updateKind = updateKindTrait.updateKind(); + if (merged == null) { + merged = updateKind; + } else if (merged == UpdateKind.NONE) { + merged = updateKind; + } else if (updateKind == UpdateKind.NONE) { + // merged stays unchanged + } else if (merged == updateKind) { + // merged stays unchanged + } else { + // UNION doesn't support to union ONLY_UPDATE_AFTER and BEFORE_AND_AFTER inputs + return Optional.empty(); + } + } + providedTrait = new UpdateKindTrait(merged); + } + return createNewNode(union, Optional.of(childRels), providedTrait); + } else if (rel instanceof StreamPhysicalChangelogNormalize) { + StreamPhysicalChangelogNormalize normalize = (StreamPhysicalChangelogNormalize) rel; + // changelog normalize currently only supports input only sending UPDATE_AFTER + Optional> children = + visitChildren(normalize, UpdateKindTrait.ONLY_UPDATE_AFTER()); + // use requiredTrait as providedTrait, + // because changelog normalize supports all kinds of UpdateKind + return createNewNode(rel, children, requiredUpdateTrait); + } else if (rel instanceof StreamPhysicalTableSourceScan) { + // currently only support BEFORE_AND_AFTER if source produces updates + StreamPhysicalTableSourceScan ts = (StreamPhysicalTableSourceScan) rel; + UpdateKindTrait providedTrait = + UpdateKindTrait.fromChangelogMode(ts.tableSource().getChangelogMode()); + Optional newSource = + createNewNode(rel, Optional.of(Collections.emptyList()), providedTrait); + if (providedTrait.equals(UpdateKindTrait.BEFORE_AND_AFTER()) + && requiredUpdateTrait.equals(UpdateKindTrait.ONLY_UPDATE_AFTER())) { + // requiring only-after, but the source is CDC source, then drop update_before manually + StreamPhysicalDropUpdateBefore dropUB = + new StreamPhysicalDropUpdateBefore(rel.getCluster(), rel.getTraitSet(), rel); + return createNewNode( + dropUB, newSource.map(Collections::singletonList), requiredUpdateTrait); + } else { + return newSource; + } + } else if (rel instanceof StreamPhysicalDataStreamScan + || rel instanceof StreamPhysicalLegacyTableSourceScan + || rel instanceof StreamPhysicalValues) { + return createNewNode(rel, Optional.of(Collections.emptyList()), UpdateKindTrait.NONE()); + } else if (rel instanceof StreamPhysicalIntermediateTableScan) { + StreamPhysicalIntermediateTableScan scan = (StreamPhysicalIntermediateTableScan) rel; + final UpdateKindTrait providedTrait; + if (scan.intermediateTable().isUpdateBeforeRequired()) { + // we can't drop UPDATE_BEFORE if it is required by other parent blocks + providedTrait = UpdateKindTrait.BEFORE_AND_AFTER(); + } else { + providedTrait = requiredUpdateTrait; + } + if (!providedTrait.satisfies(requiredUpdateTrait)) { + // require ONLY_AFTER but can only provide BEFORE_AND_AFTER + return Optional.empty(); + } + return createNewNode(rel, Optional.of(Collections.emptyList()), providedTrait); + } else if (rel instanceof StreamPhysicalProcessTableFunction) { + // Required update traits depend on the table argument declaration, + // input traits, partition keys, and upsert keys + StreamPhysicalProcessTableFunction process = (StreamPhysicalProcessTableFunction) rel; + List> inputArgs = + StreamPhysicalProcessTableFunction.getProvidedInputArgs(process.getCall()); + List children = new ArrayList<>(); + List inputs = process.getInputs(); + for (int inputIndex = 0; inputIndex < inputs.size(); inputIndex++) { + StreamPhysicalRel child = (StreamPhysicalRel) inputs.get(inputIndex); + final Optional visited; + // For PTF without table arguments (i.e. values child) + if (inputArgs.isEmpty()) { + visited = visit(child, UpdateKindTrait.NONE()); + } else { + // Derive the required update trait for table arguments + Ord inputArg = inputArgs.get(inputIndex); + PtfTableArgComponents components = + extractPtfTableArgComponents(process, child, inputArg); + StaticArgument tableArg = components.tableArg; + final UpdateKindTrait childRequiredTrait; + if (!components.modifyKindSet.isInsertOnly() + && tableArg.is(StaticArgumentTrait.SUPPORT_UPDATES)) { + childRequiredTrait = + ptfRequiresUpdateBefore(tableArg, components.tableArgCall, child) + ? UpdateKindTrait.BEFORE_AND_AFTER() + : UpdateKindTrait.ONLY_UPDATE_AFTER(); + } else { + childRequiredTrait = UpdateKindTrait.NONE(); + } + visited = visit(child, childRequiredTrait); + } + visited.ifPresent(children::add); + } + // Query PTF for upsert vs. retract + UpdateKindTrait providedUpdateTrait = + queryPtfChangelogMode( + process, + children, + toChangelogMode(process, requiredUpdateTrait, null), + UpdateKindTrait::fromChangelogMode, + UpdateKindTrait.NONE()); + return createNewNode(rel, Optional.of(children), providedUpdateTrait); + } else if (rel instanceof StreamPhysicalMultiJoin) { + StreamPhysicalMultiJoin multiJoin = (StreamPhysicalMultiJoin) rel; + boolean onlyAfterByParent = + requiredUpdateTrait.updateKind() == UpdateKind.ONLY_UPDATE_AFTER; + List> children = new ArrayList<>(); + for (int childOrdinal = 0; childOrdinal < multiJoin.getInputs().size(); childOrdinal++) { + StreamPhysicalRel physicalChild = (StreamPhysicalRel) multiJoin.getInput(childOrdinal); + boolean supportOnlyAfter = multiJoin.inputUniqueKeyContainsCommonJoinKey(childOrdinal); + ModifyKindSet inputModifyKindSet = getModifyKindSet(physicalChild); + if (onlyAfterByParent) { + if (inputModifyKindSet.contains(ModifyKind.UPDATE) && !supportOnlyAfter) { + // the parent requires only-after, however, the multi-join doesn't support + // this for this input + children.add(Optional.empty()); + } else { + children.add( + visit(physicalChild, UpdateKindTrait.onlyAfterOrNone(inputModifyKindSet))); + } + } else { + children.add( + visit(physicalChild, UpdateKindTrait.beforeAfterOrNone(inputModifyKindSet))); + } + } + if (children.stream().anyMatch(c -> !c.isPresent())) { + return Optional.empty(); + } + return createNewNode(multiJoin, Optional.of(present(children)), requiredUpdateTrait); + } else { + throw new UnsupportedOperationException( + "Unsupported visit for " + rel.getClass().getSimpleName()); + } + } + + /** + * Aggregate, TableAggregate, OverAggregate, Limit, GroupWindowAggregate, WindowAggregate, and + * WindowTableAggregate require update_before if there are updates. + */ + private static boolean requiresUpdateBeforeFromChildren(StreamPhysicalRel rel) { + return rel instanceof StreamPhysicalGroupAggregate + || rel instanceof StreamPhysicalGroupTableAggregate + || rel instanceof StreamPhysicalLimit + || rel instanceof StreamPhysicalPythonGroupAggregate + || rel instanceof StreamPhysicalPythonGroupTableAggregate + || rel instanceof StreamPhysicalGroupWindowAggregateBase + || rel instanceof StreamPhysicalWindowAggregate + || rel instanceof StreamPhysicalOverAggregate; + } + + /** + * WindowRank, WindowDeduplicate, TemporalSort, CEP, IntervalJoin and WindowJoin require nothing + * about UpdateKind. + */ + private static boolean requiresNoUpdateFromChildren(StreamPhysicalRel rel) { + return rel instanceof StreamPhysicalWindowRank + || rel instanceof StreamPhysicalWindowDeduplicate + || rel instanceof StreamPhysicalTemporalSort + || rel instanceof StreamPhysicalMatch + || rel instanceof StreamPhysicalIntervalJoin + || rel instanceof StreamPhysicalPythonOverAggregate + || rel instanceof StreamPhysicalWindowJoin; + } + + /** Operators that transparently forward the required UpdateKindTrait to their children. */ + private static boolean isTransparentForwardOperator(StreamPhysicalRel rel) { + return rel instanceof StreamPhysicalCorrelateBase + || rel instanceof StreamPhysicalLookupJoin + || rel instanceof StreamPhysicalExchange + || rel instanceof StreamPhysicalExpand + || rel instanceof StreamPhysicalMiniBatchAssigner + || rel instanceof StreamPhysicalWatermarkAssigner + || rel instanceof StreamPhysicalWindowTableFunction + || rel instanceof StreamPhysicalMLPredictTableFunction + || rel instanceof StreamPhysicalVectorSearchTableFunction; + } + + private Optional> visitChildren( + StreamPhysicalRel parent, UpdateKindTrait requiredChildrenUpdateTrait) { + List newChildren = new ArrayList<>(); + for (RelNode childNode : parent.getInputs()) { + Optional newChild = + visit((StreamPhysicalRel) childNode, requiredChildrenUpdateTrait); + if (!newChild.isPresent()) { + // return empty if one of the children can't satisfy + return Optional.empty(); + } + UpdateKindTrait providedUpdateTrait = + newChild.get().getTraitSet().getTrait(UpdateKindTraitDef.INSTANCE()); + if (!providedUpdateTrait.satisfies(requiredChildrenUpdateTrait)) { + // the provided trait can't satisfy required trait, thus we should return empty. + return Optional.empty(); + } + newChildren.add(newChild.get()); + } + return Optional.of(newChildren); + } + + private Optional createNewNode( + StreamPhysicalRel node, + Optional> childrenOption, + UpdateKindTrait providedUpdateTrait) { + if (!childrenOption.isPresent()) { + return Optional.empty(); + } + List children = childrenOption.get(); + ModifyKindSetTrait modifyKindSetTrait = + node.getTraitSet().getTrait(ModifyKindSetTraitDef.INSTANCE()); + String nodeDescription = node.getRelDetailedDescription(); + final boolean isUpdateKindValid; + switch (providedUpdateTrait.updateKind()) { + case NONE: + isUpdateKindValid = !modifyKindSetTrait.modifyKindSet().contains(ModifyKind.UPDATE); + break; + case BEFORE_AND_AFTER: + case ONLY_UPDATE_AFTER: + isUpdateKindValid = modifyKindSetTrait.modifyKindSet().contains(ModifyKind.UPDATE); + break; + default: + isUpdateKindValid = false; + } + if (!isUpdateKindValid) { + throw new TableException( + "UpdateKindTrait " + + providedUpdateTrait + + " conflicts with ModifyKindSetTrait " + + modifyKindSetTrait + + ". This is a bug in planner, please file an issue. \n" + + "Current node is " + + nodeDescription + + "."); + } + RelTraitSet newTraitSet = node.getTraitSet().plus(providedUpdateTrait); + return Optional.of((StreamPhysicalRel) node.copy(newTraitSet, new ArrayList(children))); + } + + /** + * Try all possible rank strategies and return the first viable new node. + * + * @param rankStrategies all possible supported rank strategy by current node + * @param requiredUpdateKindTrait the required UpdateKindTrait by parent of rank node + * @param applyRankStrategy a function to apply rank strategy to get a new copied rank node + */ + private Optional visitRankStrategies( + List rankStrategies, + UpdateKindTrait requiredUpdateKindTrait, + Function applyRankStrategy) { + // go pass every RankProcessStrategy, apply the rank strategy to get a new copied rank node, + // return the first satisfied converted node + for (RankProcessStrategy strategy : rankStrategies) { + final UpdateKindTrait requiredChildrenTrait; + if (strategy instanceof RankProcessStrategy.UpdateFastStrategy) { + requiredChildrenTrait = UpdateKindTrait.ONLY_UPDATE_AFTER(); + } else if (strategy instanceof RankProcessStrategy.RetractStrategy) { + requiredChildrenTrait = UpdateKindTrait.BEFORE_AND_AFTER(); + } else if (strategy instanceof RankProcessStrategy.AppendFastStrategy) { + requiredChildrenTrait = UpdateKindTrait.NONE(); + } else { + throw new IllegalStateException( + "Unsupported rank strategy: " + strategy.getClass().getSimpleName()); + } + StreamPhysicalRel node = applyRankStrategy.apply(strategy); + Optional> children = visitChildren(node, requiredChildrenTrait); + Optional newNode = + createNewNode(node, children, requiredUpdateKindTrait); + if (newNode.isPresent()) { + return newNode; + } + } + return Optional.empty(); + } + + private Optional visitSink( + StreamPhysicalRel sink, List sinkRequiredTraits) { + List> satisfiedChildren = new ArrayList<>(); + for (UpdateKindTrait requiredTrait : sinkRequiredTraits) { + visitChildren(sink, requiredTrait).ifPresent(satisfiedChildren::add); + } + if (satisfiedChildren.isEmpty()) { + return Optional.empty(); + } + RelTraitSet sinkTrait = sink.getTraitSet().plus(UpdateKindTrait.NONE()); + return Optional.of( + (StreamPhysicalRel) + sink.copy(sinkTrait, new ArrayList(satisfiedChildren.get(0)))); + } + + /** + * Infer sink required traits by the sink node and its input. Sink required traits is based on + * the sink node's changelog mode, the only exception is when sink's pk(s) are not satisfied by + * the input's upsert keys (considering immutable columns) and sink's changelog mode is + * ONLY_UPDATE_AFTER. + */ + private List inferSinkRequiredTraits(StreamPhysicalSink sink) { + ModifyKindSet childModifyKindSet = getModifyKindSet(sink.getInput()); + UpdateKindTrait onlyAfter = UpdateKindTrait.onlyAfterOrNone(childModifyKindSet); + UpdateKindTrait beforeAndAfter = UpdateKindTrait.beforeAfterOrNone(childModifyKindSet); + UpdateKindTrait sinkTrait = + UpdateKindTrait.fromChangelogMode( + sink.tableSink().getChangelogMode(childModifyKindSet.toDefaultChangelogMode())); + + if (sinkTrait.equals(UpdateKindTrait.ONLY_UPDATE_AFTER())) { + // if sink's pk(s) are not satisfied by input upsert keys (considering immutable columns), + // fallback to beforeAndAfter mode for correctness + boolean requireBeforeAndAfter = !canUpsertKeysWithImmutableColsSatisfyPk(sink); + if (requireBeforeAndAfter) { + return Collections.singletonList(beforeAndAfter); + } else { + return Arrays.asList(onlyAfter, beforeAndAfter); + } + } else if (sinkTrait.equals(UpdateKindTrait.BEFORE_AND_AFTER())) { + return Collections.singletonList(beforeAndAfter); + } else { + return Collections.singletonList(UpdateKindTrait.NONE()); + } + } + + /** + * Check whether input's upsert keys (together with immutable columns) can satisfy sink's primary + * keys. + * + *

A sink pk is considered "satisfied" when there exists an upsert key {@code uk} such that: + * + *

    + *
  • {@code uk} is a subset of sink pk (no extra columns that could cause key collision) + *
  • the remaining sink pk columns not in {@code uk} are all immutable (immutable columns + * never change, so they effectively act as part of the key for upsert semantics) + *
+ * + *

Example: sink pk = {a, b, c}, uk = {a, b}, immutable columns = {a, b, c, d}. + * + *

    + *
  • Step 1: uk {a, b} ⊆ sink pk {a, b, c} → true + *
  • Step 2: sink pk \ uk = {c}, immutable columns contain {c} → true + *
  • Result: satisfied + *
+ * + *

Notice: even if sink pk is a subset of the upsert key, the pk is NOT considered satisfied + * when the upsert key has columns outside sink pk. This differs from batch job's unique key + * inference. + */ + private boolean canUpsertKeysWithImmutableColsSatisfyPk(StreamPhysicalSink sink) { + int[] sinkDefinedPks = sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes(); + if (sinkDefinedPks.length == 0) { + return true; + } + ImmutableBitSet sinkPks = ImmutableBitSet.of(sinkDefinedPks); + FlinkRelMetadataQuery fmq = + FlinkRelMetadataQuery.reuseOrCreate(sink.getCluster().getMetadataQuery()); + Set changeLogUpsertKeys = fmq.getUpsertKeys(sink.getInput()); + // if upsert key is null, pk cannot be satisfied, should fall back to beforeAndAfter + if (changeLogUpsertKeys == null) { + return false; + } + ImmutableBitSet immutableCols = fmq.getImmutableColumns(sink.getInput()); + if (immutableCols == null) { + immutableCols = ImmutableBitSet.of(); + } + final ImmutableBitSet immutableColumns = immutableCols; + + // when input immutableCols is empty, this degrades to uk.equals(sinkPks) + return changeLogUpsertKeys.stream() + .anyMatch( + uk -> { + // 1. uk ⊆ sinkPks + boolean isSinkPkContainsUk = sinkPks.contains(uk); + // 2. (sinkPks \ uk) ⊆ immutableCols + ImmutableBitSet extraSinkPkCols = sinkPks.except(uk); + boolean areExtraSinkPkColsImmutable = + immutableColumns.contains(extraSinkPkCols); + return isSinkPkContainsUk && areExtraSinkPkColsImmutable; + }); + } + + /** + * Analyze whether to enable upsertMaterialize or not. In these case will return true: + * + *

    + *
  1. when {@code TABLE_EXEC_SINK_UPSERT_MATERIALIZE} set to FORCE and sink's primary key + * nonempty. + *
  2. when {@code TABLE_EXEC_SINK_UPSERT_MATERIALIZE} set to AUTO and sink's primary key + * doesn't contain upsertKeys of the input update stream. + *
+ * + *

Also validates that ON CONFLICT clause is specified when upsert key differs from primary + * key. + */ + private boolean analyzeUpsertMaterializeStrategy(StreamPhysicalSink sink) { + TableConfig tableConfig = ShortcutUtils.unwrapTableConfig(sink); + ChangelogMode inputChangelogMode = + ChangelogPlanUtils.getChangelogMode((StreamPhysicalRel) sink.getInput()).get(); + int[] primaryKeys = sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes(); + ChangelogMode sinkChangelogMode = sink.tableSink().getChangelogMode(inputChangelogMode); + boolean inputIsAppend = inputChangelogMode.containsOnly(RowKind.INSERT); + boolean sinkIsAppend = sinkChangelogMode.containsOnly(RowKind.INSERT); + boolean sinkIsRetract = sinkChangelogMode.contains(RowKind.UPDATE_BEFORE); + + // Validate ON CONFLICT is only allowed for upsert sinks + if (sink.conflictStrategy() != null) { + boolean isUpsertSink = !sinkIsAppend && !sinkIsRetract; + if (!isUpsertSink) { + String reason = + sinkIsAppend + ? "it only accepts INSERT (append-only) changes" + : "it requires UPDATE_BEFORE (retract mode)"; + throw new ValidationException( + "ON CONFLICT clause is only allowed for upsert sinks. The sink '" + + sink.contextResolvedTable().getIdentifier().asSummaryString() + + "' is not an upsert sink because " + + reason + + "."); + } + } + + // Validate that sources have watermarks when using ERROR or NOTHING strategy + if (sink.conflictStrategy() != null + && (sink.conflictStrategy().getBehavior() == ConflictBehavior.ERROR + || sink.conflictStrategy().getBehavior() == ConflictBehavior.NOTHING)) { + validateSourcesHaveWatermarks(sink); + } + + switch (tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE)) { + case FORCE: + return primaryKeys.length > 0 && !sinkIsRetract; + case NONE: + return false; + case AUTO: + // if the sink is not an UPSERT sink (has no PK, or is an APPEND or RETRACT sink) + // we don't need to materialize results + if (primaryKeys.length == 0 || sinkIsAppend || sinkIsRetract) { + return false; + } + + // For a DEDUPLICATE strategy and INSERT only input, we simply let the inserts be + // handled as UPSERT_AFTER and overwrite previous value + if (inputIsAppend && sink.isDeduplicateConflictStrategy()) { + return false; + } + + // if input has updates and primary key != upsert key we should enable + // upsertMaterialize. + // + // An optimize is: do not enable upsertMaterialize when sink pk(s) contains input + // changeLogUpsertKeys + boolean upsertKeyDiffersFromPk = !sink.primaryKeysContainsUpsertKey(); + + // Validate that ON CONFLICT is specified when upsert key differs from primary key + boolean requireOnConflict = + tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_REQUIRE_ON_CONFLICT); + if (requireOnConflict && upsertKeyDiffersFromPk && sink.conflictStrategy() == null) { + String pkNames = sink.getPrimaryKeyNames(); + String upsertKeyNames = sink.getUpsertKeyNames(); + throw new ValidationException( + "The query has an upsert key that differs from the primary key of the " + + "sink table '" + + sink.contextResolvedTable().getIdentifier().asSummaryString() + + "'. Primary key: " + + pkNames + + ", upsert key: " + + upsertKeyNames + + ". This can lead to non-deterministic results when multiple " + + "records with different upsert keys map to the same primary " + + "key. Please specify an ON CONFLICT clause to define how " + + "conflicts should be handled: ON CONFLICT DO DEDUPLICATE " + + "(update to the latest record, state intensive, since we need " + + "to keep the entire history), or ON CONFLICT DO ERROR (fail on " + + "conflict), or ON CONFLICT DO NOTHING (keep first record)."); + } + + return upsertKeyDiffersFromPk; + default: + throw new IllegalStateException( + "Unsupported upsert materialize strategy: " + + tableConfig.get( + ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE)); + } + } + + private void validateSourcesHaveWatermarks(StreamPhysicalSink sink) { + List sourcesWithoutWatermarks = new ArrayList<>(); + collectSourcesWithoutWatermarks(sink.getInput(), sourcesWithoutWatermarks); + if (!sourcesWithoutWatermarks.isEmpty()) { + throw new ValidationException( + "ON CONFLICT DO " + + sink.conflictStrategy().getBehavior() + + " requires all source tables to define watermarks, but the following " + + "source(s) do not: " + + String.join(", ", sourcesWithoutWatermarks) + + ". Please add a WATERMARK declaration to these tables."); + } + } + + private void collectSourcesWithoutWatermarks(RelNode rel, List result) { + if (rel instanceof StreamPhysicalTableSourceScan) { + StreamPhysicalTableSourceScan ts = (StreamPhysicalTableSourceScan) rel; + TableSourceTable table = ts.getTable().unwrap(TableSourceTable.class); + if (table != null + && table.contextResolvedTable().getResolvedSchema().getWatermarkSpecs().isEmpty()) { + result.add(table.contextResolvedTable().getIdentifier().asSummaryString()); + } + } else { + rel.getInputs().forEach(input -> collectSourcesWithoutWatermarks(input, result)); + } + } + + private static List present(List> options) { + List result = new ArrayList<>(options.size()); + for (Optional option : options) { + result.add(option.get()); + } + return result; + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala index d1fb4d17956fc..683fa8661f065 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala @@ -18,29 +18,20 @@ package org.apache.flink.table.planner.plan.optimize.program import org.apache.flink.legacy.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, StreamTableSink, UpsertStreamTableSink} -import org.apache.flink.table.api.{TableException, ValidationException} -import org.apache.flink.table.api.InsertConflictStrategy.ConflictBehavior -import org.apache.flink.table.api.config.ExecutionConfigOptions -import org.apache.flink.table.api.config.ExecutionConfigOptions.UpsertMaterialize +import org.apache.flink.table.api.TableException import org.apache.flink.table.connector.ChangelogMode import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexTableArgCall} import org.apache.flink.table.planner.plan.`trait`._ -import org.apache.flink.table.planner.plan.`trait`.UpdateKindTrait.{beforeAfterOrNone, onlyAfterOrNone, BEFORE_AND_AFTER, ONLY_UPDATE_AFTER} -import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery import org.apache.flink.table.planner.plan.nodes.physical.stream._ -import org.apache.flink.table.planner.plan.schema.TableSourceTable import org.apache.flink.table.planner.plan.utils._ -import org.apache.flink.table.planner.plan.utils.RankProcessStrategy.{AppendFastStrategy, RetractStrategy, UpdateFastStrategy} import org.apache.flink.table.planner.sinks.DataStreamTableSink import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig import org.apache.flink.table.runtime.operators.join.FlinkJoinType import org.apache.flink.table.types.inference.{StaticArgument, StaticArgumentTrait} -import org.apache.flink.types.RowKind import org.apache.calcite.linq4j.Ord import org.apache.calcite.rel.RelNode import org.apache.calcite.rel.core.JoinRelType -import org.apache.calcite.util.ImmutableBitSet import scala.collection.JavaConversions._ @@ -77,7 +68,8 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti val updateKindTraitVisitor = new SatisfyUpdateKindTraitVisitor(context) val updateRoot = requiredUpdateKindTraits.flatMap { requiredUpdateKindTrait => - updateKindTraitVisitor.visit(rootWithModifyKindSet, requiredUpdateKindTrait) + val updated = updateKindTraitVisitor.visit(rootWithModifyKindSet, requiredUpdateKindTrait) + if (updated.isPresent) Some(updated.get) else None } // step3: satisfy DeleteKind trait @@ -544,646 +536,6 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti } } - /** - * A visitor which will try to satisfy the required [[UpdateKindTrait]] from root. - * - *

After traversed by this visitor, every node should have a correct [[UpdateKindTrait]] or - * returns None if the planner doesn't support to satisfy the required [[UpdateKindTrait]]. - */ - private class SatisfyUpdateKindTraitVisitor(private val context: StreamOptimizeContext) { - - /** - * Try to satisfy the required [[UpdateKindTrait]] from root. - * - *

Each node will first require a UpdateKindTrait to its children. The required - * UpdateKindTrait may come from the node's parent, or come from the node itself, depending on - * whether the node will destroy the trait provided by children or pass the trait from children. - * - *

If the node will pass the children's UpdateKindTrait without destroying it, then return a - * new node with new inputs and forwarded UpdateKindTrait. - * - *

If the node will destroy the children's UpdateKindTrait, then the node itself needs to be - * converted, or a new node should be generated to satisfy the required trait, such as marking - * itself not to generate UPDATE_BEFORE, or generating a new node to filter UPDATE_BEFORE. - * - * @param rel - * the node who should satisfy the requiredTrait - * @param requiredUpdateTrait - * the required UpdateKindTrait - * @return - * A converted node which satisfies required traits by input nodes of current node. Or None if - * required traits cannot be satisfied. - */ - def visit( - rel: StreamPhysicalRel, - requiredUpdateTrait: UpdateKindTrait): Option[StreamPhysicalRel] = - rel match { - case sink: StreamPhysicalSink => - val sinkRequiredTraits = inferSinkRequiredTraits(sink) - val upsertMaterialize = analyzeUpsertMaterializeStrategy(sink) - visitSink(sink.copy(upsertMaterialize), sinkRequiredTraits) - - case sink: StreamPhysicalLegacySink[_] => - val childModifyKindSet = getModifyKindSet(sink.getInput) - val onlyAfter = onlyAfterOrNone(childModifyKindSet) - val beforeAndAfter = beforeAfterOrNone(childModifyKindSet) - val sinkRequiredTraits = sink.sink match { - case _: UpsertStreamTableSink[_] => - // support both ONLY_AFTER and BEFORE_AFTER, but prefer ONLY_AFTER - Seq(onlyAfter, beforeAndAfter) - case _: RetractStreamTableSink[_] => - Seq(beforeAndAfter) - case _: AppendStreamTableSink[_] | _: StreamTableSink[_] => - Seq(UpdateKindTrait.NONE) - case ds: DataStreamTableSink[_] => - if (ds.withChangeFlag) { - if (ds.needUpdateBefore) { - Seq(beforeAndAfter) - } else { - // support both ONLY_AFTER and BEFORE_AFTER, but prefer ONLY_AFTER - Seq(onlyAfter, beforeAndAfter) - } - } else { - Seq(UpdateKindTrait.NONE) - } - } - visitSink(sink, sinkRequiredTraits) - - case _: StreamPhysicalGroupAggregate | _: StreamPhysicalGroupTableAggregate | - _: StreamPhysicalLimit | _: StreamPhysicalPythonGroupAggregate | - _: StreamPhysicalPythonGroupTableAggregate | _: StreamPhysicalGroupWindowAggregateBase | - _: StreamPhysicalWindowAggregate | _: StreamPhysicalOverAggregate => - // Aggregate, TableAggregate, OverAggregate, Limit, GroupWindowAggregate, WindowAggregate, - // and WindowTableAggregate requires update_before if there are updates - val requiredChildUpdateTrait = beforeAfterOrNone(getModifyKindSet(rel.getInput(0))) - val children = visitChildren(rel, requiredChildUpdateTrait) - // use requiredTrait as providedTrait, because they should support all kinds of UpdateKind - createNewNode(rel, children, requiredUpdateTrait) - - case _: StreamPhysicalWindowRank | _: StreamPhysicalWindowDeduplicate | - _: StreamPhysicalTemporalSort | _: StreamPhysicalMatch | _: StreamPhysicalIntervalJoin | - _: StreamPhysicalPythonOverAggregate | _: StreamPhysicalWindowJoin => - // WindowRank, WindowDeduplicate, Deduplicate, TemporalSort, CEP, - // and IntervalJoin, WindowJoin require nothing about UpdateKind. - val children = visitChildren(rel, UpdateKindTrait.NONE) - createNewNode(rel, children, requiredUpdateTrait) - - case rank: StreamPhysicalRank => - val rankStrategies = - RankProcessStrategy.analyzeRankProcessStrategies(rank, rank.partitionKey, rank.orderKey) - visitRankStrategies( - rankStrategies, - requiredUpdateTrait, - rankStrategy => rank.copy(rankStrategy)) - - case sortLimit: StreamPhysicalSortLimit => - val rankStrategies = RankProcessStrategy.analyzeRankProcessStrategies( - sortLimit, - ImmutableBitSet.of(), - sortLimit.getCollation) - visitRankStrategies( - rankStrategies, - requiredUpdateTrait, - rankStrategy => sortLimit.copy(rankStrategy)) - - case sort: StreamPhysicalSort => - val requiredChildTrait = beforeAfterOrNone(getModifyKindSet(sort.getInput)) - val children = visitChildren(sort, requiredChildTrait) - createNewNode(sort, children, requiredUpdateTrait) - - case join: StreamPhysicalJoin => - val onlyAfterByParent = requiredUpdateTrait.updateKind == UpdateKind.ONLY_UPDATE_AFTER - val children = join.getInputs.zipWithIndex.map { - case (child, childOrdinal) => - val physicalChild = child.asInstanceOf[StreamPhysicalRel] - val supportOnlyAfter = join.inputUniqueKeyContainsJoinKey(childOrdinal) - val inputModifyKindSet = getModifyKindSet(physicalChild) - if (onlyAfterByParent) { - if (inputModifyKindSet.contains(ModifyKind.UPDATE) && !supportOnlyAfter) { - // the parent requires only-after, however, the join doesn't support this - None - } else { - this.visit(physicalChild, onlyAfterOrNone(inputModifyKindSet)) - } - } else { - this.visit(physicalChild, beforeAfterOrNone(inputModifyKindSet)) - } - } - if (children.exists(_.isEmpty)) { - None - } else { - createNewNode(join, Some(children.flatten.toList), requiredUpdateTrait) - } - - case temporalJoin: StreamPhysicalTemporalJoin => - val left = temporalJoin.getLeft.asInstanceOf[StreamPhysicalRel] - val right = temporalJoin.getRight.asInstanceOf[StreamPhysicalRel] - - // the left input required trait depends on it's parent in temporal join - // the left input will send message to parent - val requiredUpdateBeforeByParent = - requiredUpdateTrait.updateKind == UpdateKind.BEFORE_AND_AFTER - val leftInputModifyKindSet = getModifyKindSet(left) - val leftRequiredTrait = if (requiredUpdateBeforeByParent) { - beforeAfterOrNone(leftInputModifyKindSet) - } else { - onlyAfterOrNone(leftInputModifyKindSet) - } - val newLeftOption = this.visit(left, leftRequiredTrait) - - val rightInputModifyKindSet = getModifyKindSet(right) - // currently temporal join support changelog stream as the right side - // so it supports both ONLY_AFTER and BEFORE_AFTER, but prefer ONLY_AFTER - val newRightOption = this.visit(right, onlyAfterOrNone(rightInputModifyKindSet)) match { - case Some(newRight) => Some(newRight) - case None => - val beforeAfter = beforeAfterOrNone(rightInputModifyKindSet) - this.visit(right, beforeAfter) - } - - (newLeftOption, newRightOption) match { - case (Some(newLeft), Some(newRight)) => - val leftTrait = newLeft.getTraitSet.getTrait(UpdateKindTraitDef.INSTANCE) - createNewNode(temporalJoin, Some(List(newLeft, newRight)), leftTrait) - case _ => - None - } - - // if the condition is applied on the upsert key, we can emit whatever the requiredTrait - // is, because we will filter all records based on the condition that applies to that key - case calc: StreamPhysicalCalcBase => - if ( - requiredUpdateTrait == UpdateKindTrait.ONLY_UPDATE_AFTER && - isNonUpsertKeyCondition(calc) - ) { - // we don't expect filter to satisfy ONLY_UPDATE_AFTER update kind, - // to solve the bad case like a single 'cnt < 10' condition after aggregation. - // See FLINK-9528. - None - } else { - // otherwise, forward UpdateKind requirement - visitChildren(rel, requiredUpdateTrait) match { - case None => None - case Some(children) => - val childTrait = children.head.getTraitSet.getTrait(UpdateKindTraitDef.INSTANCE) - createNewNode(rel, Some(children), childTrait) - } - } - - case _: StreamPhysicalCorrelateBase | _: StreamPhysicalLookupJoin | - _: StreamPhysicalExchange | _: StreamPhysicalExpand | - _: StreamPhysicalMiniBatchAssigner | _: StreamPhysicalWatermarkAssigner | - _: StreamPhysicalWindowTableFunction | _: StreamPhysicalMLPredictTableFunction | - _: StreamPhysicalVectorSearchTableFunction => - // transparent forward requiredTrait to children - visitChildren(rel, requiredUpdateTrait) match { - case None => None - case Some(children) => - val childTrait = children.head.getTraitSet.getTrait(UpdateKindTraitDef.INSTANCE) - createNewNode(rel, Some(children), childTrait) - } - - case union: StreamPhysicalUnion => - val children = union.getInputs.map { - case child: StreamPhysicalRel => - val childModifyKindSet = getModifyKindSet(child) - val requiredChildTrait = if (childModifyKindSet.isInsertOnly) { - UpdateKindTrait.NONE - } else { - requiredUpdateTrait - } - this.visit(child, requiredChildTrait) - }.toList - - if (children.exists(_.isEmpty)) { - None - } else { - val updateKinds = children.flatten - .map(_.getTraitSet.getTrait(UpdateKindTraitDef.INSTANCE)) - // union can just forward changes, can't actively satisfy to another changelog mode - val providedTrait = if (updateKinds.forall(k => UpdateKindTrait.NONE == k)) { - // if all the children is NO_UPDATE, union is NO_UPDATE - UpdateKindTrait.NONE - } else { - // otherwise, merge update kinds. - val merged = updateKinds - .map(_.updateKind) - .reduce { - (l, r) => - (l, r) match { - case (UpdateKind.NONE, r: UpdateKind) => r - case (l: UpdateKind, UpdateKind.NONE) => l - case (l: UpdateKind, r: UpdateKind) if l == r => l - // UNION doesn't support to union ONLY_UPDATE_AFTER and BEFORE_AND_AFTER inputs - case (_, _) => return None - } - } - new UpdateKindTrait(merged) - } - createNewNode(union, Some(children.flatten), providedTrait) - } - - case normalize: StreamPhysicalChangelogNormalize => - // changelog normalize currently only supports input only sending UPDATE_AFTER - val children = visitChildren(normalize, UpdateKindTrait.ONLY_UPDATE_AFTER) - // use requiredTrait as providedTrait, - // because changelog normalize supports all kinds of UpdateKind - createNewNode(rel, children, requiredUpdateTrait) - - case ts: StreamPhysicalTableSourceScan => - // currently only support BEFORE_AND_AFTER if source produces updates - val providedTrait = UpdateKindTrait.fromChangelogMode(ts.tableSource.getChangelogMode) - val newSource = createNewNode(rel, Some(List()), providedTrait) - if ( - providedTrait.equals(UpdateKindTrait.BEFORE_AND_AFTER) && - requiredUpdateTrait.equals(UpdateKindTrait.ONLY_UPDATE_AFTER) - ) { - // requiring only-after, but the source is CDC source, then drop update_before manually - val dropUB = new StreamPhysicalDropUpdateBefore(rel.getCluster, rel.getTraitSet, rel) - createNewNode(dropUB, newSource.map(s => List(s)), requiredUpdateTrait) - } else { - newSource - } - - case _: StreamPhysicalDataStreamScan | _: StreamPhysicalLegacyTableSourceScan | - _: StreamPhysicalValues => - createNewNode(rel, Some(List()), UpdateKindTrait.NONE) - - case scan: StreamPhysicalIntermediateTableScan => - val providedTrait = if (scan.intermediateTable.isUpdateBeforeRequired) { - // we can't drop UPDATE_BEFORE if it is required by other parent blocks - UpdateKindTrait.BEFORE_AND_AFTER - } else { - requiredUpdateTrait - } - if (!providedTrait.satisfies(requiredUpdateTrait)) { - // require ONLY_AFTER but can only provide BEFORE_AND_AFTER - None - } else { - createNewNode(rel, Some(List()), providedTrait) - } - - case process: StreamPhysicalProcessTableFunction => - // Required update traits depend on the table argument declaration, - // input traits, partition keys, and upsert keys - val inputArgs = StreamPhysicalProcessTableFunction - .getProvidedInputArgs(process.getCall) - val children = process.getInputs - .map(_.asInstanceOf[StreamPhysicalRel]) - .zipWithIndex - .map { - case (child, inputIndex) => - // For PTF without table arguments (i.e. values child) - if (inputArgs.isEmpty) { - this.visit(child, UpdateKindTrait.NONE) - } - // Derive the required update trait for table arguments - else { - val inputArg = inputArgs.get(inputIndex) - val (tableArg, tableArgCall, modifyKindSet) = - extractPtfTableArgComponents(process, child, inputArg) - val requiredUpdateTrait = - if ( - !modifyKindSet.isInsertOnly && tableArg.is( - StaticArgumentTrait.SUPPORT_UPDATES) - ) { - if (ptfRequiresUpdateBefore(tableArg, tableArgCall, child)) { - UpdateKindTrait.BEFORE_AND_AFTER - } else { - UpdateKindTrait.ONLY_UPDATE_AFTER - } - } else { - UpdateKindTrait.NONE - } - this.visit(child, requiredUpdateTrait) - } - } - .toList - .flatten - // Query PTF for upsert vs. retract - val providedUpdateTrait = queryPtfChangelogMode( - process, - children, - toChangelogMode(process, Some(requiredUpdateTrait), None), - UpdateKindTrait.fromChangelogMode, - UpdateKindTrait.NONE) - createNewNode(rel, Some(children), providedUpdateTrait) - - case multiJoin: StreamPhysicalMultiJoin => - val onlyAfterByParent = requiredUpdateTrait.updateKind == UpdateKind.ONLY_UPDATE_AFTER - val children = multiJoin.getInputs.zipWithIndex.map { - case (child, childOrdinal) => - val physicalChild = child.asInstanceOf[StreamPhysicalRel] - val supportOnlyAfter = multiJoin.inputUniqueKeyContainsCommonJoinKey(childOrdinal) - val inputModifyKindSet = getModifyKindSet(physicalChild) - if (onlyAfterByParent) { - if (inputModifyKindSet.contains(ModifyKind.UPDATE) && !supportOnlyAfter) { - // the parent requires only-after, however, the multi-join doesn't support this for this input - None - } else { - this.visit(physicalChild, onlyAfterOrNone(inputModifyKindSet)) - } - } else { - this.visit(physicalChild, beforeAfterOrNone(inputModifyKindSet)) - } - } - - if (children.exists(_.isEmpty)) { - None - } else { - createNewNode(multiJoin, Some(children.flatten.toList), requiredUpdateTrait) - } - - case _ => - throw new UnsupportedOperationException( - s"Unsupported visit for ${rel.getClass.getSimpleName}") - - } - - private def visitChildren( - parent: StreamPhysicalRel, - requiredChildrenUpdateTrait: UpdateKindTrait): Option[List[StreamPhysicalRel]] = { - val newChildren = for (child <- parent.getInputs) yield { - this.visit(child.asInstanceOf[StreamPhysicalRel], requiredChildrenUpdateTrait) match { - case None => - // return None if one of the children can't satisfy - return None - case Some(newChild) => - val providedUpdateTrait = newChild.getTraitSet.getTrait(UpdateKindTraitDef.INSTANCE) - if (!providedUpdateTrait.satisfies(requiredChildrenUpdateTrait)) { - // the provided trait can't satisfy required trait, thus we should return None. - return None - } - newChild - } - } - Some(newChildren.toList) - } - - private def createNewNode( - node: StreamPhysicalRel, - childrenOption: Option[List[StreamPhysicalRel]], - providedUpdateTrait: UpdateKindTrait): Option[StreamPhysicalRel] = childrenOption match { - case None => - None - case Some(children) => - val modifyKindSetTrait = node.getTraitSet.getTrait(ModifyKindSetTraitDef.INSTANCE) - val nodeDescription = node.getRelDetailedDescription - val isUpdateKindValid = providedUpdateTrait.updateKind match { - case UpdateKind.NONE => - !modifyKindSetTrait.modifyKindSet.contains(ModifyKind.UPDATE) - case UpdateKind.BEFORE_AND_AFTER | UpdateKind.ONLY_UPDATE_AFTER => - modifyKindSetTrait.modifyKindSet.contains(ModifyKind.UPDATE) - } - if (!isUpdateKindValid) { - throw new TableException( - s"UpdateKindTrait $providedUpdateTrait conflicts with " + - s"ModifyKindSetTrait $modifyKindSetTrait. " + - s"This is a bug in planner, please file an issue. \n" + - s"Current node is $nodeDescription.") - } - - val newTraitSet = node.getTraitSet.plus(providedUpdateTrait) - Some(node.copy(newTraitSet, children).asInstanceOf[StreamPhysicalRel]) - } - - /** - * Try all possible rank strategies and return the first viable new node. - * - * @param rankStrategies - * all possible supported rank strategy by current node - * @param requiredUpdateKindTrait - * the required UpdateKindTrait by parent of rank node - * @param applyRankStrategy - * a function to apply rank strategy to get a new copied rank node - */ - private def visitRankStrategies( - rankStrategies: Seq[RankProcessStrategy], - requiredUpdateKindTrait: UpdateKindTrait, - applyRankStrategy: RankProcessStrategy => StreamPhysicalRel): Option[StreamPhysicalRel] = { - // go pass every RankProcessStrategy, apply the rank strategy to get a new copied rank node, - // return the first satisfied converted node - for (strategy <- rankStrategies) { - val requiredChildrenTrait = strategy match { - case _: UpdateFastStrategy => UpdateKindTrait.ONLY_UPDATE_AFTER - case _: RetractStrategy => UpdateKindTrait.BEFORE_AND_AFTER - case _: AppendFastStrategy => UpdateKindTrait.NONE - } - val node = applyRankStrategy(strategy) - val children = visitChildren(node, requiredChildrenTrait) - val newNode = createNewNode(node, children, requiredUpdateKindTrait) - if (newNode.isDefined) { - return newNode - } - } - None - } - - private def visitSink( - sink: StreamPhysicalRel, - sinkRequiredTraits: Seq[UpdateKindTrait]): Option[StreamPhysicalRel] = { - val children = sinkRequiredTraits.flatMap(t => visitChildren(sink, t)) - if (children.isEmpty) { - None - } else { - val sinkTrait = sink.getTraitSet.plus(UpdateKindTrait.NONE) - Some(sink.copy(sinkTrait, children.head).asInstanceOf[StreamPhysicalRel]) - } - } - - /** - * Infer sink required traits by the sink node and its input. Sink required traits is based on - * the sink node's changelog mode, the only exception is when sink's pk(s) are not satisfied by - * the input's upsert keys (considering immutable columns) and sink's changelog mode is - * ONLY_UPDATE_AFTER. - */ - private def inferSinkRequiredTraits(sink: StreamPhysicalSink): Seq[UpdateKindTrait] = { - val childModifyKindSet = getModifyKindSet(sink.getInput) - val onlyAfter = onlyAfterOrNone(childModifyKindSet) - val beforeAndAfter = beforeAfterOrNone(childModifyKindSet) - val sinkTrait = UpdateKindTrait.fromChangelogMode( - sink.tableSink.getChangelogMode(childModifyKindSet.toDefaultChangelogMode)) - - val sinkRequiredTraits = if (sinkTrait.equals(ONLY_UPDATE_AFTER)) { - // if sink's pk(s) are not satisfied by input upsert keys (considering immutable columns), - // fallback to beforeAndAfter mode for correctness - val requireBeforeAndAfter = !canUpsertKeysWithImmutableColsSatisfyPk(sink) - if (requireBeforeAndAfter) { - Seq(beforeAndAfter) - } else { - Seq(onlyAfter, beforeAndAfter) - } - } else if (sinkTrait.equals(BEFORE_AND_AFTER)) { - Seq(beforeAndAfter) - } else { - Seq(UpdateKindTrait.NONE) - } - sinkRequiredTraits - } - - /** - * Check whether input's upsert keys (together with immutable columns) can satisfy sink's - * primary keys. - * - *

A sink pk is considered "satisfied" when there exists an upsert key `uk` such that: - * - `uk` is a subset of sink pk (no extra columns that could cause key collision) - * - the remaining sink pk columns not in `uk` are all immutable (immutable columns never - * change, so they effectively act as part of the key for upsert semantics) - * - *

Example: sink pk = {a, b, c}, uk = {a, b}, immutable columns = {a, b, c, d}. - * - Step 1: uk {a, b} ⊆ sink pk {a, b, c} → true - * - Step 2: sink pk \ uk = {c}, immutable columns contain {c} → true - * - Result: satisfied - * - *

Notice: even if sink pk is a subset of the upsert key, the pk is NOT considered satisfied - * when the upsert key has columns outside sink pk. This differs from batch job's unique key - * inference. - */ - private def canUpsertKeysWithImmutableColsSatisfyPk(sink: StreamPhysicalSink): Boolean = { - val sinkDefinedPks = sink.contextResolvedTable.getResolvedSchema.getPrimaryKeyIndexes - if (sinkDefinedPks.isEmpty) { - return true - } - val sinkPks = ImmutableBitSet.of(sinkDefinedPks: _*) - val fmq = FlinkRelMetadataQuery.reuseOrCreate(sink.getCluster.getMetadataQuery) - val changeLogUpsertKeys = fmq.getUpsertKeys(sink.getInput) - // if upsert key is null, pk cannot be satisfied, should fall back to beforeAndAfter - if (changeLogUpsertKeys == null) { - return false - } - val immutableCols = - Option.apply(fmq.getImmutableColumns(sink.getInput)).getOrElse(ImmutableBitSet.of()) - - // when input immutableCols is empty, this degrades to uk.equals(sinkPks) - changeLogUpsertKeys.exists( - uk => { - // 1. uk ⊆ sinkPks - val isSinkPkContainsUk = sinkPks.contains(uk) - // 2. (sinkPks \ uk) ⊆ immutableCols - val extraSinkPkCols = sinkPks.except(uk) - val areExtraSinkPkColsImmutable = immutableCols.contains(extraSinkPkCols) - isSinkPkContainsUk && areExtraSinkPkColsImmutable - }) - } - - /** - * Analyze whether to enable upsertMaterialize or not. In these case will return true: - * 1. when `TABLE_EXEC_SINK_UPSERT_MATERIALIZE` set to FORCE and sink's primary key nonempty. - * 2. when `TABLE_EXEC_SINK_UPSERT_MATERIALIZE` set to AUTO and sink's primary key doesn't - * contain upsertKeys of the input update stream. - * - * Also validates that ON CONFLICT clause is specified when upsert key differs from primary key. - */ - private def analyzeUpsertMaterializeStrategy(sink: StreamPhysicalSink): Boolean = { - val tableConfig = unwrapTableConfig(sink) - val inputChangelogMode = - ChangelogPlanUtils.getChangelogMode(sink.getInput.asInstanceOf[StreamPhysicalRel]).get - val primaryKeys = sink.contextResolvedTable.getResolvedSchema.getPrimaryKeyIndexes - val sinkChangelogMode = sink.tableSink.getChangelogMode(inputChangelogMode) - val inputIsAppend = inputChangelogMode.containsOnly(RowKind.INSERT) - val sinkIsAppend = sinkChangelogMode.containsOnly(RowKind.INSERT) - val sinkIsRetract = sinkChangelogMode.contains(RowKind.UPDATE_BEFORE) - - // Validate ON CONFLICT is only allowed for upsert sinks - if (sink.conflictStrategy != null) { - val isUpsertSink = !sinkIsAppend && !sinkIsRetract - if (!isUpsertSink) { - val reason = if (sinkIsAppend) { - "it only accepts INSERT (append-only) changes" - } else { - "it requires UPDATE_BEFORE (retract mode)" - } - throw new ValidationException( - s"ON CONFLICT clause is only allowed for upsert sinks. " + - s"The sink '${sink.contextResolvedTable.getIdentifier.asSummaryString}' " + - s"is not an upsert sink because $reason.") - } - } - - // Validate that sources have watermarks when using ERROR or NOTHING strategy - if ( - sink.conflictStrategy != null && - (sink.conflictStrategy.getBehavior == ConflictBehavior.ERROR || - sink.conflictStrategy.getBehavior == ConflictBehavior.NOTHING) - ) { - validateSourcesHaveWatermarks(sink) - } - - tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE) match { - case UpsertMaterialize.FORCE => primaryKeys.nonEmpty && !sinkIsRetract - case UpsertMaterialize.NONE => false - case UpsertMaterialize.AUTO => - // if the sink is not an UPSERT sink (has no PK, or is an APPEND or RETRACT sink) - // we don't need to materialize results - if (primaryKeys.isEmpty || sinkIsAppend || sinkIsRetract) { - return false - } - - // For a DEDUPLICATE strategy and INSERT only input, we simply let the inserts be handled - // as UPSERT_AFTER and overwrite previous value - if (inputIsAppend && sink.isDeduplicateConflictStrategy) { - return false - } - - // if input has updates and primary key != upsert key we should enable upsertMaterialize. - // - // An optimize is: do not enable upsertMaterialize when sink pk(s) contains input - // changeLogUpsertKeys - val upsertKeyDiffersFromPk = !sink.primaryKeysContainsUpsertKey - - // Validate that ON CONFLICT is specified when upsert key differs from primary key - val requireOnConflict = - tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_REQUIRE_ON_CONFLICT) - if (requireOnConflict && upsertKeyDiffersFromPk && sink.conflictStrategy == null) { - val pkNames = sink.getPrimaryKeyNames - val upsertKeyNames = sink.getUpsertKeyNames - throw new ValidationException( - "The query has an upsert key that differs from the primary key of the sink table " + - s"'${sink.contextResolvedTable.getIdentifier.asSummaryString}'. " + - s"Primary key: $pkNames, upsert key: $upsertKeyNames. " + - "This can lead to non-deterministic results when multiple records with different " + - "upsert keys map to the same primary key. " + - "Please specify an ON CONFLICT clause to define how conflicts should be handled: " + - "ON CONFLICT DO DEDUPLICATE (update to the latest record, state intensive, since we" + - " need to keep the entire history), or " + - "ON CONFLICT DO ERROR (fail on conflict), or " + - "ON CONFLICT DO NOTHING (keep first record).") - } - - upsertKeyDiffersFromPk - } - } - - private def validateSourcesHaveWatermarks(sink: StreamPhysicalSink): Unit = { - val sourcesWithoutWatermarks = new java.util.ArrayList[String]() - collectSourcesWithoutWatermarks(sink.getInput, sourcesWithoutWatermarks) - if (!sourcesWithoutWatermarks.isEmpty) { - throw new ValidationException( - s"ON CONFLICT DO ${sink.conflictStrategy.getBehavior} requires all source " + - s"tables to define watermarks, but the following source(s) do not: " + - s"${sourcesWithoutWatermarks.mkString(", ")}. " + - s"Please add a WATERMARK declaration to these tables.") - } - } - - private def collectSourcesWithoutWatermarks( - rel: RelNode, - result: java.util.List[String]): Unit = { - rel match { - case ts: StreamPhysicalTableSourceScan => - val table = ts.getTable.unwrap(classOf[TableSourceTable]) - if ( - table != null && - table.contextResolvedTable.getResolvedSchema.getWatermarkSpecs.isEmpty - ) { - result.add(table.contextResolvedTable.getIdentifier.asSummaryString()) - } - case _ => - rel.getInputs.forEach(input => collectSourcesWithoutWatermarks(input, result)) - } - } - } - private def isNonUpsertKeyCondition(calc: StreamPhysicalCalcBase): Boolean = ChangelogModeInferenceUtils.isNonUpsertKeyCondition(calc) From 7314672e8b1338ad57892a9198ab7f569fd6b960 Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Wed, 3 Jun 2026 15:06:43 -0700 Subject: [PATCH 5/6] [FLINK-XXXXX][table] Move SatisfyModifyKindSetTraitVisitor to top level Java class --- .../SatisfyModifyKindSetTraitVisitor.java | 554 ++++++++++++++++++ .../FlinkChangelogModeInferenceProgram.scala | 483 --------------- 2 files changed, 554 insertions(+), 483 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java new file mode 100644 index 0000000000000..ee9bf782de4b6 --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java @@ -0,0 +1,554 @@ +/* + * 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.optimize.program; + +import org.apache.flink.legacy.table.sinks.AppendStreamTableSink; +import org.apache.flink.legacy.table.sinks.RetractStreamTableSink; +import org.apache.flink.legacy.table.sinks.StreamTableSink; +import org.apache.flink.legacy.table.sinks.UpsertStreamTableSink; +import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.legacy.sinks.TableSink; +import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCalcBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalChangelogNormalize; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalCorrelateBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalDataStreamScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExchange; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalExpand; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupTableAggregateBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalGroupWindowAggregateBase; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalIntermediateTableScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalIntervalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacySink; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLegacyTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLimit; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalLookupJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMLPredictTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMatch; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMiniBatchAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalMultiJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalOverAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalProcessTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonGroupAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalPythonOverAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRank; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSort; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSortLimit; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTableSourceScan; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalTemporalSort; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalUnion; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalValues; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalVectorSearchTableFunction; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWatermarkAssigner; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowAggregate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowDeduplicate; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowJoin; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowRank; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalWindowTableFunction; +import org.apache.flink.table.planner.plan.trait.ModifyKind; +import org.apache.flink.table.planner.plan.trait.ModifyKindSet; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTraitDef; +import org.apache.flink.table.planner.plan.utils.ChangelogPlanUtils; +import org.apache.flink.table.planner.plan.utils.RankUtil; +import org.apache.flink.table.planner.sinks.DataStreamTableSink; +import org.apache.flink.table.planner.utils.ShortcutUtils; +import org.apache.flink.table.runtime.operators.join.FlinkJoinType; +import org.apache.flink.table.types.inference.StaticArgument; +import org.apache.flink.table.types.inference.StaticArgumentTrait; + +import org.apache.calcite.linq4j.Ord; +import org.apache.calcite.plan.RelTraitSet; +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.rel.core.JoinRelType; +import org.apache.calcite.rel.type.RelDataType; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.getModifyKindSet; +import static org.apache.flink.table.planner.plan.optimize.program.ChangelogModeInferenceUtils.queryPtfChangelogMode; + +/** + * A visitor which will try to satisfy the required {@link ModifyKindSetTrait} from root. + * + *

After traversed by this visitor, every node should have a correct {@link ModifyKindSetTrait} + * or an exception should be thrown if the planner doesn't support to satisfy the required {@link + * ModifyKindSetTrait}. + */ +class SatisfyModifyKindSetTraitVisitor { + + /** + * Try to satisfy the required {@link ModifyKindSetTrait} from root. + * + *

Each node should first require a {@link ModifyKindSetTrait} to its children. If the trait + * provided by children does not satisfy the required one, it should throw an exception and + * prompt the user that plan is not supported. The required {@link ModifyKindSetTrait} may come + * from the node's parent, or come from the node itself, depending on whether the node will + * destroy the trait provided by children or pass the trait from children. + * + *

Each node should provide {@link ModifyKindSetTrait} according to current node's behavior + * and the ModifyKindSetTrait provided by children. + * + * @param rel the node who should satisfy the requiredTrait + * @param requiredTrait the required ModifyKindSetTrait + * @param requester the requester who starts the requirement, used for better exception message + * @return A converted node which satisfy required traits by inputs node of current node. Or + * throws exception if required trait can't be satisfied. + */ + StreamPhysicalRel visit( + StreamPhysicalRel rel, ModifyKindSetTrait requiredTrait, String requester) { + if (rel instanceof StreamPhysicalSink) { + StreamPhysicalSink sink = (StreamPhysicalSink) rel; + String name = + "Table sink '" + + sink.contextResolvedTable().getIdentifier().asSummaryString() + + "'"; + ChangelogMode queryModifyKindSet = deriveQueryDefaultChangelogMode(sink.getInput(), name); + ModifyKindSetTrait sinkRequiredTrait = + ModifyKindSetTrait.fromChangelogMode( + sink.tableSink().getChangelogMode(queryModifyKindSet)); + List children = visitChildren(sink, sinkRequiredTrait, name); + RelTraitSet sinkTrait = sink.getTraitSet().plus(ModifyKindSetTrait.EMPTY()); + // ignore required trait from context, because sink is the true root + return (StreamPhysicalRel) sink.copy(sinkTrait, new ArrayList(children)); + } else if (rel instanceof StreamPhysicalLegacySink) { + StreamPhysicalLegacySink legacySink = (StreamPhysicalLegacySink) rel; + TableSink tableSink = legacySink.sink(); + final ModifyKindSetTrait sinkRequiredTrait; + final String name; + if (tableSink instanceof UpsertStreamTableSink) { + sinkRequiredTrait = ModifyKindSetTrait.ALL_CHANGES(); + name = "UpsertStreamTableSink"; + } else if (tableSink instanceof RetractStreamTableSink) { + sinkRequiredTrait = ModifyKindSetTrait.ALL_CHANGES(); + name = "RetractStreamTableSink"; + } else if (tableSink instanceof AppendStreamTableSink) { + sinkRequiredTrait = ModifyKindSetTrait.INSERT_ONLY(); + name = "AppendStreamTableSink"; + } else if (tableSink instanceof StreamTableSink) { + sinkRequiredTrait = ModifyKindSetTrait.INSERT_ONLY(); + name = "StreamTableSink"; + } else if (tableSink instanceof DataStreamTableSink) { + DataStreamTableSink ds = (DataStreamTableSink) tableSink; + if (ds.withChangeFlag()) { + sinkRequiredTrait = ModifyKindSetTrait.ALL_CHANGES(); + name = "toRetractStream"; + } else { + sinkRequiredTrait = ModifyKindSetTrait.INSERT_ONLY(); + name = "toAppendStream"; + } + } else { + throw new UnsupportedOperationException( + "Unsupported sink '" + tableSink.getClass().getSimpleName() + "'"); + } + List children = visitChildren(legacySink, sinkRequiredTrait, name); + RelTraitSet sinkTrait = legacySink.getTraitSet().plus(ModifyKindSetTrait.EMPTY()); + // ignore required trait from context, because sink is the true root + return (StreamPhysicalRel) legacySink.copy(sinkTrait, new ArrayList(children)); + } else if (rel instanceof StreamPhysicalGroupAggregate) { + // agg support all changes in input + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + ModifyKindSet inputModifyKindSet = getModifyKindSet(children.get(0)); + ModifyKindSet.Builder builder = + ModifyKindSet.newBuilder() + .addContainedKind(ModifyKind.INSERT) + .addContainedKind(ModifyKind.UPDATE); + if (inputModifyKindSet.contains(ModifyKind.UPDATE) + || inputModifyKindSet.contains(ModifyKind.DELETE)) { + builder.addContainedKind(ModifyKind.DELETE); + } + ModifyKindSetTrait providedTrait = new ModifyKindSetTrait(builder.build()); + return createNewNode(rel, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalGroupTableAggregateBase) { + // table agg support all changes in input + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + // table aggregate will produce all changes, including deletions + return createNewNode( + rel, children, ModifyKindSetTrait.ALL_CHANGES(), requiredTrait, requester); + } else if (rel instanceof StreamPhysicalPythonGroupAggregate) { + // agg support all changes in input + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + ModifyKindSet inputModifyKindSet = getModifyKindSet(children.get(0)); + ModifyKindSet.Builder builder = + ModifyKindSet.newBuilder() + .addContainedKind(ModifyKind.INSERT) + .addContainedKind(ModifyKind.UPDATE); + if (inputModifyKindSet.contains(ModifyKind.UPDATE) + || inputModifyKindSet.contains(ModifyKind.DELETE)) { + builder.addContainedKind(ModifyKind.DELETE); + } + ModifyKindSetTrait providedTrait = new ModifyKindSetTrait(builder.build()); + return createNewNode(rel, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalGroupWindowAggregateBase) { + // WindowAggregate and WindowTableAggregate support all changes in input + StreamPhysicalGroupWindowAggregateBase window = + (StreamPhysicalGroupWindowAggregateBase) rel; + List children = visitChildren(window, ModifyKindSetTrait.ALL_CHANGES()); + ModifyKindSet.Builder builder = + ModifyKindSet.newBuilder().addContainedKind(ModifyKind.INSERT); + if (window.emitStrategy().produceUpdates()) { + builder.addContainedKind(ModifyKind.UPDATE); + } + ModifyKindSetTrait providedTrait = new ModifyKindSetTrait(builder.build()); + return createNewNode(window, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalWindowAggregate) { + // WindowAggregate and WindowTableAggregate support all changes in input + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + // TODO support early / late fire and then this node may produce update records + ModifyKindSetTrait providedTrait = new ModifyKindSetTrait(ModifyKindSet.INSERT_ONLY); + return createNewNode(rel, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalWindowRank + || rel instanceof StreamPhysicalWindowDeduplicate) { + // WindowAggregate, WindowRank, WindowDeduplicate support insert-only in input + List children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY()); + ModifyKindSetTrait providedTrait = ModifyKindSetTrait.INSERT_ONLY(); + return createNewNode(rel, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalRank) { + StreamPhysicalRank rank = (StreamPhysicalRank) rel; + if (RankUtil.isDeduplication(rank)) { + List children = + visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + TableConfig tableConfig = ShortcutUtils.unwrapTableConfig(rank); + + // if the rank is deduplication and can be executed as insert-only, forward that + // information + boolean insertOnly = children.stream().allMatch(ChangelogPlanUtils::isInsertOnly); + + final ModifyKindSetTrait providedTrait; + if (insertOnly + && RankUtil.outputInsertOnlyInDeduplicate( + tableConfig, + RankUtil.keepLastDeduplicateRow(rank.orderKey()))) { + // Deduplicate outputs append only if first row is kept and mini batching is + // disabled + providedTrait = ModifyKindSetTrait.INSERT_ONLY(); + } else { + providedTrait = ModifyKindSetTrait.ALL_CHANGES(); + } + return createNewNode(rel, children, providedTrait, requiredTrait, requester); + } else { + // Rank supports consuming all changes + List children = + visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + return createNewNode( + rel, children, ModifyKindSetTrait.ALL_CHANGES(), requiredTrait, requester); + } + } else if (rel instanceof StreamPhysicalLimit) { + // limit support all changes in input + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + ModifyKindSetTrait providedTrait = + getModifyKindSet(children.get(0)).isInsertOnly() + ? ModifyKindSetTrait.INSERT_ONLY() + : ModifyKindSetTrait.ALL_CHANGES(); + return createNewNode(rel, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalSortLimit) { + // SortLimit supports consuming all changes + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + return createNewNode( + rel, children, ModifyKindSetTrait.ALL_CHANGES(), requiredTrait, requester); + } else if (rel instanceof StreamPhysicalSort) { + // Sort supports consuming all changes + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + // Sort will buffer all inputs, and produce insert-only messages when input is finished + return createNewNode( + rel, children, ModifyKindSetTrait.INSERT_ONLY(), requiredTrait, requester); + } else if (rel instanceof StreamPhysicalMatch) { + // CEP only supports consuming insert-only and producing insert-only changes + // give a better requester name for exception message + List children = + visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY(), "Match Recognize"); + return createNewNode( + rel, children, ModifyKindSetTrait.INSERT_ONLY(), requiredTrait, requester); + } else if (rel instanceof StreamPhysicalOverAggregate) { + // OverAggregate can only support insert for row-time/proc-time sort keys + StreamPhysicalOverAggregate over = (StreamPhysicalOverAggregate) rel; + ModifyKindSetTrait overRequiredTrait = ModifyKindSetTrait.INSERT_ONLY(); + ModifyKindSet.Builder builder = + ModifyKindSet.newBuilder().addContainedKind(ModifyKind.INSERT); + List groups = + over.logicWindow().groups; + + if (!groups.isEmpty() && !groups.get(0).orderKeys.getFieldCollations().isEmpty()) { + // All aggregates are computed over the same window and order by is supported for + // only 1 field + int orderKeyIndex = groups.get(0).orderKeys.getFieldCollations().get(0).getFieldIndex(); + RelDataType orderKeyType = + over.logicWindow().getRowType().getFieldList().get(orderKeyIndex).getType(); + if (!FlinkTypeFactory.isRowtimeIndicatorType(orderKeyType) + && !FlinkTypeFactory.isProctimeIndicatorType(orderKeyType)) { + // Only non row-time/proc-time sort can support UPDATES + builder.addContainedKind(ModifyKind.UPDATE); + builder.addContainedKind(ModifyKind.DELETE); + overRequiredTrait = ModifyKindSetTrait.ALL_CHANGES(); + } + } + List children = visitChildren(over, overRequiredTrait); + ModifyKindSetTrait providedTrait = new ModifyKindSetTrait(builder.build()); + return createNewNode(over, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalTemporalSort + || rel instanceof StreamPhysicalIntervalJoin + || rel instanceof StreamPhysicalPythonOverAggregate) { + // TemporalSort, IntervalJoin only support consuming insert-only + // and producing insert-only changes + List children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY()); + return createNewNode( + rel, children, ModifyKindSetTrait.INSERT_ONLY(), requiredTrait, requester); + } else if (rel instanceof StreamPhysicalMLPredictTableFunction + || rel instanceof StreamPhysicalVectorSearchTableFunction) { + // MLPredict, VectorSearch supports only support consuming insert-only + List children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY()); + return createNewNode( + rel, children, ModifyKindSetTrait.INSERT_ONLY(), requiredTrait, requester); + } else if (rel instanceof StreamPhysicalJoin) { + // join support all changes in input + StreamPhysicalJoin join = (StreamPhysicalJoin) rel; + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + ModifyKindSet leftKindSet = getModifyKindSet(children.get(0)); + ModifyKindSet rightKindSet = getModifyKindSet(children.get(children.size() - 1)); + boolean innerOrSemi = + join.joinSpec().getJoinType() == FlinkJoinType.INNER + || join.joinSpec().getJoinType() == FlinkJoinType.SEMI; + final ModifyKindSetTrait providedTrait; + if (innerOrSemi) { + // forward left and right modify operations + providedTrait = new ModifyKindSetTrait(leftKindSet.union(rightKindSet)); + } else { + // otherwise, it may produce any kinds of changes + providedTrait = ModifyKindSetTrait.ALL_CHANGES(); + } + return createNewNode(join, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalWindowJoin) { + // Currently, window join only supports INSERT_ONLY in input + List children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY()); + return createNewNode( + rel, children, ModifyKindSetTrait.INSERT_ONLY(), requiredTrait, requester); + } else if (rel instanceof StreamPhysicalTemporalJoin) { + // currently, temporal join supports all kings of changes, including right side + StreamPhysicalTemporalJoin temporalJoin = (StreamPhysicalTemporalJoin) rel; + List children = + visitChildren(temporalJoin, ModifyKindSetTrait.ALL_CHANGES()); + // forward left input changes + ModifyKindSetTrait leftTrait = + children.get(0).getTraitSet().getTrait(ModifyKindSetTraitDef.INSTANCE()); + return createNewNode(temporalJoin, children, leftTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalMultiJoin) { + // multi-join supports all changes in input + StreamPhysicalMultiJoin multiJoin = (StreamPhysicalMultiJoin) rel; + List children = + visitChildren(multiJoin, ModifyKindSetTrait.ALL_CHANGES()); + boolean allInnerJoins = + multiJoin.getJoinTypes().stream().allMatch(t -> t == JoinRelType.INNER); + final ModifyKindSetTrait providedTrait; + if (allInnerJoins) { + // if all are inner joins, forward all modify operations from children + ModifyKindSet[] kindSets = + children.stream() + .map(ChangelogModeInferenceUtils::getModifyKindSet) + .toArray(ModifyKindSet[]::new); + providedTrait = new ModifyKindSetTrait(ModifyKindSet.union(kindSets)); + } else { + // if there is any outer join, it may produce any kinds of changes + providedTrait = ModifyKindSetTrait.ALL_CHANGES(); + } + return createNewNode(multiJoin, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalCalcBase + || rel instanceof StreamPhysicalCorrelateBase + || rel instanceof StreamPhysicalLookupJoin + || rel instanceof StreamPhysicalExchange + || rel instanceof StreamPhysicalExpand + || rel instanceof StreamPhysicalMiniBatchAssigner + || rel instanceof StreamPhysicalWatermarkAssigner + || rel instanceof StreamPhysicalWindowTableFunction) { + // transparent forward requiredTrait to children + List children = visitChildren(rel, requiredTrait, requester); + ModifyKindSetTrait childrenTrait = + children.get(0).getTraitSet().getTrait(ModifyKindSetTraitDef.INSTANCE()); + // forward children mode + return createNewNode(rel, children, childrenTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalUnion) { + // transparent forward requiredTrait to children + StreamPhysicalUnion union = (StreamPhysicalUnion) rel; + List children = visitChildren(rel, requiredTrait, requester); + // union provides all possible kinds of children have + ModifyKindSet[] kindSets = + children.stream() + .map(ChangelogModeInferenceUtils::getModifyKindSet) + .toArray(ModifyKindSet[]::new); + ModifyKindSet providedKindSet = ModifyKindSet.union(kindSets); + return createNewNode( + union, + children, + new ModifyKindSetTrait(providedKindSet), + requiredTrait, + requester); + } else if (rel instanceof StreamPhysicalChangelogNormalize) { + // changelog normalize support update&delete input + List children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES()); + // changelog normalize will output all changes + ModifyKindSetTrait providedTrait = ModifyKindSetTrait.ALL_CHANGES(); + return createNewNode(rel, children, providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalTableSourceScan) { + // ScanTableSource supports produces updates and deletions + StreamPhysicalTableSourceScan ts = (StreamPhysicalTableSourceScan) rel; + ModifyKindSetTrait providedTrait = + ModifyKindSetTrait.fromChangelogMode(ts.tableSource().getChangelogMode()); + return createNewNode( + ts, Collections.emptyList(), providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalDataStreamScan + || rel instanceof StreamPhysicalLegacyTableSourceScan + || rel instanceof StreamPhysicalValues) { + // DataStream, TableSource and Values only support producing insert-only messages + return createNewNode( + rel, + Collections.emptyList(), + ModifyKindSetTrait.INSERT_ONLY(), + requiredTrait, + requester); + } else if (rel instanceof StreamPhysicalIntermediateTableScan) { + StreamPhysicalIntermediateTableScan scan = (StreamPhysicalIntermediateTableScan) rel; + ModifyKindSetTrait providedTrait = + new ModifyKindSetTrait(scan.intermediateTable().modifyKindSet()); + return createNewNode( + scan, Collections.emptyList(), providedTrait, requiredTrait, requester); + } else if (rel instanceof StreamPhysicalProcessTableFunction) { + // Accepted changes depend on table argument declaration + StreamPhysicalProcessTableFunction process = (StreamPhysicalProcessTableFunction) rel; + List> inputArgs = + StreamPhysicalProcessTableFunction.getProvidedInputArgs(process.getCall()); + List requiredChildrenTraits = new ArrayList<>(); + for (Ord inputArg : inputArgs) { + StaticArgument tableArg = inputArg.e; + requiredChildrenTraits.add( + tableArg.is(StaticArgumentTrait.SUPPORT_UPDATES) + ? ModifyKindSetTrait.ALL_CHANGES() + : ModifyKindSetTrait.INSERT_ONLY()); + } + final List children; + if (requiredChildrenTraits.isEmpty()) { + // Constant function has a single StreamPhysicalValues input + children = visitChildren(process, ModifyKindSetTrait.INSERT_ONLY()); + } else { + children = visitChildren(process, requiredChildrenTraits); + } + // Query PTF for updating vs. non-updating + ModifyKindSetTrait providedModifyTrait = + queryPtfChangelogMode( + process, + children, + requiredTrait.modifyKindSet().toChangelogModeBuilder().build(), + ModifyKindSetTrait::fromChangelogMode, + ModifyKindSetTrait.INSERT_ONLY()); + return createNewNode(process, children, providedModifyTrait, requiredTrait, requester); + } else { + throw new UnsupportedOperationException( + "Unsupported visit for " + rel.getClass().getSimpleName()); + } + } + + private List visitChildren( + StreamPhysicalRel parent, ModifyKindSetTrait requiredChildrenTrait) { + return visitChildren(parent, requiredChildrenTrait, getNodeName(parent)); + } + + private List visitChildren( + StreamPhysicalRel parent, ModifyKindSetTrait requiredChildrenTrait, String requester) { + List newChildren = new ArrayList<>(); + for (int i = 0; i < parent.getInputs().size(); i++) { + newChildren.add(visitChild(parent, i, requiredChildrenTrait, requester)); + } + return newChildren; + } + + private List visitChildren( + StreamPhysicalRel parent, List requiredChildrenTraits) { + String requester = getNodeName(parent); + List newChildren = new ArrayList<>(); + for (int i = 0; i < parent.getInputs().size(); i++) { + newChildren.add(visitChild(parent, i, requiredChildrenTraits.get(i), requester)); + } + return newChildren; + } + + private StreamPhysicalRel visitChild( + StreamPhysicalRel parent, + int childOrdinal, + ModifyKindSetTrait requiredChildTrait, + String requester) { + StreamPhysicalRel child = (StreamPhysicalRel) parent.getInput(childOrdinal); + return visit(child, requiredChildTrait, requester); + } + + private static String getNodeName(StreamPhysicalRel rel) { + String prefix = "StreamExec"; + String typeName = rel.getRelTypeName(); + if (typeName.startsWith(prefix)) { + return typeName.substring(prefix.length()); + } else { + return typeName; + } + } + + /** Derives the {@link ModifyKindSetTrait} of query plan without required ModifyKindSet validation. */ + private ChangelogMode deriveQueryDefaultChangelogMode(RelNode queryNode, String name) { + StreamPhysicalRel newNode = + visit((StreamPhysicalRel) queryNode, ModifyKindSetTrait.ALL_CHANGES(), name); + return getModifyKindSet(newNode).toDefaultChangelogMode(); + } + + private StreamPhysicalRel createNewNode( + StreamPhysicalRel node, + List children, + ModifyKindSetTrait providedTrait, + ModifyKindSetTrait requiredTrait, + String requestedOwner) { + if (!providedTrait.satisfies(requiredTrait)) { + ModifyKindSet diff = providedTrait.modifyKindSet().minus(requiredTrait.modifyKindSet()); + // for deterministic error message + List sortedKinds = new ArrayList<>(diff.getContainedKinds()); + Collections.sort(sortedKinds); + String diffString = + sortedKinds.stream() + .map(kind -> kind.toString().toLowerCase()) + .collect(Collectors.joining(" and ")); + // creates a new node based on the new children, to have a more correct node description + // e.g. description of GroupAggregate is based on the ModifyKindSetTrait of children + StreamPhysicalRel tempNode = + (StreamPhysicalRel) node.copy(node.getTraitSet(), new ArrayList(children)); + String nodeString = tempNode.getRelDetailedDescription(); + throw new TableException( + requestedOwner + + " doesn't support consuming " + + diffString + + " changes which is produced by node " + + nodeString); + } + RelTraitSet newTraitSet = node.getTraitSet().plus(providedTrait); + return (StreamPhysicalRel) node.copy(newTraitSet, new ArrayList(children)); + } +} diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala index 683fa8661f065..488e3e4f3e595 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala @@ -17,23 +17,12 @@ */ package org.apache.flink.table.planner.plan.optimize.program -import org.apache.flink.legacy.table.sinks.{AppendStreamTableSink, RetractStreamTableSink, StreamTableSink, UpsertStreamTableSink} import org.apache.flink.table.api.TableException -import org.apache.flink.table.connector.ChangelogMode -import org.apache.flink.table.planner.calcite.{FlinkTypeFactory, RexTableArgCall} import org.apache.flink.table.planner.plan.`trait`._ import org.apache.flink.table.planner.plan.nodes.physical.stream._ import org.apache.flink.table.planner.plan.utils._ -import org.apache.flink.table.planner.sinks.DataStreamTableSink -import org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTableConfig -import org.apache.flink.table.runtime.operators.join.FlinkJoinType -import org.apache.flink.table.types.inference.{StaticArgument, StaticArgumentTrait} -import org.apache.calcite.linq4j.Ord import org.apache.calcite.rel.RelNode -import org.apache.calcite.rel.core.JoinRelType - -import scala.collection.JavaConversions._ /** An optimize program to infer ChangelogMode for every physical node. */ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOptimizeContext] { @@ -108,478 +97,6 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti } } - /** - * A visitor which will try to satisfy the required [[ModifyKindSetTrait]] from root. - * - *

After traversed by this visitor, every node should have a correct [[ModifyKindSetTrait]] or - * an exception should be thrown if the planner doesn't support to satisfy the required - * [[ModifyKindSetTrait]]. - */ - private class SatisfyModifyKindSetTraitVisitor { - - /** - * Try to satisfy the required [[ModifyKindSetTrait]] from root. - * - *

Each node should first require a [[ModifyKindSetTrait]] to its children. If the trait - * provided by children does not satisfy the required one, it should throw an exception and - * prompt the user that plan is not supported. The required [[ModifyKindSetTrait]] may come from - * the node's parent, or come from the node itself, depending on whether the node will destroy - * the trait provided by children or pass the trait from children. - * - *

Each node should provide [[ModifyKindSetTrait]] according to current node's behavior and - * the ModifyKindSetTrait provided by children. - * - * @param rel - * the node who should satisfy the requiredTrait - * @param requiredTrait - * the required ModifyKindSetTrait - * @param requester - * the requester who starts the requirement, used for better exception message - * @return - * A converted node which satisfy required traits by inputs node of current node. Or throws - * exception if required trait can’t be satisfied. - */ - def visit( - rel: StreamPhysicalRel, - requiredTrait: ModifyKindSetTrait, - requester: String): StreamPhysicalRel = rel match { - case sink: StreamPhysicalSink => - val name = s"Table sink '${sink.contextResolvedTable.getIdentifier.asSummaryString()}'" - val queryModifyKindSet = deriveQueryDefaultChangelogMode(sink.getInput, name) - val sinkRequiredTrait = - ModifyKindSetTrait.fromChangelogMode(sink.tableSink.getChangelogMode(queryModifyKindSet)) - val children = visitChildren(sink, sinkRequiredTrait, name) - val sinkTrait = sink.getTraitSet.plus(ModifyKindSetTrait.EMPTY) - // ignore required trait from context, because sink is the true root - sink.copy(sinkTrait, children).asInstanceOf[StreamPhysicalRel] - - case sink: StreamPhysicalLegacySink[_] => - val (sinkRequiredTrait, name) = sink.sink match { - case _: UpsertStreamTableSink[_] => - (ModifyKindSetTrait.ALL_CHANGES, "UpsertStreamTableSink") - case _: RetractStreamTableSink[_] => - (ModifyKindSetTrait.ALL_CHANGES, "RetractStreamTableSink") - case _: AppendStreamTableSink[_] => - (ModifyKindSetTrait.INSERT_ONLY, "AppendStreamTableSink") - case _: StreamTableSink[_] => - (ModifyKindSetTrait.INSERT_ONLY, "StreamTableSink") - case ds: DataStreamTableSink[_] => - if (ds.withChangeFlag) { - (ModifyKindSetTrait.ALL_CHANGES, "toRetractStream") - } else { - (ModifyKindSetTrait.INSERT_ONLY, "toAppendStream") - } - case _ => - throw new UnsupportedOperationException( - s"Unsupported sink '${sink.sink.getClass.getSimpleName}'") - } - val children = visitChildren(sink, sinkRequiredTrait, name) - val sinkTrait = sink.getTraitSet.plus(ModifyKindSetTrait.EMPTY) - // ignore required trait from context, because sink is the true root - sink.copy(sinkTrait, children).asInstanceOf[StreamPhysicalRel] - - case agg: StreamPhysicalGroupAggregate => - // agg support all changes in input - val children = visitChildren(agg, ModifyKindSetTrait.ALL_CHANGES) - val inputModifyKindSet = getModifyKindSet(children.head) - val builder = ModifyKindSet - .newBuilder() - .addContainedKind(ModifyKind.INSERT) - .addContainedKind(ModifyKind.UPDATE) - if ( - inputModifyKindSet.contains(ModifyKind.UPDATE) || - inputModifyKindSet.contains(ModifyKind.DELETE) - ) { - builder.addContainedKind(ModifyKind.DELETE) - } - val providedTrait = new ModifyKindSetTrait(builder.build()) - createNewNode(agg, children, providedTrait, requiredTrait, requester) - - case tagg: StreamPhysicalGroupTableAggregateBase => - // table agg support all changes in input - val children = visitChildren(tagg, ModifyKindSetTrait.ALL_CHANGES) - // table aggregate will produce all changes, including deletions - createNewNode(tagg, children, ModifyKindSetTrait.ALL_CHANGES, requiredTrait, requester) - - case agg: StreamPhysicalPythonGroupAggregate => - // agg support all changes in input - val children = visitChildren(agg, ModifyKindSetTrait.ALL_CHANGES) - val inputModifyKindSet = getModifyKindSet(children.head) - val builder = ModifyKindSet - .newBuilder() - .addContainedKind(ModifyKind.INSERT) - .addContainedKind(ModifyKind.UPDATE) - if ( - inputModifyKindSet.contains(ModifyKind.UPDATE) || - inputModifyKindSet.contains(ModifyKind.DELETE) - ) { - builder.addContainedKind(ModifyKind.DELETE) - } - val providedTrait = new ModifyKindSetTrait(builder.build()) - createNewNode(agg, children, providedTrait, requiredTrait, requester) - - case window: StreamPhysicalGroupWindowAggregateBase => - // WindowAggregate and WindowTableAggregate support all changes in input - val children = visitChildren(window, ModifyKindSetTrait.ALL_CHANGES) - val builder = ModifyKindSet - .newBuilder() - .addContainedKind(ModifyKind.INSERT) - if (window.emitStrategy.produceUpdates) { - builder.addContainedKind(ModifyKind.UPDATE) - } - val providedTrait = new ModifyKindSetTrait(builder.build()) - createNewNode(window, children, providedTrait, requiredTrait, requester) - - case window: StreamPhysicalWindowAggregate => - // WindowAggregate and WindowTableAggregate support all changes in input - val children = visitChildren(window, ModifyKindSetTrait.ALL_CHANGES) - // TODO support early / late fire and then this node may produce update records - val providedTrait = new ModifyKindSetTrait(ModifyKindSet.INSERT_ONLY) - createNewNode(window, children, providedTrait, requiredTrait, requester) - - case _: StreamPhysicalWindowRank | _: StreamPhysicalWindowDeduplicate => - // WindowAggregate, WindowRank, WindowDeduplicate support insert-only in input - val children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY) - val providedTrait = ModifyKindSetTrait.INSERT_ONLY - createNewNode(rel, children, providedTrait, requiredTrait, requester) - - case rank: StreamPhysicalRank if RankUtil.isDeduplication(rank) => - val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) - val tableConfig = unwrapTableConfig(rank) - - // if the rank is deduplication and can be executed as insert-only, forward that information - val insertOnly = children.forall(ChangelogPlanUtils.isInsertOnly) - - val providedTrait = { - if ( - insertOnly && RankUtil.outputInsertOnlyInDeduplicate( - tableConfig, - RankUtil.keepLastDeduplicateRow(rank.orderKey)) - ) { - // Deduplicate outputs append only if first row is kept and mini batching is disabled - ModifyKindSetTrait.INSERT_ONLY - } else { - ModifyKindSetTrait.ALL_CHANGES - } - } - - createNewNode(rel, children, providedTrait, requiredTrait, requester) - - case rank: StreamPhysicalRank if !RankUtil.isDeduplication(rank) => - // Rank supports consuming all changes - val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) - createNewNode(rel, children, ModifyKindSetTrait.ALL_CHANGES, requiredTrait, requester) - - case limit: StreamPhysicalLimit => - // limit support all changes in input - val children = visitChildren(limit, ModifyKindSetTrait.ALL_CHANGES) - val providedTrait = if (getModifyKindSet(children.head).isInsertOnly) { - ModifyKindSetTrait.INSERT_ONLY - } else { - ModifyKindSetTrait.ALL_CHANGES - } - createNewNode(limit, children, providedTrait, requiredTrait, requester) - - case _: StreamPhysicalSortLimit => - // SortLimit supports consuming all changes - val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) - createNewNode(rel, children, ModifyKindSetTrait.ALL_CHANGES, requiredTrait, requester) - - case sort: StreamPhysicalSort => - // Sort supports consuming all changes - val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) - // Sort will buffer all inputs, and produce insert-only messages when input is finished - createNewNode(sort, children, ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) - - case cep: StreamPhysicalMatch => - // CEP only supports consuming insert-only and producing insert-only changes - // give a better requester name for exception message - val children = visitChildren(cep, ModifyKindSetTrait.INSERT_ONLY, "Match Recognize") - createNewNode(cep, children, ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) - - case over: StreamPhysicalOverAggregate => - // OverAggregate can only support insert for row-time/proc-time sort keys - var overRequiredTrait = ModifyKindSetTrait.INSERT_ONLY - val builder = ModifyKindSet - .newBuilder() - .addContainedKind(ModifyKind.INSERT) - val groups = over.logicWindow.groups - - if (!groups.isEmpty && !groups.get(0).orderKeys.getFieldCollations.isEmpty) { - // All aggregates are computed over the same window and order by is supported for only 1 field - val orderKeyIndex = groups.get(0).orderKeys.getFieldCollations.get(0).getFieldIndex - val orderKeyType = over.logicWindow.getRowType.getFieldList.get(orderKeyIndex).getType - if ( - !FlinkTypeFactory.isRowtimeIndicatorType(orderKeyType) - && !FlinkTypeFactory.isProctimeIndicatorType(orderKeyType) - ) { - // Only non row-time/proc-time sort can support UPDATES - builder.addContainedKind(ModifyKind.UPDATE) - builder.addContainedKind(ModifyKind.DELETE) - overRequiredTrait = ModifyKindSetTrait.ALL_CHANGES - } - } - val children = visitChildren(over, overRequiredTrait) - val providedTrait = new ModifyKindSetTrait(builder.build()) - createNewNode(over, children, providedTrait, requiredTrait, requester) - - case _: StreamPhysicalTemporalSort | _: StreamPhysicalIntervalJoin | - _: StreamPhysicalPythonOverAggregate => - // TemporalSort, IntervalJoin only support consuming insert-only - // and producing insert-only changes - val children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY) - createNewNode(rel, children, ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) - - case _: StreamPhysicalMLPredictTableFunction | _: StreamPhysicalVectorSearchTableFunction => - // MLPredict, VectorSearch supports only support consuming insert-only - val children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY) - createNewNode(rel, children, ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) - - case join: StreamPhysicalJoin => - // join support all changes in input - val children = visitChildren(rel, ModifyKindSetTrait.ALL_CHANGES) - val leftKindSet = getModifyKindSet(children.head) - val rightKindSet = getModifyKindSet(children.last) - val innerOrSemi = join.joinSpec.getJoinType == FlinkJoinType.INNER || - join.joinSpec.getJoinType == FlinkJoinType.SEMI - val providedTrait = if (innerOrSemi) { - // forward left and right modify operations - new ModifyKindSetTrait(leftKindSet.union(rightKindSet)) - } else { - // otherwise, it may produce any kinds of changes - ModifyKindSetTrait.ALL_CHANGES - } - createNewNode(join, children, providedTrait, requiredTrait, requester) - - case windowJoin: StreamPhysicalWindowJoin => - // Currently, window join only supports INSERT_ONLY in input - val children = visitChildren(rel, ModifyKindSetTrait.INSERT_ONLY) - createNewNode( - windowJoin, - children, - ModifyKindSetTrait.INSERT_ONLY, - requiredTrait, - requester) - - case temporalJoin: StreamPhysicalTemporalJoin => - // currently, temporal join supports all kings of changes, including right side - val children = visitChildren(temporalJoin, ModifyKindSetTrait.ALL_CHANGES) - // forward left input changes - val leftTrait = children.head.getTraitSet.getTrait(ModifyKindSetTraitDef.INSTANCE) - createNewNode(temporalJoin, children, leftTrait, requiredTrait, requester) - - case multiJoin: StreamPhysicalMultiJoin => - // multi-join supports all changes in input - val children = visitChildren(multiJoin, ModifyKindSetTrait.ALL_CHANGES) - val allInnerJoins = multiJoin.getJoinTypes.forall(_ == JoinRelType.INNER) - val providedTrait = if (allInnerJoins) { - // if all are inner joins, forward all modify operations from children - val kindSets = children.map(getModifyKindSet) - new ModifyKindSetTrait(ModifyKindSet.union(kindSets: _*)) - } else { - // if there is any outer join, it may produce any kinds of changes - ModifyKindSetTrait.ALL_CHANGES - } - createNewNode(multiJoin, children, providedTrait, requiredTrait, requester) - - case _: StreamPhysicalCalcBase | _: StreamPhysicalCorrelateBase | - _: StreamPhysicalLookupJoin | _: StreamPhysicalExchange | _: StreamPhysicalExpand | - _: StreamPhysicalMiniBatchAssigner | _: StreamPhysicalWatermarkAssigner | - _: StreamPhysicalWindowTableFunction => - // transparent forward requiredTrait to children - val children = visitChildren(rel, requiredTrait, requester) - val childrenTrait = children.head.getTraitSet.getTrait(ModifyKindSetTraitDef.INSTANCE) - // forward children mode - createNewNode(rel, children, childrenTrait, requiredTrait, requester) - - case union: StreamPhysicalUnion => - // transparent forward requiredTrait to children - val children = visitChildren(rel, requiredTrait, requester) - // union provides all possible kinds of children have - val providedKindSet = ModifyKindSet.union(children.map(getModifyKindSet): _*) - createNewNode( - union, - children, - new ModifyKindSetTrait(providedKindSet), - requiredTrait, - requester) - - case normalize: StreamPhysicalChangelogNormalize => - // changelog normalize support update&delete input - val children = visitChildren(normalize, ModifyKindSetTrait.ALL_CHANGES) - // changelog normalize will output all changes - val providedTrait = ModifyKindSetTrait.ALL_CHANGES - createNewNode(normalize, children, providedTrait, requiredTrait, requester) - - case ts: StreamPhysicalTableSourceScan => - // ScanTableSource supports produces updates and deletions - val providedTrait = ModifyKindSetTrait.fromChangelogMode(ts.tableSource.getChangelogMode) - createNewNode(ts, List(), providedTrait, requiredTrait, requester) - - case _: StreamPhysicalDataStreamScan | _: StreamPhysicalLegacyTableSourceScan | - _: StreamPhysicalValues => - // DataStream, TableSource and Values only support producing insert-only messages - createNewNode(rel, List(), ModifyKindSetTrait.INSERT_ONLY, requiredTrait, requester) - - case scan: StreamPhysicalIntermediateTableScan => - val providedTrait = new ModifyKindSetTrait(scan.intermediateTable.modifyKindSet) - createNewNode(scan, List(), providedTrait, requiredTrait, requester) - - case process: StreamPhysicalProcessTableFunction => - // Accepted changes depend on table argument declaration - val requiredChildrenTraits = StreamPhysicalProcessTableFunction - .getProvidedInputArgs(process.getCall) - .map(arg => arg.e) - .map( - arg => - if (arg.is(StaticArgumentTrait.SUPPORT_UPDATES)) { - ModifyKindSetTrait.ALL_CHANGES - } else { - ModifyKindSetTrait.INSERT_ONLY - }) - .toList - val children = if (requiredChildrenTraits.isEmpty) { - // Constant function has a single StreamPhysicalValues input - visitChildren(process, ModifyKindSetTrait.INSERT_ONLY) - } else { - visitChildren(process, requiredChildrenTraits) - } - // Query PTF for updating vs. non-updating - val providedModifyTrait = queryPtfChangelogMode( - process, - children, - requiredTrait.modifyKindSet.toChangelogModeBuilder.build(), - ModifyKindSetTrait.fromChangelogMode, - ModifyKindSetTrait.INSERT_ONLY) - createNewNode(process, children, providedModifyTrait, requiredTrait, requester) - - case _ => - throw new UnsupportedOperationException( - s"Unsupported visit for ${rel.getClass.getSimpleName}") - } - - private def visitChildren( - parent: StreamPhysicalRel, - requiredChildrenTrait: ModifyKindSetTrait): List[StreamPhysicalRel] = { - visitChildren(parent, requiredChildrenTrait, getNodeName(parent)) - } - - private def visitChildren( - parent: StreamPhysicalRel, - requiredChildrenTrait: ModifyKindSetTrait, - requester: String): List[StreamPhysicalRel] = { - val newChildren = for (i <- 0 until parent.getInputs.size()) yield { - visitChild(parent, i, requiredChildrenTrait, requester) - } - newChildren.toList - } - - private def visitChildren( - parent: StreamPhysicalRel, - requiredChildrenTraits: List[ModifyKindSetTrait]): List[StreamPhysicalRel] = { - val requester = getNodeName(parent) - val newChildren = for (i <- 0 until parent.getInputs.size()) yield { - visitChild(parent, i, requiredChildrenTraits(i), requester) - } - newChildren.toList - } - - private def visitChild( - parent: StreamPhysicalRel, - childOrdinal: Int, - requiredChildTrait: ModifyKindSetTrait, - requester: String): StreamPhysicalRel = { - val child = parent.getInput(childOrdinal).asInstanceOf[StreamPhysicalRel] - this.visit(child, requiredChildTrait, requester) - } - - private def getNodeName(rel: StreamPhysicalRel): String = { - val prefix = "StreamExec" - val typeName = rel.getRelTypeName - if (typeName.startsWith(prefix)) { - typeName.substring(prefix.length) - } else { - typeName - } - } - - /** - * Derives the [[ModifyKindSetTrait]] of query plan without required ModifyKindSet validation. - */ - private def deriveQueryDefaultChangelogMode(queryNode: RelNode, name: String): ChangelogMode = { - val newNode = - visit(queryNode.asInstanceOf[StreamPhysicalRel], ModifyKindSetTrait.ALL_CHANGES, name) - getModifyKindSet(newNode).toDefaultChangelogMode - } - - private def createNewNode( - node: StreamPhysicalRel, - children: List[StreamPhysicalRel], - providedTrait: ModifyKindSetTrait, - requiredTrait: ModifyKindSetTrait, - requestedOwner: String): StreamPhysicalRel = { - if (!providedTrait.satisfies(requiredTrait)) { - val diff = providedTrait.modifyKindSet.minus(requiredTrait.modifyKindSet) - val diffString = diff.getContainedKinds.toList.sorted // for deterministic error message - .map(_.toString.toLowerCase) - .mkString(" and ") - // creates a new node based on the new children, to have a more correct node description - // e.g. description of GroupAggregate is based on the ModifyKindSetTrait of children - val tempNode = node.copy(node.getTraitSet, children).asInstanceOf[StreamPhysicalRel] - val nodeString = tempNode.getRelDetailedDescription - throw new TableException( - s"$requestedOwner doesn't support consuming $diffString changes " + - s"which is produced by node $nodeString") - } - val newTraitSet = node.getTraitSet.plus(providedTrait) - node.copy(newTraitSet, children).asInstanceOf[StreamPhysicalRel] - } - } - - private def isNonUpsertKeyCondition(calc: StreamPhysicalCalcBase): Boolean = - ChangelogModeInferenceUtils.isNonUpsertKeyCondition(calc) - private def getModifyKindSet(node: RelNode): ModifyKindSet = ChangelogModeInferenceUtils.getModifyKindSet(node) - - // ---------------------------------------------------------------------------------------------- - // PTF helper methods - // ---------------------------------------------------------------------------------------------- - - private def toChangelogMode( - node: StreamPhysicalRel, - updateKindTrait: Option[UpdateKindTrait], - deleteKindTrait: Option[DeleteKindTrait]): ChangelogMode = - ChangelogModeInferenceUtils.toChangelogMode( - node, - updateKindTrait.orNull, - deleteKindTrait.orNull) - - private def ptfRequiresUpdateBefore( - tableArg: StaticArgument, - tableArgCall: RexTableArgCall, - input: StreamPhysicalRel): Boolean = - ChangelogModeInferenceUtils.ptfRequiresUpdateBefore(tableArg, tableArgCall, input) - - private def extractPtfTableArgComponents( - process: StreamPhysicalProcessTableFunction, - child: StreamPhysicalRel, - inputArg: Ord[StaticArgument]): (StaticArgument, RexTableArgCall, ModifyKindSet) = { - val components = - ChangelogModeInferenceUtils.extractPtfTableArgComponents(process, child, inputArg) - (components.tableArg, components.tableArgCall, components.modifyKindSet) - } - - private def queryPtfChangelogMode[T]( - process: StreamPhysicalProcessTableFunction, - children: List[StreamPhysicalRel], - requiredChangelogMode: ChangelogMode, - toTraitSet: ChangelogMode => T, - defaultTraitSet: T): T = - ChangelogModeInferenceUtils.queryPtfChangelogMode[T]( - process, - children, - requiredChangelogMode, - (mode: ChangelogMode) => toTraitSet(mode), - defaultTraitSet) } From 4aeb20cbd67a3d7b655f5aeebccef252262ec627 Mon Sep 17 00:00:00 2001 From: bvarghese1 Date: Wed, 3 Jun 2026 15:26:37 -0700 Subject: [PATCH 6/6] [FLINK-XXXXX][table] Port FlinkChangelogModeInferenceProgram to java --- .../program/ChangelogModeInferenceUtils.java | 17 ++- .../FlinkChangelogModeInferenceProgram.java | 135 ++++++++++++++++++ .../SatisfyDeleteKindTraitVisitor.java | 52 +++++-- .../SatisfyModifyKindSetTraitVisitor.java | 20 ++- .../SatisfyUpdateKindTraitVisitor.java | 97 ++++++++----- .../FlinkChangelogModeInferenceProgram.scala | 102 ------------- 6 files changed, 260 insertions(+), 163 deletions(-) create mode 100644 flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.java delete mode 100644 flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java index c3c9dc6caa92a..1582801da5271 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/ChangelogModeInferenceUtils.java @@ -215,15 +215,19 @@ static T queryPtfChangelogMode( if (definition instanceof ChangelogFunction) { ChangelogFunction changelogFunction = (ChangelogFunction) definition; ChangelogContext changelogContext = - toPtfChangelogContext(process, toInputChangelogModes(children), requiredChangelogMode); + toPtfChangelogContext( + process, toInputChangelogModes(children), requiredChangelogMode); ChangelogMode changelogMode = changelogFunction.getChangelogMode(changelogContext); verifyPtfTableArgsForUpdates(call, changelogMode); return toTraitSet.apply(changelogMode); } else if (definition instanceof BuiltInFunctionDefinition - && ((BuiltInFunctionDefinition) definition).getChangelogModeStrategy().isPresent()) { + && ((BuiltInFunctionDefinition) definition) + .getChangelogModeStrategy() + .isPresent()) { BuiltInFunctionDefinition builtIn = (BuiltInFunctionDefinition) definition; ChangelogContext changelogContext = - toPtfChangelogContext(process, toInputChangelogModes(children), requiredChangelogMode); + toPtfChangelogContext( + process, toInputChangelogModes(children), requiredChangelogMode); ChangelogMode changelogMode = builtIn.getChangelogModeStrategy().get().inferChangelogMode(changelogContext); verifyPtfTableArgsForUpdates(call, changelogMode); @@ -253,7 +257,8 @@ private static void verifyPtfTableArgsForUpdates(RexCall call, ChangelogMode cha || changelogMode.contains(RowKind.UPDATE_BEFORE)) { return; } - for (Ord inputArg : StreamPhysicalProcessTableFunction.getProvidedInputArgs(call)) { + for (Ord inputArg : + StreamPhysicalProcessTableFunction.getProvidedInputArgs(call)) { StaticArgument tableArg = inputArg.e; if (tableArg.is(StaticArgumentTrait.ROW_SEMANTIC_TABLE)) { throw new ValidationException( @@ -274,7 +279,9 @@ static final class PtfTableArgComponents { final ModifyKindSet modifyKindSet; PtfTableArgComponents( - StaticArgument tableArg, RexTableArgCall tableArgCall, ModifyKindSet modifyKindSet) { + StaticArgument tableArg, + RexTableArgCall tableArgCall, + ModifyKindSet modifyKindSet) { this.tableArg = tableArg; this.tableArgCall = tableArgCall; this.modifyKindSet = modifyKindSet; diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.java new file mode 100644 index 0000000000000..d50fb037ee26f --- /dev/null +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.java @@ -0,0 +1,135 @@ +/* + * 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.optimize.program; + +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalRel; +import org.apache.flink.table.planner.plan.nodes.physical.stream.StreamPhysicalSink; +import org.apache.flink.table.planner.plan.trait.DeleteKindTrait; +import org.apache.flink.table.planner.plan.trait.ModifyKind; +import org.apache.flink.table.planner.plan.trait.ModifyKindSet; +import org.apache.flink.table.planner.plan.trait.ModifyKindSetTrait; +import org.apache.flink.table.planner.plan.trait.UpdateKindTrait; +import org.apache.flink.table.planner.plan.utils.FlinkRelOptUtil; + +import org.apache.calcite.rel.RelNode; +import org.apache.calcite.sql.SqlExplainLevel; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +/** An optimize program to infer ChangelogMode for every physical node. */ +public class FlinkChangelogModeInferenceProgram + implements FlinkOptimizeProgram { + + @Override + public RelNode optimize(RelNode root, StreamOptimizeContext context) { + // step1: satisfy ModifyKindSet trait + StreamPhysicalRel physicalRoot = (StreamPhysicalRel) root; + StreamPhysicalRel rootWithModifyKindSet = + new SatisfyModifyKindSetTraitVisitor() + .visit( + physicalRoot, + // we do not propagate the ModifyKindSet requirement and requester + // among blocks; set default ModifyKindSet requirement and requester + // for root + ModifyKindSetTrait.ALL_CHANGES(), + "ROOT"); + + // step2: satisfy UpdateKind trait + ModifyKindSet rootModifyKindSet = + ChangelogModeInferenceUtils.getModifyKindSet(rootWithModifyKindSet); + // use the required UpdateKindTrait from parent blocks + final List requiredUpdateKindTraits; + if (rootModifyKindSet.contains(ModifyKind.UPDATE)) { + if (context.isUpdateBeforeRequired()) { + requiredUpdateKindTraits = + Collections.singletonList(UpdateKindTrait.BEFORE_AND_AFTER()); + } else { + // update_before is not required, and input contains updates + // try ONLY_UPDATE_AFTER first, and then BEFORE_AND_AFTER + requiredUpdateKindTraits = + Arrays.asList( + UpdateKindTrait.ONLY_UPDATE_AFTER(), + UpdateKindTrait.BEFORE_AND_AFTER()); + } + } else { + // there is no updates + requiredUpdateKindTraits = Collections.singletonList(UpdateKindTrait.NONE()); + } + + SatisfyUpdateKindTraitVisitor updateKindTraitVisitor = + new SatisfyUpdateKindTraitVisitor(context); + List updateRoot = new ArrayList<>(); + for (UpdateKindTrait requiredUpdateKindTrait : requiredUpdateKindTraits) { + updateKindTraitVisitor + .visit(rootWithModifyKindSet, requiredUpdateKindTrait) + .ifPresent(updateRoot::add); + } + + // step3: satisfy DeleteKind trait + final List requiredDeleteKindTraits; + if (rootModifyKindSet.contains(ModifyKind.DELETE)) { + if (root instanceof StreamPhysicalSink) { + // try DELETE_BY_KEY first, and then FULL_DELETE + requiredDeleteKindTraits = + Arrays.asList( + DeleteKindTrait.DELETE_BY_KEY(), DeleteKindTrait.FULL_DELETE()); + } else { + // for non-sink nodes prefer full deletes + requiredDeleteKindTraits = Collections.singletonList(DeleteKindTrait.FULL_DELETE()); + } + } else { + // there is no deletes + requiredDeleteKindTraits = Collections.singletonList(DeleteKindTrait.NONE()); + } + + SatisfyDeleteKindTraitVisitor deleteKindTraitVisitor = + new SatisfyDeleteKindTraitVisitor(context); + List finalRoot = new ArrayList<>(); + if (!updateRoot.isEmpty()) { + StreamPhysicalRel updated = updateRoot.get(0); + for (DeleteKindTrait requiredDeleteKindTrait : requiredDeleteKindTraits) { + deleteKindTraitVisitor + .visit(updated, requiredDeleteKindTrait) + .ifPresent(finalRoot::add); + } + } + + // step4: sanity check and return non-empty root + if (finalRoot.isEmpty()) { + String plan = + FlinkRelOptUtil.toString( + root, + SqlExplainLevel.DIGEST_ATTRIBUTES, + false, + true, + false, + false, + false, + false); + throw new TableException( + "Can't generate a valid execution plan for the given query:\n" + plan); + } else { + return finalRoot.get(0); + } + } +} diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java index f567e87b2b322..2abe26ed9eec6 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyDeleteKindTraitVisitor.java @@ -144,11 +144,15 @@ Optional visit(StreamPhysicalRel rel, DeleteKindTrait require // if not explicitly supported, all operators require full deletes if there are updates List children = new ArrayList<>(); for (RelNode child : rel.getInputs()) { - visit((StreamPhysicalRel) child, DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(child))) + visit( + (StreamPhysicalRel) child, + DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(child))) .ifPresent(children::add); } return createNewNode( - rel, Optional.of(children), DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); + rel, + Optional.of(children), + DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); } else if (rel instanceof StreamPhysicalProcessTableFunction) { // Required delete traits depend on the table argument declaration, // input traits, partition keys, and upsert keys @@ -211,7 +215,9 @@ Optional visit(StreamPhysicalRel rel, DeleteKindTrait require final ModifyKindSet inputModifyKindSet = getModifyKindSet(physicalChild); if (supportsDeleteByKey && DeleteKindTrait.DELETE_BY_KEY().equals(requiredTrait)) { children.add( - visit(physicalChild, DeleteKindTrait.deleteOnKeyOrNone(inputModifyKindSet)) + visit( + physicalChild, + DeleteKindTrait.deleteOnKeyOrNone(inputModifyKindSet)) .or( () -> visit( @@ -220,7 +226,9 @@ Optional visit(StreamPhysicalRel rel, DeleteKindTrait require inputModifyKindSet)))); } else { children.add( - visit(physicalChild, DeleteKindTrait.fullDeleteOrNone(inputModifyKindSet))); + visit( + physicalChild, + DeleteKindTrait.fullDeleteOrNone(inputModifyKindSet))); } } if (children.stream().anyMatch(c -> !c.isPresent())) { @@ -229,14 +237,19 @@ Optional visit(StreamPhysicalRel rel, DeleteKindTrait require List childRels = present(children); if (childRels.stream().anyMatch(r -> getDeleteKind(r) == DeleteKind.DELETE_BY_KEY)) { return createNewNode( - join, Optional.of(childRels), DeleteKindTrait.deleteOnKeyOrNone(getModifyKindSet(rel))); + join, + Optional.of(childRels), + DeleteKindTrait.deleteOnKeyOrNone(getModifyKindSet(rel))); } else { return createNewNode( - join, Optional.of(childRels), DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); + join, + Optional.of(childRels), + DeleteKindTrait.fullDeleteOrNone(getModifyKindSet(rel))); } } else if (rel instanceof StreamPhysicalCalcBase) { // if the condition is applied on the upsert key, we can emit whatever the requiredTrait - // is, because we will filter all records based on the condition that applies to that key + // is, because we will filter all records based on the condition that applies to that + // key StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel; if (DeleteKindTrait.DELETE_BY_KEY().equals(requiredTrait) && isNonUpsertKeyCondition(calc)) { @@ -306,7 +319,8 @@ && isNonUpsertKeyCondition(calc)) { // 4. we don't use metadata columns // we can skip ChangelogNormalize if (!ChangelogNormalizeRequirementResolver.isRequired(normalize)) { - Optional> children = visitChildren(normalize, requiredTrait); + Optional> children = + visitChildren(normalize, requiredTrait); if (children.isPresent()) { StreamPhysicalRel first = children.get().get(0); RelNode input = @@ -320,7 +334,9 @@ && isNonUpsertKeyCondition(calc)) { // prefer delete by key, but accept both Optional> children = - visitChildren(normalize, DeleteKindTrait.deleteOnKeyOrNone(childModifyKindTrait)) + visitChildren( + normalize, + DeleteKindTrait.deleteOnKeyOrNone(childModifyKindTrait)) .or( () -> visitChildren( @@ -349,7 +365,9 @@ && isNonUpsertKeyCondition(calc)) { } else if (rel instanceof StreamPhysicalMultiJoin) { StreamPhysicalMultiJoin multiJoin = (StreamPhysicalMultiJoin) rel; List> children = new ArrayList<>(); - for (int childOrdinal = 0; childOrdinal < multiJoin.getInputs().size(); childOrdinal++) { + for (int childOrdinal = 0; + childOrdinal < multiJoin.getInputs().size(); + childOrdinal++) { final StreamPhysicalRel physicalChild = (StreamPhysicalRel) multiJoin.getInput(childOrdinal); boolean supportsDeleteByKey = @@ -357,7 +375,9 @@ && isNonUpsertKeyCondition(calc)) { final ModifyKindSet inputModifyKindSet = getModifyKindSet(physicalChild); if (supportsDeleteByKey && DeleteKindTrait.DELETE_BY_KEY().equals(requiredTrait)) { children.add( - visit(physicalChild, DeleteKindTrait.deleteOnKeyOrNone(inputModifyKindSet)) + visit( + physicalChild, + DeleteKindTrait.deleteOnKeyOrNone(inputModifyKindSet)) .or( () -> visit( @@ -366,7 +386,9 @@ && isNonUpsertKeyCondition(calc)) { inputModifyKindSet)))); } else { children.add( - visit(physicalChild, DeleteKindTrait.fullDeleteOrNone(inputModifyKindSet))); + visit( + physicalChild, + DeleteKindTrait.fullDeleteOrNone(inputModifyKindSet))); } } if (children.stream().anyMatch(c -> !c.isPresent())) { @@ -493,7 +515,8 @@ private Optional createNewNode( + "."); } RelTraitSet newTraitSet = node.getTraitSet().plus(providedDeleteTrait); - return Optional.of((StreamPhysicalRel) node.copy(newTraitSet, new ArrayList(children))); + return Optional.of( + (StreamPhysicalRel) node.copy(newTraitSet, new ArrayList(children))); } private Optional visitSink( @@ -539,7 +562,8 @@ private boolean areUpsertKeysDifferentFromPk(StreamPhysicalSink sink) { // if sink's pk(s) are not exactly match input changeLogUpsertKeys then it will fallback // to beforeAndAfter mode for the correctness boolean upsertKeyDifferentFromPk = false; - int[] sinkDefinedPks = sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes(); + int[] sinkDefinedPks = + sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes(); if (sinkDefinedPks.length > 0) { ImmutableBitSet sinkPks = ImmutableBitSet.of(sinkDefinedPks); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java index ee9bf782de4b6..194820c4c6348 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyModifyKindSetTraitVisitor.java @@ -129,7 +129,8 @@ StreamPhysicalRel visit( "Table sink '" + sink.contextResolvedTable().getIdentifier().asSummaryString() + "'"; - ChangelogMode queryModifyKindSet = deriveQueryDefaultChangelogMode(sink.getInput(), name); + ChangelogMode queryModifyKindSet = + deriveQueryDefaultChangelogMode(sink.getInput(), name); ModifyKindSetTrait sinkRequiredTrait = ModifyKindSetTrait.fromChangelogMode( sink.tableSink().getChangelogMode(queryModifyKindSet)); @@ -209,7 +210,8 @@ StreamPhysicalRel visit( // WindowAggregate and WindowTableAggregate support all changes in input StreamPhysicalGroupWindowAggregateBase window = (StreamPhysicalGroupWindowAggregateBase) rel; - List children = visitChildren(window, ModifyKindSetTrait.ALL_CHANGES()); + List children = + visitChildren(window, ModifyKindSetTrait.ALL_CHANGES()); ModifyKindSet.Builder builder = ModifyKindSet.newBuilder().addContainedKind(ModifyKind.INSERT); if (window.emitStrategy().produceUpdates()) { @@ -243,8 +245,7 @@ StreamPhysicalRel visit( final ModifyKindSetTrait providedTrait; if (insertOnly && RankUtil.outputInsertOnlyInDeduplicate( - tableConfig, - RankUtil.keepLastDeduplicateRow(rank.orderKey()))) { + tableConfig, RankUtil.keepLastDeduplicateRow(rank.orderKey()))) { // Deduplicate outputs append only if first row is kept and mini batching is // disabled providedTrait = ModifyKindSetTrait.INSERT_ONLY(); @@ -297,7 +298,8 @@ StreamPhysicalRel visit( if (!groups.isEmpty() && !groups.get(0).orderKeys.getFieldCollations().isEmpty()) { // All aggregates are computed over the same window and order by is supported for // only 1 field - int orderKeyIndex = groups.get(0).orderKeys.getFieldCollations().get(0).getFieldIndex(); + int orderKeyIndex = + groups.get(0).orderKeys.getFieldCollations().get(0).getFieldIndex(); RelDataType orderKeyType = over.logicWindow().getRowType().getFieldList().get(orderKeyIndex).getType(); if (!FlinkTypeFactory.isRowtimeIndicatorType(orderKeyType) @@ -514,7 +516,10 @@ private static String getNodeName(StreamPhysicalRel rel) { } } - /** Derives the {@link ModifyKindSetTrait} of query plan without required ModifyKindSet validation. */ + /** + * Derives the {@link ModifyKindSetTrait} of query plan without required ModifyKindSet + * validation. + */ private ChangelogMode deriveQueryDefaultChangelogMode(RelNode queryNode, String name) { StreamPhysicalRel newNode = visit((StreamPhysicalRel) queryNode, ModifyKindSetTrait.ALL_CHANGES(), name); @@ -539,7 +544,8 @@ private StreamPhysicalRel createNewNode( // creates a new node based on the new children, to have a more correct node description // e.g. description of GroupAggregate is based on the ModifyKindSetTrait of children StreamPhysicalRel tempNode = - (StreamPhysicalRel) node.copy(node.getTraitSet(), new ArrayList(children)); + (StreamPhysicalRel) + node.copy(node.getTraitSet(), new ArrayList(children)); String nodeString = tempNode.getRelDetailedDescription(); throw new TableException( requestedOwner diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java index 7a36e7b03622a..72ac9b93cca97 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/optimize/program/SatisfyUpdateKindTraitVisitor.java @@ -22,12 +22,11 @@ import org.apache.flink.legacy.table.sinks.RetractStreamTableSink; import org.apache.flink.legacy.table.sinks.StreamTableSink; import org.apache.flink.legacy.table.sinks.UpsertStreamTableSink; +import org.apache.flink.table.api.InsertConflictStrategy.ConflictBehavior; import org.apache.flink.table.api.TableConfig; import org.apache.flink.table.api.TableException; import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.InsertConflictStrategy.ConflictBehavior; import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.api.config.ExecutionConfigOptions.UpsertMaterialize; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.table.legacy.sinks.TableSink; import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery; @@ -129,9 +128,9 @@ class SatisfyUpdateKindTraitVisitor { /** * Try to satisfy the required {@link UpdateKindTrait} from root. * - *

Each node will first require a UpdateKindTrait to its children. The required UpdateKindTrait - * may come from the node's parent, or come from the node itself, depending on whether the node - * will destroy the trait provided by children or pass the trait from children. + *

Each node will first require a UpdateKindTrait to its children. The required + * UpdateKindTrait may come from the node's parent, or come from the node itself, depending on + * whether the node will destroy the trait provided by children or pass the trait from children. * *

If the node will pass the children's UpdateKindTrait without destroying it, then return a * new node with new inputs and forwarded UpdateKindTrait. @@ -145,8 +144,7 @@ class SatisfyUpdateKindTraitVisitor { * @return A converted node which satisfies required traits by input nodes of current node. Or * {@link Optional#empty()} if required traits cannot be satisfied. */ - Optional visit( - StreamPhysicalRel rel, UpdateKindTrait requiredUpdateTrait) { + Optional visit(StreamPhysicalRel rel, UpdateKindTrait requiredUpdateTrait) { if (rel instanceof StreamPhysicalSink) { StreamPhysicalSink sink = (StreamPhysicalSink) rel; List sinkRequiredTraits = inferSinkRequiredTraits(sink); @@ -185,12 +183,15 @@ Optional visit( } return visitSink(legacySink, sinkRequiredTraits); } else if (requiresUpdateBeforeFromChildren(rel)) { - // Aggregate, TableAggregate, OverAggregate, Limit, GroupWindowAggregate, WindowAggregate, + // Aggregate, TableAggregate, OverAggregate, Limit, GroupWindowAggregate, + // WindowAggregate, // and WindowTableAggregate requires update_before if there are updates UpdateKindTrait requiredChildUpdateTrait = UpdateKindTrait.beforeAfterOrNone(getModifyKindSet(rel.getInput(0))); - Optional> children = visitChildren(rel, requiredChildUpdateTrait); - // use requiredTrait as providedTrait, because they should support all kinds of UpdateKind + Optional> children = + visitChildren(rel, requiredChildUpdateTrait); + // use requiredTrait as providedTrait, because they should support all kinds of + // UpdateKind return createNewNode(rel, children, requiredUpdateTrait); } else if (requiresNoUpdateFromChildren(rel)) { // WindowRank, WindowDeduplicate, Deduplicate, TemporalSort, CEP, @@ -202,15 +203,13 @@ Optional visit( List rankStrategies = RankProcessStrategy.analyzeRankProcessStrategies( rank, rank.partitionKey(), rank.orderKey()); - return visitRankStrategies( - rankStrategies, requiredUpdateTrait, rank::copy); + return visitRankStrategies(rankStrategies, requiredUpdateTrait, rank::copy); } else if (rel instanceof StreamPhysicalSortLimit) { StreamPhysicalSortLimit sortLimit = (StreamPhysicalSortLimit) rel; List rankStrategies = RankProcessStrategy.analyzeRankProcessStrategies( sortLimit, ImmutableBitSet.of(), sortLimit.getCollation()); - return visitRankStrategies( - rankStrategies, requiredUpdateTrait, sortLimit::copy); + return visitRankStrategies(rankStrategies, requiredUpdateTrait, sortLimit::copy); } else if (rel instanceof StreamPhysicalSort) { StreamPhysicalSort sort = (StreamPhysicalSort) rel; UpdateKindTrait requiredChildTrait = @@ -232,11 +231,15 @@ Optional visit( children.add(Optional.empty()); } else { children.add( - visit(physicalChild, UpdateKindTrait.onlyAfterOrNone(inputModifyKindSet))); + visit( + physicalChild, + UpdateKindTrait.onlyAfterOrNone(inputModifyKindSet))); } } else { children.add( - visit(physicalChild, UpdateKindTrait.beforeAfterOrNone(inputModifyKindSet))); + visit( + physicalChild, + UpdateKindTrait.beforeAfterOrNone(inputModifyKindSet))); } } if (children.stream().anyMatch(c -> !c.isPresent())) { @@ -265,7 +268,8 @@ Optional visit( Optional newRightOption = visit(right, UpdateKindTrait.onlyAfterOrNone(rightInputModifyKindSet)); if (!newRightOption.isPresent()) { - newRightOption = visit(right, UpdateKindTrait.beforeAfterOrNone(rightInputModifyKindSet)); + newRightOption = + visit(right, UpdateKindTrait.beforeAfterOrNone(rightInputModifyKindSet)); } if (newLeftOption.isPresent() && newRightOption.isPresent()) { @@ -280,7 +284,8 @@ Optional visit( } } else if (rel instanceof StreamPhysicalCalcBase) { // if the condition is applied on the upsert key, we can emit whatever the requiredTrait - // is, because we will filter all records based on the condition that applies to that key + // is, because we will filter all records based on the condition that applies to that + // key StreamPhysicalCalcBase calc = (StreamPhysicalCalcBase) rel; if (requiredUpdateTrait.equals(UpdateKindTrait.ONLY_UPDATE_AFTER()) && isNonUpsertKeyCondition(calc)) { @@ -313,7 +318,9 @@ && isNonUpsertKeyCondition(calc)) { StreamPhysicalRel child = (StreamPhysicalRel) childNode; ModifyKindSet childModifyKindSet = getModifyKindSet(child); UpdateKindTrait requiredChildTrait = - childModifyKindSet.isInsertOnly() ? UpdateKindTrait.NONE() : requiredUpdateTrait; + childModifyKindSet.isInsertOnly() + ? UpdateKindTrait.NONE() + : requiredUpdateTrait; children.add(visit(child, requiredChildTrait)); } if (children.stream().anyMatch(c -> !c.isPresent())) { @@ -343,7 +350,8 @@ && isNonUpsertKeyCondition(calc)) { } else if (merged == updateKind) { // merged stays unchanged } else { - // UNION doesn't support to union ONLY_UPDATE_AFTER and BEFORE_AND_AFTER inputs + // UNION doesn't support to union ONLY_UPDATE_AFTER and BEFORE_AND_AFTER + // inputs return Optional.empty(); } } @@ -367,9 +375,11 @@ && isNonUpsertKeyCondition(calc)) { createNewNode(rel, Optional.of(Collections.emptyList()), providedTrait); if (providedTrait.equals(UpdateKindTrait.BEFORE_AND_AFTER()) && requiredUpdateTrait.equals(UpdateKindTrait.ONLY_UPDATE_AFTER())) { - // requiring only-after, but the source is CDC source, then drop update_before manually + // requiring only-after, but the source is CDC source, then drop update_before + // manually StreamPhysicalDropUpdateBefore dropUB = - new StreamPhysicalDropUpdateBefore(rel.getCluster(), rel.getTraitSet(), rel); + new StreamPhysicalDropUpdateBefore( + rel.getCluster(), rel.getTraitSet(), rel); return createNewNode( dropUB, newSource.map(Collections::singletonList), requiredUpdateTrait); } else { @@ -441,9 +451,13 @@ && isNonUpsertKeyCondition(calc)) { boolean onlyAfterByParent = requiredUpdateTrait.updateKind() == UpdateKind.ONLY_UPDATE_AFTER; List> children = new ArrayList<>(); - for (int childOrdinal = 0; childOrdinal < multiJoin.getInputs().size(); childOrdinal++) { - StreamPhysicalRel physicalChild = (StreamPhysicalRel) multiJoin.getInput(childOrdinal); - boolean supportOnlyAfter = multiJoin.inputUniqueKeyContainsCommonJoinKey(childOrdinal); + for (int childOrdinal = 0; + childOrdinal < multiJoin.getInputs().size(); + childOrdinal++) { + StreamPhysicalRel physicalChild = + (StreamPhysicalRel) multiJoin.getInput(childOrdinal); + boolean supportOnlyAfter = + multiJoin.inputUniqueKeyContainsCommonJoinKey(childOrdinal); ModifyKindSet inputModifyKindSet = getModifyKindSet(physicalChild); if (onlyAfterByParent) { if (inputModifyKindSet.contains(ModifyKind.UPDATE) && !supportOnlyAfter) { @@ -452,11 +466,15 @@ && isNonUpsertKeyCondition(calc)) { children.add(Optional.empty()); } else { children.add( - visit(physicalChild, UpdateKindTrait.onlyAfterOrNone(inputModifyKindSet))); + visit( + physicalChild, + UpdateKindTrait.onlyAfterOrNone(inputModifyKindSet))); } } else { children.add( - visit(physicalChild, UpdateKindTrait.beforeAfterOrNone(inputModifyKindSet))); + visit( + physicalChild, + UpdateKindTrait.beforeAfterOrNone(inputModifyKindSet))); } } if (children.stream().anyMatch(c -> !c.isPresent())) { @@ -567,7 +585,8 @@ private Optional createNewNode( + "."); } RelTraitSet newTraitSet = node.getTraitSet().plus(providedUpdateTrait); - return Optional.of((StreamPhysicalRel) node.copy(newTraitSet, new ArrayList(children))); + return Optional.of( + (StreamPhysicalRel) node.copy(newTraitSet, new ArrayList(children))); } /** @@ -633,10 +652,12 @@ private List inferSinkRequiredTraits(StreamPhysicalSink sink) { UpdateKindTrait beforeAndAfter = UpdateKindTrait.beforeAfterOrNone(childModifyKindSet); UpdateKindTrait sinkTrait = UpdateKindTrait.fromChangelogMode( - sink.tableSink().getChangelogMode(childModifyKindSet.toDefaultChangelogMode())); + sink.tableSink() + .getChangelogMode(childModifyKindSet.toDefaultChangelogMode())); if (sinkTrait.equals(UpdateKindTrait.ONLY_UPDATE_AFTER())) { - // if sink's pk(s) are not satisfied by input upsert keys (considering immutable columns), + // if sink's pk(s) are not satisfied by input upsert keys (considering immutable + // columns), // fallback to beforeAndAfter mode for correctness boolean requireBeforeAndAfter = !canUpsertKeysWithImmutableColsSatisfyPk(sink); if (requireBeforeAndAfter) { @@ -652,8 +673,8 @@ private List inferSinkRequiredTraits(StreamPhysicalSink sink) { } /** - * Check whether input's upsert keys (together with immutable columns) can satisfy sink's primary - * keys. + * Check whether input's upsert keys (together with immutable columns) can satisfy sink's + * primary keys. * *

A sink pk is considered "satisfied" when there exists an upsert key {@code uk} such that: * @@ -676,7 +697,8 @@ private List inferSinkRequiredTraits(StreamPhysicalSink sink) { * inference. */ private boolean canUpsertKeysWithImmutableColsSatisfyPk(StreamPhysicalSink sink) { - int[] sinkDefinedPks = sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes(); + int[] sinkDefinedPks = + sink.contextResolvedTable().getResolvedSchema().getPrimaryKeyIndexes(); if (sinkDefinedPks.length == 0) { return true; } @@ -783,7 +805,9 @@ private boolean analyzeUpsertMaterializeStrategy(StreamPhysicalSink sink) { // Validate that ON CONFLICT is specified when upsert key differs from primary key boolean requireOnConflict = tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_REQUIRE_ON_CONFLICT); - if (requireOnConflict && upsertKeyDiffersFromPk && sink.conflictStrategy() == null) { + if (requireOnConflict + && upsertKeyDiffersFromPk + && sink.conflictStrategy() == null) { String pkNames = sink.getPrimaryKeyNames(); String upsertKeyNames = sink.getUpsertKeyNames(); throw new ValidationException( @@ -831,7 +855,10 @@ private void collectSourcesWithoutWatermarks(RelNode rel, List result) { StreamPhysicalTableSourceScan ts = (StreamPhysicalTableSourceScan) rel; TableSourceTable table = ts.getTable().unwrap(TableSourceTable.class); if (table != null - && table.contextResolvedTable().getResolvedSchema().getWatermarkSpecs().isEmpty()) { + && table.contextResolvedTable() + .getResolvedSchema() + .getWatermarkSpecs() + .isEmpty()) { result.add(table.contextResolvedTable().getIdentifier().asSummaryString()); } } else { diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala deleted file mode 100644 index 488e3e4f3e595..0000000000000 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/plan/optimize/program/FlinkChangelogModeInferenceProgram.scala +++ /dev/null @@ -1,102 +0,0 @@ -/* - * 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.optimize.program - -import org.apache.flink.table.api.TableException -import org.apache.flink.table.planner.plan.`trait`._ -import org.apache.flink.table.planner.plan.nodes.physical.stream._ -import org.apache.flink.table.planner.plan.utils._ - -import org.apache.calcite.rel.RelNode - -/** An optimize program to infer ChangelogMode for every physical node. */ -class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOptimizeContext] { - - override def optimize(root: RelNode, context: StreamOptimizeContext): RelNode = { - // step1: satisfy ModifyKindSet trait - val physicalRoot = root.asInstanceOf[StreamPhysicalRel] - val rootWithModifyKindSet = new SatisfyModifyKindSetTraitVisitor().visit( - physicalRoot, - // we do not propagate the ModifyKindSet requirement and requester among blocks - // set default ModifyKindSet requirement and requester for root - ModifyKindSetTrait.ALL_CHANGES, - "ROOT" - ) - - // step2: satisfy UpdateKind trait - val rootModifyKindSet = getModifyKindSet(rootWithModifyKindSet) - // use the required UpdateKindTrait from parent blocks - val requiredUpdateKindTraits = if (rootModifyKindSet.contains(ModifyKind.UPDATE)) { - if (context.isUpdateBeforeRequired) { - Seq(UpdateKindTrait.BEFORE_AND_AFTER) - } else { - // update_before is not required, and input contains updates - // try ONLY_UPDATE_AFTER first, and then BEFORE_AND_AFTER - Seq(UpdateKindTrait.ONLY_UPDATE_AFTER, UpdateKindTrait.BEFORE_AND_AFTER) - } - } else { - // there is no updates - Seq(UpdateKindTrait.NONE) - } - - val updateKindTraitVisitor = new SatisfyUpdateKindTraitVisitor(context) - val updateRoot = requiredUpdateKindTraits.flatMap { - requiredUpdateKindTrait => - val updated = updateKindTraitVisitor.visit(rootWithModifyKindSet, requiredUpdateKindTrait) - if (updated.isPresent) Some(updated.get) else None - } - - // step3: satisfy DeleteKind trait - val requiredDeleteKindTraits = if (rootModifyKindSet.contains(ModifyKind.DELETE)) { - root match { - case _: StreamPhysicalSink => - // try DELETE_BY_KEY first, and then FULL_DELETE - Seq(DeleteKindTrait.DELETE_BY_KEY, DeleteKindTrait.FULL_DELETE) - case _ => - // for non-sink nodes prefer full deletes - Seq(DeleteKindTrait.FULL_DELETE) - } - } else { - // there is no deletes - Seq(DeleteKindTrait.NONE) - } - - val deleteKindTraitVisitor = new SatisfyDeleteKindTraitVisitor(context) - val finalRoot = if (updateRoot.isEmpty) { - updateRoot - } else { - requiredDeleteKindTraits.flatMap { - requiredDeleteKindTrait => - val deleteRoot = deleteKindTraitVisitor.visit(updateRoot.head, requiredDeleteKindTrait) - if (deleteRoot.isPresent) Some(deleteRoot.get) else None - } - } - - // step4: sanity check and return non-empty root - if (finalRoot.isEmpty) { - val plan = FlinkRelOptUtil.toString(root, withChangelogTraits = true) - throw new TableException( - "Can't generate a valid execution plan for the given query:\n" + plan) - } else { - finalRoot.head - } - } - - private def getModifyKindSet(node: RelNode): ModifyKindSet = - ChangelogModeInferenceUtils.getModifyKindSet(node) -}