From 674b86697cc9ff65ce0d679721420b3972859506 Mon Sep 17 00:00:00 2001 From: xuyang Date: Tue, 10 Mar 2026 15:59:25 +0800 Subject: [PATCH 1/3] [FLINK-39236][table] Adapt binary delta join runtime operator to structure of cascade delta join --- .../nodes/exec/spec/DeltaJoinAssociation.java | 9 + .../nodes/exec/spec/DeltaJoinLookupChain.java | 24 + .../plan/nodes/exec/spec/DeltaJoinTree.java | 72 + .../exec/stream/StreamExecDeltaJoin.java | 1065 ++++++-- .../stream/StreamPhysicalDeltaJoin.java | 13 +- .../table/planner/utils/ShortcutUtils.java | 5 + .../planner/codegen/FilterCodeGenerator.scala | 17 +- .../codegen/LookupJoinCodeGenerator.scala | 24 +- .../planner/plan/stream/sql/DeltaJoinTest.xml | 2 +- ...stCalcBetweenCascadedDeltaJoinJsonPlan.out | 2415 +++++++++++++++++ .../testJsonPlanWithTableHints.out | 427 ++- .../plan/stream/sql/DeltaJoinTest.scala | 28 +- .../StreamingDeltaJoinOperatorFactory.java | 24 +- .../TableAbstractCoUdfStreamOperator.java | 10 +- .../join/deltajoin/AsyncDeltaJoinRunner.java | 506 ++-- .../join/deltajoin/BinaryLookupHandler.java | 105 + .../join/deltajoin/DeltaJoinCache.java | 24 +- .../join/deltajoin/DeltaJoinHandlerBase.java | 112 + .../join/deltajoin/DeltaJoinHandlerChain.java | 131 + .../join/deltajoin/DeltaJoinOpenContext.java | 58 + .../join/deltajoin/DeltaJoinRuntimeTree.java | 221 ++ .../join/deltajoin/LookupHandlerBase.java | 424 +++ .../deltajoin/MultiInputRowDataBuffer.java | 298 ++ .../deltajoin/StreamingDeltaJoinOperator.java | 52 +- ...StreamingBinaryDeltaJoinOperatorTest.java} | 954 +++---- .../StreamingDeltaJoinOperatorTestBase.java | 729 +++++ 26 files changed, 6431 insertions(+), 1318 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest_jsonplan/testCalcBetweenCascadedDeltaJoinJsonPlan.out create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerBase.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerChain.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinOpenContext.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinRuntimeTree.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/LookupHandlerBase.java create mode 100644 flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/MultiInputRowDataBuffer.java rename flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/{StreamingDeltaJoinOperatorTest.java => StreamingBinaryDeltaJoinOperatorTest.java} (62%) create mode 100644 flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperatorTestBase.java diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinAssociation.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinAssociation.java index a46ad62ef600c..a205a8dfe4075 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinAssociation.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinAssociation.java @@ -319,6 +319,15 @@ public List getAllBinaryInputOrdinalsWithOffset(int offset) { .collect(Collectors.toList()); } + public List getAllBinaryInputTableSourceSpecs() { + return IntStream.range(0, binaryInputInfos.size()) + .mapToObj( + i -> + new TemporalTableSourceSpec( + binaryInputInfos.get(i).tableScan.getTable())) + .collect(Collectors.toList()); + } + /** * Returns a concise summary of all source-to-target association pairs in both {@link * #binary2BinaryJoinAssociation} and {@link #compositeBinary2BinaryJoinAssociation}. diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinLookupChain.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinLookupChain.java index 1fe7dcc60050d..22664283fa1fb 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinLookupChain.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinLookupChain.java @@ -157,5 +157,29 @@ public static Node of( deltaJoinSpec, joinType); } + + /** + * Return whether all source ordinals are in left side, and the lookup ordinal is in right + * side. + */ + @JsonIgnore + public boolean isLeftLookupRight() { + if (Arrays.stream(inputTableBinaryInputOrdinals) + .allMatch(i -> i < lookupTableBinaryInputOrdinal)) { + return true; + } else if (Arrays.stream(inputTableBinaryInputOrdinals) + .allMatch(i -> i > lookupTableBinaryInputOrdinal)) { + return false; + } else { + // should not happen + throw new IllegalStateException( + String.format( + "Could not judge the direction of this lookup.\n" + + "All input ordinals in the stream side is %s\n" + + "The input ordinals in the lookup side is '%d'", + Arrays.toString(inputTableBinaryInputOrdinals), + lookupTableBinaryInputOrdinal)); + } + } } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinTree.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinTree.java index 6458ffee62285..f7555e06822e4 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinTree.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/spec/DeltaJoinTree.java @@ -18,10 +18,21 @@ package org.apache.flink.table.planner.plan.nodes.exec.spec; +import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.table.data.RowData; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.codegen.FilterCodeGenerator; +import org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeConfig; import org.apache.flink.table.planner.plan.utils.DeltaJoinUtil; +import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; +import org.apache.flink.table.runtime.generated.GeneratedFilterCondition; +import org.apache.flink.table.runtime.generated.GeneratedFunction; import org.apache.flink.table.runtime.operators.join.FlinkJoinType; +import org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinRuntimeTree; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.util.Preconditions; @@ -50,6 +61,7 @@ import static org.apache.flink.table.planner.plan.utils.DeltaJoinUtil.combineOutputRowType; import static org.apache.flink.table.planner.plan.utils.DeltaJoinUtil.splitProjectionAndFilter; +import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTypeFactory; /** * A delta join tree used to describe the relationships among one or more joins in the input. @@ -189,6 +201,66 @@ private RowType getOutputTypeOnNodeInternal( return getOutputTypeOnNodeInternal(caresInputOrdinals, joinNode.right, typeFactory); } + /** Convert this {@link DeltaJoinTree} to {@link DeltaJoinRuntimeTree}. */ + public DeltaJoinRuntimeTree convert2RuntimeTree(PlannerBase planner, ExecNodeConfig config) { + return new DeltaJoinRuntimeTree(convert2RuntimeTreeInternal(planner, config, root)); + } + + private DeltaJoinRuntimeTree.Node convert2RuntimeTreeInternal( + PlannerBase planner, ExecNodeConfig config, Node node) { + ClassLoader classLoader = planner.getFlinkContext().getClassLoader(); + FlinkTypeFactory typeFactory = unwrapTypeFactory(planner); + RowType rowTypeBeforeCalc = node.getRowTypeBeforeCalc(typeFactory); + RowType rowTypePassThroughCalc = node.getRowTypeAfterCalc(typeFactory); + + String generatedCalcName = + node instanceof BinaryInputNode + ? "BinaryInputNodeCalcFunction" + : "JoinNodeCalcFunction"; + GeneratedFunction> generatedCalc = + Optional.ofNullable(node.projection) + .map( + projection -> + LookupJoinCodeGenerator.generateCalcMapFunction( + config, + classLoader, + JavaScalaConversionUtil.toScala(node.projection), + node.filter, + rowTypePassThroughCalc, + rowTypeBeforeCalc, + generatedCalcName)) + .orElse(null); + + if (node instanceof BinaryInputNode) { + return new DeltaJoinRuntimeTree.BinaryInputNode( + ((BinaryInputNode) node).inputOrdinal, + generatedCalc, + InternalSerializers.create(rowTypePassThroughCalc)); + } + JoinNode joinNode = (JoinNode) node; + + DeltaJoinRuntimeTree.Node newLeft = + convert2RuntimeTreeInternal(planner, config, joinNode.left); + DeltaJoinRuntimeTree.Node newRight = + convert2RuntimeTreeInternal(planner, config, joinNode.right); + + GeneratedFilterCondition generatedJoinCondition = + FilterCodeGenerator.generateFilterCondition( + config, + classLoader, + joinNode.condition, + joinNode.getRowTypeBeforeCalc(typeFactory), + "JoinCondition"); + + return new DeltaJoinRuntimeTree.JoinNode( + joinNode.joinType, + generatedJoinCondition, + generatedCalc, + newLeft, + newRight, + InternalSerializers.create(rowTypePassThroughCalc)); + } + private static List getAllInputOrdinals(Node node) { List collector = new ArrayList<>(); collectAllInputOrdinals(node, collector); diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java index 2836f0b601597..463a37596a652 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java @@ -27,13 +27,11 @@ import org.apache.flink.streaming.api.operators.StreamOperatorFactory; import org.apache.flink.streaming.api.transformations.TwoInputTransformation; import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.conversion.DataStructureConverter; -import org.apache.flink.table.data.conversion.DataStructureConverters; import org.apache.flink.table.functions.AsyncTableFunction; import org.apache.flink.table.functions.UserDefinedFunctionHelper; import org.apache.flink.table.planner.calcite.FlinkTypeFactory; +import org.apache.flink.table.planner.codegen.FilterCodeGenerator; import org.apache.flink.table.planner.codegen.FunctionCallCodeGenerator; import org.apache.flink.table.planner.codegen.LookupJoinCodeGenerator; import org.apache.flink.table.planner.delegation.PlannerBase; @@ -44,26 +42,32 @@ import org.apache.flink.table.planner.plan.nodes.exec.ExecNodeMetadata; import org.apache.flink.table.planner.plan.nodes.exec.InputProperty; import org.apache.flink.table.planner.plan.nodes.exec.SingleTransformationTranslator; +import org.apache.flink.table.planner.plan.nodes.exec.spec.DeltaJoinLookupChain; import org.apache.flink.table.planner.plan.nodes.exec.spec.DeltaJoinSpec; +import org.apache.flink.table.planner.plan.nodes.exec.spec.DeltaJoinTree; +import org.apache.flink.table.planner.plan.nodes.exec.spec.TemporalTableSourceSpec; import org.apache.flink.table.planner.plan.nodes.exec.utils.ExecNodeUtil; +import org.apache.flink.table.planner.plan.schema.TableSourceTable; import org.apache.flink.table.planner.plan.utils.DeltaJoinUtil; -import org.apache.flink.table.planner.plan.utils.FunctionCallUtil; import org.apache.flink.table.planner.plan.utils.FunctionCallUtil.AsyncOptions; -import org.apache.flink.table.planner.plan.utils.FunctionCallUtil.FunctionParam; import org.apache.flink.table.planner.plan.utils.KeySelectorUtil; import org.apache.flink.table.planner.plan.utils.LookupJoinUtil; import org.apache.flink.table.planner.utils.JavaScalaConversionUtil; -import org.apache.flink.table.planner.utils.ShortcutUtils; -import org.apache.flink.table.runtime.collector.TableFunctionResultFuture; +import org.apache.flink.table.runtime.generated.GeneratedFilterCondition; import org.apache.flink.table.runtime.generated.GeneratedFunction; -import org.apache.flink.table.runtime.generated.GeneratedResultFuture; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.StreamingDeltaJoinOperatorFactory; import org.apache.flink.table.runtime.operators.join.FlinkJoinType; import org.apache.flink.table.runtime.operators.join.deltajoin.AsyncDeltaJoinRunner; +import org.apache.flink.table.runtime.operators.join.deltajoin.BinaryLookupHandler; +import org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinHandlerChain; +import org.apache.flink.table.runtime.operators.join.deltajoin.DeltaJoinRuntimeTree; +import org.apache.flink.table.runtime.operators.join.deltajoin.LookupHandlerBase; import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.util.Preconditions; import org.apache.flink.shaded.guava33.com.google.common.collect.Lists; import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator; @@ -72,30 +76,106 @@ import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty; import org.apache.calcite.plan.RelOptTable; -import org.apache.calcite.plan.RelOptUtil; -import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rex.RexBuilder; -import org.apache.calcite.rex.RexInputRef; import org.apache.calcite.rex.RexNode; -import org.apache.calcite.rex.RexShuttle; -import org.apache.calcite.tools.RelBuilder; +import org.apache.calcite.rex.RexUtil; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import javax.annotation.Nullable; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; +import static java.util.Objects.requireNonNull; import static org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecDeltaJoin.DELTA_JOIN_TRANSFORMATION; +import static org.apache.flink.table.planner.plan.utils.DeltaJoinUtil.combineOutputRowType; import static org.apache.flink.table.planner.plan.utils.DeltaJoinUtil.getUnwrappedAsyncLookupFunction; -import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapTypeFactory; +import static org.apache.flink.table.planner.plan.utils.DeltaJoinUtil.swapJoinType; +import static org.apache.flink.table.planner.utils.ShortcutUtils.unwrapDataTypeFactory; -/** {@link StreamExecNode} for delta join. */ +/** + * {@link StreamExecNode} for delta join. + * + *

This node has two versions with different capabilities: + * + *

Version 1 (Binary Delta Join)

+ * + *

Introduced in Flink v2.1. This version only supports a simple two-table delta join scenario. + * It uses two {@link DeltaJoinSpec} fields ({@link #lookupRightTableJoinSpec} and {@link + * #lookupLeftTableJoinSpec}) to describe how each streaming side looks up the other side's + * dimension table: + * + *

The {@link DeltaJoinTree} is built internally from these two specs during translation. The + * operator factory is built via {@link DeltaJoinOperatorFactoryBuilderV1}. + * + *

{@code
+ * Example (v1 - Binary Delta Join):
+ *
+ *     Left Stream (A)  ──┐
+ *                        ├──  DeltaJoin  (each side looks up the other's dimension table)
+ *     Right Stream (B) ──┘
+ * }
+ * + *

Version 2 (Cascaded Delta Join)

+ * + *

Introduced in Flink v2.3. This version extends delta join to support multi-table (cascaded) + * scenarios where each side may involve multiple dimension tables that need to be looked up in a + * specific order. It uses the following additional structures: + * + *

    + *
  • {@link DeltaJoinLookupChain}: an ordered chain of lookup operations. Each {@link + * DeltaJoinLookupChain.Node} represents a single lookup step, using one or more already + * resolved inputs to look up the next dimension table. + *
  • {@link DeltaJoinTree}: a tree structure describing the relationships among all joins. Leaf + * nodes ({@link DeltaJoinTree.BinaryInputNode}) represent source tables, and non-leaf nodes + * ({@link DeltaJoinTree.JoinNode}) represent join operations. + *
  • {@link #allBinaryInputTables}: the list of all binary input (dimension) table source specs. + *
  • {@link #leftAllBinaryInputOrdinals} / {@link #rightAllBinaryInputOrdinals}: the ordinals + * identifying which binary inputs belong to the left side and which to the right side. + *
  • {@link #condition}: the overall join condition on this join node. + *
+ * + *

The operator factory is built via {@code DeltaJoinOperatorFactoryBuilderV2}. + * + *

{@code
+ * Example (v2 - Cascaded Delta Join):
+ *
+ *              DeltaJoin
+ *           /            \
+ *       Calc3             \
+ *        /                 \
+ *   DeltaJoin           DeltaJoin
+ *     /    \             /     \
+ *  Calc1    \          /      Calc2
+ *   /        \       /           \
+ * #0 A     #1 B    #2 C          #3 D
+ *
+ * Left stream side owns inputs #0, #1; Right stream side owns inputs #2, #3.
+ * When the left side receives an update, it looks up #2, then #3 (cascaded).
+ * When the right side receives an update, it looks up #0, then #1 (cascaded).
+ * }
+ * + *

Conceptually, version 1 (binary two-table delta join) is a special case of version 2 (cascaded + * multi-table delta join) — it is equivalent to a v2 tree with exactly two leaf inputs and no + * cascading lookup chain. + * + *

Version 2 is backward compatible with version 1: a plan serialized with v1 can be deserialized + * and executed by v2. When v1 fields ({@link #lookupRightTableJoinSpec} and {@link + * #lookupLeftTableJoinSpec}) are present, the node falls back to the v1 code path via {@link + * DeltaJoinOperatorFactoryBuilderV1}; otherwise it uses the v2 code path via {@link + * DeltaJoinOperatorFactoryBuilderV2}. + */ @ExecNodeMetadata( name = "stream-exec-delta-join", version = 1, @@ -106,6 +186,16 @@ }, minPlanVersion = FlinkVersion.v2_1, minStateVersion = FlinkVersion.v2_1) +@ExecNodeMetadata( + name = "stream-exec-delta-join", + version = 2, + producedTransformations = DELTA_JOIN_TRANSFORMATION, + consumedOptions = { + "table.exec.async-lookup.buffer-capacity", + "table.exec.async-lookup.timeout" + }, + minPlanVersion = FlinkVersion.v2_3, + minStateVersion = FlinkVersion.v2_3) @JsonIgnoreProperties(ignoreUnknown = true) public class StreamExecDeltaJoin extends ExecNodeBase implements StreamExecNode, SingleTransformationTranslator { @@ -114,21 +204,31 @@ public class StreamExecDeltaJoin extends ExecNodeBase public static final String DELTA_JOIN_TRANSFORMATION = "delta-join"; + private static final String GENERATED_JOIN_CONDITION_CLASS_NAME = "JoinCondition"; + private static final String FIELD_NAME_LEFT_JOIN_KEYS = "leftJoinKeys"; private static final String FIELD_NAME_RIGHT_JOIN_KEYS = "rightJoinKeys"; - private static final String FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC = - "lookupRightTableJoinSpec"; - private static final String FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC = "lookupLeftTableJoinSpec"; - private static final String FIELD_NAME_LEFT_UPSERT_KEY = "leftUpsertKey"; private static final String FIELD_NAME_RIGHT_UPSERT_KEY = "rightUpsertKey"; - private static final String FIELD_NAME_JOIN_TYPE = "joinType"; + private static final String FIELD_NAME_ASYNC_OPTIONS = "asyncOptions"; - public static final String FIELD_NAME_ASYNC_OPTIONS = "asyncOptions"; + // v1 (binary delta join) field names + private static final String FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC = + "lookupRightTableJoinSpec"; + private static final String FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC = "lookupLeftTableJoinSpec"; - // ===== common ===== + // v2 (cascaded delta join) field names + private static final String FIELD_NAME_CONDITION = "condition"; + private static final String FIELD_NAME_LEFT_ALL_BINARY_INPUT_ORDINALS = + "leftAllBinaryInputOrdinals"; + private static final String FIELD_NAME_RIGHT_ALL_BINARY_INPUT_ORDINALS = + "rightAllBinaryInputOrdinals"; + private static final String FIELD_NAME_LEFT_2_RIGHT_LOOKUP_CHAIN = "left2RightLookupChain"; + private static final String FIELD_NAME_RIGHT_2_LEFT_LOOKUP_CHAIN = "right2LeftLookupChain"; + private static final String FIELD_NAME_ALL_BINARY_INPUT_TABLES = "allBinaryInputTables"; + private static final String FIELD_NAME_DELTA_JOIN_TREE = "deltaJoinTree"; @JsonProperty(FIELD_NAME_JOIN_TYPE) private final FlinkJoinType flinkJoinType; @@ -137,8 +237,6 @@ public class StreamExecDeltaJoin extends ExecNodeBase @JsonInclude(JsonInclude.Include.NON_NULL) private final AsyncOptions asyncLookupOptions; - // ===== related LEFT side ===== - @JsonProperty(FIELD_NAME_LEFT_JOIN_KEYS) private final int[] leftJoinKeys; @@ -147,12 +245,6 @@ public class StreamExecDeltaJoin extends ExecNodeBase @Nullable private final int[] leftUpsertKeys; - // left (streaming) side join right (lookup) side - @JsonProperty(FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC) - private final DeltaJoinSpec lookupRightTableJoinSpec; - - // ===== related RIGHT side ===== - @JsonProperty(FIELD_NAME_RIGHT_JOIN_KEYS) private final int[] rightJoinKeys; @@ -161,21 +253,64 @@ public class StreamExecDeltaJoin extends ExecNodeBase @Nullable private final int[] rightUpsertKeys; + // ===== v1 (binary delta join) fields ===== + + // left (streaming) side join right (lookup) side + @JsonProperty(FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC) + @Nullable + private final DeltaJoinSpec lookupRightTableJoinSpec; + // right (streaming) side join left (lookup) side @JsonProperty(FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC) + @Nullable private final DeltaJoinSpec lookupLeftTableJoinSpec; + // ===== v2 (cascaded delta join) fields ===== + + @JsonProperty(FIELD_NAME_CONDITION) + @Nullable + private final RexNode condition; + + // based on 0 + @JsonProperty(FIELD_NAME_LEFT_ALL_BINARY_INPUT_ORDINALS) + @Nullable + private final List leftAllBinaryInputOrdinals; + + // based on 0 + @JsonProperty(FIELD_NAME_RIGHT_ALL_BINARY_INPUT_ORDINALS) + @Nullable + private final List rightAllBinaryInputOrdinals; + + @JsonProperty(FIELD_NAME_LEFT_2_RIGHT_LOOKUP_CHAIN) + @Nullable + private final DeltaJoinLookupChain left2RightLookupChain; + + @JsonProperty(FIELD_NAME_RIGHT_2_LEFT_LOOKUP_CHAIN) + @Nullable + private final DeltaJoinLookupChain right2LeftLookupChain; + + @JsonProperty(FIELD_NAME_ALL_BINARY_INPUT_TABLES) + @Nullable + private final List allBinaryInputTables; + + @JsonProperty(FIELD_NAME_DELTA_JOIN_TREE) + @Nullable + private final DeltaJoinTree deltaJoinTree; + public StreamExecDeltaJoin( ReadableConfig tableConfig, FlinkJoinType flinkJoinType, - // delta join args related with the left side + RexNode condition, int[] leftJoinKeys, @Nullable int[] leftUpsertKeys, - DeltaJoinSpec lookupRightTableJoinSpec, - // delta join args related with the right side int[] rightJoinKeys, @Nullable int[] rightUpsertKeys, - DeltaJoinSpec lookupLeftTableJoinSpec, + List leftAllBinaryInputOrdinals, + List rightAllBinaryInputOrdinals, + DeltaJoinLookupChain left2RightLookupChain, + DeltaJoinLookupChain right2LeftLookupChain, + List allBinaryInputTables, + DeltaJoinTree deltaJoinTree, InputProperty leftInputProperty, InputProperty rightInputProperty, RowType outputType, @@ -188,10 +323,17 @@ public StreamExecDeltaJoin( flinkJoinType, leftJoinKeys, leftUpsertKeys, - lookupRightTableJoinSpec, rightJoinKeys, rightUpsertKeys, - lookupLeftTableJoinSpec, + null, // v1 lookupRightTableJoinSpec + null, // v1 lookupLeftTableJoinSpec + condition, + leftAllBinaryInputOrdinals, + rightAllBinaryInputOrdinals, + left2RightLookupChain, + right2LeftLookupChain, + allBinaryInputTables, + deltaJoinTree, Lists.newArrayList(leftInputProperty, rightInputProperty), outputType, description, @@ -206,26 +348,67 @@ public StreamExecDeltaJoin( @JsonProperty(FIELD_NAME_JOIN_TYPE) FlinkJoinType flinkJoinType, @JsonProperty(FIELD_NAME_LEFT_JOIN_KEYS) int[] leftJoinKeys, @JsonProperty(FIELD_NAME_LEFT_UPSERT_KEY) @Nullable int[] leftUpsertKeys, - @JsonProperty(FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC) - DeltaJoinSpec lookupRightTableJoinSpec, @JsonProperty(FIELD_NAME_RIGHT_JOIN_KEYS) int[] rightJoinKeys, @JsonProperty(FIELD_NAME_RIGHT_UPSERT_KEY) @Nullable int[] rightUpsertKeys, - @JsonProperty(FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC) + // v1 (binary delta join) fields + @JsonProperty(FIELD_NAME_LOOKUP_RIGHT_TABLE_JOIN_SPEC) @Nullable + DeltaJoinSpec lookupRightTableJoinSpec, + @JsonProperty(FIELD_NAME_LOOKUP_LEFT_TABLE_JOIN_SPEC) @Nullable DeltaJoinSpec lookupLeftTableJoinSpec, + // v2 (cascaded delta join) fields + @JsonProperty(FIELD_NAME_CONDITION) @Nullable RexNode condition, + @JsonProperty(FIELD_NAME_LEFT_ALL_BINARY_INPUT_ORDINALS) @Nullable + List leftAllBinaryInputOrdinals, + @JsonProperty(FIELD_NAME_RIGHT_ALL_BINARY_INPUT_ORDINALS) @Nullable + List rightAllBinaryInputOrdinals, + @JsonProperty(FIELD_NAME_LEFT_2_RIGHT_LOOKUP_CHAIN) @Nullable + DeltaJoinLookupChain left2RightLookupChain, + @JsonProperty(FIELD_NAME_RIGHT_2_LEFT_LOOKUP_CHAIN) @Nullable + DeltaJoinLookupChain right2LeftLookupChain, + @JsonProperty(FIELD_NAME_ALL_BINARY_INPUT_TABLES) @Nullable + List allBinaryInputTables, + @JsonProperty(FIELD_NAME_DELTA_JOIN_TREE) @Nullable DeltaJoinTree deltaJoinTree, @JsonProperty(FIELD_NAME_INPUT_PROPERTIES) List inputProperties, @JsonProperty(FIELD_NAME_OUTPUT_TYPE) RowType outputType, @JsonProperty(FIELD_NAME_DESCRIPTION) String description, @JsonProperty(FIELD_NAME_ASYNC_OPTIONS) AsyncOptions asyncLookupOptions) { super(id, context, persistedConfig, inputProperties, outputType, description); - this.flinkJoinType = flinkJoinType; this.leftJoinKeys = leftJoinKeys; this.leftUpsertKeys = leftUpsertKeys; - this.lookupRightTableJoinSpec = lookupRightTableJoinSpec; this.rightJoinKeys = rightJoinKeys; this.rightUpsertKeys = rightUpsertKeys; + this.lookupRightTableJoinSpec = lookupRightTableJoinSpec; this.lookupLeftTableJoinSpec = lookupLeftTableJoinSpec; + this.condition = condition; + this.leftAllBinaryInputOrdinals = leftAllBinaryInputOrdinals; + this.rightAllBinaryInputOrdinals = rightAllBinaryInputOrdinals; + this.left2RightLookupChain = left2RightLookupChain; + this.right2LeftLookupChain = right2LeftLookupChain; + this.allBinaryInputTables = allBinaryInputTables; + this.deltaJoinTree = deltaJoinTree; this.asyncLookupOptions = asyncLookupOptions; + + if (isDeltaJoinV1()) { + Preconditions.checkArgument(leftAllBinaryInputOrdinals == null); + Preconditions.checkArgument(rightAllBinaryInputOrdinals == null); + Preconditions.checkArgument(left2RightLookupChain == null); + Preconditions.checkArgument(right2LeftLookupChain == null); + Preconditions.checkArgument(deltaJoinTree == null); + + } else { + Preconditions.checkArgument(lookupRightTableJoinSpec == null); + Preconditions.checkArgument(lookupLeftTableJoinSpec == null); + Preconditions.checkArgument(leftAllBinaryInputOrdinals != null); + Preconditions.checkArgument(rightAllBinaryInputOrdinals != null); + Preconditions.checkArgument(left2RightLookupChain != null); + Preconditions.checkArgument(right2LeftLookupChain != null); + Preconditions.checkArgument(deltaJoinTree != null); + } + } + + private boolean isDeltaJoinV1() { + return lookupRightTableJoinSpec != null && lookupLeftTableJoinSpec != null; } @SuppressWarnings("unchecked") @@ -242,15 +425,6 @@ protected Transformation translateToPlanInternal( final RowType leftStreamType = (RowType) leftInputEdge.getOutputType(); final RowType rightStreamType = (RowType) rightInputEdge.getOutputType(); - RelOptTable leftTemporalTable = - lookupLeftTableJoinSpec - .getLookupTable() - .getTemporalTable(planner.getFlinkContext(), unwrapTypeFactory(planner)); - RelOptTable rightTemporalTable = - lookupRightTableJoinSpec - .getLookupTable() - .getTemporalTable(planner.getFlinkContext(), unwrapTypeFactory(planner)); - Transformation leftInputTransformation = (Transformation) leftInputEdge.translateToPlan(planner); Transformation rightInputTransformation = @@ -272,22 +446,40 @@ protected Transformation translateToPlanInternal( RowDataKeySelector rightUpsertKeySelector = getUpsertKeySelector(rightUpsertKeys, rightStreamType, classLoader); - StreamOperatorFactory operatorFactory = - createAsyncLookupDeltaJoin( - planner, - config, - leftTemporalTable, - rightTemporalTable, - lookupLeftTableJoinSpec.getLookupKeyMap(), - lookupRightTableJoinSpec.getLookupKeyMap(), - planner.createRelBuilder(), - leftStreamType, - rightStreamType, - leftJoinKeySelector, - leftUpsertKeySelector, - rightJoinKeySelector, - rightUpsertKeySelector, - classLoader); + DeltaJoinOperatorFactoryBuilder builder; + if (isDeltaJoinV1()) { + builder = + new DeltaJoinOperatorFactoryBuilderV1( + planner, + config, + leftStreamType, + rightStreamType, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector, + requireNonNull(lookupRightTableJoinSpec), + requireNonNull(lookupLeftTableJoinSpec)); + } else { + builder = + new DeltaJoinOperatorFactoryBuilderV2( + planner, + config, + leftStreamType, + rightStreamType, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector, + requireNonNull(condition), + requireNonNull(leftAllBinaryInputOrdinals), + requireNonNull(rightAllBinaryInputOrdinals), + requireNonNull(left2RightLookupChain), + requireNonNull(right2LeftLookupChain), + requireNonNull(allBinaryInputTables), + requireNonNull(deltaJoinTree)); + } + StreamOperatorFactory operatorFactory = builder.build(); final TwoInputTransformation transform = ExecNodeUtil.createTwoInputTransformation( @@ -305,244 +497,106 @@ protected Transformation translateToPlanInternal( return transform; } - private StreamOperatorFactory createAsyncLookupDeltaJoin( - PlannerBase planner, - ExecNodeConfig config, - RelOptTable leftTempTable, - RelOptTable rightTempTable, - Map leftLookupKeys, - Map rightLookupKeys, - RelBuilder relBuilder, - RowType leftStreamType, - RowType rightStreamType, - RowDataKeySelector leftJoinKeySelector, - RowDataKeySelector leftUpsertKeySelector, - RowDataKeySelector rightJoinKeySelector, - RowDataKeySelector rightUpsertKeySelector, - ClassLoader classLoader) { - - DataTypeFactory dataTypeFactory = - ShortcutUtils.unwrapContext(relBuilder).getCatalogManager().getDataTypeFactory(); - - AsyncDeltaJoinRunner leftLookupTableAsyncFunction = - createAsyncDeltaJoinRunner( - planner, - config, - classLoader, - dataTypeFactory, - leftTempTable, - rightTempTable, - leftStreamType, - rightStreamType, - leftLookupKeys, - leftJoinKeySelector, - leftUpsertKeySelector, - rightJoinKeySelector, - rightUpsertKeySelector, - false); - - AsyncDeltaJoinRunner rightLookupTableAsyncFunction = - createAsyncDeltaJoinRunner( - planner, - config, - classLoader, - dataTypeFactory, - leftTempTable, - rightTempTable, - leftStreamType, - rightStreamType, - rightLookupKeys, - leftJoinKeySelector, - leftUpsertKeySelector, - rightJoinKeySelector, - rightUpsertKeySelector, - true); - - Tuple2 leftRightCacheSize = getCacheSize(config); - - return new StreamingDeltaJoinOperatorFactory( - rightLookupTableAsyncFunction, - leftLookupTableAsyncFunction, - leftJoinKeySelector, - rightJoinKeySelector, - asyncLookupOptions.asyncTimeout, - asyncLookupOptions.asyncBufferCapacity, - leftRightCacheSize.f0, - leftRightCacheSize.f1, - leftStreamType, - rightStreamType); - } - - @SuppressWarnings("unchecked") - private AsyncDeltaJoinRunner createAsyncDeltaJoinRunner( + private static LookupHandlerBase generateLookupHandler( + boolean isBinaryLookup, + DeltaJoinLookupChain.Node node, + Map>> + generatedFetcherCollector, + DeltaJoinTree deltaJoinTree, PlannerBase planner, - ExecNodeConfig config, + FlinkTypeFactory typeFactory, ClassLoader classLoader, - DataTypeFactory dataTypeFactory, - RelOptTable leftTempTable, - RelOptTable rightTempTable, - RowType leftStreamSideType, - RowType rightStreamSideType, - Map lookupKeys, - RowDataKeySelector leftJoinKeySelector, - RowDataKeySelector leftUpsertKeySelector, - RowDataKeySelector rightJoinKeySelector, - RowDataKeySelector rightUpsertKeySelector, - boolean treatRightAsLookupTable) { - RelOptTable lookupTable = treatRightAsLookupTable ? rightTempTable : leftTempTable; - RowType streamSideType = treatRightAsLookupTable ? leftStreamSideType : rightStreamSideType; - RowType lookupSideType = treatRightAsLookupTable ? rightStreamSideType : leftStreamSideType; - + ExecNodeConfig config) { + final int[] sourceInputOrdinals = node.inputTableBinaryInputOrdinals; + final int lookupTableOrdinal = node.lookupTableBinaryInputOrdinal; + final RowType sourceStreamType = + deltaJoinTree.getOutputRowTypeOnNode(sourceInputOrdinals, typeFactory); + + final TableSourceTable lookupTable = + (TableSourceTable) + node.deltaJoinSpec + .getLookupTable() + .getTemporalTable(planner.getFlinkContext(), typeFactory); + + final Map lookupKeyMap = + node.deltaJoinSpec.getLookupKeyMap(); AsyncTableFunction lookupSideAsyncTableFunction = - getUnwrappedAsyncLookupFunction(lookupTable, lookupKeys.keySet(), classLoader); + getUnwrappedAsyncLookupFunction(lookupTable, lookupKeyMap.keySet(), classLoader); UserDefinedFunctionHelper.prepareInstance(config, lookupSideAsyncTableFunction); - RowType lookupTableSourceRowType = + final RowType lookupTableSourceRowType = FlinkTypeFactory.toLogicalRowType(lookupTable.getRowType()); - RowType resultRowType = (RowType) getOutputType(); + final RowType lookupResultRowType = + combineOutputRowType( + sourceStreamType, lookupTableSourceRowType, node.joinType, typeFactory); - List convertedKeys = - Arrays.stream(LookupJoinUtil.getOrderedLookupKeys(lookupKeys.keySet())) - .mapToObj(lookupKeys::get) + List lookupKeysOnInputSide = + Arrays.stream(LookupJoinUtil.getOrderedLookupKeys(lookupKeyMap.keySet())) + .mapToObj(lookupKeyMap::get) .collect(Collectors.toList()); FunctionCallCodeGenerator.GeneratedTableFunctionWithDataType> - lookupSideGeneratedFuncWithType = + lookupSideGeneratedFetcherWithType = LookupJoinCodeGenerator.generateAsyncLookupFunction( config, classLoader, - dataTypeFactory, - streamSideType, + unwrapDataTypeFactory(planner.createRelBuilder()), + sourceStreamType, lookupTableSourceRowType, - resultRowType, - convertedKeys, + lookupResultRowType, + lookupKeysOnInputSide, lookupSideAsyncTableFunction, String.join(".", lookupTable.getQualifiedName())); - DataStructureConverter lookupSideFetcherConverter = - DataStructureConverters.getConverter(lookupSideGeneratedFuncWithType.dataType()); - - GeneratedResultFuture> lookupSideGeneratedResultFuture; - if (treatRightAsLookupTable) { - lookupSideGeneratedResultFuture = - LookupJoinCodeGenerator.generateTableAsyncCollector( - config, - classLoader, - "TableFunctionResultFuture", - streamSideType, - lookupTableSourceRowType, - JavaScalaConversionUtil.toScala( - lookupRightTableJoinSpec.getRemainingCondition())); - } else { - RexBuilder rexBuilder = new RexBuilder(planner.getTypeFactory()); - - Optional newCond = - lookupLeftTableJoinSpec - .getRemainingCondition() - .map( - con -> - swapInputRefsInCondition( - rexBuilder, - con, - leftStreamSideType, - rightStreamSideType)); - lookupSideGeneratedResultFuture = - LookupJoinCodeGenerator.generateTableAsyncCollector( - config, - classLoader, - "TableFunctionResultFuture", - streamSideType, - lookupTableSourceRowType, - JavaScalaConversionUtil.toScala(newCond)); - } + final RowType lookupSidePassThroughCalcRowType = + deltaJoinTree.getOutputRowTypeOnNode(new int[] {lookupTableOrdinal}, typeFactory); GeneratedFunction> lookupSideGeneratedCalc = null; - if ((treatRightAsLookupTable - && lookupRightTableJoinSpec.getProjectionOnTemporalTable().isPresent()) - || (!treatRightAsLookupTable - && lookupLeftTableJoinSpec.getProjectionOnTemporalTable().isPresent())) { - // a projection or filter after lookup table + if (node.deltaJoinSpec.getProjectionOnTemporalTable().isPresent()) { + // a projection or filter after table source scan List projectionOnTemporalTable = - treatRightAsLookupTable - ? lookupRightTableJoinSpec.getProjectionOnTemporalTable().get() - : lookupLeftTableJoinSpec.getProjectionOnTemporalTable().get(); + node.deltaJoinSpec.getProjectionOnTemporalTable().get(); RexNode filterOnTemporalTable = - treatRightAsLookupTable - ? lookupRightTableJoinSpec.getFilterOnTemporalTable().orElse(null) - : lookupLeftTableJoinSpec.getFilterOnTemporalTable().orElse(null); + node.deltaJoinSpec.getFilterOnTemporalTable().orElse(null); lookupSideGeneratedCalc = LookupJoinCodeGenerator.generateCalcMapFunction( config, - planner.getFlinkContext().getClassLoader(), + classLoader, JavaScalaConversionUtil.toScala(projectionOnTemporalTable), filterOnTemporalTable, - lookupSideType, + lookupSidePassThroughCalcRowType, lookupTableSourceRowType); } - return new AsyncDeltaJoinRunner( - lookupSideGeneratedFuncWithType.tableFunc(), - (DataStructureConverter) lookupSideFetcherConverter, - lookupSideGeneratedCalc, - lookupSideGeneratedResultFuture, - InternalSerializers.create(lookupSideType), - leftJoinKeySelector, - leftUpsertKeySelector, - rightJoinKeySelector, - rightUpsertKeySelector, - asyncLookupOptions.asyncBufferCapacity, - treatRightAsLookupTable, - enableCache(config)); - } + Preconditions.checkState(!generatedFetcherCollector.containsKey(lookupTableOrdinal)); + generatedFetcherCollector.put( + lookupTableOrdinal, lookupSideGeneratedFetcherWithType.tableFunc()); + + if (isBinaryLookup) { + return new BinaryLookupHandler( + TypeConversions.fromLogicalToDataType(sourceStreamType), + lookupSideGeneratedFetcherWithType.dataType(), + TypeConversions.fromLogicalToDataType(lookupSidePassThroughCalcRowType), + InternalSerializers.create(lookupSidePassThroughCalcRowType), + lookupSideGeneratedCalc, + node.inputTableBinaryInputOrdinals, + node.lookupTableBinaryInputOrdinal); + } - /** - * When swapping the left and right row type, all input references in the condition should be - * shifted accordingly. Input references that originally pointed to the left will now point to - * the right, and those that originally pointed to the right will point to the left. - * - *

For example, origin left type: [int, double]; origin right type: [double, int]; origin - * condition: [$1 = $2]. After this shifting, the condition will be [$0 = $3]. - * - *

Mainly inspired by {@link RelOptUtil.RexInputConverter}. - */ - private RexNode swapInputRefsInCondition( - RexBuilder rexBuilder, RexNode condition, RowType leftType, RowType rightType) { - int leftFieldCount = leftType.getFieldCount(); - int rightFieldCount = rightType.getFieldCount(); - int[] adjustments = new int[leftFieldCount + rightFieldCount]; - // all input references on the left will be shifted to the right by `rightFieldCount` - Arrays.fill(adjustments, 0, leftFieldCount, rightFieldCount); - // all input references on the right will be shifted to the left by `leftFieldCount` - Arrays.fill( - adjustments, leftFieldCount, leftFieldCount + rightFieldCount, leftFieldCount * -1); - - RexShuttle converter = - new RexShuttle() { - - @Override - public RexNode visitInputRef(RexInputRef inputRef) { - int srcIndex = inputRef.getIndex(); - int destIndex = srcIndex + adjustments[srcIndex]; - RelDataType type = inputRef.getType(); - - return rexBuilder.makeInputRef(type, destIndex); - } - }; - - return condition.accept(converter); + throw new IllegalStateException("Support later"); } - private RowDataKeySelector getUpsertKeySelector( + private static RowDataKeySelector getUpsertKeySelector( @Nullable int[] upsertKey, RowType rowType, ClassLoader classLoader) { - final int[] rightUpsertKeys; + final int[] finalUpsertKeys; if (upsertKey != null && upsertKey.length > 0) { - rightUpsertKeys = upsertKey; + finalUpsertKeys = upsertKey; } else { - rightUpsertKeys = IntStream.range(0, rowType.getFields().size()).toArray(); + finalUpsertKeys = IntStream.range(0, rowType.getFields().size()).toArray(); } return KeySelectorUtil.getRowDataSelector( - classLoader, rightUpsertKeys, InternalTypeInfo.of(rowType)); + classLoader, finalUpsertKeys, InternalTypeInfo.of(rowType)); } private boolean enableCache(ReadableConfig config) { @@ -561,4 +615,483 @@ private Tuple2 getCacheSize(ReadableConfig config) { } return Tuple2.of(leftCacheSize, rightCacheSize); } + + private abstract static class DeltaJoinOperatorFactoryBuilder { + protected final PlannerBase planner; + protected final ExecNodeConfig config; + protected final RowType leftStreamType; + protected final RowType rightStreamType; + protected final RowDataKeySelector leftJoinKeySelector; + protected final RowDataKeySelector leftUpsertKeySelector; + protected final RowDataKeySelector rightJoinKeySelector; + protected final RowDataKeySelector rightUpsertKeySelector; + protected final ClassLoader classLoader; + protected final FlinkTypeFactory typeFactory; + + public DeltaJoinOperatorFactoryBuilder( + PlannerBase planner, + ExecNodeConfig config, + RowType leftStreamType, + RowType rightStreamType, + RowDataKeySelector leftJoinKeySelector, + RowDataKeySelector leftUpsertKeySelector, + RowDataKeySelector rightJoinKeySelector, + RowDataKeySelector rightUpsertKeySelector) { + this.planner = planner; + this.config = config; + this.leftStreamType = leftStreamType; + this.rightStreamType = rightStreamType; + this.leftJoinKeySelector = leftJoinKeySelector; + this.leftUpsertKeySelector = leftUpsertKeySelector; + this.rightJoinKeySelector = rightJoinKeySelector; + this.rightUpsertKeySelector = rightUpsertKeySelector; + this.classLoader = planner.getFlinkContext().getClassLoader(); + this.typeFactory = planner.getTypeFactory(); + } + + protected abstract StreamOperatorFactory build(); + } + + private class DeltaJoinOperatorFactoryBuilderV1 extends DeltaJoinOperatorFactoryBuilder { + + // left (streaming) side join right (lookup) side + private final DeltaJoinSpec lookupRightTableJoinSpec; + // right (streaming) side join left (lookup) side + private final DeltaJoinSpec lookupLeftTableJoinSpec; + + public DeltaJoinOperatorFactoryBuilderV1( + PlannerBase planner, + ExecNodeConfig config, + RowType leftStreamType, + RowType rightStreamType, + RowDataKeySelector leftJoinKeySelector, + RowDataKeySelector leftUpsertKeySelector, + RowDataKeySelector rightJoinKeySelector, + RowDataKeySelector rightUpsertKeySelector, + DeltaJoinSpec lookupRightTableJoinSpec, + DeltaJoinSpec lookupLeftTableJoinSpec) { + super( + planner, + config, + leftStreamType, + rightStreamType, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector); + this.lookupRightTableJoinSpec = lookupRightTableJoinSpec; + this.lookupLeftTableJoinSpec = lookupLeftTableJoinSpec; + } + + @Override + public StreamOperatorFactory build() { + RelOptTable leftTempTable = + lookupLeftTableJoinSpec + .getLookupTable() + .getTemporalTable(planner.getFlinkContext(), typeFactory); + RelOptTable rightTempTable = + lookupRightTableJoinSpec + .getLookupTable() + .getTemporalTable(planner.getFlinkContext(), typeFactory); + + int[] eachBinaryInputFieldSize = new int[2]; + eachBinaryInputFieldSize[0] = + lookupLeftTableJoinSpec.getProjectionOnTemporalTable().isEmpty() + ? leftTempTable.getRowType().getFieldCount() + : leftStreamType.getFieldCount(); + eachBinaryInputFieldSize[1] = + lookupRightTableJoinSpec.getProjectionOnTemporalTable().isEmpty() + ? rightTempTable.getRowType().getFieldCount() + : rightStreamType.getFieldCount(); + + // collect all lookup functions of each source table + // + Map>> + generatedFetcherCollector = new HashMap<>(); + + AsyncDeltaJoinRunner left2RightRunner = + createAsyncDeltaJoinRunner( + eachBinaryInputFieldSize, generatedFetcherCollector, true); + AsyncDeltaJoinRunner right2LeftRunner = + createAsyncDeltaJoinRunner( + eachBinaryInputFieldSize, generatedFetcherCollector, false); + + Tuple2 leftRightCacheSize = getCacheSize(config); + + return new StreamingDeltaJoinOperatorFactory( + left2RightRunner, + right2LeftRunner, + generatedFetcherCollector, + leftJoinKeySelector, + rightJoinKeySelector, + asyncLookupOptions.asyncTimeout, + asyncLookupOptions.asyncBufferCapacity, + leftRightCacheSize.f0, + leftRightCacheSize.f1, + leftStreamType, + rightStreamType); + } + + private AsyncDeltaJoinRunner createAsyncDeltaJoinRunner( + int[] eachBinaryInputFieldSize, + Map>> + generatedFetcherCollector, + boolean treatRightAsLookupTable) { + RexNode remainingCondition; + if (treatRightAsLookupTable) { + remainingCondition = lookupRightTableJoinSpec.getRemainingCondition().orElse(null); + } else { + remainingCondition = lookupLeftTableJoinSpec.getRemainingCondition().orElse(null); + } + GeneratedFilterCondition generatedRemainingJoinCondition = + Optional.ofNullable(remainingCondition) + .map( + rexNode -> + FilterCodeGenerator.generateFilterCondition( + config, + classLoader, + rexNode, + getOutputType(), + GENERATED_JOIN_CONDITION_CLASS_NAME)) + .orElse(null); + + DeltaJoinTree deltaJoinTree = buildDeltaJoinTree(); + return new AsyncDeltaJoinRunner( + eachBinaryInputFieldSize, + generatedRemainingJoinCondition, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector, + buildBinaryLookupHandlerChain( + generatedFetcherCollector, deltaJoinTree, treatRightAsLookupTable), + deltaJoinTree.convert2RuntimeTree(planner, config), + Set.of(Set.of(0), Set.of(1)), + treatRightAsLookupTable, + asyncLookupOptions.asyncBufferCapacity, + enableCache(config)); + } + + private DeltaJoinHandlerChain buildBinaryLookupHandlerChain( + Map>> + generatedFetcherCollector, + DeltaJoinTree deltaJoinTree, + boolean treatRightAsLookupTable) { + DeltaJoinLookupChain.Node node; + if (treatRightAsLookupTable) { + node = + DeltaJoinLookupChain.Node.of( + 0, // inputTableBinaryInputOrdinal + 1, // lookupTableBinaryInputOrdinal + lookupRightTableJoinSpec, + flinkJoinType); + } else { + node = + DeltaJoinLookupChain.Node.of( + 1, // inputTableBinaryInputOrdinal + 0, // lookupTableBinaryInputOrdinal + lookupLeftTableJoinSpec, + swapJoinType(flinkJoinType)); + } + return DeltaJoinHandlerChain.build( + Collections.singletonList( + generateLookupHandler( + true, // isBinaryLookup + node, + generatedFetcherCollector, + deltaJoinTree, + planner, + typeFactory, + classLoader, + config)), + new int[] {treatRightAsLookupTable ? 0 : 1}); + } + + private DeltaJoinTree buildDeltaJoinTree() { + RowType leftTablePassThroughCalcRowType = null; + if (lookupLeftTableJoinSpec.getProjectionOnTemporalTable().isPresent()) { + leftTablePassThroughCalcRowType = leftStreamType; + } + DeltaJoinTree.BinaryInputNode leftInputNode = + new DeltaJoinTree.BinaryInputNode( + 0, // inputOrdinal + lookupLeftTableJoinSpec.getProjectionOnTemporalTable().orElse(null), + lookupLeftTableJoinSpec.getFilterOnTemporalTable().orElse(null), + leftTablePassThroughCalcRowType, + FlinkTypeFactory.toLogicalRowType( + lookupLeftTableJoinSpec.getLookupTable().getOutputType())); + + RowType rightTablePassThroughCalcRowType = null; + if (lookupRightTableJoinSpec.getProjectionOnTemporalTable().isPresent()) { + rightTablePassThroughCalcRowType = rightStreamType; + } + DeltaJoinTree.BinaryInputNode rightInputNode = + new DeltaJoinTree.BinaryInputNode( + 1, // inputOrdinal + lookupRightTableJoinSpec.getProjectionOnTemporalTable().orElse(null), + lookupRightTableJoinSpec.getFilterOnTemporalTable().orElse(null), + rightTablePassThroughCalcRowType, + FlinkTypeFactory.toLogicalRowType( + lookupRightTableJoinSpec.getLookupTable().getOutputType())); + + DeltaJoinTree.JoinNode joinNode = + new DeltaJoinTree.JoinNode( + flinkJoinType, + buildJoinCondition(), + leftJoinKeys, + rightJoinKeys, + leftInputNode, + rightInputNode, + null // `rexProgram`: calc on this join + ); + return new DeltaJoinTree(joinNode); + } + + private RexNode buildJoinCondition() { + RexBuilder rexBuilder = planner.createRelBuilder().getRexBuilder(); + int leftFieldCount = leftStreamType.getFieldCount(); + List conditions = new ArrayList<>(); + + for (int i = 0; i < leftJoinKeys.length; i++) { + int leftIdx = leftJoinKeys[i]; + int rightIdx = rightJoinKeys[i]; + RexNode leftRef = + rexBuilder.makeInputRef( + typeFactory.createFieldTypeFromLogicalType( + leftStreamType.getFields().get(leftIdx).getType()), + leftIdx); + RexNode rightRef = + rexBuilder.makeInputRef( + typeFactory.createFieldTypeFromLogicalType( + rightStreamType.getFields().get(rightIdx).getType()), + rightIdx + leftFieldCount); + + conditions.add(rexBuilder.makeCall(SqlStdOperatorTable.EQUALS, leftRef, rightRef)); + } + lookupRightTableJoinSpec.getRemainingCondition().ifPresent(conditions::add); + + return RexUtil.composeConjunction(rexBuilder, conditions); + } + } + + private class DeltaJoinOperatorFactoryBuilderV2 extends DeltaJoinOperatorFactoryBuilder { + + private final RexNode condition; + private final List leftAllBinaryInputOrdinals; + private final List rightAllBinaryInputOrdinals; + private final DeltaJoinLookupChain left2RightLookupChain; + private final DeltaJoinLookupChain right2LeftLookupChain; + private final List allBinaryInputTables; + private final DeltaJoinTree deltaJoinTree; + + public DeltaJoinOperatorFactoryBuilderV2( + PlannerBase planner, + ExecNodeConfig config, + RowType leftStreamType, + RowType rightStreamType, + RowDataKeySelector leftJoinKeySelector, + RowDataKeySelector leftUpsertKeySelector, + RowDataKeySelector rightJoinKeySelector, + RowDataKeySelector rightUpsertKeySelector, + RexNode condition, + List leftAllBinaryInputOrdinals, + List rightAllBinaryInputOrdinals, + DeltaJoinLookupChain left2RightLookupChain, + DeltaJoinLookupChain right2LeftLookupChain, + List allBinaryInputTables, + DeltaJoinTree deltaJoinTree) { + super( + planner, + config, + leftStreamType, + rightStreamType, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector); + this.condition = condition; + this.leftAllBinaryInputOrdinals = leftAllBinaryInputOrdinals; + this.rightAllBinaryInputOrdinals = rightAllBinaryInputOrdinals; + this.left2RightLookupChain = left2RightLookupChain; + this.right2LeftLookupChain = right2LeftLookupChain; + this.allBinaryInputTables = allBinaryInputTables; + this.deltaJoinTree = deltaJoinTree; + } + + @Override + public StreamOperatorFactory build() { + // collect all lookup functions of each source table + // + Map>> + generatedFetcherCollector = new HashMap<>(); + DeltaJoinHandlerChain left2RightHandlerChain = + generateDeltaJoinHandlerChain(true, generatedFetcherCollector); + DeltaJoinHandlerChain right2LeftHandlerChain = + generateDeltaJoinHandlerChain(false, generatedFetcherCollector); + Preconditions.checkState( + generatedFetcherCollector.size() + == leftAllBinaryInputOrdinals.size() + + rightAllBinaryInputOrdinals.size()); + + int[] eachBinaryInputFieldSize = + IntStream.range(0, allBinaryInputTables.size()) + .map( + i -> + deltaJoinTree + .getOutputRowTypeOnNode( + new int[] {i}, typeFactory) + .getFieldCount()) + .toArray(); + + RowType remainingJoinConditionInputRowType = + combineOutputRowType( + leftStreamType, rightStreamType, flinkJoinType, typeFactory); + GeneratedFilterCondition left2RightGeneratedRemainingJoinCondition = + generateRemainingJoinCondition(true, remainingJoinConditionInputRowType); + GeneratedFilterCondition right2LeftGeneratedRemainingJoinCondition = + generateRemainingJoinCondition(false, remainingJoinConditionInputRowType); + + DeltaJoinRuntimeTree joinRuntimeTree = + deltaJoinTree.convert2RuntimeTree(planner, config); + + Set> left2RightAllDrivenInputsWhenLookup = + getAllDrivenInputsWhenLookup(true); + Set> right2LeftAllDrivenInputsWhenLookup = + getAllDrivenInputsWhenLookup(false); + + AsyncDeltaJoinRunner left2RightAsyncRunner = + new AsyncDeltaJoinRunner( + eachBinaryInputFieldSize, + left2RightGeneratedRemainingJoinCondition, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector, + left2RightHandlerChain, + joinRuntimeTree, + left2RightAllDrivenInputsWhenLookup, + true, + asyncLookupOptions.asyncBufferCapacity, + enableCache(config)); + + AsyncDeltaJoinRunner right2LeftAsyncRunner = + new AsyncDeltaJoinRunner( + eachBinaryInputFieldSize, + right2LeftGeneratedRemainingJoinCondition, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector, + right2LeftHandlerChain, + joinRuntimeTree, + right2LeftAllDrivenInputsWhenLookup, + false, + asyncLookupOptions.asyncBufferCapacity, + enableCache(config)); + + Tuple2 cacheSize = getCacheSize(config); + + return new StreamingDeltaJoinOperatorFactory( + left2RightAsyncRunner, + right2LeftAsyncRunner, + generatedFetcherCollector, + leftJoinKeySelector, + rightJoinKeySelector, + asyncLookupOptions.asyncTimeout, + asyncLookupOptions.asyncBufferCapacity, + cacheSize.f0, + cacheSize.f1, + leftStreamType, + rightStreamType); + } + + private DeltaJoinHandlerChain generateDeltaJoinHandlerChain( + boolean lookupRight, + Map>> + generatedFetcherCollector) { + int[] streamOwnedSourceOrdinals = + lookupRight + ? leftAllBinaryInputOrdinals.stream().mapToInt(i -> i).toArray() + : rightAllBinaryInputOrdinals.stream() + .mapToInt(i -> i + leftAllBinaryInputOrdinals.size()) + .toArray(); + + DeltaJoinLookupChain lookupChain = + lookupRight ? left2RightLookupChain : right2LeftLookupChain; + + List nodes = lookupChain.getNodes(); + Preconditions.checkArgument(!nodes.isEmpty()); + + boolean isBinaryLookup = isBinaryLookup(lookupRight); + if (isBinaryLookup) { + return DeltaJoinHandlerChain.build( + Collections.singletonList( + generateLookupHandler( + true, // isBinaryLookup + nodes.get(0), + generatedFetcherCollector, + deltaJoinTree, + planner, + typeFactory, + classLoader, + config)), + streamOwnedSourceOrdinals); + } + + throw new UnsupportedOperationException("Support cascaded delta join operator later"); + } + + private Set> getAllDrivenInputsWhenLookup(boolean lookupRight) { + Set> result = new HashSet<>(); + + DeltaJoinLookupChain lookupChain = + lookupRight ? left2RightLookupChain : right2LeftLookupChain; + + for (DeltaJoinLookupChain.Node node : lookupChain.getNodes()) { + Set drivenInputs = + Arrays.stream(node.inputTableBinaryInputOrdinals) + .boxed() + .collect(Collectors.toSet()); + + result.add(drivenInputs); + } + return result; + } + + private boolean isBinaryLookup(boolean lookupRight) { + if (lookupRight) { + return requireNonNull(left2RightLookupChain).getNodes().size() == 1; + } else { + return requireNonNull(right2LeftLookupChain).getNodes().size() == 1; + } + } + + @Nullable + private GeneratedFilterCondition generateRemainingJoinCondition( + boolean lookupRight, RowType conditionInputRowType) { + boolean isBinaryLookup = isBinaryLookup(lookupRight); + final Optional remainingJoinCondition; + if (isBinaryLookup) { + DeltaJoinLookupChain lookupChain = + lookupRight ? left2RightLookupChain : right2LeftLookupChain; + remainingJoinCondition = + lookupChain.getNodes().get(0).deltaJoinSpec.getRemainingCondition(); + } else { + // Even if we push down conditions into multi lookup handlers, the original + // conditions in the join node must still be applied again to filter the retrieved + // results to ensure correctness + remainingJoinCondition = Optional.of(condition); + } + return remainingJoinCondition + .map( + cond -> + FilterCodeGenerator.generateFilterCondition( + config, + classLoader, + cond, + conditionInputRowType, + GENERATED_JOIN_CONDITION_CLASS_NAME)) + .orElse(null); + } + } } diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeltaJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeltaJoin.java index f383f195f05ed..88a0bceecffe1 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeltaJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/physical/stream/StreamPhysicalDeltaJoin.java @@ -122,10 +122,6 @@ public StreamPhysicalDeltaJoin( @Override public ExecNode translateToExecNode() { - if (!isBinaryDeltaJoin()) { - throw new UnsupportedOperationException("Support cascaded delta join later"); - } - TableConfig config = unwrapTableConfig(this); FunctionCallUtil.AsyncOptions asyncLookupOptions = new FunctionCallUtil.AsyncOptions( @@ -146,12 +142,17 @@ public ExecNode translateToExecNode() { return new StreamExecDeltaJoin( config, JoinTypeUtil.getFlinkJoinType(joinType), + condition, joinInfo.leftKeys.toIntArray(), leftUpsertKey, - left2RightLookupChain.getNodes().get(0).deltaJoinSpec, joinInfo.rightKeys.toIntArray(), rightUpsertKey, - right2LeftLookupChain.getNodes().get(0).deltaJoinSpec, + leftAllBinaryInputOrdinals, + rightAllBinaryInputOrdinals, + left2RightLookupChain, + right2LeftLookupChain, + deltaJoinAssociation.getAllBinaryInputTableSourceSpecs(), + deltaJoinAssociation.getJoinTree(), InputProperty.DEFAULT, InputProperty.DEFAULT, FlinkTypeFactory.toLogicalRowType(rowType), diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java index 9ebe5e05aec63..38bc5eb038ddf 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/utils/ShortcutUtils.java @@ -20,6 +20,7 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.table.api.TableConfig; +import org.apache.flink.table.catalog.DataTypeFactory; import org.apache.flink.table.delegation.Planner; import org.apache.flink.table.expressions.CallExpression; import org.apache.flink.table.expressions.ResolvedExpression; @@ -117,6 +118,10 @@ public static ClassLoader unwrapClassLoader(RelNode relNode) { return unwrapContext(relNode).getClassLoader(); } + public static DataTypeFactory unwrapDataTypeFactory(RelBuilder relBuilder) { + return unwrapContext(relBuilder).getCatalogManager().getDataTypeFactory(); + } + public static @Nullable FunctionDefinition unwrapFunctionDefinition( ResolvedExpression expression) { // Table API expression diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FilterCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FilterCodeGenerator.scala index c84225f9b2fd1..d56693534bc7c 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FilterCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/FilterCodeGenerator.scala @@ -35,6 +35,21 @@ object FilterCodeGenerator { classLoader: ClassLoader, filterCondition: RexNode, inputType: LogicalType): GeneratedFilterCondition = { + generateFilterCondition( + tableConfig, + classLoader, + filterCondition, + inputType, + "PreFilterCondition") + } + + /** Generates filter condition runner. */ + def generateFilterCondition( + tableConfig: ReadableConfig, + classLoader: ClassLoader, + filterCondition: RexNode, + inputType: LogicalType, + name: String): GeneratedFilterCondition = { val ctx = new CodeGeneratorContext(tableConfig, classLoader) // should consider null fields val exprGenerator = @@ -50,6 +65,6 @@ object FilterCodeGenerator { |""".stripMargin } - FunctionCodeGenerator.generateFilterCondition(ctx, "PreFilterCondition", bodyCode) + FunctionCodeGenerator.generateFilterCondition(ctx, name, bodyCode) } } diff --git a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala index 13b817b1fca4e..db158572df9e9 100644 --- a/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala +++ b/flink-table/flink-table-planner/src/main/scala/org/apache/flink/table/planner/codegen/LookupJoinCodeGenerator.scala @@ -376,9 +376,31 @@ object LookupJoinCodeGenerator { condition: RexNode, outputType: RowType, tableSourceRowType: RowType): GeneratedFunction[FlatMapFunction[RowData, RowData]] = { + generateCalcMapFunction( + tableConfig, + classLoader, + projection, + condition, + outputType, + tableSourceRowType, + "TableCalcMapFunction") + } + + /** + * Generates calculate flatmap function for temporal join which is used to projection/filter the + * dimension table results + */ + def generateCalcMapFunction( + tableConfig: ReadableConfig, + classLoader: ClassLoader, + projection: Seq[RexNode], + condition: RexNode, + outputType: RowType, + tableSourceRowType: RowType, + name: String): GeneratedFunction[FlatMapFunction[RowData, RowData]] = { CalcCodeGenerator.generateFunction( tableSourceRowType, - "TableCalcMapFunction", + name, outputType, classOf[GenericRowData], projection, diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml index 3138e4c153255..247b216d2e6a8 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.xml @@ -240,7 +240,7 @@ Sink(table=[default_catalog.default_database.snk_for_cdc_src], fields=[a0, a1, a ]]> - + ", + "description" : "TableSourceScan(table=[[default_catalog, default_database, no_delete_src1]], fields=[a0, a1, a2, a3])" + }, { + "id" : 2, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT>", + "description" : "DropUpdateBefore" + }, { + "id" : 3, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT>", + "description" : "Exchange(distribution=[hash[a1, a2]])" + }, { + "id" : 4, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src2`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "INT" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "b1", + "dataType" : "DOUBLE NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b1_b2", + "type" : "PRIMARY_KEY", + "columns" : [ "b1", "b2" ] + }, + "indexes" : [ { + "name" : "INDEX_b2", + "columns" : [ "b2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, no_delete_src2]], fields=[b0, b2, b1])" + }, { + "id" : 5, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 6, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL>", + "description" : "Exchange(distribution=[hash[b1, b2]])" + }, { + "id" : 7, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1, 2 ], + "leftUpsertKey" : [ 0, 1, 2 ], + "rightJoinKeys" : [ 2, 1 ], + "rightUpsertKey" : [ 1, 2 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src2`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "INT" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "b1", + "dataType" : "DOUBLE NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b1_b2", + "type" : "PRIMARY_KEY", + "columns" : [ "b1", "b2" ] + }, + "indexes" : [ { + "name" : "INDEX_b2", + "columns" : [ "b2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "1" : { + "type" : "FieldRef", + "index" : 2 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src1`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a0", + "dataType" : "INT NOT NULL" + }, { + "name" : "a1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "a3", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_a1_a2", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "a1", "a2" ] + }, + "indexes" : [ { + "name" : "INDEX_a1_a2", + "columns" : [ "a1", "a2" ] + }, { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT> NOT NULL" + }, + "lookupKeys" : { + "1" : { + "type" : "FieldRef", + "index" : 2 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src1`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a0", + "dataType" : "INT NOT NULL" + }, { + "name" : "a1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "a3", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_a1_a2", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "a1", "a2" ] + }, + "indexes" : [ { + "name" : "INDEX_a1_a2", + "columns" : [ "a1", "a2" ] + }, { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src2`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "INT" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "b1", + "dataType" : "DOUBLE NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b1_b2", + "type" : "PRIMARY_KEY", + "columns" : [ "b1", "b2" ] + }, + "indexes" : [ { + "name" : "INDEX_b2", + "columns" : [ "b2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 1, 2 ], + "rightJoinKey" : [ 2, 1 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT, `b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a2 = b2) AND (a0 <> CAST(b1 AS INTEGER)))], leftToRight=[Binary], rightToLeft=[Binary], select=[a0, a1, a2, a3, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 8, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT" + } ], + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` INT, `b1` DOUBLE NOT NULL, `b2` VARCHAR(2147483647) NOT NULL, `a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT, `mix_ab` INT>", + "description" : "Calc(select=[(b0 + 1) AS b0, b1, b2, a0, a1, a2, a3, (a3 + b0) AS mix_ab])" + }, { + "id" : 9, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 4, 5 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` INT, `b1` DOUBLE NOT NULL, `b2` VARCHAR(2147483647) NOT NULL, `a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT, `mix_ab` INT>", + "description" : "Exchange(distribution=[hash[a1, a2]])" + }, { + "id" : 10, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src3`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c0", + "dataType" : "INT" + }, { + "name" : "c1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_c1_c2", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c2" ] + }, + "indexes" : [ { + "name" : "INDEX_c1_c2", + "columns" : [ "c1", "c2" ] + }, { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, no_delete_src3, project=[c1, c2], metadata=[]]], fields=[c1, c2])" + }, { + "id" : 11, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 12, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL>", + "description" : "Exchange(distribution=[hash[c1, c2]])" + }, { + "id" : 13, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src4`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "d0", + "dataType" : "INT" + }, { + "name" : "d1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "d2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_d1_d2", + "type" : "PRIMARY_KEY", + "columns" : [ "d1", "d2" ] + }, + "indexes" : [ { + "name" : "INDEX_d1_d2", + "columns" : [ "d1", "d2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, no_delete_src4, project=[d1, d2], metadata=[]]], fields=[d1, d2])" + }, { + "id" : 14, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 15, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL>", + "description" : "Exchange(distribution=[hash[d1, d2]])" + }, { + "id" : 16, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0, 1 ], + "leftUpsertKey" : [ 0, 1 ], + "rightJoinKeys" : [ 0, 1 ], + "rightUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src4`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "d0", + "dataType" : "INT" + }, { + "name" : "d1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "d2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_d1_d2", + "type" : "PRIMARY_KEY", + "columns" : [ "d1", "d2" ] + }, + "indexes" : [ { + "name" : "INDEX_d1_d2", + "columns" : [ "d1", "d2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "1" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src3`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c0", + "dataType" : "INT" + }, { + "name" : "c1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_c1_c2", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c2" ] + }, + "indexes" : [ { + "name" : "INDEX_c1_c2", + "columns" : [ "c1", "c2" ] + }, { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "1" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src3`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c0", + "dataType" : "INT" + }, { + "name" : "c1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_c1_c2", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c2" ] + }, + "indexes" : [ { + "name" : "INDEX_c1_c2", + "columns" : [ "c1", "c2" ] + }, { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src4`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "d0", + "dataType" : "INT" + }, { + "name" : "d1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "d2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_d1_d2", + "type" : "PRIMARY_KEY", + "columns" : [ "d1", "d2" ] + }, + "indexes" : [ { + "name" : "INDEX_d1_d2", + "columns" : [ "d1", "d2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0, 1 ], + "rightJoinKey" : [ 0, 1 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL, `d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((c1 = d1) AND (c2 = d2) AND (c1 <> CAST(d2 AS DOUBLE)))], leftToRight=[Binary], rightToLeft=[Binary], select=[c1, c2, d1, d2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 17, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL, `mix_cd` DOUBLE NOT NULL>", + "description" : "Calc(select=[d1, d2, (c1 + d1) AS mix_cd])" + }, { + "id" : 18, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL, `mix_cd` DOUBLE NOT NULL>", + "description" : "Exchange(distribution=[hash[d1, d2]])" + }, { + "id" : 19, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 4, 5 ], + "leftUpsertKey" : [ 3, 4, 5 ], + "rightJoinKeys" : [ 0, 1 ], + "rightUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 8, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 9, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0, 1 ], + "rightAllBinaryInputOrdinals" : [ 0, 1 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0, 1 ], + "lookupTableBinaryInputOrdinal" : 3, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src4`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "d0", + "dataType" : "INT" + }, { + "name" : "d1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "d2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_d1_d2", + "type" : "PRIMARY_KEY", + "columns" : [ "d1", "d2" ] + }, + "indexes" : [ { + "name" : "INDEX_d1_d2", + "columns" : [ "d1", "d2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 4 + }, + "1" : { + "type" : "FieldRef", + "index" : 5 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + }, { + "inputTableBinaryInputOrdinals" : [ 3 ], + "lookupTableBinaryInputOrdinal" : 2, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src3`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c0", + "dataType" : "INT" + }, { + "name" : "c1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_c1_c2", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c2" ] + }, + "indexes" : [ { + "name" : "INDEX_c1_c2", + "columns" : [ "c1", "c2" ] + }, { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "1" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 2, 3 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src1`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a0", + "dataType" : "INT NOT NULL" + }, { + "name" : "a1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "a3", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_a1_a2", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "a1", "a2" ] + }, + "indexes" : [ { + "name" : "INDEX_a1_a2", + "columns" : [ "a1", "a2" ] + }, { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT> NOT NULL" + }, + "lookupKeys" : { + "1" : { + "type" : "FieldRef", + "index" : 0 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + }, { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src2`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "INT" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "b1", + "dataType" : "DOUBLE NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b1_b2", + "type" : "PRIMARY_KEY", + "columns" : [ "b1", "b2" ] + }, + "indexes" : [ { + "name" : "INDEX_b2", + "columns" : [ "b2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "1" : { + "type" : "FieldRef", + "index" : 2 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src1`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a0", + "dataType" : "INT NOT NULL" + }, { + "name" : "a1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "a3", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_a1_a2", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "a1", "a2" ] + }, + "indexes" : [ { + "name" : "INDEX_a1_a2", + "columns" : [ "a1", "a2" ] + }, { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src2`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "INT" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "b1", + "dataType" : "DOUBLE NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b1_b2", + "type" : "PRIMARY_KEY", + "columns" : [ "b1", "b2" ] + }, + "indexes" : [ { + "name" : "INDEX_b2", + "columns" : [ "b2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src3`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c0", + "dataType" : "INT" + }, { + "name" : "c1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_c1_c2", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c2" ] + }, + "indexes" : [ { + "name" : "INDEX_c1_c2", + "columns" : [ "c1", "c2" ] + }, { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`no_delete_src4`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "d0", + "dataType" : "INT" + }, { + "name" : "d1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "d2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_d1_d2", + "type" : "PRIMARY_KEY", + "columns" : [ "d1", "d2" ] + }, + "indexes" : [ { + "name" : "INDEX_d1_d2", + "columns" : [ "d1", "d2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "changelog-mode" : "I,UA,UB", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + }, + "abilities" : [ { + "type" : "ProjectPushDown", + "projectedFields" : [ [ 1 ], [ 2 ] ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + }, { + "type" : "ReadingMetadata", + "metadataKeys" : [ ], + "producedType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ] + }, + "outputType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 8, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 9, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 4, 5 ], + "rightJoinKey" : [ 0, 1 ], + "left" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT" + } ], + "type" : "INT" + } ], + "filter" : null, + "rowTypeAfterCalc" : "ROW<`b0` INT, `b1` DOUBLE NOT NULL, `b2` VARCHAR(2147483647) NOT NULL, `a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT, `mix_ab` INT>", + "leftJoinKey" : [ 1, 2 ], + "rightJoinKey" : [ 2, 1 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL>" + } + }, + "right" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "DOUBLE NOT NULL" + } ], + "filter" : null, + "rowTypeAfterCalc" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL, `mix_cd` DOUBLE NOT NULL>", + "leftJoinKey" : [ 0, 1 ], + "rightJoinKey" : [ 0, 1 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 2, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`c1` DOUBLE NOT NULL, `c2` VARCHAR(2147483647) NOT NULL>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 3, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL>" + } + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` INT, `b1` DOUBLE NOT NULL, `b2` VARCHAR(2147483647) NOT NULL, `a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT, `mix_ab` INT, `d1` DOUBLE NOT NULL, `d2` VARCHAR(2147483647) NOT NULL, `mix_cd` DOUBLE NOT NULL>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = d1) AND (a2 = d2))], leftToRight=[[{round=[1], sourceTables=[default_catalog.default_database.no_delete_src1, default_catalog.default_database.no_delete_src2], lookupTable=[default_catalog.default_database.no_delete_src4], lookupKeys=[d1=a1, d2=a2]}, {round=[2], sourceTables=[default_catalog.default_database.no_delete_src4], lookupTable=[default_catalog.default_database.no_delete_src3], lookupKeys=[c1=d1, c2=d2], remaining=[(c1 <> CAST(d2 AS DOUBLE))]}]], rightToLeft=[[{round=[1], sourceTables=[default_catalog.default_database.no_delete_src3, default_catalog.default_database.no_delete_src4], lookupTable=[default_catalog.default_database.no_delete_src1], lookupKeys=[a1=d1, a2=d2]}, {round=[2], sourceTables=[default_catalog.default_database.no_delete_src1], lookupTable=[default_catalog.default_database.no_delete_src2], lookupKeys=[b1=a1, b2=a2], remaining=[(a0 <> CAST(b1 AS INTEGER))]}]], select=[b0, b1, b2, a0, a1, a2, a3, mix_ab, d1, d2, mix_cd])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 20, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647) NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "SPECIAL", + "internalName" : "$CAST$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 10, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT, `b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL, `EXPR$7` INT>", + "description" : "Calc(select=[a0, a1, a2, a3, b0, b2, b1, (mix_ab + CAST(mix_cd AS INTEGER)) AS EXPR$7])" + }, { + "id" : 21, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk_for_cdc_src`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "l0", + "dataType" : "INT NOT NULL" + }, { + "name" : "l1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "l2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "l3", + "dataType" : "INT" + }, { + "name" : "r0", + "dataType" : "INT" + }, { + "name" : "r2", + "dataType" : "VARCHAR(2147483647) NOT NULL" + }, { + "name" : "r1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "mix_ab_cd", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_l0_l1_l2_r1_r2", + "type" : "PRIMARY_KEY", + "columns" : [ "l0", "l1", "l2", "r1", "r2" ] + } + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "connector" : "values", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "inputUpsertKey" : [ 0, 1, 2 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647) NOT NULL, `a3` INT, `b0` INT, `b2` VARCHAR(2147483647) NOT NULL, `b1` DOUBLE NOT NULL, `EXPR$7` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk_for_cdc_src], fields=[a0, a1, a2, a3, b0, b2, b1, EXPR$7])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 4, + "target" : 5, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 5, + "target" : 6, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 3, + "target" : 7, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 6, + "target" : 7, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 9, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 13, + "target" : 14, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 14, + "target" : 15, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 12, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 16, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 17, + "target" : 18, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 9, + "target" : 19, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 18, + "target" : 19, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 21, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest_jsonplan/testJsonPlanWithTableHints.out b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest_jsonplan/testJsonPlanWithTableHints.out index de2cff243dce6..fecac42784ba3 100644 --- a/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest_jsonplan/testJsonPlanWithTableHints.out +++ b/flink-table/flink-table-planner/src/test/resources/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest_jsonplan/testJsonPlanWithTableHints.out @@ -105,60 +105,49 @@ "description" : "Exchange(distribution=[hash[b1, b2]])" }, { "id" : 5, - "type" : "stream-exec-delta-join_1", + "type" : "stream-exec-delta-join_2", "configuration" : { "table.exec.async-lookup.buffer-capacity" : "100", "table.exec.async-lookup.timeout" : "3 min" }, "joinType" : "INNER", "leftJoinKeys" : [ 1, 2 ], - "lookupRightTableJoinSpec" : { - "lookupTable" : { - "lookupTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`src2`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "b0", - "dataType" : "INT NOT NULL" - }, { - "name" : "b2", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "b1", - "dataType" : "DOUBLE" - } ], - "indexes" : [ { - "name" : "INDEX_b2", - "columns" : [ "b2" ] - } ] - }, - "comment" : "test comment", - "options" : { - "async" : "true", - "bounded" : "false", - "connector" : "values", - "failing-source" : "true", - "sink-changelog-mode-enforced" : "I,UA,D", - "sink-insert-only" : "false" - } - } - } - }, - "outputType" : "ROW<`b0` INT NOT NULL, `b2` VARCHAR(2147483647), `b1` DOUBLE> NOT NULL" - }, - "lookupKeys" : { - "1" : { - "type" : "FieldRef", - "index" : 2 - }, - "2" : { - "type" : "FieldRef", - "index" : 1 - } - }, - "remainingCondition" : { + "rightJoinKeys" : [ 2, 1 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + }, { "kind" : "CALL", "syntax" : "BINARY", "internalName" : "$>$1", @@ -172,72 +161,298 @@ "type" : "INT NOT NULL" } ], "type" : "BOOLEAN NOT NULL" - } + } ], + "type" : "BOOLEAN" }, - "rightJoinKeys" : [ 2, 1 ], - "lookupLeftTableJoinSpec" : { - "lookupTable" : { - "lookupTableSource" : { - "table" : { - "identifier" : "`default_catalog`.`default_database`.`src1`", - "resolvedTable" : { - "schema" : { - "columns" : [ { - "name" : "a0", - "dataType" : "INT NOT NULL" - }, { - "name" : "a1", - "dataType" : "DOUBLE NOT NULL" - }, { - "name" : "a2", - "dataType" : "VARCHAR(2147483647)" - }, { - "name" : "a3", - "dataType" : "INT" - } ], - "indexes" : [ { - "name" : "INDEX_a1_a2", - "columns" : [ "a1", "a2" ] - } ] - }, - "comment" : "test comment", - "options" : { - "async" : "true", - "bounded" : "false", - "connector" : "values", - "failing-source" : "true", - "sink-changelog-mode-enforced" : "I,UA,D", - "sink-insert-only" : "false" + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`src2`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "INT NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "DOUBLE" + } ], + "indexes" : [ { + "name" : "INDEX_b2", + "columns" : [ "b2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "connector" : "values", + "failing-source" : "true", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } } + }, + "outputType" : "ROW<`b0` INT NOT NULL, `b2` VARCHAR(2147483647), `b1` DOUBLE> NOT NULL" + }, + "lookupKeys" : { + "1" : { + "type" : "FieldRef", + "index" : 2 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" } }, - "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `a3` INT> NOT NULL" - }, - "lookupKeys" : { - "1" : { - "type" : "FieldRef", - "index" : 2 + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`src1`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a0", + "dataType" : "INT NOT NULL" + }, { + "name" : "a1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a3", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_a1_a2", + "columns" : [ "a1", "a2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "connector" : "values", + "failing-source" : "true", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `a3` INT> NOT NULL" + }, + "lookupKeys" : { + "1" : { + "type" : "FieldRef", + "index" : 2 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } }, - "2" : { - "type" : "FieldRef", - "index" : 1 + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`src1`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a0", + "dataType" : "INT NOT NULL" + }, { + "name" : "a1", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "a3", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_a1_a2", + "columns" : [ "a1", "a2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "connector" : "values", + "failing-source" : "true", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } } }, - "remainingCondition" : { - "kind" : "CALL", - "syntax" : "BINARY", - "internalName" : "$>$1", - "operands" : [ { - "kind" : "INPUT_REF", - "inputIndex" : 4, - "type" : "INT NOT NULL" - }, { - "kind" : "INPUT_REF", - "inputIndex" : 0, - "type" : "INT NOT NULL" - } ], - "type" : "BOOLEAN NOT NULL" + "outputType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `a3` INT> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`src2`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "INT NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "DOUBLE" + } ], + "indexes" : [ { + "name" : "INDEX_b2", + "columns" : [ "b2" ] + } ] + }, + "comment" : "test comment", + "options" : { + "async" : "true", + "bounded" : "false", + "connector" : "values", + "failing-source" : "true", + "sink-changelog-mode-enforced" : "I,UA,D", + "sink-insert-only" : "false" + } + } + } + }, + "outputType" : "ROW<`b0` INT NOT NULL, `b2` VARCHAR(2147483647), `b1` DOUBLE> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 1, 2 ], + "rightJoinKey" : [ 2, 1 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a0` INT NOT NULL, `a1` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `a3` INT>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` INT NOT NULL, `b2` VARCHAR(2147483647), `b1` DOUBLE>" + } } }, "inputProperties" : [ { diff --git a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.scala b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.scala index be6416abce1d2..ae4f95ed7225d 100644 --- a/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.scala +++ b/flink-table/flink-table-planner/src/test/scala/org/apache/flink/table/planner/plan/stream/sql/DeltaJoinTest.scala @@ -1536,7 +1536,16 @@ class DeltaJoinTest extends TableTestBase { } @Test - def testCalcBetweenDeltaJoin(): Unit = { + def testCalcBetweenCascadedDeltaJoin(): Unit = { + testCalcBetweenCascadedDeltaJoinInner(dml => util.verifyRelPlanInsert(dml)) + } + + @Test + def testCalcBetweenCascadedDeltaJoinJsonPlan(): Unit = { + testCalcBetweenCascadedDeltaJoinInner(dml => util.verifyJsonPlan(dml)) + } + + private def testCalcBetweenCascadedDeltaJoinInner(verifyFunc: String => Unit): Unit = { // DT-3 // / \ // DT-1 DT-2 @@ -1565,14 +1574,15 @@ class DeltaJoinTest extends TableTestBase { tEnv.executeSql(""" |alter table snk_for_cdc_src add (mix_ab_cd int) |""".stripMargin) - util.verifyRelPlanInsert( - """ - |insert into snk_for_cdc_src - | select a0, a1, a2, a3, b0, b2, b1, mix_ab + cast(mix_cd as int) from tmp_ab - | join tmp_cd - | on a1 = d1 and a2 = d2 - |""".stripMargin - ) + + val dml = """ + |insert into snk_for_cdc_src + | select a0, a1, a2, a3, b0, b2, b1, mix_ab + cast(mix_cd as int) from tmp_ab + | join tmp_cd + | on a1 = d1 and a2 = d2 + |""".stripMargin + + verifyFunc.apply(dml) } @Test diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/StreamingDeltaJoinOperatorFactory.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/StreamingDeltaJoinOperatorFactory.java index 9dabd3176780f..bac6a7b85ee84 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/StreamingDeltaJoinOperatorFactory.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/StreamingDeltaJoinOperatorFactory.java @@ -19,24 +19,29 @@ package org.apache.flink.table.runtime.operators; import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorParameters; import org.apache.flink.streaming.api.operators.TwoInputStreamOperatorFactory; import org.apache.flink.streaming.api.operators.legacy.YieldingOperatorFactory; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedFunction; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.join.deltajoin.AsyncDeltaJoinRunner; import org.apache.flink.table.runtime.operators.join.deltajoin.StreamingDeltaJoinOperator; import org.apache.flink.table.types.logical.RowType; +import java.util.Map; + /** The factory of {@link StreamingDeltaJoinOperator}. */ public class StreamingDeltaJoinOperatorFactory extends AbstractStreamOperatorFactory implements TwoInputStreamOperatorFactory, YieldingOperatorFactory { - private final AsyncDeltaJoinRunner rightLookupTableAsyncFunction; - private final AsyncDeltaJoinRunner leftLookupTableAsyncFunction; + private final AsyncDeltaJoinRunner left2RightRunner; + private final AsyncDeltaJoinRunner right2LeftRunner; + private final Map>> generatedFetchers; private final RowDataKeySelector leftJoinKeySelector; private final RowDataKeySelector rightJoinKeySelector; @@ -51,8 +56,9 @@ public class StreamingDeltaJoinOperatorFactory extends AbstractStreamOperatorFac private final RowType rightStreamType; public StreamingDeltaJoinOperatorFactory( - AsyncDeltaJoinRunner rightLookupTableAsyncFunction, - AsyncDeltaJoinRunner leftLookupTableAsyncFunction, + AsyncDeltaJoinRunner left2RightRunner, + AsyncDeltaJoinRunner right2LeftRunner, + Map>> generatedFetchers, RowDataKeySelector leftJoinKeySelector, RowDataKeySelector rightJoinKeySelector, long timeout, @@ -61,8 +67,9 @@ public StreamingDeltaJoinOperatorFactory( long rightSideCacheSize, RowType leftStreamType, RowType rightStreamType) { - this.rightLookupTableAsyncFunction = rightLookupTableAsyncFunction; - this.leftLookupTableAsyncFunction = leftLookupTableAsyncFunction; + this.left2RightRunner = left2RightRunner; + this.right2LeftRunner = right2LeftRunner; + this.generatedFetchers = generatedFetchers; this.leftJoinKeySelector = leftJoinKeySelector; this.rightJoinKeySelector = rightJoinKeySelector; this.timeout = timeout; @@ -79,8 +86,9 @@ public > T createStreamOperator( MailboxExecutor mailboxExecutor = getMailboxExecutor(); StreamingDeltaJoinOperator deltaJoinOperator = new StreamingDeltaJoinOperator( - rightLookupTableAsyncFunction, - leftLookupTableAsyncFunction, + left2RightRunner, + right2LeftRunner, + generatedFetchers, leftJoinKeySelector, rightJoinKeySelector, timeout, diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableAbstractCoUdfStreamOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableAbstractCoUdfStreamOperator.java index 1a219c08076a6..3702c2bbf4815 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableAbstractCoUdfStreamOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/TableAbstractCoUdfStreamOperator.java @@ -21,6 +21,7 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.DefaultOpenContext; import org.apache.flink.api.common.functions.Function; +import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.state.CheckpointListener; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -116,10 +117,13 @@ public void initializeState(StateInitializationContext context) throws Exception @Override public void open() throws Exception { + this.open(DefaultOpenContext.INSTANCE); + } + + public void open(OpenContext openContext) throws Exception { super.open(); - super.open(); - FunctionUtils.openFunction(leftTriggeredUserFunction, DefaultOpenContext.INSTANCE); - FunctionUtils.openFunction(rightTriggeredUserFunction, DefaultOpenContext.INSTANCE); + FunctionUtils.openFunction(leftTriggeredUserFunction, openContext); + FunctionUtils.openFunction(rightTriggeredUserFunction, openContext); } @Override diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/AsyncDeltaJoinRunner.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/AsyncDeltaJoinRunner.java index 41bcf8746a9fb..7c184b31522ad 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/AsyncDeltaJoinRunner.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/AsyncDeltaJoinRunner.java @@ -19,22 +19,19 @@ package org.apache.flink.table.runtime.operators.join.deltajoin; import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.api.common.functions.util.FunctionUtils; +import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.api.functions.async.CollectionSupplier; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.conversion.DataStructureConverter; import org.apache.flink.table.data.utils.JoinedRowData; -import org.apache.flink.table.runtime.collector.TableFunctionResultFuture; -import org.apache.flink.table.runtime.generated.GeneratedFunction; -import org.apache.flink.table.runtime.generated.GeneratedResultFuture; +import org.apache.flink.table.runtime.generated.FilterCondition; +import org.apache.flink.table.runtime.generated.GeneratedFilterCondition; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; -import org.apache.flink.table.runtime.operators.join.lookup.CalcCollectionCollector; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; import org.apache.flink.util.Preconditions; import org.slf4j.Logger; @@ -47,11 +44,13 @@ import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.BlockingQueue; -/** The async join runner to look up the dimension table in delta join. */ +/** The async join runner to look up one or multi dimension tables in delta join. */ public class AsyncDeltaJoinRunner extends RichAsyncFunction { private static final Logger LOG = LoggerFactory.getLogger(AsyncDeltaJoinRunner.class); @@ -61,20 +60,6 @@ public class AsyncDeltaJoinRunner extends RichAsyncFunction { "deltaJoinLeftCallAsyncFetchCostTime"; private static final String METRIC_DELTA_JOIN_RIGHT_CALL_ASYNC_FETCH_COST_TIME = "deltaJoinRightCallAsyncFetchCostTime"; - private final GeneratedFunction> generatedFetcher; - private final DataStructureConverter fetcherConverter; - private final @Nullable GeneratedFunction> - lookupSideGeneratedCalc; - private final GeneratedResultFuture> generatedResultFuture; - private final int asyncBufferCapacity; - - private transient AsyncFunction fetcher; - - protected final RowDataSerializer lookupSideRowSerializer; - - private final boolean treatRightAsLookupTable; - - private final boolean enableCache; /** Selector to get join key from left input. */ private final RowDataKeySelector leftJoinKeySelector; @@ -88,101 +73,118 @@ public class AsyncDeltaJoinRunner extends RichAsyncFunction { /** Selector to get upsert key from right input. */ private final RowDataKeySelector rightUpsertKeySelector; + private final int[] eachBinaryInputFieldSize; + private final @Nullable GeneratedFilterCondition generatedRemainingJoinCondition; + private final DeltaJoinHandlerChain handlerChainTemplate; + private final DeltaJoinRuntimeTree joinTreeTemplate; + private final Set> allDrivenInputsWhenLookup; + + private final boolean lookupRight; + private final int asyncBufferCapacity; + private final boolean enableCache; + private transient DeltaJoinCache cache; /** - * Buffers {@link ResultFuture} to avoid newInstance cost when processing elements every time. - * We use {@link BlockingQueue} to make sure the head {@link ResultFuture}s are available. + * Buffers {@link DeltaJoinProcessor} to avoid newInstance cost when processing elements every + * time. We use {@link BlockingQueue} to make sure the head {@link DeltaJoinProcessor}s are + * available. */ - private transient BlockingQueue resultFutureBuffer; + private transient BlockingQueue processorBuffer; /** - * A Collection contains all ResultFutures in the runner which is used to invoke {@code close()} - * on every ResultFuture. {@link #resultFutureBuffer} may not contain all the ResultFutures - * because ResultFutures will be polled from the buffer when processing. + * A Collection contains all DeltaJoinProcessors in the runner which is used to invoke {@code + * close()} on every {@link DeltaJoinProcessor}. {@link #processorBuffer} may not contain all + * the DeltaJoinProcessors because DeltaJoinProcessors will be polled from the buffer when + * processing. */ - private transient List allResultFutures; + private transient List allProcessors; // metrics private transient long callAsyncFetchCostTime = 0L; public AsyncDeltaJoinRunner( - GeneratedFunction> generatedFetcher, - DataStructureConverter fetcherConverter, - @Nullable GeneratedFunction> lookupSideGeneratedCalc, - GeneratedResultFuture> generatedResultFuture, - RowDataSerializer lookupSideRowSerializer, + int[] eachBinaryInputFieldSize, + @Nullable GeneratedFilterCondition generatedRemainingJoinCondition, RowDataKeySelector leftJoinKeySelector, RowDataKeySelector leftUpsertKeySelector, RowDataKeySelector rightJoinKeySelector, RowDataKeySelector rightUpsertKeySelector, + DeltaJoinHandlerChain handlerChainTemplate, + DeltaJoinRuntimeTree joinTreeTemplate, + Set> allDrivenInputsWhenLookup, + boolean lookupRight, int asyncBufferCapacity, - boolean treatRightAsLookupTable, boolean enableCache) { - this.generatedFetcher = generatedFetcher; - this.fetcherConverter = fetcherConverter; - this.lookupSideGeneratedCalc = lookupSideGeneratedCalc; - this.generatedResultFuture = generatedResultFuture; - this.lookupSideRowSerializer = lookupSideRowSerializer; + this.eachBinaryInputFieldSize = eachBinaryInputFieldSize; + this.generatedRemainingJoinCondition = generatedRemainingJoinCondition; + this.leftJoinKeySelector = leftJoinKeySelector; this.leftUpsertKeySelector = leftUpsertKeySelector; this.rightJoinKeySelector = rightJoinKeySelector; this.rightUpsertKeySelector = rightUpsertKeySelector; + + this.handlerChainTemplate = handlerChainTemplate; + this.joinTreeTemplate = joinTreeTemplate; + this.allDrivenInputsWhenLookup = allDrivenInputsWhenLookup; + this.asyncBufferCapacity = asyncBufferCapacity; - this.treatRightAsLookupTable = treatRightAsLookupTable; + this.lookupRight = lookupRight; this.enableCache = enableCache; } - public void setCache(DeltaJoinCache cache) { - this.cache = cache; - } - @Override public void open(OpenContext openContext) throws Exception { super.open(openContext); - this.fetcher = generatedFetcher.newInstance(getRuntimeContext().getUserCodeClassLoader()); - FunctionUtils.setFunctionRuntimeContext(fetcher, getRuntimeContext()); - FunctionUtils.openFunction(fetcher, openContext); - - // try to compile the generated Calc and ResultFuture, fail fast if the code is corrupt. - if (lookupSideGeneratedCalc != null) { - lookupSideGeneratedCalc.compile(getRuntimeContext().getUserCodeClassLoader()); - } - generatedResultFuture.compile(getRuntimeContext().getUserCodeClassLoader()); + DeltaJoinOpenContext deltaJoinOpenContext = (DeltaJoinOpenContext) openContext; + this.cache = deltaJoinOpenContext.getCache(); - fetcherConverter.open(getRuntimeContext().getUserCodeClassLoader()); + RuntimeContext runtimeContext = getRuntimeContext(); + ClassLoader cl = runtimeContext.getUserCodeClassLoader(); // asyncBufferCapacity + 1 as the queue size in order to avoid // blocking on the queue when taking a collector. - this.resultFutureBuffer = new ArrayBlockingQueue<>(asyncBufferCapacity + 1); - this.allResultFutures = new ArrayList<>(); - LOG.info( - "Begin to initialize reusable result futures with size {}", - asyncBufferCapacity + 1); + this.processorBuffer = new ArrayBlockingQueue<>(asyncBufferCapacity + 1); + this.allProcessors = new ArrayList<>(); + LOG.info("Begin to initialize reusable processors with size {}", asyncBufferCapacity + 1); for (int i = 0; i < asyncBufferCapacity + 1; i++) { - JoinedRowResultFuture rf = - new JoinedRowResultFuture( - resultFutureBuffer, - createCalcFunction(openContext), - createFetcherResultFuture(openContext), - fetcherConverter, - treatRightAsLookupTable, - leftUpsertKeySelector, - rightUpsertKeySelector, - lookupSideRowSerializer, + DeltaJoinRuntimeTree joinRuntimeTree = joinTreeTemplate.copy(); + MultiInputRowDataBuffer multiInputRowDataBuffer = + new MultiInputRowDataBuffer( + eachBinaryInputFieldSize, joinRuntimeTree, allDrivenInputsWhenLookup); + DeltaJoinHandlerChain chain = handlerChainTemplate.copy(); + + FilterCondition remainingJoinCondition = + Optional.ofNullable(this.generatedRemainingJoinCondition) + .map(c -> c.newInstance(cl)) + .orElse(null); + + DeltaJoinProcessor processor = + new DeltaJoinProcessor( + processorBuffer, + multiInputRowDataBuffer, + chain, + remainingJoinCondition, enableCache, - cache); + cache, + lookupRight, + leftUpsertKeySelector.copy(), + rightUpsertKeySelector.copy(), + runtimeContext); + + processor.open(deltaJoinOpenContext); + // add will throw exception immediately if the queue is full which should never happen - resultFutureBuffer.add(rf); - allResultFutures.add(rf); + processorBuffer.add(processor); + allProcessors.add(processor); } - LOG.info("Finish initializing reusable result futures"); + LOG.info("Finish initializing reusable processors"); getRuntimeContext() .getMetricGroup() .gauge( - treatRightAsLookupTable + lookupRight ? METRIC_DELTA_JOIN_LEFT_CALL_ASYNC_FETCH_COST_TIME : METRIC_DELTA_JOIN_RIGHT_CALL_ASYNC_FETCH_COST_TIME, () -> callAsyncFetchCostTime); @@ -190,11 +192,11 @@ public void open(OpenContext openContext) throws Exception { @Override public void asyncInvoke(RowData input, ResultFuture resultFuture) throws Exception { - JoinedRowResultFuture outResultFuture = resultFutureBuffer.take(); + DeltaJoinProcessor deltaJoinProcessor = processorBuffer.take(); RowData streamJoinKey = null; if (enableCache) { - if (treatRightAsLookupTable) { + if (lookupRight) { streamJoinKey = leftJoinKeySelector.getKey(input); cache.requestRightCache(); } else { @@ -204,79 +206,48 @@ public void asyncInvoke(RowData input, ResultFuture resultFuture) throw } // the input row is copied when object reuse in StreamDeltaJoinOperator - outResultFuture.reset(streamJoinKey, input, resultFuture); + deltaJoinProcessor.reset(streamJoinKey, input, resultFuture); if (enableCache) { - Optional> dataFromCache = tryGetDataFromCache(streamJoinKey); + Optional> dataFromCache = tryGetDataFromCache(streamJoinKey); if (dataFromCache.isPresent()) { - outResultFuture.complete(dataFromCache.get(), true); + deltaJoinProcessor.complete(dataFromCache.get()); return; } } long startTime = System.currentTimeMillis(); // fetcher has copied the input field when object reuse is enabled - fetcher.asyncInvoke(input, outResultFuture); + deltaJoinProcessor.asyncHandle(input); callAsyncFetchCostTime = System.currentTimeMillis() - startTime; } - @Nullable - private FlatMapFunction createCalcFunction(OpenContext openContext) - throws Exception { - FlatMapFunction calc = null; - if (lookupSideGeneratedCalc != null) { - calc = - lookupSideGeneratedCalc.newInstance( - getRuntimeContext().getUserCodeClassLoader()); - FunctionUtils.setFunctionRuntimeContext(calc, getRuntimeContext()); - FunctionUtils.openFunction(calc, openContext); - } - return calc; - } - - public TableFunctionResultFuture createFetcherResultFuture(OpenContext openContext) - throws Exception { - TableFunctionResultFuture resultFuture = - generatedResultFuture.newInstance(getRuntimeContext().getUserCodeClassLoader()); - FunctionUtils.setFunctionRuntimeContext(resultFuture, getRuntimeContext()); - FunctionUtils.openFunction(resultFuture, openContext); - return resultFuture; - } - @Override public void close() throws Exception { super.close(); - if (fetcher != null) { - FunctionUtils.closeFunction(fetcher); - } - if (allResultFutures != null) { - for (JoinedRowResultFuture rf : allResultFutures) { - rf.close(); + if (allProcessors != null) { + for (DeltaJoinProcessor processor : allProcessors) { + processor.close(); } } } - @VisibleForTesting - public AsyncFunction getFetcher() { - return fetcher; - } - @VisibleForTesting public DeltaJoinCache getCache() { return cache; } - private Optional> tryGetDataFromCache(RowData joinKey) { + private Optional> tryGetDataFromCache(RowData joinKey) { Preconditions.checkState(enableCache); - if (treatRightAsLookupTable) { - LinkedHashMap rightRows = cache.getData(joinKey, true); + if (lookupRight) { + LinkedHashMap rightRows = cache.getData(joinKey, true); if (rightRows != null) { cache.hitRightCache(); return Optional.of(rightRows.values()); } } else { - LinkedHashMap leftRows = cache.getData(joinKey, false); + LinkedHashMap leftRows = cache.getData(joinKey, false); if (leftRows != null) { cache.hitLeftCache(); return Optional.of(leftRows.values()); @@ -286,26 +257,32 @@ private Optional> tryGetDataFromCache(RowData joinKey) { } /** - * The {@link JoinedRowResultFuture} is used to combine left {@link RowData} and right {@link - * RowData} into {@link JoinedRowData}. + * The processor to handle the inputs of the delta join runner. + * + *

It mainly does the following things: * - *

There are 3 phases in this collector similar with {@see - * AsyncLookupJoinRunner#JoinedRowResultFuture}. Furthermore, this {@link JoinedRowResultFuture} - * also handles logic about bidirectional lookup join processing. + *

    + *
  1. Trigger to lookup by invoking {@link DeltaJoinHandlerChain#asyncHandle} and get all + * lookup results back. + *
  2. Build and update the cache if necessary. + *
  3. Do the final filter for each lookup result. + *
*/ @VisibleForTesting - public static final class JoinedRowResultFuture implements ResultFuture { - private final BlockingQueue resultFutureBuffer; - private final @Nullable FlatMapFunction calcFunction; - private final CalcCollectionCollector calcCollector; - private final TableFunctionResultFuture joinConditionResultFuture; - private final DataStructureConverter resultConverter; + public static final class DeltaJoinProcessor implements ResultFuture { + + private static final Logger LOG = LoggerFactory.getLogger(DeltaJoinProcessor.class); + + private final BlockingQueue processorBuffer; + + private final DeltaJoinHandlerChain handlerChain; + private final MultiInputRowDataBuffer multiInputRowDataBuffer; + private final @Nullable FilterCondition remainingJoinCondition; private final boolean enableCache; private final DeltaJoinCache cache; - - private final DelegateResultFuture delegate; - private final boolean treatRightAsLookupTable; + private final boolean lookupRight; + private final RuntimeContext runtimeContext; private final RowDataKeySelector leftUpsertKeySelector; private final RowDataKeySelector rightUpsertKeySelector; @@ -314,28 +291,55 @@ public static final class JoinedRowResultFuture implements ResultFuture private RowData streamRow; private ResultFuture realOutput; - private JoinedRowResultFuture( - BlockingQueue resultFutureBuffer, - @Nullable FlatMapFunction calcFunction, - TableFunctionResultFuture joinConditionResultFuture, - DataStructureConverter resultConverter, - boolean treatRightAsLookupTable, + private DeltaJoinProcessor( + BlockingQueue processorBuffer, + MultiInputRowDataBuffer multiInputRowDataBuffer, + DeltaJoinHandlerChain handlerChain, + @Nullable FilterCondition remainingJoinCondition, + boolean enableCache, + DeltaJoinCache cache, + boolean lookupRight, RowDataKeySelector leftUpsertKeySelector, RowDataKeySelector rightUpsertKeySelector, - RowDataSerializer lookupSideRowSerializer, - boolean enableCache, - DeltaJoinCache cache) { - this.resultFutureBuffer = resultFutureBuffer; - this.calcFunction = calcFunction; - this.calcCollector = new CalcCollectionCollector(lookupSideRowSerializer); - this.joinConditionResultFuture = joinConditionResultFuture; - this.resultConverter = resultConverter; + RuntimeContext runtimeContext) { + this.processorBuffer = processorBuffer; + this.multiInputRowDataBuffer = multiInputRowDataBuffer; + this.handlerChain = handlerChain; + this.remainingJoinCondition = remainingJoinCondition; this.enableCache = enableCache; this.cache = cache; - this.delegate = new DelegateResultFuture(); - this.treatRightAsLookupTable = treatRightAsLookupTable; + this.lookupRight = lookupRight; this.leftUpsertKeySelector = leftUpsertKeySelector; this.rightUpsertKeySelector = rightUpsertKeySelector; + this.runtimeContext = runtimeContext; + } + + public void open(DeltaJoinOpenContext openContext) throws Exception { + multiInputRowDataBuffer.open(openContext, runtimeContext); + + HandlerChainContextImpl context = + HandlerChainContextImpl.create( + multiInputRowDataBuffer, + this, + runtimeContext, + openContext.getMailboxExecutor(), + openContext.getLookupFunctions(), + lookupRight); + handlerChain.open(openContext, context); + + if (remainingJoinCondition != null) { + FunctionUtils.setFunctionRuntimeContext(remainingJoinCondition, runtimeContext); + FunctionUtils.openFunction(remainingJoinCondition, openContext); + } + } + + public void close() throws Exception { + multiInputRowDataBuffer.close(); + handlerChain.close(); + + if (remainingJoinCondition != null) { + remainingJoinCondition.close(); + } } public void reset( @@ -346,75 +350,56 @@ public void reset( this.streamJoinKey = joinKey; this.streamRow = row; - joinConditionResultFuture.setInput(row); - joinConditionResultFuture.setResultFuture(delegate); - delegate.reset(); - calcCollector.reset(); + this.multiInputRowDataBuffer.reset(); + this.handlerChain.reset(); } - @Override - public void complete(Collection result) { - complete(result, false); + public void asyncHandle(RowData input) throws Exception { + handlerChain.asyncHandle(input); } - public void complete(Collection result, boolean fromCache) { - if (result == null) { - result = Collections.emptyList(); - } - - Collection rowDataCollection = convertToInternalData(result); - - Collection lookupRowsAfterCalc = rowDataCollection; - if (!fromCache && calcFunction != null && rowDataCollection != null) { - for (RowData row : rowDataCollection) { - try { - calcFunction.flatMap(row, calcCollector); - } catch (Exception e) { - completeExceptionally(e); - } - } - lookupRowsAfterCalc = calcCollector.getCollection(); + @Override + public void complete(Collection lookupRows) { + if (lookupRows == null) { + lookupRows = Collections.emptyList(); } - // now we have received the rows from the lookup table, try to set them to the cache + // now we have received the rows from the lookup tables, + // try to set them to the cache try { - updateCacheIfNecessary(lookupRowsAfterCalc); + updateCacheIfNecessary(lookupRows); } catch (Throwable t) { LOG.info("Failed to update the cache", t); completeExceptionally(t); return; } - // call join condition collector, - // the filtered result will be routed to the delegateCollector - try { - joinConditionResultFuture.complete(lookupRowsAfterCalc); - } catch (Throwable t) { - // we should catch the exception here to let the framework know - completeExceptionally(t); - return; - } - - Collection lookupRowsAfterJoin = delegate.collection; - if (lookupRowsAfterJoin == null || lookupRowsAfterJoin.isEmpty()) { - realOutput.complete(Collections.emptyList()); + List outRows; + if (lookupRows.isEmpty()) { + outRows = Collections.emptyList(); } else { - List outRows = new ArrayList<>(); - for (RowData lookupRow : lookupRowsAfterJoin) { - RowData outRow; - if (treatRightAsLookupTable) { - outRow = new JoinedRowData(streamRow.getRowKind(), streamRow, lookupRow); - } else { - outRow = new JoinedRowData(streamRow.getRowKind(), lookupRow, streamRow); + outRows = new ArrayList<>(); + for (RowData lookupRow : lookupRows) { + JoinedRowData outRow = + lookupRight + ? new JoinedRowData( + streamRow.getRowKind(), streamRow, lookupRow) + : new JoinedRowData( + streamRow.getRowKind(), lookupRow, streamRow); + if (remainingJoinCondition != null + && !remainingJoinCondition.apply( + FilterCondition.Context.INVALID_CONTEXT, outRow)) { + continue; } outRows.add(outRow); } - realOutput.complete(outRows); } + + realOutput.complete(outRows); try { // put this collector to the queue to avoid this collector is used // again before outRows in the collector is not consumed. - resultFutureBuffer.put(this); + processorBuffer.put(this); } catch (InterruptedException e) { completeExceptionally(e); } @@ -430,14 +415,10 @@ public void completeExceptionally(Throwable error) { * the mailbox to invoke from the caller thread. */ @Override - public void complete(CollectionSupplier supplier) { + public void complete(CollectionSupplier supplier) { throw new UnsupportedOperationException(); } - public void close() throws Exception { - joinConditionResultFuture.close(); - } - private void updateCacheIfNecessary(Collection lookupRows) throws Exception { if (!enableCache) { return; @@ -445,12 +426,12 @@ private void updateCacheIfNecessary(Collection lookupRows) throws Excep // 1. build the cache in lookup side if not exists // 2. update the cache in stream side if exists - if (treatRightAsLookupTable) { + if (lookupRight) { if (cache.getData(streamJoinKey, true) == null) { cache.buildCache(streamJoinKey, buildMapWithUkAsKeys(lookupRows, true), true); } - LinkedHashMap leftCacheData = cache.getData(streamJoinKey, false); + LinkedHashMap leftCacheData = cache.getData(streamJoinKey, false); if (leftCacheData != null) { RowData uk = leftUpsertKeySelector.getKey(streamRow); cache.upsertCache(streamJoinKey, uk, streamRow, false); @@ -460,7 +441,7 @@ private void updateCacheIfNecessary(Collection lookupRows) throws Excep cache.buildCache(streamJoinKey, buildMapWithUkAsKeys(lookupRows, false), false); } - LinkedHashMap rightCacheData = cache.getData(streamJoinKey, true); + LinkedHashMap rightCacheData = cache.getData(streamJoinKey, true); if (rightCacheData != null) { RowData uk = rightUpsertKeySelector.getKey(streamRow); cache.upsertCache(streamJoinKey, uk, streamRow, true); @@ -468,70 +449,97 @@ private void updateCacheIfNecessary(Collection lookupRows) throws Excep } } - private LinkedHashMap buildMapWithUkAsKeys( + private LinkedHashMap buildMapWithUkAsKeys( Collection lookupRows, boolean treatRightAsLookupTable) throws Exception { - LinkedHashMap map = new LinkedHashMap<>(); - for (Object lookupRow : lookupRows) { - RowData rowData = convertToInternalData(lookupRow); + LinkedHashMap map = new LinkedHashMap<>(); + for (RowData lookupRow : lookupRows) { RowData uk; if (treatRightAsLookupTable) { - uk = rightUpsertKeySelector.getKey(rowData); + uk = rightUpsertKeySelector.getKey(lookupRow); map.put(uk, lookupRow); } else { - uk = leftUpsertKeySelector.getKey(rowData); + uk = leftUpsertKeySelector.getKey(lookupRow); map.put(uk, lookupRow); } } return map; } + } - private RowData convertToInternalData(Object data) { - if (resultConverter.isIdentityConversion()) { - return (RowData) data; - } else { - return resultConverter.toInternal(data); - } + /** + * A {@link DeltaJoinHandlerBase.DeltaJoinHandlerContext} that exposes some context for the + * lookup chain. + */ + private static class HandlerChainContextImpl + implements DeltaJoinHandlerBase.DeltaJoinHandlerContext { + + private final MultiInputRowDataBuffer sharedMultiInputRowDataBuffer; + private final ResultFuture realOutput; + private final RuntimeContext runtimeContext; + private final MailboxExecutor mailboxExecutor; + private final Map> lookupFunctions; + private final boolean inLeft2RightLookupChain; + + private HandlerChainContextImpl( + MultiInputRowDataBuffer sharedMultiInputRowDataBuffer, + ResultFuture realOutput, + RuntimeContext runtimeContext, + MailboxExecutor mailboxExecutor, + Map> lookupFunctions, + boolean inLeft2RightLookupChain) { + this.sharedMultiInputRowDataBuffer = sharedMultiInputRowDataBuffer; + this.realOutput = realOutput; + this.runtimeContext = runtimeContext; + this.mailboxExecutor = mailboxExecutor; + this.lookupFunctions = lookupFunctions; + this.inLeft2RightLookupChain = inLeft2RightLookupChain; } - @SuppressWarnings({"unchecked", "rawtypes"}) - private Collection convertToInternalData(Collection data) { - if (resultConverter.isIdentityConversion()) { - return (Collection) data; - } else { - Collection result = new ArrayList<>(data.size()); - for (Object element : data) { - result.add(resultConverter.toInternal(element)); - } - return result; - } + public static HandlerChainContextImpl create( + MultiInputRowDataBuffer sharedMultiInputRowDataBuffer, + ResultFuture realOutput, + RuntimeContext runtimeContext, + MailboxExecutor mailboxExecutor, + Map> lookupFunctions, + boolean inLeft2RightLookupChain) { + return new HandlerChainContextImpl( + sharedMultiInputRowDataBuffer, + realOutput, + runtimeContext, + mailboxExecutor, + lookupFunctions, + inLeft2RightLookupChain); } - private final class DelegateResultFuture implements ResultFuture { + @Override + public MultiInputRowDataBuffer getSharedMultiInputRowDataBuffer() { + return sharedMultiInputRowDataBuffer; + } - private Collection collection; + @Override + public ResultFuture getRealOutputResultFuture() { + return realOutput; + } - public void reset() { - this.collection = null; - } + @Override + public RuntimeContext getRuntimeContext() { + return runtimeContext; + } - @Override - public void complete(Collection result) { - this.collection = result; - } + @Override + public MailboxExecutor getMailboxExecutor() { + return mailboxExecutor; + } - @Override - public void completeExceptionally(Throwable error) { - JoinedRowResultFuture.this.completeExceptionally(error); - } + @Override + public AsyncFunction getLookupFunction(int lookupOrdinal) { + Preconditions.checkArgument(lookupFunctions.containsKey(lookupOrdinal)); + return lookupFunctions.get(lookupOrdinal); + } - /** - * Unsupported, because the containing classes are AsyncFunctions which don't have - * access to the mailbox to invoke from the caller thread. - */ - @Override - public void complete(CollectionSupplier supplier) { - throw new UnsupportedOperationException(); - } + @Override + public boolean inLeft2RightLookupChain() { + return inLeft2RightLookupChain; } } } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java new file mode 100644 index 0000000000000..3f366dce14ca1 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java @@ -0,0 +1,105 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedFunction; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.util.Collection; + +/** + * A lookup handler implement to do a binary lookup on a dim table. + * + *

Different with {@code CascadedLookupHandler}, the {@link BinaryLookupHandler} only looks up a + * single table and do not be chained with other {@link DeltaJoinHandlerBase}s. Therefore, it allows + * more optimizations compared to the {@code CascadedLookupHandler}. Specifically: + * + *

    + *
  1. There only be a single input row. + *
  2. There is no need to deduplicate results using upsert keys. + *
  3. There is no need to filter the result with remaining conditions, and the lookup results + * will be directly returned. + *
+ */ +public class BinaryLookupHandler extends LookupHandlerBase { + + private static final long serialVersionUID = 1L; + + public BinaryLookupHandler( + DataType streamSideType, + DataType lookupResultType, + DataType lookupSidePassThroughCalcType, + RowDataSerializer lookupSidePassThroughCalcRowSerializer, + @Nullable GeneratedFunction> lookupSideGeneratedCalc, + int[] ownedSourceOrdinals, + int ownedLookupOrdinal) { + super( + streamSideType, + lookupResultType, + lookupSidePassThroughCalcType, + lookupSidePassThroughCalcRowSerializer, + lookupSideGeneratedCalc, + ownedSourceOrdinals, + ownedLookupOrdinal, + "BinaryLookupHandler"); + } + + @Override + public void setNext(@Nullable DeltaJoinHandlerBase next) { + super.setNext(next); + + Preconditions.checkArgument( + next == null, "This binary handler should not have a concrete handler after it"); + } + + @Override + public void asyncHandle() throws Exception { + Collection allSourceRowData = + handlerContext.getSharedMultiInputRowDataBuffer().getData(ownedSourceOrdinals); + + Preconditions.checkState(allSourceRowData.size() == 1); + + RowData input = allSourceRowData.stream().findFirst().get(); + + fetcher.asyncInvoke(input, createLookupResultFuture(input)); + } + + @Override + protected void completeResultsInMailbox(RowData input, Collection lookupResult) { + handlerContext.getRealOutputResultFuture().complete(lookupResult); + } + + @Override + protected DeltaJoinHandlerBase copyInternal() { + return new BinaryLookupHandler( + streamSideType, + lookupResultType, + lookupSidePassThroughCalcType, + lookupSidePassThroughCalcRowSerializer, + lookupSideGeneratedCalc, + ownedSourceOrdinals, + ownedLookupOrdinal); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java index ffaca5736bd70..61816739c4d06 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinCache.java @@ -53,8 +53,8 @@ public class DeltaJoinCache { private static final String METRIC_TOTAL_NON_EMPTY_VALUE_SIZE = "totalNonEmptyValues"; // use LinkedHashMap to keep order - private final Cache> leftCache; - private final Cache> rightCache; + private final Cache> leftCache; + private final Cache> rightCache; // metrics private final AtomicLong leftTotalSize = new AtomicLong(0L); @@ -115,12 +115,12 @@ public void registerMetrics(MetricGroup metricGroup) { } @Nullable - public LinkedHashMap getData(RowData key, boolean requestRightCache) { + public LinkedHashMap getData(RowData key, boolean requestRightCache) { return requestRightCache ? rightCache.getIfPresent(key) : leftCache.getIfPresent(key); } public void buildCache( - RowData key, LinkedHashMap ukDataMap, boolean buildRightCache) { + RowData key, LinkedHashMap ukDataMap, boolean buildRightCache) { Preconditions.checkState(getData(key, buildRightCache) == null); if (buildRightCache) { rightCache.put(key, ukDataMap); @@ -131,7 +131,7 @@ public void buildCache( } } - public void upsertCache(RowData key, RowData uk, Object data, boolean upsertRightCache) { + public void upsertCache(RowData key, RowData uk, RowData data, boolean upsertRightCache) { if (upsertRightCache) { upsert(rightCache, key, uk, data, rightTotalSize); } else { @@ -140,16 +140,16 @@ public void upsertCache(RowData key, RowData uk, Object data, boolean upsertRigh } private void upsert( - Cache> cache, + Cache> cache, RowData key, RowData uk, - Object data, + RowData data, AtomicLong cacheTotalSize) { cache.asMap() .computeIfPresent( key, (k, v) -> { - Object oldData = v.put(uk, data); + RowData oldData = v.put(uk, data); if (oldData == null) { cacheTotalSize.incrementAndGet(); } @@ -174,7 +174,7 @@ public void hitRightCache() { } private class DeltaJoinCacheRemovalListener - implements RemovalListener> { + implements RemovalListener> { private final boolean isLeftCache; @@ -184,7 +184,7 @@ public DeltaJoinCacheRemovalListener(boolean isLeftCache) { @Override public void onRemoval( - RemovalNotification> removalNotification) { + RemovalNotification> removalNotification) { if (removalNotification.getValue() == null) { return; } @@ -200,12 +200,12 @@ public void onRemoval( // ===== visible for test ===== @VisibleForTesting - public Cache> getLeftCache() { + public Cache> getLeftCache() { return leftCache; } @VisibleForTesting - public Cache> getRightCache() { + public Cache> getRightCache() { return rightCache; } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerBase.java new file mode 100644 index 0000000000000..0a66e77c43e86 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerBase.java @@ -0,0 +1,112 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.data.RowData; + +import javax.annotation.Nullable; + +import java.io.Serializable; + +/** A base class for different type delta join handler. */ +public abstract class DeltaJoinHandlerBase implements Serializable { + + private static final long serialVersionUID = 1L; + + @Nullable protected DeltaJoinHandlerBase next; + + protected transient OpenContext openContext; + protected transient DeltaJoinHandlerContext handlerContext; + + public void setNext(@Nullable DeltaJoinHandlerBase next) { + this.next = next; + } + + public void open(OpenContext openContext, DeltaJoinHandlerContext handlerContext) + throws Exception { + this.openContext = openContext; + this.handlerContext = handlerContext; + + if (next != null) { + next.open(openContext, handlerContext); + } + } + + /** Start to handle this {@link DeltaJoinHandlerBase}. */ + public abstract void asyncHandle() throws Exception; + + public void completeExceptionally(Throwable error) { + handlerContext.getRealOutputResultFuture().completeExceptionally(error); + } + + public final DeltaJoinHandlerBase copy() { + DeltaJoinHandlerBase newNext = null; + if (next != null) { + newNext = next.copy(); + } + + DeltaJoinHandlerBase newThis = this.copyInternal(); + + newThis.setNext(newNext); + return newThis; + } + + protected abstract DeltaJoinHandlerBase copyInternal(); + + public void reset() { + if (next != null) { + next.reset(); + } + } + + public void close() throws Exception { + if (next != null) { + next.close(); + } + } + + @VisibleForTesting + @Nullable + public DeltaJoinHandlerBase getNext() { + return next; + } + + /** A context for delta join handler. */ + public interface DeltaJoinHandlerContext { + + /** Get the buffer shared on all lookup handlers within a single delta join chain. */ + MultiInputRowDataBuffer getSharedMultiInputRowDataBuffer(); + + ResultFuture getRealOutputResultFuture(); + + RuntimeContext getRuntimeContext(); + + MailboxExecutor getMailboxExecutor(); + + AsyncFunction getLookupFunction(int lookupOrdinal); + + boolean inLeft2RightLookupChain(); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerChain.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerChain.java new file mode 100644 index 0000000000000..70dc4e8b12fdc --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinHandlerChain.java @@ -0,0 +1,131 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; + +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + +/** + * A runtime chain for delta join to lookup tables. + * + *

It serves as the entry point that triggers multiple chained handlers when data arrives at the + * stream side. + */ +public class DeltaJoinHandlerChain implements Serializable { + + private static final long serialVersionUID = 1L; + + private final DeltaJoinHandlerBase head; + private final int[] streamInputOrdinals; + + private transient MultiInputRowDataBuffer sharedMultiInputRowDataBuffer; + + private DeltaJoinHandlerChain(DeltaJoinHandlerBase head, int[] streamInputOrdinals) { + this.head = head; + + this.streamInputOrdinals = streamInputOrdinals; + + Preconditions.checkArgument(streamInputOrdinals.length > 0); + } + + /** Chain the handlers together to form a processing pipeline. */ + public static DeltaJoinHandlerChain build( + List handlers, int[] streamInputOrdinals) { + Preconditions.checkArgument(!handlers.isEmpty()); + + DeltaJoinHandlerBase header = null; + for (int i = 0; i < handlers.size(); i++) { + DeltaJoinHandlerBase handler = handlers.get(i); + if (i == 0) { + header = handler; + } + if (i != handlers.size() - 1) { + handler.setNext(handlers.get(i + 1)); + } + } + + return new DeltaJoinHandlerChain(header, streamInputOrdinals); + } + + public void open( + OpenContext openContext, DeltaJoinHandlerBase.DeltaJoinHandlerContext handlerContext) + throws Exception { + this.sharedMultiInputRowDataBuffer = handlerContext.getSharedMultiInputRowDataBuffer(); + + head.open(openContext, handlerContext); + } + + /** Start handing one input row from stream side to trigger the chained handlers. */ + public void asyncHandle(RowData input) throws Exception { + prepareDataInBuffer(input); + head.asyncHandle(); + } + + public DeltaJoinHandlerChain copy() { + DeltaJoinHandlerBase newHead = head.copy(); + return new DeltaJoinHandlerChain(newHead, streamInputOrdinals); + } + + public void reset() { + head.reset(); + } + + public void close() throws Exception { + head.close(); + } + + @VisibleForTesting + public DeltaJoinHandlerBase getHead() { + return head; + } + + /** + * Place the data arriving from the stream side into the {@link MultiInputRowDataBuffer}. + * + *

If the upstream of this stream side is directly a binary input, the data will be placed + * into the single corresponding slot. For example, if we are processing data from the left + * (stream) side A in {@code DeltaJoin1}, the data fills slot 0 only in {@link + * MultiInputRowDataBuffer}. + * + *

If the upstream of this stream side is a delta join, the prepared data will represent the + * dataset of binary inputs A and B after being processed by the upstream delta join, and it + * will span multiple input ordinals in the buffer. For example, if we are processing data from + * the left (stream) side {@code DeltaJoin1} in {@code DeltaJoin2}, the data will span the slots + * 0 and 1 in {@link MultiInputRowDataBuffer}: + * + *

{@code
+     *         DeltaJoin2
+     *       /              \
+     *  DeltaJoin1       #2 C
+     *    /      \
+     * #0 A    #1 B
+     * }
+ */ + private void prepareDataInBuffer(RowData data) { + sharedMultiInputRowDataBuffer.prepareDataInBuffer( + data, Arrays.stream(streamInputOrdinals).boxed().collect(Collectors.toSet())); + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinOpenContext.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinOpenContext.java new file mode 100644 index 0000000000000..8dd82a2888a36 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinOpenContext.java @@ -0,0 +1,58 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.table.data.RowData; + +import java.util.Map; + +/** + * A {@link OpenContext} defines shared resources between {@link StreamingDeltaJoinOperator} and two + * {@link AsyncDeltaJoinRunner}. + */ +public class DeltaJoinOpenContext implements OpenContext { + + private final DeltaJoinCache cache; + private final MailboxExecutor mailboxExecutor; + private final Map> lookupFunctions; + + public DeltaJoinOpenContext( + DeltaJoinCache cache, + MailboxExecutor mailboxExecutor, + Map> lookupFunctions) { + this.cache = cache; + this.mailboxExecutor = mailboxExecutor; + this.lookupFunctions = lookupFunctions; + } + + public DeltaJoinCache getCache() { + return cache; + } + + public MailboxExecutor getMailboxExecutor() { + return mailboxExecutor; + } + + public Map> getLookupFunctions() { + return lookupFunctions; + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinRuntimeTree.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinRuntimeTree.java new file mode 100644 index 0000000000000..eb02727e78f83 --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/DeltaJoinRuntimeTree.java @@ -0,0 +1,221 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.functions.util.FunctionUtils; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.FilterCondition; +import org.apache.flink.table.runtime.generated.GeneratedFilterCondition; +import org.apache.flink.table.runtime.generated.GeneratedFunction; +import org.apache.flink.table.runtime.operators.join.FlinkJoinType; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.util.Preconditions; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +/** + * A runtime delta join tree converted from {@code DeltaJoinTree}. + * + *

Its structure is fundamentally similar to {@code DeltaJoinTree}. The only difference is that + * this tree resides within the runtime module, and holds the function generated by the calc on the + * {@link Node}. + * + *

See {@code DeltaJoinTree} for more details. + */ +public class DeltaJoinRuntimeTree implements Serializable { + + public final Node root; + + public DeltaJoinRuntimeTree(Node root) { + this.root = root; + + // validate + List allInputOrdinals = + root.getAllInputOrdinals().stream().sorted().collect(Collectors.toList()); + Preconditions.checkArgument( + allInputOrdinals.equals( + IntStream.range(0, allInputOrdinals.size()) + .boxed() + .collect(Collectors.toList()))); + } + + public void open(RuntimeContext runtimeContext, OpenContext openContext) throws Exception { + root.open(runtimeContext, openContext); + } + + public void close() throws Exception { + root.close(); + } + + public DeltaJoinRuntimeTree copy() { + Node newRoot = root.copy(); + return new DeltaJoinRuntimeTree(newRoot); + } + + /** + * An abstract node for {@link BinaryInputNode} and {@link JoinNode}. + * + *

See {@code DeltaJoinTree#Node} for more details. + */ + public abstract static class Node implements Serializable { + + protected final @Nullable GeneratedFunction> + generatedCalc; + public final RowDataSerializer rowDataSerializerPassThroughCalc; + + public transient @Nullable FlatMapFunction calc; + + public Node( + @Nullable GeneratedFunction> generatedCalc, + RowDataSerializer rowDataSerializerPassThroughCalc) { + this.generatedCalc = generatedCalc; + this.rowDataSerializerPassThroughCalc = rowDataSerializerPassThroughCalc; + } + + public void open(RuntimeContext runtimeContext, OpenContext openContext) throws Exception { + if (generatedCalc != null) { + calc = generatedCalc.newInstance(runtimeContext.getUserCodeClassLoader()); + FunctionUtils.setFunctionRuntimeContext(calc, runtimeContext); + FunctionUtils.openFunction(calc, openContext); + } + } + + public void close() throws Exception { + if (calc != null) { + FunctionUtils.closeFunction(calc); + } + } + + public abstract Set getAllInputOrdinals(); + + public abstract Node copy(); + } + + /** + * A {@link Node} in the tree representing a source used for delta join to scan and lookup. + * + *

See {@code DeltaJoinTree#BinaryInputNode} for more details. + */ + public static class BinaryInputNode extends Node { + public final int inputOrdinal; + + public BinaryInputNode( + int inputOrdinal, + @Nullable GeneratedFunction> generatedCalc, + RowDataSerializer rowDataSerializerPassThroughCalc) { + super(generatedCalc, rowDataSerializerPassThroughCalc); + this.inputOrdinal = inputOrdinal; + } + + @Override + public Set getAllInputOrdinals() { + return Collections.singleton(inputOrdinal); + } + + @Override + public Node copy() { + return new BinaryInputNode( + inputOrdinal, generatedCalc, rowDataSerializerPassThroughCalc); + } + } + + /** + * A {@link Node} in the tree representing a join operation between two inputs. + * + *

See {@code DeltaJoinTree#JoinNode} for more details. + */ + public static class JoinNode extends Node { + public final FlinkJoinType joinType; + public final GeneratedFilterCondition generatedJoinCondition; + public final Node left; + public final Node right; + + public transient FilterCondition joinCondition; + + public JoinNode( + FlinkJoinType joinType, + GeneratedFilterCondition generatedJoinCondition, + @Nullable GeneratedFunction> generatedCalc, + Node left, + Node right, + RowDataSerializer rowDataSerializerPassThroughCalc) { + super(generatedCalc, rowDataSerializerPassThroughCalc); + this.joinType = joinType; + this.generatedJoinCondition = generatedJoinCondition; + this.left = left; + this.right = right; + } + + @Override + public void open(RuntimeContext runtimeContext, OpenContext openContext) throws Exception { + super.open(runtimeContext, openContext); + joinCondition = + generatedJoinCondition.newInstance(runtimeContext.getUserCodeClassLoader()); + FunctionUtils.setFunctionRuntimeContext(joinCondition, runtimeContext); + FunctionUtils.openFunction(joinCondition, openContext); + + left.open(runtimeContext, openContext); + right.open(runtimeContext, openContext); + } + + @Override + public void close() throws Exception { + super.close(); + + if (joinCondition != null) { + FunctionUtils.closeFunction(joinCondition); + } + + left.close(); + right.close(); + } + + @Override + public Set getAllInputOrdinals() { + return Stream.concat( + left.getAllInputOrdinals().stream(), + right.getAllInputOrdinals().stream()) + .collect(Collectors.toSet()); + } + + @Override + public Node copy() { + Node newLeft = left.copy(); + Node newRight = right.copy(); + return new JoinNode( + joinType, + generatedJoinCondition, + generatedCalc, + newLeft, + newRight, + rowDataSerializerPassThroughCalc); + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/LookupHandlerBase.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/LookupHandlerBase.java new file mode 100644 index 0000000000000..3890e79fce7ad --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/LookupHandlerBase.java @@ -0,0 +1,424 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.functions.util.FunctionUtils; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.functions.async.CollectionSupplier; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.runtime.generated.GeneratedFunction; +import org.apache.flink.table.runtime.operators.join.lookup.CalcCollectionCollector; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.DataType; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; + +/** + * A {@link DeltaJoinHandlerBase} that used to lookup dim tables. + * + *

This type of {@link DeltaJoinHandlerBase} always holds a {@link AsyncFunction} fetcher to do + * the lookup and get the lookup result by {@link #complete}. + * + *

The {@link #ownedSourceOrdinals} is the source ordinal of this lookup handler on the driving + * side, and the {@link #ownedLookupOrdinal} is the target table ordinal that is used to lookup. + * + *

Take the following pattern as an example. Image that we are in the top `DeltaJoin2`. + * + *

{@code
+ *       DeltaJoin2
+ *      /        \
+ *   DeltaJoin1  #2 C
+ *    /      \
+ * #0 A     #1 B
+ * }
+ * + *

If the stream side is `C`, and the two lookup handlers are `[(C -> A), (A -> B)]`, then in + * these two lookup handlers, {@link #ownedSourceOrdinals} and {@link #ownedLookupOrdinal} are `[2], + * 0` and `[0], 1`. + * + *

If the stream side is `DeltaJoin1`, the lookup handler is [(A, B -> C)], and the {@link + * #ownedSourceOrdinals} and {@link #ownedLookupOrdinal} is `[0, 1], 2`. + */ +public abstract class LookupHandlerBase extends DeltaJoinHandlerBase { + + private static final long serialVersionUID = 1L; + + private static final Logger LOG = LoggerFactory.getLogger(LookupHandlerBase.class); + + @Nullable + protected final GeneratedFunction> lookupSideGeneratedCalc; + + protected final DataType streamSideType; + protected final DataType lookupResultType; + protected final DataType lookupSidePassThroughCalcType; + protected final RowDataSerializer lookupSidePassThroughCalcRowSerializer; + + protected final int[] ownedSourceOrdinals; + protected final int ownedLookupOrdinal; + + // used for debug + private final String lookupHandlerDescription; + + protected transient AsyncFunction fetcher; + protected @Nullable transient FlatMapFunction lookupSideCalc; + + public LookupHandlerBase( + DataType streamSideType, + DataType lookupResultType, + DataType lookupSidePassThroughCalcType, + RowDataSerializer lookupSidePassThroughCalcRowSerializer, + @Nullable GeneratedFunction> lookupSideGeneratedCalc, + int[] ownedSourceOrdinals, + int ownedLookupOrdinal, + String lookupHandlerDescription) { + this.lookupSideGeneratedCalc = lookupSideGeneratedCalc; + this.streamSideType = streamSideType; + this.lookupResultType = lookupResultType; + this.lookupSidePassThroughCalcType = lookupSidePassThroughCalcType; + this.lookupSidePassThroughCalcRowSerializer = lookupSidePassThroughCalcRowSerializer; + this.ownedSourceOrdinals = ownedSourceOrdinals; + this.ownedLookupOrdinal = ownedLookupOrdinal; + this.lookupHandlerDescription = lookupHandlerDescription; + } + + @Override + public void open(OpenContext openContext, DeltaJoinHandlerContext handlerContext) + throws Exception { + super.open(openContext, handlerContext); + + RuntimeContext runtimeContext = handlerContext.getRuntimeContext(); + ClassLoader cl = runtimeContext.getUserCodeClassLoader(); + + fetcher = handlerContext.getLookupFunction(ownedLookupOrdinal); + + if (lookupSideGeneratedCalc != null) { + lookupSideCalc = lookupSideGeneratedCalc.newInstance(cl); + FunctionUtils.setFunctionRuntimeContext(lookupSideCalc, runtimeContext); + FunctionUtils.openFunction(lookupSideCalc, openContext); + } + } + + /** + * Complete the result after lookup. + * + *

Different with {@link #completeResultsInMailbox}, this method is called by lookup source. + * It is likely to be executed within the connector's own thread pool. + * + * @param input the input row used to trigger lookup + * @param lookupResult the lookup result + */ + public final void complete(RowData input, Collection lookupResult) { + handlerContext + .getMailboxExecutor() + .execute( + () -> completeResultsInMailbox(input, lookupResult), + "Lookup results in LookupHandlerBase with input %s are %s", + input, + lookupResult); + } + + /** + * Complete the result after lookup in mailbox thread. + * + *

Different with {@link #complete}, this method is executed in mailbox thread. + * + * @param input the input row used to trigger lookup + * @param lookupResult the lookup result + */ + protected abstract void completeResultsInMailbox( + RowData input, Collection lookupResult); + + protected Object2RowDataConverterResultFuture createLookupResultFuture(RowData input) + throws Exception { + return new Object2RowDataConverterResultFuture( + input, + createConverter( + lookupResultType, + handlerContext.getRuntimeContext().getUserCodeClassLoader()), + createCalcFunction(), + createCalcCollector(), + this, + lookupHandlerDescription, + streamSideType, + lookupSidePassThroughCalcType); + } + + @Nullable + private FlatMapFunction createCalcFunction() throws Exception { + FlatMapFunction calc = null; + if (lookupSideGeneratedCalc != null) { + calc = + lookupSideGeneratedCalc.newInstance( + handlerContext.getRuntimeContext().getUserCodeClassLoader()); + FunctionUtils.setFunctionRuntimeContext(calc, handlerContext.getRuntimeContext()); + FunctionUtils.openFunction(calc, openContext); + } + return calc; + } + + @Nullable + private CalcCollectionCollector createCalcCollector() { + CalcCollectionCollector calcCollector = null; + if (lookupSideGeneratedCalc != null) { + calcCollector = new CalcCollectionCollector(lookupSidePassThroughCalcRowSerializer); + calcCollector.reset(); + } + return calcCollector; + } + + @VisibleForTesting + public AsyncFunction getFetcher() { + return fetcher; + } + + /** + * A result future helps to convert {@link Object} to {@link RowData} after looking up and apply + * the calc on the lookup table if necessary. + */ + @VisibleForTesting + public static class Object2RowDataConverterResultFuture implements ResultFuture { + + private final RowData input; + private final DataStructureConverter lookupResultConverter; + private final @Nullable FlatMapFunction calcFunction; + private final @Nullable CalcCollectionCollector calcCollector; + private final LookupHandlerBase lookupResultCallBack; + + // used for debug + private final String lookupHandlerDescription; + private final DataType streamSideType; + private final DataType lookupSidePassThroughCalcType; + + private @Nullable transient DataStructureConverter streamSideConverter; + private @Nullable transient DataStructureConverter + lookupSidePassThroughCalcConverter; + + private Object2RowDataConverterResultFuture( + RowData input, + DataStructureConverter lookupResultConverter, + @Nullable FlatMapFunction calcFunction, + @Nullable CalcCollectionCollector calcCollector, + LookupHandlerBase lookupResultCallBack, + String lookupHandlerDescription, + DataType streamSideType, + DataType lookupSidePassThroughCalcType) { + this.input = input; + this.lookupResultConverter = lookupResultConverter; + this.calcFunction = calcFunction; + this.calcCollector = calcCollector; + this.lookupResultCallBack = lookupResultCallBack; + + this.lookupHandlerDescription = lookupHandlerDescription; + this.streamSideType = streamSideType; + this.lookupSidePassThroughCalcType = lookupSidePassThroughCalcType; + + Preconditions.checkArgument( + (calcFunction == null && calcCollector == null) + || (calcFunction != null && calcCollector != null)); + } + + @Override + public void complete(Collection result) { + if (result == null) { + result = Collections.emptyList(); + } + Collection lookupRows = convertToInternalData(lookupResultConverter, result); + Collection lookupRowsAfterCalc = passThroughCalc(lookupRows); + + printDebugMessageIfNecessary(lookupRows, lookupRowsAfterCalc); + + lookupResultCallBack.complete(input, lookupRowsAfterCalc); + } + + @Override + public void completeExceptionally(Throwable error) { + lookupResultCallBack.completeExceptionally(error); + } + + /** + * Unsupported, because the containing classes are AsyncFunctions which don't have access to + * the mailbox to invoke from the caller thread. + */ + @Override + public void complete(CollectionSupplier supplier) { + throw new UnsupportedOperationException(); + } + + @VisibleForTesting + @Nullable + public CalcCollectionCollector getCalcCollector() { + return calcCollector; + } + + @VisibleForTesting + @Nullable + public FlatMapFunction getCalcFunction() { + return calcFunction; + } + + private Collection passThroughCalc(Collection data) { + if (calcFunction == null) { + return data; + } + for (RowData row : data) { + try { + calcFunction.flatMap(row, calcCollector); + } catch (Exception e) { + completeExceptionally(e); + return Collections.emptyList(); + } + } + return calcCollector.getCollection(); + } + + private void printDebugMessageIfNecessary( + Collection lookupRows, Collection lookupRowsAfterCalc) { + if (!LOG.isDebugEnabled()) { + return; + } + + ClassLoader cl = + lookupResultCallBack + .handlerContext + .getRuntimeContext() + .getUserCodeClassLoader(); + + // init converter for first debug + if (null == streamSideConverter) { + streamSideConverter = createConverter(streamSideType, cl); + } + if (null == lookupSidePassThroughCalcConverter) { + lookupSidePassThroughCalcConverter = + createConverter(lookupSidePassThroughCalcType, cl); + } + + LOG.debug( + DebugInfo.of( + lookupHandlerDescription, + lookupResultCallBack.handlerContext.inLeft2RightLookupChain(), + convertToExternalData(streamSideConverter, input), + convertToExternalData(lookupResultConverter, lookupRows), + convertToExternalData( + lookupSidePassThroughCalcConverter, + lookupRowsAfterCalc)) + .toString()); + } + } + + private static DataStructureConverter createConverter( + DataType dataType, ClassLoader cl) { + DataStructureConverter lookupResultFetcherConverter = + DataStructureConverters.getConverter(dataType); + lookupResultFetcherConverter.open(cl); + return (DataStructureConverter) lookupResultFetcherConverter; + } + + private static Collection convertToInternalData( + DataStructureConverter converter, Collection data) { + if (converter.isIdentityConversion()) { + return (Collection) data; + } else { + Collection result = new ArrayList<>(data.size()); + for (Object element : data) { + result.add(converter.toInternal(element)); + } + return result; + } + } + + private static Object convertToExternalData( + DataStructureConverter converter, RowData data) { + return convertToExternalData(converter, Collections.singletonList(data)).iterator().next(); + } + + private static Collection convertToExternalData( + DataStructureConverter converter, Collection data) { + Collection result = new ArrayList<>(data.size()); + for (RowData element : data) { + result.add(converter.toExternal(element)); + } + return result; + } + + /** A class used to format debug information in a structured manner when printed. */ + private static class DebugInfo { + private final String lookupHandlerDescription; + private final boolean inLeftSideLookupChain; + private final Object input; + private final Collection lookupResults; + private final Collection lookupRowsAfterCalc; + + public DebugInfo( + String lookupHandlerDescription, + boolean inLeftSideLookupChain, + Object input, + Collection lookupResults, + Collection lookupRowsAfterCalc) { + this.lookupHandlerDescription = lookupHandlerDescription; + this.inLeftSideLookupChain = inLeftSideLookupChain; + this.input = input; + this.lookupResults = lookupResults; + this.lookupRowsAfterCalc = lookupRowsAfterCalc; + } + + public static DebugInfo of( + String lookupHandlerDescription, + boolean inLeftSideLookupChain, + Object input, + Collection lookupResults, + Collection lookupRowsAfterCalc) { + return new DebugInfo( + lookupHandlerDescription, + inLeftSideLookupChain, + input, + lookupResults, + lookupRowsAfterCalc); + } + + @Override + public String toString() { + return "DebugInfo{chainPosition=[" + + (inLeftSideLookupChain ? "left" : "right") + + "], lookupHandlerDescription=[" + + lookupHandlerDescription + + "], input=[" + + input + + "], lookupResult=[" + + lookupResults + + "], lookupRowsAfterCalc=[" + + lookupRowsAfterCalc + + "]}"; + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/MultiInputRowDataBuffer.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/MultiInputRowDataBuffer.java new file mode 100644 index 0000000000000..ba2845607ae0e --- /dev/null +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/MultiInputRowDataBuffer.java @@ -0,0 +1,298 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.utils.JoinedRowData; +import org.apache.flink.table.runtime.generated.FilterCondition; +import org.apache.flink.table.runtime.operators.join.FlinkJoinType; +import org.apache.flink.table.runtime.operators.join.lookup.CalcCollectionCollector; +import org.apache.flink.util.Preconditions; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * A shared buffer used in delta join to manage data for all binary inputs (source tables). It + * maintains one slot per binary input, indexed by input ordinal. + * + *

The buffer provides the following capabilities: + * + *

    + *
  1. Store data for each binary input: data is placed into slots either by the stream + * side via {@link #prepareDataInBuffer} or by lookup handlers via {@link #setRowData} after a + * lookup completes. + *
  2. Compute and save data on demand: when a {@link LookupHandlerBase} needs source data + * that spans multiple binary inputs (i.e., the result of a sub-join in the {@link + * DeltaJoinRuntimeTree}), the internal {@link DataProvider} joins the individual slot data + * on-the-fly using the join tree, saving the result, and returns it. + *
+ * + *

For example, given the following job topology: + * + *

{@code
+ *         DeltaJoin2
+ *       /              \
+ *  DeltaJoin1       #2 C
+ *    /      \
+ * #0 A    #1 B
+ * }
+ * + *

The buffer has 3 slots (slot 0 for A, slot 1 for B, slot 2 for C). When processing data from + * the left stream side ({@code DeltaJoin1}) in {@code DeltaJoin2}: + * + *

    + *
  1. The joined result of A and B from the upstream delta join is placed into slots 0 and 1 via + * {@link #prepareDataInBuffer}. + *
  2. The lookup handler then looks up C, and the result is placed into slot 2 via {@link + * #setRowData}. + *
+ * + *

Note: + * + *

    + *
  1. This buffer is not thread safe. + *
  2. The data of each slot can be set only once per processing cycle. Call {@link #reset()} + * before the next cycle. + *
+ */ +public class MultiInputRowDataBuffer { + + private final int inputCnt; + private final Collection[] subRowsPerInput; + private final DeltaJoinRuntimeTree joinTree; + // + private final Set> allDrivenInputsWhenLookup; + private final DataProvider dataProvider; + + public MultiInputRowDataBuffer( + int[] eachInputFieldSize, + DeltaJoinRuntimeTree joinTree, + Set> allDrivenInputsWhenLookup) { + this.inputCnt = eachInputFieldSize.length; + this.subRowsPerInput = new Collection[inputCnt]; + + this.joinTree = joinTree; + this.allDrivenInputsWhenLookup = allDrivenInputsWhenLookup; + this.dataProvider = new DataProvider(); + } + + public void open(OpenContext openContext, RuntimeContext runtimeContext) throws Exception { + joinTree.open(runtimeContext, openContext); + } + + public void close() throws Exception { + joinTree.close(); + } + + public Collection getData(int[] inputOrdinals) throws Exception { + return dataProvider.getData(inputOrdinals); + } + + public void setRowData(Collection data, int inputIdx) { + validateInputIsNotSet(inputIdx); + this.subRowsPerInput[inputIdx] = data; + } + + public void prepareDataInBuffer(RowData data, Set inputIdxes) { + // set a fake data set to tag the input is ready + inputIdxes.forEach(input -> setRowData(Collections.emptySet(), input)); + dataProvider.providedData.put(inputIdxes, Collections.singleton(data)); + } + + public void reset() { + for (Collection rows : subRowsPerInput) { + if (rows != null) { + rows.clear(); + } + } + Arrays.fill(subRowsPerInput, null); + dataProvider.reset(); + } + + public void validateInputIsSet(int inputIdx) { + Preconditions.checkArgument(inputIdx >= 0 && inputIdx < inputCnt); + Preconditions.checkArgument( + subRowsPerInput[inputIdx] != null, "This input row has not been set: " + inputIdx); + } + + @VisibleForTesting + public DeltaJoinRuntimeTree getJoinTree() { + return joinTree; + } + + private void validateInputIsNotSet(int inputIdx) { + Preconditions.checkArgument(inputIdx >= 0 && inputIdx < inputCnt); + Preconditions.checkArgument( + subRowsPerInput[inputIdx] == null, "This input row has been set"); + } + + private Collection getBinaryInputRowData(int inputIdx) { + validateInputIsSet(inputIdx); + return this.subRowsPerInput[inputIdx]; + } + + /** + * This {@link DataProvider} is used to directly get or compute source data on the spot for each + * node on {@link DeltaJoinRuntimeTree}. + * + *

For {@link DeltaJoinRuntimeTree.BinaryInputNode}, the data is retrieved directly from the + * node. For {@link DeltaJoinRuntimeTree.JoinNode}, if it has been computed previously, the data + * is retrieved from the {@link #providedData}; if it has not been computed before, it is + * calculated on-site. + */ + private class DataProvider { + + private final Map, Collection> providedData = new HashMap<>(); + + public void reset() { + providedData.clear(); + } + + /** + * Get the data for the given input ordinals. + * + *

Each node on {@link DeltaJoinRuntimeTree} represents a collection of different input + * ordinals. The method first determines which node's data to retrieve using {@code + * caresInputOrdinals}, and then retrieves or calculates the data from that node. + */ + public Collection getData(int[] caresInputOrdinals) throws Exception { + Preconditions.checkArgument(caresInputOrdinals.length > 0); + + DeltaJoinRuntimeTree.Node node = + findNodeToGetData( + joinTree.root, + Arrays.stream(caresInputOrdinals).boxed().collect(Collectors.toSet())); + return visit(node); + } + + private DeltaJoinRuntimeTree.Node findNodeToGetData( + DeltaJoinRuntimeTree.Node node, Set caresInputOrdinals) { + Set allInputOrdinalsInThisSubTree = node.getAllInputOrdinals(); + Preconditions.checkArgument( + allInputOrdinalsInThisSubTree.containsAll(caresInputOrdinals)); + + if (allInputOrdinalsInThisSubTree.equals(caresInputOrdinals)) { + return node; + } + + Preconditions.checkArgument(node instanceof DeltaJoinRuntimeTree.JoinNode); + DeltaJoinRuntimeTree.JoinNode joinNode = (DeltaJoinRuntimeTree.JoinNode) node; + if (joinNode.left.getAllInputOrdinals().containsAll(caresInputOrdinals)) { + return findNodeToGetData(joinNode.left, caresInputOrdinals); + } + + Preconditions.checkArgument( + joinNode.right.getAllInputOrdinals().containsAll(caresInputOrdinals)); + return findNodeToGetData(joinNode.right, caresInputOrdinals); + } + + private Collection visit(DeltaJoinRuntimeTree.Node node) throws Exception { + Set inputOrdinals = node.getAllInputOrdinals(); + if (providedData.containsKey(inputOrdinals)) { + return providedData.get(inputOrdinals); + } + + if (node instanceof DeltaJoinRuntimeTree.BinaryInputNode) { + return visit((DeltaJoinRuntimeTree.BinaryInputNode) node); + } + if (node instanceof DeltaJoinRuntimeTree.JoinNode) { + return visit((DeltaJoinRuntimeTree.JoinNode) node); + } + throw new IllegalStateException( + "Unknown node type: " + node.getClass().getSimpleName()); + } + + private Collection visit(DeltaJoinRuntimeTree.BinaryInputNode node) { + Collection results = getBinaryInputRowData(node.inputOrdinal); + // the data from binary input is no need to be processed again by calc + providedData.put(Collections.singleton(node.inputOrdinal), results); + return results; + } + + private Collection visit(DeltaJoinRuntimeTree.JoinNode node) throws Exception { + boolean lookupRight; + if (allDrivenInputsWhenLookup.contains(node.left.getAllInputOrdinals())) { + lookupRight = true; + } else if (allDrivenInputsWhenLookup.contains(node.right.getAllInputOrdinals())) { + lookupRight = false; + } else { + throw new IllegalStateException( + String.format( + "Can't find driven side inputs %s in Join Node. " + + "All inputs in left are %s, and those in right are %s", + allDrivenInputsWhenLookup, + node.left.getAllInputOrdinals(), + node.right.getAllInputOrdinals())); + } + + Preconditions.checkArgument( + node.joinType == FlinkJoinType.INNER, "Only INNER JOIN is supported"); + + Collection leftData = visit(node.left); + Collection rightData = visit(node.right); + + if (leftData.isEmpty() && rightData.isEmpty()) { + return Collections.emptySet(); + } + + Collection inputData = lookupRight ? leftData : rightData; + Collection lookupData = lookupRight ? rightData : leftData; + + FilterCondition condition = node.joinCondition; + + Collection results = new ArrayList<>(); + for (RowData input : inputData) { + for (RowData lookup : lookupData) { + JoinedRowData joinedRowData = + lookupRight + ? new JoinedRowData(input.getRowKind(), input, lookup) + : new JoinedRowData(input.getRowKind(), lookup, input); + if (!condition.apply(FilterCondition.Context.INVALID_CONTEXT, joinedRowData)) { + continue; + } + results.add(joinedRowData); + } + } + + if (node.calc != null) { + CalcCollectionCollector calcCollector = + new CalcCollectionCollector(node.rowDataSerializerPassThroughCalc); + calcCollector.reset(); + for (RowData row : results) { + node.calc.flatMap(row, calcCollector); + } + results = calcCollector.getCollection(); + } + + Set allInvolvingInputs = node.getAllInputOrdinals(); + providedData.put(allInvolvingInputs, results); + return results; + } + } +} diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperator.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperator.java index 2505550104bc7..a1475062f8763 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperator.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperator.java @@ -19,6 +19,8 @@ package org.apache.flink.table.runtime.operators.join.deltajoin; import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.functions.DefaultOpenContext; +import org.apache.flink.api.common.functions.util.FunctionUtils; import org.apache.flink.api.common.operators.MailboxExecutor; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -33,6 +35,7 @@ import org.apache.flink.runtime.state.CheckpointStreamFactory; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.VoidNamespace; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.api.functions.async.CollectionSupplier; import org.apache.flink.streaming.api.functions.async.ResultFuture; import org.apache.flink.streaming.api.graph.StreamConfig; @@ -50,6 +53,7 @@ import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.GeneratedFunction; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; import org.apache.flink.table.runtime.operators.TableAbstractCoUdfStreamOperator; import org.apache.flink.table.runtime.operators.join.lookup.keyordered.AecRecord; @@ -68,6 +72,7 @@ import java.util.Collection; import java.util.Collections; import java.util.Deque; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; @@ -101,6 +106,10 @@ public class StreamingDeltaJoinOperator "deltaJoinOpTotalInFlightNum"; private static final String METRIC_DELTA_JOIN_ASYNC_IO_TIME = "deltaJoinAsyncIOTime"; + // + private final Map>> + generatedLookupFunctions; + private final StreamRecord leftEmptyStreamRecord; private final StreamRecord rightEmptyStreamRecord; @@ -142,8 +151,6 @@ public class StreamingDeltaJoinOperator private transient TableAsyncExecutionController asyncExecutionController; - private transient DeltaJoinCache cache; - /** Mailbox executor used to yield while waiting for buffers to empty. */ private final transient MailboxExecutor mailboxExecutor; @@ -169,9 +176,13 @@ public class StreamingDeltaJoinOperator */ private transient List allResultHandlers; + private transient Map> lookupFunctions; + public StreamingDeltaJoinOperator( - AsyncDeltaJoinRunner rightLookupTableAsyncFunction, - AsyncDeltaJoinRunner leftLookupTableAsyncFunction, + AsyncDeltaJoinRunner left2RightAsyncRunner, + AsyncDeltaJoinRunner right2LeftAsyncRunner, + Map>> + generatedLookupFunctions, RowDataKeySelector leftJoinKeySelector, RowDataKeySelector rightJoinKeySelector, long timeout, @@ -182,9 +193,10 @@ public StreamingDeltaJoinOperator( long rightSideCacheSize, RowType leftStreamType, RowType rightStreamType) { - // rightLookupTableAsyncFunction is an udx used for left records - // leftLookupTableAsyncFunction is an udx used for right records - super(rightLookupTableAsyncFunction, leftLookupTableAsyncFunction); + // left2RightAsyncRunner is an udx used for left records + // right2LeftAsyncRunner is an udx used for right records + super(left2RightAsyncRunner, right2LeftAsyncRunner); + this.generatedLookupFunctions = generatedLookupFunctions; this.leftJoinKeySelector = leftJoinKeySelector; this.rightJoinKeySelector = rightJoinKeySelector; this.timeout = timeout; @@ -234,11 +246,6 @@ public void setup( isLeft(inputIndex) ? leftJoinKeySelector : rightJoinKeySelector; return keySelector.getKey(record.getValue()); }); - - this.cache = new DeltaJoinCache(leftSideCacheSize, rightSideCacheSize); - - leftTriggeredUserFunction.setCache(cache); - rightTriggeredUserFunction.setCache(cache); } @Override @@ -276,7 +283,21 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void open() throws Exception { - super.open(); + lookupFunctions = new HashMap<>(); + for (Map.Entry>> entry : + generatedLookupFunctions.entrySet()) { + AsyncFunction lookupFunc = + entry.getValue().newInstance(getUserCodeClassloader()); + FunctionUtils.setFunctionRuntimeContext(lookupFunc, getRuntimeContext()); + FunctionUtils.openFunction(lookupFunc, DefaultOpenContext.INSTANCE); + lookupFunctions.put(entry.getKey(), lookupFunc); + } + DeltaJoinCache cache = new DeltaJoinCache(leftSideCacheSize, rightSideCacheSize); + + DeltaJoinOpenContext openContext = + new DeltaJoinOpenContext(cache, mailboxExecutor, lookupFunctions); + + super.open(openContext); this.needDeepCopy = getExecutionConfig().isObjectReuseEnabled() && !config.isChainStart(); @@ -485,6 +506,11 @@ public void close() throws Exception { handler.close(); } } + if (lookupFunctions != null) { + for (AsyncFunction func : lookupFunctions.values()) { + FunctionUtils.closeFunction(func); + } + } } public void waitInFlightInputsFinished() throws InterruptedException { diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperatorTest.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingBinaryDeltaJoinOperatorTest.java similarity index 62% rename from flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperatorTest.java rename to flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingBinaryDeltaJoinOperatorTest.java index bae1d5e1096e3..cc6d8b173c720 100644 --- a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperatorTest.java +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingBinaryDeltaJoinOperatorTest.java @@ -19,41 +19,29 @@ package org.apache.flink.table.runtime.operators.join.deltajoin; import org.apache.flink.api.common.functions.FlatMapFunction; -import org.apache.flink.api.common.functions.OpenContext; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.async.ResultFuture; -import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; -import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor; -import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; -import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness; +import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.conversion.DataStructureConverter; -import org.apache.flink.table.data.conversion.DataStructureConverters; -import org.apache.flink.table.runtime.collector.TableFunctionCollector; -import org.apache.flink.table.runtime.collector.TableFunctionResultFuture; +import org.apache.flink.table.runtime.generated.GeneratedFilterCondition; import org.apache.flink.table.runtime.generated.GeneratedFunction; -import org.apache.flink.table.runtime.generated.GeneratedResultFutureWrapper; import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; +import org.apache.flink.table.runtime.operators.join.FlinkJoinType; import org.apache.flink.table.runtime.operators.join.lookup.keyordered.AecRecord; import org.apache.flink.table.runtime.operators.join.lookup.keyordered.RecordsBuffer; import org.apache.flink.table.runtime.operators.join.lookup.keyordered.TableAsyncExecutionController; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; import org.apache.flink.table.runtime.util.RowDataHarnessAssertor; import org.apache.flink.table.types.logical.BooleanType; import org.apache.flink.table.types.logical.IntType; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.logical.VarCharType; -import org.apache.flink.table.utils.HandwrittenSelectorUtil; import org.apache.flink.testutils.junit.extensions.parameterized.Parameter; import org.apache.flink.testutils.junit.extensions.parameterized.ParameterizedTestExtension; import org.apache.flink.testutils.junit.extensions.parameterized.Parameters; -import org.apache.flink.util.Preconditions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -64,23 +52,16 @@ import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Optional; +import java.util.Set; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; -import java.util.stream.IntStream; -import java.util.stream.Stream; import static org.apache.flink.table.runtime.util.StreamRecordUtils.binaryrow; import static org.apache.flink.table.runtime.util.StreamRecordUtils.insertRecord; @@ -88,17 +69,15 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -/** Test class for {@link StreamingDeltaJoinOperator}. */ +/** Test class for {@link StreamingDeltaJoinOperator} with two binary inputs. */ @ExtendWith(ParameterizedTestExtension.class) -public class StreamingDeltaJoinOperatorTest { - - private static final int AEC_CAPACITY = 100; - private static final int CACHE_SIZE = 10; +public class StreamingBinaryDeltaJoinOperatorTest extends StreamingDeltaJoinOperatorTestBase { // the data snapshot of the left/right table when joining - // - private static final HashMap leftTableCurrentData = new HashMap<>(); - private static final HashMap rightTableCurrentData = new HashMap<>(); + // > + // left table index is 0, right table index is 1 + private final Map> tableCurrentDataMap = + new HashMap<>(); @Parameters(name = "EnableCache = {0}") public static List parameters() { @@ -115,9 +94,7 @@ public static List parameters() { @BeforeEach public void beforeEach() throws Exception { - MyAsyncFunction.leftInvokeCount.set(0); - MyAsyncFunction.rightInvokeCount.set(0); - MyAsyncExecutionControllerDelegate.insertTableDataAfterEmit = true; + MyAsyncFunction.getLookupInvokeCount().clear(); } @AfterEach @@ -125,10 +102,8 @@ public void afterEach() throws Exception { if (assertor != null) { testHarness.close(); } - leftTableCurrentData.clear(); - rightTableCurrentData.clear(); + tableCurrentDataMap.clear(); latestException = Optional.empty(); - MyAsyncFunction.clearExpectedThrownException(); } @TestTemplate @@ -195,10 +170,10 @@ void testJoinBothLogTables() throws Exception { if (enableCache) { RowType leftRowType = testSpec.getLeftInputRowType(); RowType rightRowType = testSpec.getRightInputRowType(); - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(leftRecord1.getValue(), leftRowType), leftRecord1.getValue(), toBinary(leftRecord3.getValue(), leftRowType), @@ -206,7 +181,7 @@ void testJoinBothLogTables() throws Exception { toBinary(leftRecord5.getValue(), leftRowType), leftRecord5.getValue()), binaryrow(false, "jklk2"), - newHashMap( + Map.of( toBinary(leftRecord2.getValue(), leftRowType), leftRecord2.getValue(), toBinary(leftRecord4.getValue(), leftRowType), @@ -216,28 +191,30 @@ void testJoinBothLogTables() throws Exception { binaryrow(false, "unknown"), Collections.emptyMap()); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(rightRecord1.getValue(), rightRowType), rightRecord1.getValue(), toBinary(rightRecord4.getValue(), rightRowType), rightRecord4.getValue()), binaryrow(false, "jklk2"), - newHashMap( + Map.of( toBinary(rightRecord2.getValue(), rightRowType), rightRecord2.getValue(), toBinary(rightRecord5.getValue(), rightRowType), rightRecord5.getValue())); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 5, 2, 6, 4); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(2); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(3); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 5, 2, 6, 4); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(3); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(6); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(5); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(6); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(5); } } @@ -313,10 +290,10 @@ void testJoinBothLogTablesWhileFilterExistsOnBothTable() throws Exception { if (enableCache) { RowType leftRowType = testSpec.getLeftInputRowType(); RowType rightRowType = testSpec.getRightInputRowType(); - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(leftRecord1.getValue(), leftRowType), leftRecord1.getValue(), toBinary(leftRecord3.getValue(), leftRowType), @@ -324,10 +301,10 @@ void testJoinBothLogTablesWhileFilterExistsOnBothTable() throws Exception { toBinary(leftRecord6.getValue(), leftRowType), leftRecord6.getValue())); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(rightRecord1.getValue(), rightRowType), rightRecord1.getValue(), toBinary(rightRecord4.getValue(), rightRowType), @@ -335,13 +312,15 @@ void testJoinBothLogTablesWhileFilterExistsOnBothTable() throws Exception { binaryrow(true, "unknown1"), Collections.emptyMap()); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 2, 1, 4, 2); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(2); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(1); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 2, 1, 4, 2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(1); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(4); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(2); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(4); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(2); } } @@ -404,10 +383,10 @@ void testJoinBothPkTables() throws Exception { DeltaJoinCache cache = unwrapCache(testHarness); if (enableCache) { - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow("Tom"), - newHashMap( + Map.of( binaryrow(true, "Tom"), leftRecordK1V2.getValue(), binaryrow(false, "Tom"), @@ -415,23 +394,25 @@ void testJoinBothPkTables() throws Exception { binaryrow("Sam"), Collections.emptyMap()); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow("Tom"), - newHashMap( + Map.of( binaryrow("Tom", true), rightRecordK1V2.getValue(), binaryrow("Tom", false), rightRecordK2V2.getValue()), binaryrow("Jim"), Collections.emptyMap()); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 5, 3, 4, 2); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(2); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(2); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 5, 3, 4, 2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(2); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(4); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(5); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(4); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(5); } } @@ -495,24 +476,26 @@ void testJoinBothPkTablesWhileFilterExistsOnBothTable() throws Exception { DeltaJoinCache cache = unwrapCache(testHarness); if (enableCache) { - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow("Tom"), - newHashMap(binaryrow(true, "Tom"), leftRecordK1V2.getValue()), + Map.of(binaryrow(true, "Tom"), leftRecordK1V2.getValue()), binaryrow("Sam"), Collections.emptyMap()); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow("Tom"), - newHashMap(binaryrow("Tom", false), rightRecordK2V2.getValue())); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 3, 1, 2, 1); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(1); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(2); + Map.of(binaryrow("Tom", false), rightRecordK2V2.getValue())); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 3, 1, 2, 1); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(1); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(2); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(2); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(3); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(3); } } @@ -597,16 +580,16 @@ void testBlockingWithSameJoinKey() throws Exception { if (enableCache) { RowType leftRowType = testSpec.getLeftInputRowType(); RowType rightRowType = testSpec.getRightInputRowType(); - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(leftRecord1.getValue(), leftRowType), leftRecord1.getValue(), toBinary(leftRecord3.getValue(), leftRowType), leftRecord3.getValue()), binaryrow(false, "jklk2"), - newHashMap( + Map.of( toBinary(leftRecord2.getValue(), leftRowType), leftRecord2.getValue(), toBinary(leftRecord4.getValue(), leftRowType), @@ -616,24 +599,26 @@ void testBlockingWithSameJoinKey() throws Exception { binaryrow(false, "unknown"), Collections.emptyMap()); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(rightRecord1.getValue(), rightRowType), rightRecord1.getValue()), binaryrow(false, "jklk2"), - newHashMap( + Map.of( toBinary(rightRecord2.getValue(), rightRowType), rightRecord2.getValue())); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 3, 0, 5, 3); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(2); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(3); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 3, 0, 5, 3); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(3); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(5); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(3); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(5); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(3); } } @@ -645,11 +630,9 @@ void testBlockingWithSameJoinKey() throws Exception { @TestTemplate void testLogTableDataVisibleBeforeJoin() throws Exception { LogLogTableJoinTestSpec testSpec = LogLogTableJoinTestSpec.WITHOUT_FILTER_ON_TABLE; - initTestHarness(testSpec); + initTestHarness(testSpec, null, false); initAssertor(testSpec); - MyAsyncExecutionControllerDelegate.insertTableDataAfterEmit = false; - // prepare the data first to mock all following requests were in flight between source and // delta-join final StreamRecord leftRecord1 = insertRecord(100, true, "jklk1"); @@ -717,10 +700,10 @@ void testLogTableDataVisibleBeforeJoin() throws Exception { if (enableCache) { RowType leftRowType = testSpec.getLeftInputRowType(); RowType rightRowType = testSpec.getRightInputRowType(); - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(leftRecord1.getValue(), leftRowType), leftRecord1.getValue(), toBinary(leftRecord2.getValue(), leftRowType), @@ -730,22 +713,24 @@ void testLogTableDataVisibleBeforeJoin() throws Exception { binaryrow(false, "jklk2"), Collections.emptyMap()); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(rightRecord1.getValue(), rightRowType), rightRecord1.getValue(), toBinary(rightRecord3.getValue(), rightRowType), rightRecord3.getValue())); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 3, 1, 3, 2); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(1); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(2); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 3, 1, 3, 2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(1); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(2); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(3); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(3); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(3); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(3); } } @@ -757,11 +742,9 @@ void testLogTableDataVisibleBeforeJoin() throws Exception { @TestTemplate void testPkTableDataVisibleBeforeJoin() throws Exception { PkPkTableJoinTestSpec testSpec = PkPkTableJoinTestSpec.WITHOUT_FILTER_ON_TABLE; - initTestHarness(testSpec); + initTestHarness(testSpec, null, false); initAssertor(testSpec); - MyAsyncExecutionControllerDelegate.insertTableDataAfterEmit = false; - // prepare the data first to mock all following requests were in flight between source and // delta-join final StreamRecord leftRecordK1V1 = insertRecord(100, true, "Tom"); @@ -839,10 +822,10 @@ void testPkTableDataVisibleBeforeJoin() throws Exception { DeltaJoinCache cache = unwrapCache(testHarness); if (enableCache) { - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow("Tom"), - newHashMap( + Map.of( binaryrow(true, "Tom"), leftRecordK1V2.getValue(), binaryrow(false, "Tom"), @@ -850,10 +833,10 @@ void testPkTableDataVisibleBeforeJoin() throws Exception { binaryrow("Sam"), Collections.emptyMap()); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow("Tom"), - newHashMap( + Map.of( binaryrow("Tom", true), rightRecordK1V3.getValue(), binaryrow("Tom", false), @@ -861,13 +844,15 @@ void testPkTableDataVisibleBeforeJoin() throws Exception { binaryrow("Jim"), Collections.emptyMap()); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 5, 3, 5, 3); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(2); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(2); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 5, 3, 5, 3); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(2); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(5); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(5); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(5); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(5); } } @@ -912,19 +897,23 @@ void testCheckpointAndRestore() throws Exception { MyAsyncFunction.release(); testHarness.close(); - MyAsyncFunction.leftInvokeCount.set(0); - MyAsyncFunction.rightInvokeCount.set(0); + MyAsyncFunction.getLookupInvokeCount().clear(); MyAsyncFunction.block(); // restoring - testHarness = createDeltaJoinOperatorTestHarness(testSpec); + testHarness = createBinaryDeltaJoinOperatorTestHarness(testSpec, null); testHarness.setup(); StreamingDeltaJoinOperator operator = unwrapOperator(testHarness); operator.setAsyncExecutionController( new MyAsyncExecutionControllerDelegate( - testSpec, operator.getAsyncExecutionController())); + operator.getAsyncExecutionController(), + true, + (inputIdx, rowData) -> { + // split A and B + insertTableData(testSpec, rowData, inputIdx == 0); + })); latestException = Optional.empty(); testHarness.initializeState(snapshot); @@ -961,10 +950,10 @@ void testCheckpointAndRestore() throws Exception { if (enableCache) { RowType leftRowType = testSpec.getLeftInputRowType(); RowType rightRowType = testSpec.getRightInputRowType(); - Map> expectedLeftCacheData = - newHashMap( + Map> expectedLeftCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(leftRecord1.getValue(), leftRowType), toBinary(leftRecord1.getValue(), leftRowType), toBinary(leftRecord2.getValue(), leftRowType), @@ -972,20 +961,22 @@ void testCheckpointAndRestore() throws Exception { binaryrow(false, "unknown"), Collections.emptyMap()); - Map> expectedRightCacheData = - newHashMap( + Map> expectedRightCacheData = + Map.of( binaryrow(true, "jklk1"), - newHashMap( + Map.of( toBinary(rightRecord1.getValue(), rightRowType), toBinary(rightRecord1.getValue(), rightRowType))); - verifyCacheData(cache, expectedLeftCacheData, expectedRightCacheData, 2, 0, 2, 1); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(1); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(2); + verifyCacheData( + testSpec, cache, expectedLeftCacheData, expectedRightCacheData, 2, 0, 2, 1); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(1); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(2); } else { - verifyCacheData(cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); - assertThat(MyAsyncFunction.leftInvokeCount.get()).isEqualTo(2); - assertThat(MyAsyncFunction.rightInvokeCount.get()).isEqualTo(2); + verifyCacheData( + testSpec, cache, Collections.emptyMap(), Collections.emptyMap(), 0, 0, 0, 0); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(1).get()).isEqualTo(2); + assertThat(MyAsyncFunction.getLookupInvokeCount().get(0).get()).isEqualTo(2); } } @@ -1047,11 +1038,9 @@ void testClearLegacyStateWhenCheckpointing() throws Exception { @TestTemplate void testMeetExceptionWhenLookup() throws Exception { LogLogTableJoinTestSpec testSpec = LogLogTableJoinTestSpec.WITHOUT_FILTER_ON_TABLE; - initTestHarness(testSpec); - initAssertor(testSpec); - Throwable expectedException = new IllegalStateException("Mock to fail"); - MyAsyncFunction.setExpectedThrownException(expectedException); + initTestHarness(testSpec, expectedException, true); + initAssertor(testSpec); StreamRecord record = insertRecord(100, true, "jklk1"); testHarness.processElement1(record); @@ -1067,8 +1056,16 @@ void testMeetExceptionWhenLookup() throws Exception { .isEqualTo(expectedException); } - private void initTestHarness(AbstractTestSpec testSpec) throws Exception { - testHarness = createDeltaJoinOperatorTestHarness(testSpec); + private void initTestHarness(AbstractBinaryTestSpec testSpec) throws Exception { + initTestHarness(testSpec, null, true); + } + + private void initTestHarness( + AbstractBinaryTestSpec testSpec, + @Nullable Throwable expectedThrownException, + boolean insertTableDataAfterEmit) + throws Exception { + testHarness = createBinaryDeltaJoinOperatorTestHarness(testSpec, expectedThrownException); testHarness.setup(); testHarness.open(); StreamingDeltaJoinOperator operator = unwrapOperator(testHarness); @@ -1082,41 +1079,23 @@ private void initTestHarness(AbstractTestSpec testSpec) throws Exception { }); operator.setAsyncExecutionController( new MyAsyncExecutionControllerDelegate( - testSpec, operator.getAsyncExecutionController())); + operator.getAsyncExecutionController(), + insertTableDataAfterEmit, + (inputIdx, rowData) -> { + // split A and B + insertTableData(testSpec, rowData, inputIdx == 0); + })); } - private void initAssertor(AbstractTestSpec testSpec) { - RowType outputRowType = testSpec.getOutputRowType(); - assertor = - new RowDataHarnessAssertor( - outputRowType.getChildren().toArray(new LogicalType[0]), - // sort the result by the output upsert key - (o1, o2) -> { - for (int keyIndex : testSpec.getOutputFieldIndices()) { - LogicalType type = outputRowType.getChildren().get(keyIndex); - RowData.FieldGetter getter = - RowData.createFieldGetter(type, keyIndex); - - int compareResult = - Objects.requireNonNull(getter.getFieldOrNull(o1)) - .toString() - .compareTo( - Objects.requireNonNull( - getter.getFieldOrNull(o2)) - .toString()); - - if (compareResult != 0) { - return compareResult; - } - } - return o1.toString().compareTo(o2.toString()); - }); + private void initAssertor(AbstractBinaryTestSpec testSpec) { + assertor = createAssertor(testSpec.getOutputRowType()); } private void verifyCacheData( + AbstractBinaryTestSpec testSpec, DeltaJoinCache actualCache, - Map> expectedLeftCacheData, - Map> expectedRightCacheData, + Map> expectedLeftCacheData, + Map> expectedRightCacheData, long expectedLeftCacheRequestCount, long expectedLeftCacheHitCount, long expectedRightCacheRequestCount, @@ -1127,6 +1106,9 @@ private void verifyCacheData( expectedLeftCacheData, expectedLeftCacheRequestCount, expectedLeftCacheHitCount, + testSpec.getLeftJoinKeySelector().getProducedType().toRowType(), + testSpec.getLeftUpsertKeySelector().getProducedType().toRowType(), + testSpec.getLeftInputRowType(), true); // assert right cache @@ -1135,56 +1117,12 @@ private void verifyCacheData( expectedRightCacheData, expectedRightCacheRequestCount, expectedRightCacheHitCount, + testSpec.getRightJoinKeySelector().getProducedType().toRowType(), + testSpec.getRightUpsertKeySelector().getProducedType().toRowType(), + testSpec.getRightInputRowType(), false); } - private void verifyCacheData( - DeltaJoinCache actualCache, - Map> expectedCacheData, - long expectedCacheRequestCount, - long expectedCacheHitCount, - boolean testLeftCache) { - String errorPrefix = testLeftCache ? "left cache " : "right cache "; - - Map> actualCacheData = - testLeftCache - ? actualCache.getLeftCache().asMap() - : actualCache.getRightCache().asMap(); - assertThat(actualCacheData).as(errorPrefix + "data mismatch").isEqualTo(expectedCacheData); - - long actualCacheSize = - testLeftCache - ? actualCache.getLeftCache().size() - : actualCache.getRightCache().size(); - assertThat(actualCacheSize) - .as(errorPrefix + "size mismatch") - .isEqualTo(expectedCacheData.size()); - - long actualTotalSize = - testLeftCache - ? actualCache.getLeftTotalSize().get() - : actualCache.getRightTotalSize().get(); - assertThat(actualTotalSize) - .as(errorPrefix + "total size mismatch") - .isEqualTo(expectedCacheData.values().stream().mapToInt(Map::size).sum()); - - long actualRequestCount = - testLeftCache - ? actualCache.getLeftRequestCount().get() - : actualCache.getRightRequestCount().get(); - assertThat(actualRequestCount) - .as(errorPrefix + "request count mismatch") - .isEqualTo(expectedCacheRequestCount); - - long actualHitCount = - testLeftCache - ? actualCache.getLeftHitCount().get() - : actualCache.getRightHitCount().get(); - assertThat(actualHitCount) - .as(errorPrefix + "hit count mismatch") - .isEqualTo(expectedCacheHitCount); - } - private void waitAllDataProcessed() throws Exception { testHarness.endAllInputs(); if (latestException.isPresent()) { @@ -1194,140 +1132,137 @@ private void waitAllDataProcessed() throws Exception { } private KeyedTwoInputStreamOperatorTestHarness - createDeltaJoinOperatorTestHarness(AbstractTestSpec testSpec) throws Exception { - TaskMailbox mailbox = new TaskMailboxImpl(); - MailboxProcessor mailboxProcessor = - new MailboxProcessor(controller -> {}, mailbox, StreamTaskActionExecutor.IMMEDIATE); - - DataStructureConverter leftFetcherConverter = - (DataStructureConverter) - DataStructureConverters.getConverter( - testSpec.getLeftTypeInfo().getDataType()); - - AsyncDeltaJoinRunner leftAsyncFunction = - new AsyncDeltaJoinRunner( - new GeneratedFunction<>("", "", new Object[0]) { - @Override - public MyAsyncFunction newInstance(ClassLoader classLoader) { - return new MyAsyncFunction(testSpec, false); - } - }, - leftFetcherConverter, - new MockGeneratedFlatMapFunction( - testSpec.getFilterOnLeftTable().orElse(null)), - new GeneratedResultFutureWrapper<>(new TestingFetcherResultFuture()), - testSpec.getLeftTypeInfo().toRowSerializer(), - testSpec.getLeftJoinKeySelector(), - testSpec.getLeftUpsertKeySelector(), - testSpec.getRightJoinKeySelector(), - testSpec.getRightUpsertKeySelector(), - AEC_CAPACITY, - false, - enableCache); - - DataStructureConverter rightFetcherConverter = - (DataStructureConverter) - DataStructureConverters.getConverter( - testSpec.getRightTypeInfo().getDataType()); - - AsyncDeltaJoinRunner rightAsyncFunction = - new AsyncDeltaJoinRunner( - new GeneratedFunction<>("", "", new Object[0]) { - @Override - public MyAsyncFunction newInstance(ClassLoader classLoader) { - return new MyAsyncFunction(testSpec, true); - } - }, - rightFetcherConverter, - new MockGeneratedFlatMapFunction( - testSpec.getFilterOnRightTable().orElse(null)), - new GeneratedResultFutureWrapper<>(new TestingFetcherResultFuture()), - testSpec.getRightTypeInfo().toRowSerializer(), - testSpec.getLeftJoinKeySelector(), - testSpec.getLeftUpsertKeySelector(), - testSpec.getRightJoinKeySelector(), - testSpec.getRightUpsertKeySelector(), - AEC_CAPACITY, - true, - enableCache); - - InternalTypeInfo joinKeyTypeInfo = - testSpec.getLeftJoinKeySelector().getProducedType(); - - StreamingDeltaJoinOperator operator = - new StreamingDeltaJoinOperator( - rightAsyncFunction, - leftAsyncFunction, - testSpec.getLeftJoinKeySelector(), - testSpec.getRightJoinKeySelector(), - -1L, - AEC_CAPACITY, - new TestProcessingTimeService(), - new MailboxExecutorImpl( - mailbox, 0, StreamTaskActionExecutor.IMMEDIATE, mailboxProcessor), - CACHE_SIZE, - CACHE_SIZE, - testSpec.getLeftInputRowType(), - testSpec.getRightInputRowType()); - - return new KeyedTwoInputStreamOperatorTestHarness<>( - operator, + createBinaryDeltaJoinOperatorTestHarness( + AbstractBinaryTestSpec testSpec, @Nullable Throwable expectedThrownException) + throws Exception { + int[] eachBinaryInputFieldSize = + new int[] { + testSpec.getLeftInputTypeInfo().getTotalFields(), + testSpec.getRightInputTypeInfo().getTotalFields() + }; + + GeneratedFunction> generatedCalcOnLeft = + testSpec.getFilterOnLeftTable().map(MockGeneratedFlatMapFunction::new).orElse(null); + + GeneratedFunction> generatedCalcOnRight = + testSpec.getFilterOnRightTable() + .map(MockGeneratedFlatMapFunction::new) + .orElse(null); + + BinaryLookupHandler left2RightBinaryLookupHandler = + new BinaryLookupHandler( + testSpec.getLeftInputTypeInfo().getDataType(), + testSpec.getRightInputTypeInfo().getDataType(), + testSpec.getRightInputTypeInfo().getDataType(), + InternalSerializers.create( + (RowType) testSpec.getRightInputTypeInfo().toLogicalType()), + generatedCalcOnRight, + new int[] {0}, + 1); + + DeltaJoinHandlerChain left2RightHandlerChain = + DeltaJoinHandlerChain.build( + Collections.singletonList(left2RightBinaryLookupHandler), new int[] {0}); + + BinaryLookupHandler right2LeftBinaryLookupHandler = + new BinaryLookupHandler( + testSpec.getRightInputTypeInfo().getDataType(), + testSpec.getLeftInputTypeInfo().getDataType(), + testSpec.getLeftInputTypeInfo().getDataType(), + InternalSerializers.create( + (RowType) testSpec.getLeftInputTypeInfo().toLogicalType()), + generatedCalcOnLeft, + new int[] {1}, + 0); + + DeltaJoinHandlerChain right2LeftHandlerChain = + DeltaJoinHandlerChain.build( + Collections.singletonList(right2LeftBinaryLookupHandler), new int[] {1}); + + DeltaJoinRuntimeTree.BinaryInputNode left = + new DeltaJoinRuntimeTree.BinaryInputNode( + 0, generatedCalcOnLeft, testSpec.getLeftInputTypeInfo().toRowSerializer()); + DeltaJoinRuntimeTree.BinaryInputNode right = + new DeltaJoinRuntimeTree.BinaryInputNode( + 1, + generatedCalcOnRight, + testSpec.getRightInputTypeInfo().toRowSerializer()); + DeltaJoinRuntimeTree.JoinNode joinNode = + new DeltaJoinRuntimeTree.JoinNode( + FlinkJoinType.INNER, + testSpec.getGeneratedJoinCondition(), + null, // calc on the join + left, + right, + InternalSerializers.create( + combineRowTypes( + testSpec.getLeftInputTypeInfo().toRowType(), + testSpec.getRightInputTypeInfo().toRowType()))); + DeltaJoinRuntimeTree joinRuntimeTree = new DeltaJoinRuntimeTree(joinNode); + + Set> left2RightDrivenSideInfo = new HashSet<>(); + left2RightDrivenSideInfo.add(Collections.singleton(0)); + + Set> right2LeftDrivenSideInfo = new HashSet<>(); + right2LeftDrivenSideInfo.add(Collections.singleton(1)); + + Map>> fetchers = + Map.of( + 0, + createFetcherFunction( + tableCurrentDataMap, + testSpec.getRightJoinKeySelector(), + testSpec.getLeftJoinKeySelector(), + 0, + expectedThrownException), + 1, + createFetcherFunction( + tableCurrentDataMap, + testSpec.getLeftJoinKeySelector(), + testSpec.getRightJoinKeySelector(), + 1, + expectedThrownException)); + + return createDeltaJoinOperatorTestHarness( + eachBinaryInputFieldSize, + left2RightHandlerChain, + right2LeftHandlerChain, + null, // remainingJoinCondition + joinRuntimeTree, + left2RightDrivenSideInfo, + right2LeftDrivenSideInfo, testSpec.getLeftJoinKeySelector(), + testSpec.getLeftUpsertKeySelector(), testSpec.getRightJoinKeySelector(), - joinKeyTypeInfo, - 1, - 1, - 0, - testSpec.getLeftTypeInfo().toSerializer(), - testSpec.getRightTypeInfo().toSerializer()); - } - - private TableAsyncExecutionController unwrapAEC( - KeyedTwoInputStreamOperatorTestHarness - testHarness) { - return unwrapOperator(testHarness).getAsyncExecutionController(); - } - - private StreamingDeltaJoinOperator unwrapOperator( - KeyedTwoInputStreamOperatorTestHarness - testHarness) { - return (StreamingDeltaJoinOperator) testHarness.getOperator(); + testSpec.getRightUpsertKeySelector(), + fetchers, + testSpec.getLeftInputTypeInfo(), + testSpec.getRightInputTypeInfo(), + enableCache); } - private DeltaJoinCache unwrapCache( - KeyedTwoInputStreamOperatorTestHarness - testHarness) { - DeltaJoinCache cacheInLeftRunner = - unwrapOperator(testHarness).getLeftTriggeredUserFunction().getCache(); - DeltaJoinCache cacheInRightRunner = - unwrapOperator(testHarness).getRightTriggeredUserFunction().getCache(); - - // the object ref must be the same - assertThat(cacheInLeftRunner == cacheInRightRunner).isTrue(); - return cacheInLeftRunner; + private void insertLeftTable(AbstractBinaryTestSpec testSpec, StreamRecord record) { + insertTableData(testSpec, record.getValue(), true); } - private void insertLeftTable(AbstractTestSpec testSpec, StreamRecord record) { - insertTableData(testSpec, record, true); + private void insertRightTable(AbstractBinaryTestSpec testSpec, StreamRecord record) { + insertTableData(testSpec, record.getValue(), false); } - private void insertRightTable(AbstractTestSpec testSpec, StreamRecord record) { - insertTableData(testSpec, record, false); - } - - private static void insertTableData( - AbstractTestSpec testSpec, StreamRecord record, boolean insertLeftTable) { - RowData rowData = record.getValue(); + private void insertTableData( + AbstractBinaryTestSpec testSpec, RowData rowData, boolean insertLeftTable) { try { - if (insertLeftTable) { - synchronized (leftTableCurrentData) { + synchronized (tableCurrentDataMap) { + if (insertLeftTable) { RowData upsertKey = testSpec.getLeftUpsertKeySelector().getKey(rowData); - leftTableCurrentData.put(upsertKey, rowData); - } - } else { - synchronized (rightTableCurrentData) { + tableCurrentDataMap + .computeIfAbsent(0, k -> new LinkedHashMap<>()) + .put(upsertKey, rowData); + } else { RowData upsertKey = testSpec.getRightUpsertKeySelector().getKey(rowData); - rightTableCurrentData.put(upsertKey, rowData); + tableCurrentDataMap + .computeIfAbsent(1, k -> new LinkedHashMap<>()) + .put(upsertKey, rowData); } } } catch (Exception e) { @@ -1335,19 +1270,6 @@ private static void insertTableData( } } - private Map newHashMap(Object... data) { - Preconditions.checkArgument(data.length % 2 == 0); - Map map = new HashMap<>(); - for (int i = 0; i < data.length; i = i + 2) { - Preconditions.checkArgument( - data[i] instanceof RowData, "The key of the map must be RowData"); - RowData key = (RowData) data[i]; - Preconditions.checkArgument(!map.containsKey(key), "Duplicate key"); - map.put(key, (T) data[i + 1]); - } - return map; - } - private RowData toBinary(RowData row, RowType rowType) { int size = row.getArity(); Object[] fields = new Object[size]; @@ -1357,285 +1279,31 @@ private RowData toBinary(RowData row, RowType rowType) { return binaryrow(fields); } - /** An async function used for test. */ - public static class MyAsyncFunction extends RichAsyncFunction { - - private static final long serialVersionUID = 1L; - - private static final long TERMINATION_TIMEOUT = 5000L; - private static final int THREAD_POOL_SIZE = 10; - - private static ExecutorService executorService; - - private static @Nullable CountDownLatch lock; - - private static final AtomicInteger leftInvokeCount = new AtomicInteger(0); - - private static final AtomicInteger rightInvokeCount = new AtomicInteger(0); - - private static Optional expectedThrownException = Optional.empty(); - - private final AbstractTestSpec testSpec; - private final boolean treatRightAsLookupTable; - - private MyAsyncFunction(AbstractTestSpec testSpec, boolean treatRightAsLookupTable) { - this.testSpec = testSpec; - this.treatRightAsLookupTable = treatRightAsLookupTable; - } - - public static void block() throws Exception { - lock = new CountDownLatch(1); - } - - public static void release() { - Objects.requireNonNull(lock).countDown(); - } - - public static void setExpectedThrownException(Throwable t) { - expectedThrownException = Optional.of(t); - } - - public static void clearExpectedThrownException() { - expectedThrownException = Optional.empty(); - } - - @Override - public void asyncInvoke(final RowData input, final ResultFuture resultFuture) { - executorService.submit( - () -> { - try { - if (expectedThrownException.isPresent()) { - throw expectedThrownException.get(); - } - - if (lock != null) { - lock.await(); - } - - HashMap lookupTableData; - RowDataKeySelector streamSideJoinKeySelector; - RowDataKeySelector lookupSideJoinKeySelector; - if (treatRightAsLookupTable) { - synchronized (rightTableCurrentData) { - lookupTableData = new HashMap<>(rightTableCurrentData); - } - - streamSideJoinKeySelector = testSpec.getLeftJoinKeySelector(); - lookupSideJoinKeySelector = testSpec.getRightJoinKeySelector(); - leftInvokeCount.incrementAndGet(); - } else { - synchronized (leftTableCurrentData) { - lookupTableData = new HashMap<>(leftTableCurrentData); - } - - streamSideJoinKeySelector = testSpec.getRightJoinKeySelector(); - lookupSideJoinKeySelector = testSpec.getLeftJoinKeySelector(); - rightInvokeCount.incrementAndGet(); - } - - List results = new ArrayList<>(); - for (RowData row : lookupTableData.values()) { - if (streamSideJoinKeySelector - .getKey(input) - .equals(lookupSideJoinKeySelector.getKey(row))) { - results.add(row); - } - } - - resultFuture.complete(results); - } catch (Throwable e) { - resultFuture.completeExceptionally( - new RuntimeException("Failed to look up table", e)); - } - }); - } - - @Override - public void open(OpenContext openContext) throws Exception { - super.open(openContext); - - synchronized (MyAsyncFunction.class) { - if (executorService == null) { - executorService = Executors.newFixedThreadPool(THREAD_POOL_SIZE); - } - } - } - - @Override - public void close() throws Exception { - super.close(); - freeExecutor(); - } - - private void freeExecutor() { - synchronized (MyAsyncFunction.class) { - if (executorService == null) { - return; - } - - executorService.shutdown(); - - try { - if (!executorService.awaitTermination( - TERMINATION_TIMEOUT, TimeUnit.MILLISECONDS)) { - executorService.shutdownNow(); - } - } catch (InterruptedException interrupted) { - executorService.shutdownNow(); - - Thread.currentThread().interrupt(); - } - executorService = null; - } - } - } - - /** Provide some callback methods for test. */ - private static class MyAsyncExecutionControllerDelegate - extends TableAsyncExecutionController { - - private static boolean insertTableDataAfterEmit = true; - - public MyAsyncExecutionControllerDelegate( - AbstractTestSpec testSpec, - TableAsyncExecutionController innerAec) { - super( - innerAec.getAsyncInvoke(), - innerAec.getEmitWatermark(), - entry -> { - if (insertTableDataAfterEmit) { - StreamingDeltaJoinOperator.InputIndexAwareStreamRecordQueueEntry - inputIndexAwareEntry = - ((StreamingDeltaJoinOperator - .InputIndexAwareStreamRecordQueueEntry) - entry); - int inputIndex = inputIndexAwareEntry.getInputIndex(); - //noinspection unchecked - insertTableData( - testSpec, - (StreamRecord) inputIndexAwareEntry.getInputElement(), - inputIndex == 0); - } - - innerAec.getEmitResult().accept(entry); - }, - innerAec.getInferDrivenInputIndex(), - innerAec.getInferBlockingKey()); - } - } - - /** - * The {@link TestingFetcherResultFuture} is a simple implementation of {@link - * TableFunctionCollector} which forwards the collected collection. - */ - public static final class TestingFetcherResultFuture - extends TableFunctionResultFuture { - private static final long serialVersionUID = -312754413938303160L; - - @Override - public void complete(Collection result) { - //noinspection unchecked - getResultFuture().complete((Collection) result); - } - } - - private static class MockGeneratedFlatMapFunction - extends GeneratedFunction> { - - private static final long serialVersionUID = 1L; - - private final @Nullable Function condition; - - public MockGeneratedFlatMapFunction(@Nullable Function condition) { - super("", "", new Object[0]); - this.condition = condition; - } - - @Override - public FlatMapFunction newInstance(ClassLoader classLoader) { - return (value, out) -> { - if (condition == null || condition.apply(value)) { - out.collect(value); - } - }; - } - - @Override - public Class> compile(ClassLoader classLoader) { - // just avoid exceptions - return null; - } - } - - private abstract static class AbstractTestSpec { - - abstract RowType getLeftInputRowType(); - - final InternalTypeInfo getLeftTypeInfo() { - return InternalTypeInfo.of(getLeftInputRowType()); - } - - abstract Optional getLeftUpsertKey(); - - final RowDataKeySelector getLeftUpsertKeySelector() { - return getUpsertKeySelector(getLeftInputRowType(), getLeftUpsertKey().orElse(null)); - } - - abstract RowType getRightInputRowType(); - - final InternalTypeInfo getRightTypeInfo() { - return InternalTypeInfo.of(getRightInputRowType()); - } - - abstract Optional getRightUpsertKey(); - - final RowDataKeySelector getRightUpsertKeySelector() { - return getUpsertKeySelector(getRightInputRowType(), getRightUpsertKey().orElse(null)); - } - - abstract int[] getLeftJoinKeyIndices(); - - final RowDataKeySelector getLeftJoinKeySelector() { - return HandwrittenSelectorUtil.getRowDataSelector( - getLeftJoinKeyIndices(), - getLeftInputRowType().getChildren().toArray(new LogicalType[0])); - } - - abstract int[] getRightJoinKeyIndices(); - - final RowDataKeySelector getRightJoinKeySelector() { - return HandwrittenSelectorUtil.getRowDataSelector( - getRightJoinKeyIndices(), - getRightInputRowType().getChildren().toArray(new LogicalType[0])); - } - - final RowType getOutputRowType() { - return RowType.of( - Stream.concat( - getLeftInputRowType().getChildren().stream(), - getRightInputRowType().getChildren().stream()) - .toArray(LogicalType[]::new), - Stream.concat( - getLeftInputRowType().getFieldNames().stream(), - getRightInputRowType().getFieldNames().stream()) - .toArray(String[]::new)); - } - - final int[] getOutputFieldIndices() { - return IntStream.range(0, getOutputRowType().getFieldCount()).toArray(); - } + private abstract static class AbstractBinaryTestSpec extends AbstractBaseTestSpec { abstract Optional> getFilterOnLeftTable(); abstract Optional> getFilterOnRightTable(); - private RowDataKeySelector getUpsertKeySelector( - RowType rowType, @Nullable int[] upsertKey) { - if (upsertKey == null) { - upsertKey = IntStream.range(0, rowType.getFieldCount()).toArray(); + final GeneratedFilterCondition getGeneratedJoinCondition() { + int leftFieldCount = getLeftInputRowType().getFieldCount(); + RowType appliedRowType = combineRowTypes(getLeftInputRowType(), getRightInputRowType()); + int[] allJoinKeyFields = + new int[getLeftJoinKeyIndices().length + getRightJoinKeyIndices().length]; + List> conditionsPerJoinKey = new ArrayList<>(); + for (int i = 0; i < getLeftJoinKeyIndices().length; i++) { + allJoinKeyFields[2 * i] = getLeftJoinKeyIndices()[i]; + allJoinKeyFields[2 * i + 1] = leftFieldCount + getRightJoinKeyIndices()[i]; + int finalI = i; + conditionsPerJoinKey.add( + row -> row.getField(finalI).equals(row.getField(finalI + 1))); } - return HandwrittenSelectorUtil.getRowDataSelector( - upsertKey, rowType.getChildren().toArray(new LogicalType[0])); + return getFilterCondition( + appliedRowType, + allJoinKeyFields, + row -> + conditionsPerJoinKey.stream() + .allMatch(condition -> condition.apply(row))); } } @@ -1679,7 +1347,7 @@ private RowDataKeySelector getUpsertKeySelector( * and left_jk2_index = right_jk2 * */ - private static class LogLogTableJoinTestSpec extends AbstractTestSpec { + private static class LogLogTableJoinTestSpec extends AbstractBinaryTestSpec { private static final LogLogTableJoinTestSpec WITHOUT_FILTER_ON_TABLE = new LogLogTableJoinTestSpec(false); @@ -1783,7 +1451,7 @@ Optional> getFilterOnRightTable() { * ) on left_pk2_jk_index = right_pk2_jk_index * */ - private static class PkPkTableJoinTestSpec extends AbstractTestSpec { + private static class PkPkTableJoinTestSpec extends AbstractBinaryTestSpec { private static final PkPkTableJoinTestSpec WITHOUT_FILTER_ON_TABLE = new PkPkTableJoinTestSpec(false); diff --git a/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperatorTestBase.java b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperatorTestBase.java new file mode 100644 index 0000000000000..50b8d85297c67 --- /dev/null +++ b/flink-table/flink-table-runtime/src/test/java/org/apache/flink/table/runtime/operators/join/deltajoin/StreamingDeltaJoinOperatorTestBase.java @@ -0,0 +1,729 @@ +/* + * 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.runtime.operators.join.deltajoin; + +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.functions.IterationRuntimeContext; +import org.apache.flink.api.common.functions.OpenContext; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.streaming.api.functions.async.AsyncFunction; +import org.apache.flink.streaming.api.functions.async.ResultFuture; +import org.apache.flink.streaming.api.functions.async.RichAsyncFunction; +import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; +import org.apache.flink.streaming.runtime.tasks.TestProcessingTimeService; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorImpl; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor; +import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailbox; +import org.apache.flink.streaming.runtime.tasks.mailbox.TaskMailboxImpl; +import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.generated.FilterCondition; +import org.apache.flink.table.runtime.generated.GeneratedFilterCondition; +import org.apache.flink.table.runtime.generated.GeneratedFunction; +import org.apache.flink.table.runtime.keyselector.RowDataKeySelector; +import org.apache.flink.table.runtime.operators.join.lookup.keyordered.TableAsyncExecutionController; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.runtime.util.RowDataHarnessAssertor; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.HandwrittenSelectorUtil; + +import javax.annotation.Nullable; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; + +import static org.assertj.core.api.Assertions.assertThat; + +/** A base class used to test {@link StreamingDeltaJoinOperator}. */ +public class StreamingDeltaJoinOperatorTestBase { + + protected static final int AEC_CAPACITY = 100; + protected static final int CACHE_SIZE = 10; + + protected KeyedTwoInputStreamOperatorTestHarness + createDeltaJoinOperatorTestHarness( + int[] eachBinaryInputFieldSize, + DeltaJoinHandlerChain left2RightHandlerChain, + DeltaJoinHandlerChain right2LeftHandlerChain, + @Nullable GeneratedFilterCondition remainingJoinCondition, + DeltaJoinRuntimeTree joinRuntimeTree, + Set> left2RightDrivenSideInfo, + Set> right2LeftDrivenSideInfo, + RowDataKeySelector leftJoinKeySelector, + RowDataKeySelector leftUpsertKeySelector, + RowDataKeySelector rightJoinKeySelector, + RowDataKeySelector rightUpsertKeySelector, + Map>> lookupFunctions, + InternalTypeInfo leftTypeInfo, + InternalTypeInfo rightTypeInfo, + boolean enableCache) + throws Exception { + TaskMailbox mailbox = new TaskMailboxImpl(); + MailboxProcessor mailboxProcessor = + new MailboxProcessor(controller -> {}, mailbox, StreamTaskActionExecutor.IMMEDIATE); + + AsyncDeltaJoinRunner left2RightRunner = + new AsyncDeltaJoinRunner( + eachBinaryInputFieldSize, + remainingJoinCondition, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector, + left2RightHandlerChain, + joinRuntimeTree, + left2RightDrivenSideInfo, + true, + AEC_CAPACITY, + enableCache); + + AsyncDeltaJoinRunner right2LeftRunner = + new AsyncDeltaJoinRunner( + eachBinaryInputFieldSize, + remainingJoinCondition, + leftJoinKeySelector, + leftUpsertKeySelector, + rightJoinKeySelector, + rightUpsertKeySelector, + right2LeftHandlerChain, + joinRuntimeTree, + right2LeftDrivenSideInfo, + false, + AEC_CAPACITY, + enableCache); + + InternalTypeInfo joinKeyTypeInfo = leftJoinKeySelector.getProducedType(); + + StreamingDeltaJoinOperator operator = + new StreamingDeltaJoinOperator( + left2RightRunner, + right2LeftRunner, + lookupFunctions, + leftJoinKeySelector, + rightJoinKeySelector, + -1L, + AEC_CAPACITY, + new TestProcessingTimeService(), + new MailboxExecutorImpl( + mailbox, 0, StreamTaskActionExecutor.IMMEDIATE, mailboxProcessor), + CACHE_SIZE, + CACHE_SIZE, + (RowType) leftTypeInfo.toLogicalType(), + (RowType) rightTypeInfo.toLogicalType()); + + return new KeyedTwoInputStreamOperatorTestHarness<>( + operator, + leftJoinKeySelector, + rightJoinKeySelector, + joinKeyTypeInfo, + 1, + 1, + 0, + leftTypeInfo.toSerializer(), + rightTypeInfo.toSerializer()); + } + + protected RowDataHarnessAssertor createAssertor(RowType outputRowType) { + final int[] outputFieldIndices = + IntStream.range(0, outputRowType.getFieldCount()).toArray(); + return new RowDataHarnessAssertor( + outputRowType.getChildren().toArray(new LogicalType[0]), + // sort the result by the output upsert key + (o1, o2) -> { + for (int keyIndex : outputFieldIndices) { + LogicalType type = outputRowType.getChildren().get(keyIndex); + RowData.FieldGetter getter = RowData.createFieldGetter(type, keyIndex); + + int compareResult = + Objects.requireNonNull(getter.getFieldOrNull(o1)) + .toString() + .compareTo( + Objects.requireNonNull(getter.getFieldOrNull(o2)) + .toString()); + + if (compareResult != 0) { + return compareResult; + } + } + return o1.toString().compareTo(o2.toString()); + }); + } + + protected void verifyCacheData( + DeltaJoinCache actualCache, + Map> expectedCacheData, + long expectedCacheRequestCount, + long expectedCacheHitCount, + RowType joinKeyRowType, + RowType upsertKeyRowType, + RowType valueRowType, + boolean testLeftCache) { + String errorPrefix = testLeftCache ? "left cache " : "right cache "; + + @SuppressWarnings("unchecked") + Map> actualCacheData = + ((Map>) + (Map) + (testLeftCache + ? actualCache.getLeftCache().asMap() + : actualCache.getRightCache().asMap())); + assertThat( + convertCacheMapToGenericRowDataMap( + actualCacheData, joinKeyRowType, upsertKeyRowType, valueRowType)) + .as(errorPrefix + "data mismatch") + .isEqualTo( + convertCacheMapToGenericRowDataMap( + expectedCacheData, joinKeyRowType, upsertKeyRowType, valueRowType)); + + long actualCacheSize = + testLeftCache + ? actualCache.getLeftCache().size() + : actualCache.getRightCache().size(); + assertThat(actualCacheSize) + .as(errorPrefix + "size mismatch") + .isEqualTo(expectedCacheData.size()); + + long actualTotalSize = + testLeftCache + ? actualCache.getLeftTotalSize().get() + : actualCache.getRightTotalSize().get(); + assertThat(actualTotalSize) + .as(errorPrefix + "total size mismatch") + .isEqualTo(expectedCacheData.values().stream().mapToInt(Map::size).sum()); + + long actualRequestCount = + testLeftCache + ? actualCache.getLeftRequestCount().get() + : actualCache.getRightRequestCount().get(); + assertThat(actualRequestCount) + .as(errorPrefix + "request count mismatch") + .isEqualTo(expectedCacheRequestCount); + + long actualHitCount = + testLeftCache + ? actualCache.getLeftHitCount().get() + : actualCache.getRightHitCount().get(); + assertThat(actualHitCount) + .as(errorPrefix + "hit count mismatch") + .isEqualTo(expectedCacheHitCount); + } + + private GenericRowData convert2GenericRowData(RowData row, RowType rowType) { + if (row instanceof GenericRowData) { + return (GenericRowData) row; + } + + List fieldGetters = new ArrayList<>(); + for (int i = 0; i < rowType.getChildren().size(); i++) { + LogicalType type = rowType.getChildren().get(i); + fieldGetters.add(RowData.createFieldGetter(type, i)); + } + GenericRowData newRow = new GenericRowData(row.getArity()); + for (int i = 0; i < row.getArity(); i++) { + newRow.setField(i, fieldGetters.get(i).getFieldOrNull(row)); + } + return newRow; + } + + private Map> convertCacheMapToGenericRowDataMap( + Map> originalMap, + RowType joinKeyRowType, + RowType uniqueKeyRowType, + RowType valueRowType) { + Map> convertedMap = new HashMap<>(); + + for (Map.Entry> entry : originalMap.entrySet()) { + GenericRowData genericKey = convert2GenericRowData(entry.getKey(), joinKeyRowType); + Map innerMap = new HashMap<>(); + + for (Map.Entry innerEntry : entry.getValue().entrySet()) { + GenericRowData genericInnerKey = + convert2GenericRowData(innerEntry.getKey(), uniqueKeyRowType); + GenericRowData genericInnerValue = + convert2GenericRowData(innerEntry.getValue(), valueRowType); + innerMap.put(genericInnerKey, genericInnerValue); + } + + convertedMap.put(genericKey, innerMap); + } + + return convertedMap; + } + + protected TableAsyncExecutionController unwrapAEC( + KeyedTwoInputStreamOperatorTestHarness + testHarness) { + return unwrapOperator(testHarness).getAsyncExecutionController(); + } + + protected DeltaJoinCache unwrapCache( + KeyedTwoInputStreamOperatorTestHarness + testHarness) { + DeltaJoinCache cacheInLeftRunner = + unwrapOperator(testHarness).getLeftTriggeredUserFunction().getCache(); + DeltaJoinCache cacheInRightRunner = + unwrapOperator(testHarness).getRightTriggeredUserFunction().getCache(); + + // the object ref must be the same + assertThat(cacheInLeftRunner == cacheInRightRunner).isTrue(); + return cacheInLeftRunner; + } + + protected StreamingDeltaJoinOperator unwrapOperator( + KeyedTwoInputStreamOperatorTestHarness + testHarness) { + return (StreamingDeltaJoinOperator) testHarness.getOperator(); + } + + protected static GeneratedFilterCondition getFilterCondition( + RowType rowType, + int[] allInvolvingFields, + Function condition) { + return new GeneratedFilterCondition("", "", new Object[0]) { + @Override + public FilterCondition newInstance(ClassLoader classLoader) { + return new MockFilterCondition() { + @Override + public boolean apply(RowData in) { + GenericRowData rowData = new GenericRowData(allInvolvingFields.length); + for (int i = 0; i < allInvolvingFields.length; i++) { + int leftField = allInvolvingFields[i]; + RowData.FieldGetter fieldGetter = + RowData.createFieldGetter( + rowType.getChildren().get(leftField), leftField); + rowData.setField(i, fieldGetter.getFieldOrNull(in)); + } + return condition.apply(rowData); + } + }; + } + + @Override + public Class getClass(ClassLoader classLoader) { + return FilterCondition.class; + } + }; + } + + protected static RowType combineRowTypes(RowType... rowTypes) { + return new RowType( + Arrays.stream(rowTypes) + .flatMap(i -> i.getFields().stream()) + .collect(Collectors.toList())); + } + + protected static RowDataKeySelector getKeySelector(int[] keys, RowType rowType) { + return HandwrittenSelectorUtil.getRowDataSelector( + keys, rowType.getChildren().toArray(new LogicalType[0])); + } + + protected static void validateCalcFunctionAndCollectorWhenLookup( + @Nullable ResultFuture resultFuture, boolean shouldOmit) { + assertThat(resultFuture).isNotNull(); + assertThat(resultFuture) + .isInstanceOf(LookupHandlerBase.Object2RowDataConverterResultFuture.class); + LookupHandlerBase.Object2RowDataConverterResultFuture object2RowDataConverterResultFuture = + (LookupHandlerBase.Object2RowDataConverterResultFuture) resultFuture; + if (shouldOmit) { + assertThat(object2RowDataConverterResultFuture.getCalcFunction()).isNull(); + assertThat(object2RowDataConverterResultFuture.getCalcCollector()).isNull(); + } else { + assertThat(object2RowDataConverterResultFuture.getCalcFunction()).isNotNull(); + assertThat(object2RowDataConverterResultFuture.getCalcCollector()).isNotNull(); + } + } + + protected GeneratedFunction> createFlatMap( + Function> flatMapFunc) { + return new GeneratedFunction("", "", new Object[0]) { + + @Override + public org.apache.flink.api.common.functions.Function newInstance( + ClassLoader classLoader) { + return (FlatMapFunction) + (value, out) -> { + Optional result = flatMapFunc.apply(value); + result.ifPresent(out::collect); + }; + } + + @Override + public Class compile( + ClassLoader classLoader) { + return FlatMapFunction.class; + } + }; + } + + protected GeneratedFunction> createDoubleGreaterThanFilter( + double bottom, int fieldIndex) { + return createFlatMap( + rowData -> + rowData.getDouble(fieldIndex) >= bottom + ? Optional.of(rowData) + : Optional.empty()); + } + + protected GeneratedFunction> createFetcherFunction( + Map> tableCurrentDataMap, + RowDataKeySelector streamSideLookupKeySelector, + RowDataKeySelector lookupSideLookupKeySelector, + int lookupTableIdx, + @Nullable Throwable expectedThrownException) { + + return new GeneratedFunction("", "", new Object[0]) { + @Override + public MyAsyncFunction newInstance(ClassLoader classLoader) { + return new MyAsyncFunction( + tableCurrentDataMap, + streamSideLookupKeySelector, + lookupSideLookupKeySelector, + lookupTableIdx, + expectedThrownException); + } + + @Override + public Class getClass(ClassLoader classLoader) { + return MyAsyncFunction.class; + } + }; + } + + /** Provide some callback methods for test. */ + protected static class MyAsyncExecutionControllerDelegate + extends TableAsyncExecutionController { + + public MyAsyncExecutionControllerDelegate( + TableAsyncExecutionController innerAec, + boolean insertTableDataAfterEmit, + BiConsumer insertDataConsumer) { + super( + innerAec.getAsyncInvoke(), + innerAec.getEmitWatermark(), + entry -> { + if (insertTableDataAfterEmit) { + StreamingDeltaJoinOperator.InputIndexAwareStreamRecordQueueEntry + inputIndexAwareEntry = + ((StreamingDeltaJoinOperator + .InputIndexAwareStreamRecordQueueEntry) + entry); + int inputIndex = inputIndexAwareEntry.getInputIndex(); + insertDataConsumer.accept( + inputIndex, + (RowData) inputIndexAwareEntry.getInputElement().getValue()); + } + + innerAec.getEmitResult().accept(entry); + }, + innerAec.getInferDrivenInputIndex(), + innerAec.getInferBlockingKey()); + } + } + + protected abstract static class MockFilterCondition implements FilterCondition { + + private static final long serialVersionUID = 1L; + + @Override + public void open(OpenContext openContext) throws Exception {} + + protected abstract boolean apply(RowData in); + + @Override + public boolean apply(Context ctx, RowData in) { + return apply(in); + } + + @Override + public void close() throws Exception {} + + @Override + public RuntimeContext getRuntimeContext() { + return null; + } + + @Override + public IterationRuntimeContext getIterationRuntimeContext() { + return null; + } + + @Override + public void setRuntimeContext(RuntimeContext t) {} + } + + /** An async function used for test. */ + protected static class MyAsyncFunction extends RichAsyncFunction { + + private static final long serialVersionUID = 1L; + + private static final long TERMINATION_TIMEOUT = 5000L; + private static final int THREAD_POOL_SIZE = 10; + + private static ExecutorService executorService; + + private static @Nullable CountDownLatch lock; + + private static final Map lookupInvokeCount = + new ConcurrentHashMap<>(); + + public static Map getLookupInvokeCount() { + return lookupInvokeCount; + } + + private final Map> tableCurrentDataMap; + private final RowDataKeySelector streamSideLookupKeySelector; + private final RowDataKeySelector lookupSideLookupKeySelector; + private final int lookupTableIdx; + private final @Nullable Throwable expectedThrownException; + + private @Nullable ResultFuture lastResultFuture; + + public MyAsyncFunction( + Map> tableCurrentDataMap, + RowDataKeySelector streamSideLookupKeySelector, + RowDataKeySelector lookupSideLookupKeySelector, + int lookupTableIdx, + @Nullable Throwable expectedThrownException) { + this.tableCurrentDataMap = tableCurrentDataMap; + this.streamSideLookupKeySelector = streamSideLookupKeySelector; + this.lookupSideLookupKeySelector = lookupSideLookupKeySelector; + this.lookupTableIdx = lookupTableIdx; + this.expectedThrownException = expectedThrownException; + } + + public static void block() throws Exception { + lock = new CountDownLatch(1); + } + + public static void release() { + Objects.requireNonNull(lock).countDown(); + } + + @Override + public void asyncInvoke(final RowData input, final ResultFuture resultFuture) { + this.lastResultFuture = resultFuture; + executorService.submit( + () -> { + try { + if (expectedThrownException != null) { + throw expectedThrownException; + } + + if (lock != null) { + lock.await(); + } + + LinkedHashMap lookupTableData; + final RowDataKeySelector copiedStreamSideLookupKeySelector = + streamSideLookupKeySelector.copy(); + final RowDataKeySelector copiedLookupSideLookupKeySelector = + lookupSideLookupKeySelector.copy(); + + synchronized (tableCurrentDataMap) { + lookupTableData = + new LinkedHashMap<>( + tableCurrentDataMap.getOrDefault( + lookupTableIdx, new LinkedHashMap<>())); + } + + lookupInvokeCount.compute( + lookupTableIdx, + (k, v) -> { + if (v == null) { + v = new AtomicInteger(0); + } + v.incrementAndGet(); + return v; + }); + + List results = new ArrayList<>(); + for (RowData row : lookupTableData.values()) { + if (copiedStreamSideLookupKeySelector + .getKey(input) + .equals(copiedLookupSideLookupKeySelector.getKey(row))) { + results.add(row); + } + } + + resultFuture.complete(results); + } catch (Throwable e) { + resultFuture.completeExceptionally( + new RuntimeException("Failed to look up table", e)); + } + }); + } + + @Override + public void open(OpenContext openContext) throws Exception { + super.open(openContext); + + synchronized (MyAsyncFunction.class) { + if (executorService == null) { + executorService = Executors.newFixedThreadPool(THREAD_POOL_SIZE); + } + } + } + + @Override + public void close() throws Exception { + super.close(); + freeExecutor(); + } + + @Nullable + public ResultFuture getLastResultFuture() { + return lastResultFuture; + } + + private void freeExecutor() { + synchronized (MyAsyncFunction.class) { + if (executorService == null) { + return; + } + + executorService.shutdown(); + + try { + if (!executorService.awaitTermination( + TERMINATION_TIMEOUT, TimeUnit.MILLISECONDS)) { + executorService.shutdownNow(); + } + } catch (InterruptedException interrupted) { + executorService.shutdownNow(); + + Thread.currentThread().interrupt(); + } + executorService = null; + } + } + } + + /** Base test specification shared by binary and cascaded delta join test specs. */ + protected abstract static class AbstractBaseTestSpec { + + abstract RowType getLeftInputRowType(); + + final InternalTypeInfo getLeftInputTypeInfo() { + return InternalTypeInfo.of(getLeftInputRowType()); + } + + abstract RowType getRightInputRowType(); + + final InternalTypeInfo getRightInputTypeInfo() { + return InternalTypeInfo.of(getRightInputRowType()); + } + + final RowType getOutputRowType() { + return RowType.of( + Stream.concat( + getLeftInputRowType().getChildren().stream(), + getRightInputRowType().getChildren().stream()) + .toArray(LogicalType[]::new), + Stream.concat( + getLeftInputRowType().getFieldNames().stream(), + getRightInputRowType().getFieldNames().stream()) + .toArray(String[]::new)); + } + + abstract int[] getLeftJoinKeyIndices(); + + final RowDataKeySelector getLeftJoinKeySelector() { + return HandwrittenSelectorUtil.getRowDataSelector( + getLeftJoinKeyIndices(), + getLeftInputRowType().getChildren().toArray(new LogicalType[0])); + } + + abstract int[] getRightJoinKeyIndices(); + + final RowDataKeySelector getRightJoinKeySelector() { + return HandwrittenSelectorUtil.getRowDataSelector( + getRightJoinKeyIndices(), + getRightInputRowType().getChildren().toArray(new LogicalType[0])); + } + + abstract Optional getLeftUpsertKey(); + + final RowDataKeySelector getLeftUpsertKeySelector() { + return getUpsertKeySelector(getLeftInputRowType(), getLeftUpsertKey().orElse(null)); + } + + abstract Optional getRightUpsertKey(); + + final RowDataKeySelector getRightUpsertKeySelector() { + return getUpsertKeySelector(getRightInputRowType(), getRightUpsertKey().orElse(null)); + } + + private RowDataKeySelector getUpsertKeySelector( + RowType rowType, @Nullable int[] upsertKey) { + if (upsertKey == null) { + upsertKey = IntStream.range(0, rowType.getFieldCount()).toArray(); + } + return HandwrittenSelectorUtil.getRowDataSelector( + upsertKey, rowType.getChildren().toArray(new LogicalType[0])); + } + } + + protected static class MockGeneratedFlatMapFunction + extends GeneratedFunction> { + + private static final long serialVersionUID = 1L; + + private final @Nullable Function condition; + + public MockGeneratedFlatMapFunction(@Nullable Function condition) { + super("", "", new Object[0]); + this.condition = condition; + } + + @Override + public FlatMapFunction newInstance(ClassLoader classLoader) { + return (value, out) -> { + if (condition == null || condition.apply(value)) { + out.collect(value); + } + }; + } + + @Override + public Class> compile(ClassLoader classLoader) { + // just avoid exceptions + return null; + } + } +} From 86bc98e3fe4de7a336e5fce01aca9bfdaa5a7c44 Mon Sep 17 00:00:00 2001 From: xuyang Date: Thu, 12 Mar 2026 12:18:12 +0800 Subject: [PATCH 2/3] regenerate delta join restore tests --- .../exec/stream/DeltaJoinTestPrograms.java | 12 +- ...and-calc-on-cdc-source-without-delete.json | 925 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 14589 bytes ...ta-join-with-cache-and-calc-on-source.json | 935 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 15681 bytes ...h-cache-and-cdc-source-without-delete.json | 474 +++++++++ .../savepoint/_metadata | Bin 0 -> 14838 bytes .../plan/delta-join-with-cache.json | 474 +++++++++ .../delta-join-with-cache/savepoint/_metadata | Bin 0 -> 14193 bytes ...ith-calc-on-cdc-source-without-delete.json | 925 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 14544 bytes ...calc-on-source-and-filter-pushed-down.json | 935 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 15826 bytes .../plan/delta-join-with-calc-on-source.json | 935 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 15685 bytes ...a-join-with-cdc-source-without-delete.json | 474 +++++++++ .../savepoint/_metadata | Bin 0 -> 14838 bytes ...lta-join-with-join-key-contains-index.json | 454 +++++++++ .../savepoint/_metadata | Bin 0 -> 14188 bytes ...delta-join-with-join-key-equals-index.json | 406 ++++++++ .../savepoint/_metadata | Bin 0 -> 13772 bytes ...ta-join-with-with-non-equiv-condition.json | 474 +++++++++ .../savepoint/_metadata | Bin 0 -> 13778 bytes 23 files changed, 7419 insertions(+), 4 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-cdc-source-without-delete/plan/delta-join-with-cache-and-calc-on-cdc-source-without-delete.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-cdc-source-without-delete/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-source/plan/delta-join-with-cache-and-calc-on-source.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-source/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-cdc-source-without-delete/plan/delta-join-with-cache-and-cdc-source-without-delete.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-cdc-source-without-delete/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache/plan/delta-join-with-cache.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-cdc-source-without-delete/plan/delta-join-with-calc-on-cdc-source-without-delete.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-cdc-source-without-delete/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source-and-filter-pushed-down/plan/delta-join-with-calc-on-source-and-filter-pushed-down.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source-and-filter-pushed-down/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source/plan/delta-join-with-calc-on-source.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cdc-source-without-delete/plan/delta-join-with-cdc-source-without-delete.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cdc-source-without-delete/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-contains-index/plan/delta-join-with-join-key-contains-index.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-contains-index/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-equals-index/plan/delta-join-with-join-key-equals-index.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-equals-index/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-with-non-equiv-condition/savepoint/_metadata diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java index 708b4371dfc07..04b319c26a96c 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java @@ -99,7 +99,8 @@ public class DeltaJoinTestPrograms { .runSql( "insert into snk " + "select * from leftSrc join rightSrc " - + "on a1 = b1") + + "on a1 = b1 " + + "on conflict do deduplicate") .build(); public static final TableTestProgram DELTA_JOIN_WITH_JOIN_KEY_CONTAINS_INDEX = @@ -115,7 +116,8 @@ public class DeltaJoinTestPrograms { .runSql( "insert into snk " + "select * from leftSrc join rightSrc " - + "on a1 = b1 and a0 = b0") + + "on a1 = b1 and a0 = b0 " + + "on conflict do deduplicate") .build(); public static final TableTestProgram DELTA_JOIN_WITH_NON_EQUIV_CONDITION = @@ -166,7 +168,8 @@ public class DeltaJoinTestPrograms { .runSql( "insert into snk " + "select * from leftSrc join rightSrc " - + "on a1 = b1 and a2 <> b2") + + "on a1 = b1 and a2 <> b2 " + + "on conflict do deduplicate") .build(); public static final TableTestProgram DELTA_JOIN_WITH_CALC_ON_SOURCE = @@ -197,7 +200,8 @@ public class DeltaJoinTestPrograms { + " select b0, b1, b1 + 1 as new_b1, b2 from rightSrc " + " where b0 = cast(3.0 as double) or b0 = cast(5.0 as double) " + ") " - + "on a1 = b1 and a0 = b0") + + "on a1 = b1 and a0 = b0 " + + "on conflict do deduplicate") .build(); public static final TableTestProgram DELTA_JOIN_WITH_CALC_ON_SOURCE_AND_FILTER_PUSHED_DOWN = diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-cdc-source-without-delete/plan/delta-join-with-cache-and-calc-on-cdc-source-without-delete.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-cdc-source-without-delete/plan/delta-join-with-cache-and-calc-on-cdc-source-without-delete.json new file mode 100644 index 0000000000000..2e32b44ec78f8 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-cdc-source-without-delete/plan/delta-join-with-cache-and-calc-on-cdc-source-without-delete.json @@ -0,0 +1,925 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 79, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 80, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 81, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Calc(select=[a0, a1, CONCAT(a2, '-s') AS new_a2], where=[SEARCH(a0, Sarg[1.0, 2.0])])" + }, { + "id" : 82, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 83, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 84, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 85, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, CONCAT(b2, '-s') AS new_b2], where=[SEARCH(b0, Sarg[2.0, 3.0])])" + }, { + "id" : 86, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 87, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1 ], + "leftUpsertKey" : [ 0, 1 ], + "rightJoinKeys" : [ 1 ], + "rightUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 1 ], + "rightJoinKey" : [ 1 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "rowTypeAfterCalc" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "rowType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "rowTypeAfterCalc" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "rowType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], leftToRight=[Binary], rightToLeft=[Binary], select=[a0, a1, new_a2, b0, b1, new_b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 88, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Calc(select=[a1, a0, new_a2, b0, new_b2, b1])" + }, { + "id" : 89, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, new_a2, b0, new_b2, b1])" + } ], + "edges" : [ { + "source" : 79, + "target" : 80, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 80, + "target" : 81, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 81, + "target" : 82, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 83, + "target" : 84, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 84, + "target" : 85, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 85, + "target" : 86, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 82, + "target" : 87, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 86, + "target" : 87, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 87, + "target" : 88, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 88, + "target" : 89, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..ba1690cdf88f12053112f5580f14878f2352ea45 GIT binary patch literal 14589 zcmeHOe{5UD9X~rE{Q;qA8p_%Y6p7HOqtEtp{NuSn0w;0TfSW|ITS~Ixo!>hruX%pY zd+#|-DgmiBjZQGwCeYvyph(rEX+`T)At6mnut_j&64N$K{ILnKNz*1Yv}$BUf^FUR zdw$P;j-51ZNd>|u**<^o?t9<+-re`^KHs~MuTBeukPi4@@Kt`uX3{fk8l{6Jh1zV8 z6%3P2Ns%MuCMl9rAVDbe6d})(9-evng|6>^`@{GATedY#gGR zJS9m-$OcKu$?#|TQRNj;G7?L^kDl9m_SGWf#=?tO>JnU{$R*0TJlKBnuSb6M@9&&{ z{G9fSx1aj0Q+QP_ox@DC={<(3Pf773q*IFL6r>18fpl{0*_+?-zj@(}nWYQgeeJEu z6&Y@T6by^}yYekIDJNFp2x1zB8_LSjM^;w&l)kwhpj*+8Vi_Z-Byb%U@w$d*;X|xFku`UrZ&wmQ>B`EoSA(Ml#EVG z){J2T`!@La&b&vivrMvQg*uS870`@n6|8<>M7Og*aMbDZ6Ebg-(3*;-D=g}Z`zOg} z)fz2p8a60RHez)XhGE+zmIk)LN~?ps)j{6sAeSiUNEGz2#c~7lCGwC-B9|)?mUUW1 zx_PT^JWA(iK%@F}Af=aaed7TWOE%dsGMY_&IW<9g?)xv$amDkI(S}-SNe#$Gv1sU? zPRdh@p(Z`ORQIp0hkRsLWqHft@|H77%%rwq=BTxx%eiAzwR4sY%YLpR9!Sx3Z|eTk za4wTgW>dNB{o^Upo*v0$A>B{94kgnEQ@KoPVkDU!nFRTfn!F6T)2ihWyiU~4zqNBU zcvUjglr>i^_G4w#y5l%oE@`Z$l_t#nWY;PTv3P(piFo>@olG~Bv22skq6&TM)?J~P zOg2mp6kjoCzFkXk^7m@aa;Wz{5rvCR(;NhhYwP438Zg zOs84~5QoE+8bC*TYg+>jQFCa9n)SoaH$plzrcc{5pS-oicTLukL&=Gu{mF@kFJ8Rp zv0H6T*8P>&lgh-d8PC4uggPU&9xK)43I@ovuxN-X_8!bAnBHX@m1e{`nOj!8WyPE9 z#9CH-*%sK@2wv7Uwq?aD=bNihWy^}gakFK`@i5)mi(l?`vSr1u#`HK~M9Ye|tat_2 zTkpO2dYm76Mw16pnepUM13q=A;bh^SDLC{MIAo~GC@nGzAW{#0)VBh??o@%hrR{IS zSC_kQ@_4U&ar_f`}5=-pU3hGp#uUwM1rAX^Qc89=P}_FDSvl$1c%Um|A#<-)=bmrnlc z;S;wlrG7j)o9cu7D90j}a|QBM0L2?HzhE+}tl3<_j-tBn#LR?rKYi^%d?n1pd?X>T zgcKJdaWO0;7<|dB5RzjdS)x%u$-9?8Isox~WO);?g+9_dHlCVDX2&KRw00P9+KG|D zgV~X>(cFQt;S}l0jHgGkxoSd+mun8=Hcpe3+ytq4o0&S*Ja9oHscxD$*sim}+|e*0 zy;$Qum1&BF$9n{Nmd>8}!iy)K&-sZjO>Qh;nwwrcGj^PKb!1 z(mx*W;LGwbunh*j z!YtWTOO-+0?HSXR(;<#&9VP{8;PVhi));S}F99m=r{bE2W)pWLGU_A`;ZIl1O~*&- zPR>09+^^<*R$qa0Zt?IB1OJZ)4d*ras!j3W3$^O55>_|zN!uu@vTKZYp3Ln2@c3-9V{Ztf3d3g}UX{zap=`qrIy!6KgdBo?|~Zq1=dm zYOh7q!#E$08T`a)@|rsZTf-6V9^oGJ>QKejaI77FE?1KqJiCw9+HxghvB{0Gj&->) z)_B?IvJ{o50vBxy;d~(sLJB8U(eT(H@=sTKjVnqwSS@SIClOI&tD!G5TPCJ$Ib_<$wUF~@(W+un9Zw5>o5J)FUfRnAN;4R&SCr|x(EcV5l{`9S9 z9*n-PXltpX$DJ;GTH$<=`eD9E{my(DKl{?o`Q3l8UKrT*z?)C@{LkeH^CcR}hw@R8 z#-(sViN^b>*q;x_qD*E97;*(1`=65e!s5|*rtGa5 zuE~b`sHQ-?(~Mv(YtS_xSO#zTWAG6G+r+gfA>%&Dz~ZMK`TfSzhkIV;Xys;*u}Jx| z2Af11L3X$h&Vp>7$$;@1da%l9U$vNvXWFDG*4YE7yj|;rBu9vCvcIvaX07(h-3P9M z22P+0xTM", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 44, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Calc(select=[a0, a1, a2, (a1 + 1) AS new_a1], where=[SEARCH(a1, Sarg[1, 5])])" + }, { + "id" : 45, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 46, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 47, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, (b1 + 1) AS new_b1, b2], where=[SEARCH(b0, Sarg[3.0, 5.0])])" + }, { + "id" : 48, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 49, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1, 0 ], + "rightJoinKeys" : [ 1, 0 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 1, 0 ], + "rightJoinKey" : [ 1, 0 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "rowTypeAfterCalc" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "rowType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "rowTypeAfterCalc" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "rowType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT, `b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], leftToRight=[Binary], rightToLeft=[Binary], select=[a0, a1, a2, new_a1, b0, b1, new_b1, b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 50, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Calc(select=[new_a1, a0, a2, b0, b2, new_b1])" + }, { + "id" : 51, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "conflictStrategy" : { + "behavior" : "DEDUPLICATE" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[new_a1, a0, a2, b0, b2, new_b1], conflictStrategy=[DEDUPLICATE])" + } ], + "edges" : [ { + "source" : 43, + "target" : 44, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 44, + "target" : 45, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 46, + "target" : 47, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 47, + "target" : 48, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 45, + "target" : 49, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 48, + "target" : 49, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 49, + "target" : 50, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 50, + "target" : 51, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-source/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-calc-on-source/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..4a4fd1759e0df1fec28646343b5908bfca078cef GIT binary patch literal 15681 zcmeGjZHya7b?jU&Nt?@EE~kd1ZFH#DsD;>G+q?eAMI=6-L)K z@4CC|J70c;3y6NGQYeTDDg=ojKZ68{A{AADpp-(3Kx!nAP{cFum$78+FM%0s&)suS%4fW)4A$S69 z5P}5*a=y3kw?7`cZ0_0rqhEc}^NTNi({3|Q+c}LrUF>5st?$!`QB_RCRo=hPgFaW1 z6|y%lgY`g3QzW7rfshc;xuRa2m1;rZ6vqeWBC|rUW)v$r2hU_BkPl2(RoV0fBN09t z;$xu*7v^K(sKCbq2A&}mO;$~xL}D>X^bsx|mO{}|lt@A#hJ$e-#)&>Y9*Ow)m=yNK z!X?SaN%3HWi$)_j%pufY11fjf#+uJ33fb0k&BrLyTnCVecGgs_Bz{T*2ZL@s|`}mEXig zh4}TVYRYBeH|Hv(YRZb?Pt{DKt60eqUB-%his(blSveI~j490o*y~WwAJ`CFd>5@Z zboAk6Fi{MK|Da41$r!~Z&cpZLzH#xMub(+r@FGtJZ7I<*r^ql-UuGK5-~c5nkdX~@ zxTd4~uM6F2{KL=ub>J)BPnEsMn?b#zrp~~a%bJ>0umOO#T7VU-P6k-ka0cC}Xp^#t z6-t(gLK(mlu=o{V{0cCB1z0ZvcHKax-e$N~nbFbCX7X6Kv)U}MiazVSs0*;MzK+a| z;WA+Bb#?SGt_&YXy>|J8X2>QGZlZ0+T||!;r-^6+)m5x-KBXj_{s)Q=GK zY6}%jKb0VVO0AaNg%uO6%Z@(+JlxGHQTg>JHAR6TgP}Brj;Li#l1nn{6v*{^z4r$% zzjL;aaj#2(9EW#%ZVaFfc*h8ufBms*yBHXcXrjAZBV4a$4ez!(yRgnx(rZ#U>D3$vsAsip+7=fe6wX@2;K*&M%%mr;aCTPFA+o3<%%WL{Vti# znYye_QkhmkDJTc-hBO^@ zcQWW4Ko7Ll$x*Ee($*3Iau9TnjvpDxq+IYr0KYwbIEwY;6xJINz_4^HM4dFJmUiW6 zBAYyv$bMpBVWEz$$g%Vgq8;At$`7&hhdeZY&yqGlR}MDKs|{Do_0o{E9=rqTKttW*i8Szo>0 zShZXv`jZ!*zxU41|K?c=tl}n>rDFiv)x;X4`Agq_=)sf!=mN;24rEHlFl*xG&XVSq zM4tc4h1jX#OCxoROBolV&<-?1quHY@yJd@gtjOUsLIaQd6L@Sb{`qBuuz9%#Eu)Nf zt=3IB4U08svb|8ZH^aA3qjqI@&E|B^TFmLrbYXMvW~Pbv9Rt^7QM9K+n}4@v8hqbz za&1J>HQjf_(KgywYNK_;k?s)ujwSD!tXIm(4b9@%SzA`LUvSnImkrR3#+qyZT#@~v zRxWF*YrkH^2Jyox?XmzGz8Y5lX1ii}n3nbA0=qP$0rIgXYL{~`SZ37{qoq}51N_^U z^3@nex~GcKKl(WCH0x{NfZ`K;L4-c-eCzgiV?C=B{lHSr z8yze6voVPkkc=q!X8C9VmRxcBmf!vBPfzTBEY>sq+|GA@_72ph449{_oKdD}_j6Yg z{2!){Wz|fnB@LubXa3&p;O_-5on7RE-JdLBQ`2V>e}46W>9bc~zWB_@mcuWA*Yw6X zY)W}bDbFZ9u>EU4Kk>|0-~MdxTUUQS@%ZcQKpAk9LYF6}EUk!8t%%rK@!EI4@b(vH zFU4Nke4jTJc8Kh_#rq6IL^mUpdLP6kZ6u?8nV8v zpL^?h^R*|xbLFL%KN!{aJ0b$_Mg`eGbKse0|DC@M2iBAWcVAvpbRA^Na$wEP9vqcm zg69OEz=z?aG9*gAcq|h4i9wDl!9gd-6~TRf&q1J=8INAnY=$pq<~O+^>489_)xX5z z@OB)N)Zf42?YDi7@xSN^S`&Ag?Fn*izP-2E=G%jITZTu08Xg6$LUd|A7qBs>iiKG$ zn+1bzt_t?l<3W87ryfm>+E)wmk4~iER2nWAK)DySzCn;!!A^vpCf0Wy4Qg&*7H~1! zgOV*lv!fpq6vDcf;2eGAs}+S@7q}Pgz79az+|PjUE+V?nWK$d1ju2diL&?Qk5>}2* zL`|3cIaZufNSQQvUzW*l!3D320MDvUtzDhT;g-0GJATdo$ggQjU+2uHeqU(Ct9-}F z4jSygnPUpuPI67UT`eZv<^5|*{og}NGa9D&)o}Z+wv&A|Xzr}5i)U<>yfohFp_j?p zn-^W8@0f9Lzi}0=$gQQ(-P!Wpf-PUiD(P4yjg|#m@>~-wb|3xU#_Q^8tb41yR(ICb z&br#->Ura}rq0UHSsDKOD?`V~yBQBPvMWoZm?&)lVIdeO#Xs85m&5$jSCt0I9=E$JBQ zv9k_hh#rOSUguj@Wn;>nldBeuYMJIZU~*NXLR5)nb+da;kl0jJMFm5!r3BG{Z9`+(=;J{UR1|E72{|Wfl{P>B)Rm1FJ~r?oi^{qKXe?tEFJ&Sj z1Hc+%AK|=%1W1jDa1^7dfki+E7n?b8TPA85(=`Cmhh}{c5-bwVM2DJDwd%E-?jZ^cOk=#UTZ1mqvdhaO! literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-cdc-source-without-delete/plan/delta-join-with-cache-and-cdc-source-without-delete.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-cdc-source-without-delete/plan/delta-join-with-cache-and-cdc-source-without-delete.json new file mode 100644 index 0000000000000..763fa87a3f504 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-cdc-source-without-delete/plan/delta-join-with-cache-and-cdc-source-without-delete.json @@ -0,0 +1,474 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 71, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 72, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 73, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 74, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 75, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 76, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 77, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "leftUpsertKey" : [ 0, 1 ], + "rightJoinKeys" : [ 2 ], + "rightUpsertKey" : [ 0, 2 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 2 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], leftToRight=[Binary], rightToLeft=[Binary], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 78, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 71, + "target" : 72, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 72, + "target" : 73, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 74, + "target" : 75, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 75, + "target" : 76, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 73, + "target" : 77, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 76, + "target" : 77, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 77, + "target" : 78, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache-and-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..e7117a5ea9b4e55051bcb67289bb927b43f554c7 GIT binary patch literal 14838 zcmeHOTWlN06&+Htcc70D&71U4noF%w!6NXmAsI`nXND6O^J zWpr1po4qJRq&2ogBWPx^q1!hdaneiQ|Y`~)dl#ApK;OB@CA%x5h*4_ z3PLa{VIdlTtPfb(Az+VPt*0A4J!nU@?hL^a(X%EtiIwX+MOh&S{0mt37c~VE4L=|D z>w$t^Sd^B6;j@8Ia5=IV4lWsmvL1kEw(QUO&s0>|gP3qU5hJ{aBe57C^2frw5DmsK$BTgo7mCCr4##34E{Y=~&@4hEf`PQZ z!Vfsuf^gb6V9VrGG84~DX7brwJeSNTQrTQ4H8!0~O-|&GP9~CQAUl;#6$+4!(}Ya5noY})p_`keWF$>X{yg$E|ZEWD~7Lt4dTPP zj+cF7%O;u9<&td53nWgXvP74$BA+Gt@LGvk6_<@U&4hZp;8*`VIIgLNsbgSZQY~sA zF#3!TJ%v6aMDyZm61M>LNZ8c$_g;T1_x!u>`wm=qDe>l)F24+DlszjH)fNK!?tJ>P z(7pphhC0undjZ;jn9ENOdVlqU6C?j9{%i4I`A2U-5f_vtPY=-ioKB32Vj2{BYZZUz zlUDdWEBv4u?wYc~E7fptvlU8K_;xki{g4$ddC=V!O))@I)^;aV}Z7S9Gt9yc~7FPvN2In6k2r}ZKjoHG|ek%e|51g+XmMbk$m$d^Vk zvqphj|GWRu;Ol?;)&S#Pw*oOX@Alswpa#5W46R&${^lbL#v_{OK}QeQvQ@ixR}Go1 zICOk1o&Vj#-~SDMwF`MtsM~<$2iOI3)FK_;J~1bVOUc0paKZ!Iy|3 z>T*I4+ude{QrAHy^ ztT|V0if6Sc$KsjsBk|0aZ{508tF91W@56{Tc%LJlXYY@DX#D<$GC`L12rDzhs@cP0 zK3gXVYftiVQ!B}$JsI!)TK2II&wi2j!oZ<5dSRa2Q5!zcPh^(pJKwo<`R4hTqaeL0 zv6rdeNs-3j*+Ap7ZZ2qHeKu*}m zl#WqW#L1n8;#G;fbmMaL?C5J_wHg~47o}*2+-Nj9!J?a%c#pim6+yVI*4%b$c^>lVQ}Z4F7-4>GpJCaqmv1iTjg*)^up`?~|DZ zKXsh++>Px;uVcFJh@-V>U+*5RBaZZh;Int~j>&qxoNOYFowa3M=LLIhaqIzl(1dFb z&{~lMtF&VWQ1?~0`n&fPtud>+CkJd}L>=-;7qzVv4ApH_!=|NKWF7v!OZi6hBRx}{ zpnvqK*UsHINL6T2mcsrHZG+$l{Oo*GZOoPDgIuthNtb8m>LS|4zHx9%o7jA{D*Aut zbX#EU(~SCVmF27&C%!3iSy>UZOGF8HoDr`N-)v-Wz;sQ*7$m z&5yo9hb$GX6 z*^uLf2kon_Z+W`D<+U==scAWnjb&BLFJjru8}!aV-Wt6gH1KTlxn#mJyyu>qN`iF^ z%lI|2|0-9MzgLmhN%-=0^`W4wYD0g6H(J89~_+^IjN8msXGO>h;PCLuY&;3h7RDa z&H%oX0o+mcb(DRU!p?d3fUXB%*1hYxXUDYBwIS__SP7YGQQgYO?veS7zLeP_y3`vv|i=_pxksoxONxFW%XUH}`93 z+mc&-v~W^Xz|3cOCKKp0u*GXC2)2TLxq?S4hWG*AuBkSOg!Sm=uf^1%V%lQpNQB z>jN~H(6#b(Spt>o7y%z3G9&P(&~AaxOOU`-SPEBQNuR%V_3ev)d}`D6{%>47{luT> zwe9BCNJy}sCir2BA8vxb{l(bHkGEknH( z=;J|fZ9Lcx6SB6;94&t7<==058zDQL Jok~t5{tM!FK3D(% literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache/plan/delta-join-with-cache.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache/plan/delta-join-with-cache.json new file mode 100644 index 0000000000000..1006117fa53fa --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache/plan/delta-join-with-cache.json @@ -0,0 +1,474 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 37, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 38, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 39, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 40, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 41, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "rightJoinKeys" : [ 2 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 2 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a2 <> b2))], leftToRight=[Binary], rightToLeft=[Binary], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 42, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "conflictStrategy" : { + "behavior" : "DEDUPLICATE" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1], conflictStrategy=[DEDUPLICATE])" + } ], + "edges" : [ { + "source" : 37, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 39, + "target" : 40, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 38, + "target" : 41, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 40, + "target" : 41, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 41, + "target" : 42, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cache/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..71631e59f407df8423e05900029154ff4ea9b8e5 GIT binary patch literal 14193 zcmeHOZ;Ts96(9R9mw&z7<#J6)pw&rSBMPy1ZLjThu88=~4sq(-9cL#ciGn-c9p7zk zz3c9-FTQ+8E^0^-EmWfLfe(}j1wN$`QU&5$Q6(BAzEROn2q6%EgeVFsQpG=oH{)G< zeCrFBz~xWlmAqcRnR)Z(y*F=We{Y<7Za#z%8h{VI0?jY93GJRT4coD>W;j`+VpIc1-Kr|p$t8)^qIc~Gemdr z5mPfxNs-fNP>PgPN(r3-QsI?N-+BLX?Wfm&As(4el0DFn@>!t_Iv8%I6u4H{{l2|D zO72_HdV)Q`8hG^g_m}Xyuiq5Q&)*)3(>e>iDj#NR&7M$=R_&8 zoH!egH0*NC65*P!2}R*d-O!v+N|KUtJQ@{avM42!vM4778!wR>1S2ODjih2JIV2{f zh#Zp>v2rvjkeHH;Nop*l5-f!zrCbS><5DDqV+vGGDbYkFiqK<+K_M)_22s(p=EJy# zl?4*6=$f$@c5qoIVXJO9T9t&IZ~IbOO8D6MQbx46!WOw8CG;tV5Wh-FdU^;f;U;nVDmg`K+tI z^{oNNu{E`Ttxg37a%ey&#=Nueu&yP?({odY({mSY-n_}htv<97>xn-Ut-VGy-Mt-< z$5#_N0^i9srjQq5=~QaQdkCWtdRJ{!x)D40KNL~C7l&>)koV#c%&Ybeebs*0U8mN! zr{&^MZ&N`RsoV9%E^h^WUEbavl!&_#>(4_G#d|N)cl0vd+lvFnawl=8w+3#+8ra*5 z^GIUHUVI}~hSBNt(QIKhJ>|ovj{Dv$ymtx)eT9#knmUcE#0H3z!;i+-pm&UfZjg4r z3b$cztF*~`fbSBY9~$!t_!NMIKOO71&H&6jmzz9R%*{-fj?QGVXt*$&&lO9pf-Gy-9fh`hvwZD+GPW$d z9G+Zu$gHJRHAh<_X-eX-idz->A^?chx&{l7VL32n8;dM;y%|#1Hc(!A?JuwX^0{|L zzG8g&z2AQC=<86QIR|aG%G8h0`XP(hb=|SK@{y1Y7}_7s*Ggw)8xFAytaE1dlRAv)<{;diu=w`_OaJ5e!CPzfjrt#p=Dl??V#a2%~=Q(zY=H)WCF0tCvVB!Yy9>r z?~lF~e4!dd!8{s*Ll8@lVAM=wO2;-Z-q~bK^GpIGP2|ybH@QW{a!~ZP6Jd}9Te8qz zzj$ny1Q|ivRv-X&&YdQ4eE#fp*`htvfdf{Hem4Lqm|8 z1Un8e+3n}ic@((+&}DQtGL?OEGtYt!_pzKC_22N6sq0`eFr__y%&3~GR?%4c75w^N zLr+G2@aL}%Gwt>3K=E|9|GzV6Gu|_aR(|${51wFb0*Zrna~5t_Ynpdgn==dIbbd81 z{`~Q8UdP|+Ly#Wmw;`3?sDWh%&eFwn0LByddpFC8JIu%32}JYZ)F)^b{1BbdZRh9% z$5)B1SX!;=)FYfvH?e5y(*{fms)3KfGi6Pm`x`aL00(ho-LBasZB7{tqVx1?xSdXB zp1PND7lHf#f(Ek=0$vC18zRR)3jF^+S)0zDHghOqySuW;p2q{U@;;6pWzU~v^!+pg z!j(nX^Am`+8Eeq{9!B{{hbP=R$fq5%swrMD-utD(Q~x;k@hDB(QGP*)lRKIFu|z-E ziOdsgXlgd6v*jj|=OP%#vZFYGGtxhUUfHp}1+N2B6#xLbQ{qBX!&bF8H;{(Hggk#c8 z3=W{kSX`E*R7_P9$%>pvN+OmONr_5gA|etwMyoy`0Id>9mXmQQ5yH5V3`wbEG=w8! zJVeBpREg5V-Cc(40|30F5FC?N7TNjx@Koz@)}lwJ9gQ$R7W5YkqAe%U zu&UV#w$v$XCq2T=;fi}PL;yYWe zetz^vK?DfMHjuH3*^zz+#r?n?9EK(WX%Gc)BhwICj1IM$c^HXfQ|UawwXTK7 mT0B7jP{*qOXLNDJRs(8", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 61, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 62, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Calc(select=[a0, a1, CONCAT(a2, '-s') AS new_a2], where=[SEARCH(a0, Sarg[1.0, 2.0])])" + }, { + "id" : 63, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 64, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 65, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 66, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, CONCAT(b2, '-s') AS new_b2], where=[SEARCH(b0, Sarg[2.0, 3.0])])" + }, { + "id" : 67, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 68, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1 ], + "leftUpsertKey" : [ 0, 1 ], + "rightJoinKeys" : [ 1 ], + "rightUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 1 ], + "rightJoinKey" : [ 1 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "rowTypeAfterCalc" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "rowType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "rowTypeAfterCalc" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "rowType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], leftToRight=[Binary], rightToLeft=[Binary], select=[a0, a1, new_a2, b0, b1, new_b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 69, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Calc(select=[a1, a0, new_a2, b0, new_b2, b1])" + }, { + "id" : 70, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, new_a2, b0, new_b2, b1])" + } ], + "edges" : [ { + "source" : 60, + "target" : 61, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 61, + "target" : 62, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 62, + "target" : 63, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 64, + "target" : 65, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 65, + "target" : 66, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 66, + "target" : 67, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 63, + "target" : 68, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 67, + "target" : 68, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 68, + "target" : 69, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 69, + "target" : 70, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..e8da7e0b0108eed693c60e0d78619c62cfcdcd29 GIT binary patch literal 14544 zcmeHOeQevt6+g+2^HtX$andGRIu~=&wFxCrrf4Y+u*kBqxKV6FlAXpzs-4J_&6pxJ zlJeJp%)yGaK!+9Gnqot-HN*1PkOEta4(lI7fdR`fWWcZw6lnk2ied|f4ei#fLDy{A zyAx?qmh9R|oF%9zNTlv~zkB!Y-Q(Tw9S6QR!6Ag2;7bor>4dhRj($Tgj1**Skv@{s z4KgnBC((A1&&ZI$66WZ^jKiO^`E*1>Kgb&YQTfOnNw79z(0)*e@<6r zVwnD5$ZrI)Ms`M?RN91|Q9|`7iaxN@zJ%SM71ToUhVLm8wI2-PkdPq-q zFf1dqoj`A}2ZR@;bz{PZ3s{;YzMQIPQ$7o4RpK*>nx*84&zdWcqNS*&FR9P;V+*H< zp@0jYCq`FQN=m~8b5geez5%|T3x7u&Oatv-rVfg2`c!>Fk+A9;(5y5h*oNZqA~b8D zz^aNSN)VOBy<=#LVh$Enl{P4WHqq*mx^7uWOaQFE)M`I#wV$=x&m{8O68T-Qv|R6O z0i8CGuvU?1S-VxJowe%TQ+OKtRIE+-;#x89&Ma7H)gt>2R@I6N ziciw>d0lgLQWl$pYSPt975^%F$bRQRNkum;lU`F$w-(lO05{kJ*k9Gil9OP=WpM!a>qQu7>G~Lk zG2GNYG}4!d*Bu}=hZ}W(w)R%{3>d{mY!Vxl=I0qeO)AkQtjRm?S>m}Z&yvw-GIk`I z{L1y~*Ij z_u_RgUSlR!_u@;Y!1hFNP1D%A7cZ@EZbp@LFAmGix)-O5>H1uJt<%Z67rzP}YP4!);jAeb_2nDY6?qWni3t>`i$CgGCcR=+ z;mnrMrWjVGaZx;pHi}{f$WH0(55~gJlV?u_F1-+b=yLcnBey;#_|=>OdiFr4sZFuC z+D3@0nUGQ({a5m5zk2pS^6VqezmH#dsTP!{#Gc(l@8=9+7FEln(rutW2(=%y&)>Gs zS4!t?!}j^Y|AEO4@!g?Zj+3z*-~@uRTsTaETsV@AbmK@s2$K*(9h*QvV6r`EY12xq zY3I;zJQ+<7C2h*GAIQq&K;KAuU}!LNY^XnuT2sS`fplguBMyMTbJpZCt8HaDt~@?# z5kteOYt3ApRADPN!D+kMeN|TJCfp7yM3qgt#-(h>l^ba+~*#nsSbxX?R} zTI~E`-Bc_P+(O&NT|y6KPZ7xisT-#hO}0>H4RL9`eh70Mg*)KRdkZ#{Q|V4ur3n~Y zL_UJeNm~7EVby^Tz+b{zB~TNS-TV?djXZZAb#+_9APvc(x=2C3S9>-DQds3+ zm~9g@#CDjDXxs21D%JpoFB4NTltRg<+g&zYS|U-K)?iRj4SWi&DKx!qoGkz~?Zy16 zO|wQ!jsi8K)AZC@j#V*B-Nn!YfPOQJl*$V1$-4_b2>7>#5@t1d7MtQ)WS6VU2H15M zqBrk#r3cydBOaQ*pVF(YJf2-Yz{+&a(mQstl=~|zVPz*Dwe-9qInH?3Q>lmlJ^$Gt zjfzq{FZjteW_~P+2>nDRh|&J`x${@gJQ)V-9Y7mkr%jQ7^Jp#KH(1PPiSe}yXLoOX z?vGDXu!37ub{z(2SCJCeyYR}-AANZ0qeejPcaW(Zy{w2^J4=d}Eb`oY=fm^8uk@8` zT&lP*MLSfDMw9zla?KJ4S(c+|m=L%^rt29QYqB5I_0aYWg|=<&-He-9trKvqealYt zuH73@$z|LcH+aWs^0r6D>gEV*QaT5{IclwLj^*XgS~a=RHT$SPTW-V_YiyfpLhC^$ zRnIGKRp@-ZskOUT^7ZIPIvse-o)awDasJ-t?}3c4lZnuwX1G2AUuS2!N^%xgnBqB} zV@#T%OiWfJ05;5a+2NEa{aXWeaCvMSdQl$x|DjdN9b8H7+Sqrz+>Y4vY%{JA%D6@t zrO8eEeFYorlO|)MY;JZ|bG>IMCQhE{IV>r-Bdpl(Sg$wU9Y!-oc zW$7PAkG|8Ezj*)10VQ$px9QzbhCzBzXD%Y}LY4?TFUk^Z;|+znu^17gZb^m>y*ypd z)1Go$hI4ImwQZ%klLW)A;S}A5$wa#CL~Kt83q5>J;&>uuIU&bOT$m4IP6`Ba65#{g zksjKbzrvRhu{GtUZ3C^`Qa0A8B2t!j!q&3>!)w*uZJR7#kNqyHaKaQ_H%+ z_0!=P33CC=N(5>pf^8*RAHNiP=X0??Kd4^&^+)@TQOe!1l0Y^DN)@_^5Ddew801AM zCrMH+8_WuF00{28#YzGpBGCQiTu931I3Xy?Tv*5vP9y;_k|PqqJ>AfnPp*~N4k<_- zQjj^M*L-hZn}2HNC))tmCJQL)`*0<}rKYz(S#F>L2!M6F-?H5-`uY=3fLlyjaX( zSuIqPf$e};fmQB=DC4aybfmhfTCH}=-30-RDkRVa>>ur}QUaCRw5#R9Y!YPQBL6J`811GAhONe#yb`~L@f CEjH`` literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source-and-filter-pushed-down/plan/delta-join-with-calc-on-source-and-filter-pushed-down.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source-and-filter-pushed-down/plan/delta-join-with-calc-on-source-and-filter-pushed-down.json new file mode 100644 index 0000000000000..dc24c2624bb16 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source-and-filter-pushed-down/plan/delta-join-with-calc-on-source-and-filter-pushed-down.json @@ -0,0 +1,935 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 28, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 29, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Calc(select=[a0, a1, a2, (a1 + 1) AS new_a1], where=[SEARCH(a1, Sarg[1, 5])])" + }, { + "id" : 30, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 31, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 32, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, (b1 + 1) AS new_b1, b2], where=[SEARCH(b0, Sarg[3.0, 5.0])])" + }, { + "id" : 33, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 34, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1, 0 ], + "rightJoinKeys" : [ 1, 0 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 1, 0 ], + "rightJoinKey" : [ 1, 0 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "rowTypeAfterCalc" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "rowType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "rowTypeAfterCalc" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "rowType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT, `b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], leftToRight=[Binary], rightToLeft=[Binary], select=[a0, a1, a2, new_a1, b0, b1, new_b1, b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 35, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Calc(select=[new_a1, a0, a2, b0, b2, new_b1])" + }, { + "id" : 36, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "conflictStrategy" : { + "behavior" : "DEDUPLICATE" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[new_a1, a0, a2, b0, b2, new_b1], conflictStrategy=[DEDUPLICATE])" + } ], + "edges" : [ { + "source" : 28, + "target" : 29, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 29, + "target" : 30, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 31, + "target" : 32, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 32, + "target" : 33, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 30, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 33, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 34, + "target" : 35, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 35, + "target" : 36, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source-and-filter-pushed-down/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source-and-filter-pushed-down/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..6b80a55d0e78c44aa326fc1e3a69a3844d35f7a0 GIT binary patch literal 15826 zcmeGjZEPGzb?)pqPFsJ(u?k7k$f$0hCHJv+dtY(e_!v)9?% zz3%QgwuOSDs-!IvG|&p=2Q(6q%14lps!H$!2~~&>MamDTK>PuO5JjkgbyU+j*a=bs7kAyLwkL5!`fFltp zFA9FbNAg%AQ1vbcDvHrY>uP5oJIlD!1ZwLU6PvURt@Wk6tdN7gd93^LngWG}FA(zS zenA&bOAB0R$JekWR(_`_}*okBsa@|PR^cgQMVsVyu^NOsVf)Cj%YK4NPdd?s zYalQBbTE7peIyvpi7Q2(dB7uKQ`0+l4jt|L#+m2;_CjW<`%lk41<)vamf^<|^u7A& zOkf41FLn}b55#gJe7yX##6PZdpPk!%{?G-F{L3P=xG9vb3~=e-fT5mZ1a1cc4ahma zYsZVD>*lErBHXCmf1yy ziF#9Wcpm%cWbvhB!yH=B(fw;ew;TWXi|_V-&HYTljoc~J1EM|;Ymuy}2?ZMfcxM@~ zg4G!x%Nk0dI~8q42Jxk2nJAC~JihW2_OU7KWBl^5UihrKzEq{n&_Z!uM|*0?W8F?` zr>RWB)8q=*e(v?RztGFL*RDX0&AS6P2T%dLdju_CdGh)`2F4|t=x)adSE`kV zcUJ{6Q#W*eEu8)Ny%&ChzuJaeW2oH#-2?1`HR?2--W|}N(7O0$$;2SM57QZK@dkvW z4Fq2zhN#QMvQYOqWIAK&vN}U$T7^kLIdBi8>4?*G`$7@;>c-r1H2ln6|--b%gkQb8d4=Pti|@0~CG?(-B_(Mc*xM**~>NqLZ#Uwh-R2TuL7 z9Uza|kSQHQtcjC5tD2W3^5Q#}qDw=sjZ`qMW?Ym)8?J>$vqxEW-4+kABAe3)^*{ar z@VKn}vkio>R$ap^ql|WF>zbS@WeqdgQq=2P;pNgKv`@{Gfu7{!0O+9!Pln{jyVo)Buie9?9)JKjza{jFQ%}1`S2TIo{QB3?8 zxO}IB{7KXm$Ws&VY+-%Y}-SrQLSD1@n?8y_3czB-2o>)u_di*>u1o&_$B5)yu_WlYw&Q@bL zs?}?a^k&Vzo_g4-)o7K*gp;V&PMBU^_LmKh z95p<0T9N4VfHa4VMODn5#^t?&B1IJO4J3-e5oc(i1P8X?Muv-iV9g1xEt+X10ZegWk7fb z5$$NErj23h1Q)=|zjEwY&e4ge>5>=r08pG%NP$#sXBy_)(he zwQ2^|@`%@LD0LItZ+Dm{zd} zS|^S=)OQZsU}+=(eRNNdT`_SIiXbEfBfsO z&pq?>4?pweJ@?T#J$A(eLh*Rs&*deNCt*UQSTqt1MmW(Q5o19P2ZBLbb&CWjib4bn zBp`Y?Q3!eX7{_^p7!G@gNHF&EVJXIX^M4zlXsNMi%|VepWg&o4JjDV)y@{3Nq)w0H zOqsASKad8u8+D#Qy^?H*SeFu56$u2}NGDL2m30_HTq=C`*xzuM$e4BJWJ-cjD$pDo zOr~TMi7L^o4i-8Ii5(?XR4|0gNf1v}p^fUIx*=m$H++?{i5^Ir~ALh;%e>y8Vi_3yqGAI0$|nAYjD6ud}M)$aQ3IEz7;@+SDM+8 zHzukZmD3OLV`gO#;w+ZQL`P~-)$6sB?m-A@tAPZ(fJ^+r8YWP>y(@Hk>(I?u2!HE| z%i58Mk57z`CE_Uv0<=bWUru Dd>Zk5 literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source/plan/delta-join-with-calc-on-source.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source/plan/delta-join-with-calc-on-source.json new file mode 100644 index 0000000000000..2df864078e935 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source/plan/delta-join-with-calc-on-source.json @@ -0,0 +1,935 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 19, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 20, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Calc(select=[a0, a1, a2, (a1 + 1) AS new_a1], where=[SEARCH(a1, Sarg[1, 5])])" + }, { + "id" : 21, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 22, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 23, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, (b1 + 1) AS new_b1, b2], where=[SEARCH(b0, Sarg[3.0, 5.0])])" + }, { + "id" : 24, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 25, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1, 0 ], + "rightJoinKeys" : [ 1, 0 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 1, 0 ], + "rightJoinKey" : [ 1, 0 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "rowTypeAfterCalc" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "rowType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filter" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "rowTypeAfterCalc" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "rowType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT, `b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], leftToRight=[Binary], rightToLeft=[Binary], select=[a0, a1, a2, new_a1, b0, b1, new_b1, b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 26, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Calc(select=[new_a1, a0, a2, b0, b2, new_b1])" + }, { + "id" : 27, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "conflictStrategy" : { + "behavior" : "DEDUPLICATE" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[new_a1, a0, a2, b0, b2, new_b1], conflictStrategy=[DEDUPLICATE])" + } ], + "edges" : [ { + "source" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 21, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 22, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 23, + "target" : 24, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 21, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 24, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 25, + "target" : 26, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 26, + "target" : 27, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-calc-on-source/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..9028b4654ac7a08cff7e2ee7901fbb9580cfe5d4 GIT binary patch literal 15685 zcmeGje{369`PoU+bX}UX4P$M$Dq6FuWA%?a`~0Jsv^tK{dUKP^cC)QfG0%6;$u;LY zb9YXhB9&AyhK9r>V1Mk7u?iJcNS%gMikRRZkQj)e3JoOq2WdheU>eW~q)u#8_}=qf ze2$$qY3pdEKFPj!@5lGP_r34F?|nZm`MGHiLZ}OV^m0{is2g?1u_8|K{*!1k@6Ufwc)L~A^!pej>% zC-dCwi{JR;F?Updw3LQ{6wV?APzP``t$}Ni{kQ+x6xdJ-?76(5)3qmCl>#pG&4R3u zL%unz`wE&O5#8|df=~D7_58e4;e^xvAh!^j7r2U%FX?``rc1u8?^IcpO%KOM0~{9) zgn2F!76N=oh;lvy&yf;HBhym|^5KB!35ET9z7PoGC?E0p13`g{;D9F_Bn3}Uf)dV4 zf=7r7evuF{6c7psJunI=Ga2eeSxukz;u02Th_|4~>a5qqd4+iOvTDjj;x!jaq-@HH z;Z1AvBiO_lqRUv3PZNExp(dl^k};#10KNl$?u-9Gn+zQtTxAXl+wdydvxe==9$iJqcsVa#2;qDXY9@!WxD+*mu3U$-z(wb(PD3k)QzUr!dY}G!tY9GVM zXJO=XfTiPxDkaw$9R=5m5*=%;3hiL4+I9-hVXuPKX>UR;7oCkI6Kzh8WfMmdX^VQk zHO)7om0^|j6u4Un*flbKd^nXr{ZD@kq@}vqjm8?Nh)c3p)QUw-buf7z8-#MAgUjol z>Nvdd-B}%Gb$=XSS60+PcA@>%lShu8JaSrz=-AZs95xnIF*lE8GiR9CBsq(9T&VBK z#IuQ!TqYaKCUV(lClaVLmCR(>+WRJBspE-UCXr6YQppnudUSE4*g7bZk_`t*>QL?Z zYbStnIf(5-@ur{|)6+MsDo<<)lP#ANQX@PH<6% zx6YosTB8$D(~}at(tCzcq>%X z+O+c|*5fl+uh&hkB&#i3T0D z+UXmV5Gbw{2k>Th)nlxawwDsx=a}h|xuA4CV+6MEj7|iWc zNzEOy-qZ1B$EoBF)~l`1d#%>hZN1vot4+LtZM}N?^lDo(+#b!)wpZKs>J6er%}mq5 zW|b|sS6guw=pTz6OJpWu@dmH@WP?Rt^EHFR-aDzu(ikoh143>NulL|8^>x&t=EUAB z@OM8mEZ}dGU10bA!}K;y10Dxxsdn`rwtFfb!NVc+oHK>u0P3xEyyG#qrs4f#ZV|B3efdltGhj)oo(T&Mjp0TCr=X$0oDu+i?EIxAAJ=C zG02NT(1ST1dxAj;N`XMYliy07oTyB9wA;PT~v`JpRg3_8T#=H=m5 zVW-V(3MnvN;L#j@uR;t17%64due}f*8jh)UJht%yDjMYp2Xe?G4UG>$ z4C~?RLbnIM`QpF(zwQ1~(T&_G)GKQ09OMMani^NI0f4tx0n^M7AFCQlp{-Ur3PsCA z{JK+DAZ=w%N4p!*W6Opd9?Fz(b+{pA0BvP!9bTYk6chD27J3YqhNe)j)jpvapc^#R zL_4P(NRQ`F5zz#!J7;B8GSNU2bm_c4nDCGQEbf6=_{o* zXA=Q@O0eqE5M&d*KMge8&vG#+|9az^qQI8HRvLrH z)uJZJ1(_vG0bhUA`!IL;y|4E%>UAnmW7F>7jRDjE?;1vnZ+-UKUIxY`n&>{q4%fO> zw|8d^GCOzB_p5UD?FV1}1%9OiLD;y{faJ4E33j$?o*n?ZU_C-3ce5Z@(NKs1Am z@&wIW8>Me_jX64i@g-u2x?HL<^$rJ5SFIMx(<;0almdI8OhMCfXQc$u4>#u4%$iLS z-6)_Q*BN^2t0jOjPTkF*`vLl`B!aOCkUfFrQ*yxP0sigrj;vUU!=GH6F7IdMK}0+6 zchvK&{23RmKS(3vjy8^!_p-Ku3hl9r)lg%o#tGLK@-b5@%AzfdcYihW$bV1o<7uiP zh35u2xt*~eOU;9QBGW|Q@uzd=ubq890^FNKo93`mCLlceYW>E_#XQl!a`B}Hw|?uL zuTfwnC#tL*0nm;i)xhE6rJp_V$n1xm0C~iQOz{|EL!8uEF}%u==dPZQoF2L~T$^zv z;UW~;;YMh*dW2Os9q|}zvMG%mae&q)HFdT9N>o0rJ7)8LoG-c|O0nL>etX2=TE(fu zxm{C7cW&ykL)=?0^Q~yDPaF~3?09Eo(l@eWJGJ7bQY##1fUY|%m%i0;`_^R~wi@$R zL`J9*i&&a$`8&<(bN^vG_+UTlD}oXBIO~329E?&^$FgcB)Pe>92Az4~J`G+taGh99 z9ypTA68${l2mhSqewJ0)m~qx*%6X$) zq%}5}OxY+ARiag0Z1qssQdUI;!-rWCI8JJ~(PLv%-;lKoU#+Xp_K?vrU~)5ByAara z+**i`vA;i~!_Phc+wSwv^#90>z<|C7a4cf>4VQ_8M!yeyV?lhRLPT&oXsT}+(CG3o z+r&0eQ}eqY97Ser6JpG1VWOjrQ#D()6YoLrUNt}hT7U;8&_D!=w|5zDZxg&Nc3soQ zU0dR@vGK8FJeC4)pS8mS%P;`}AjgUi@|v*X@&LLEO;_W@iJE&`z^#xuo|#CDjr", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 53, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 54, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 55, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 56, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 57, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 58, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "leftUpsertKey" : [ 0, 1 ], + "rightJoinKeys" : [ 2 ], + "rightUpsertKey" : [ 0, 2 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 2 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], leftToRight=[Binary], rightToLeft=[Binary], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 59, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 52, + "target" : 53, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 53, + "target" : 54, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 55, + "target" : 56, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 56, + "target" : 57, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 54, + "target" : 58, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 57, + "target" : 58, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 58, + "target" : 59, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..4665b6f93d7c158f893184f85149a4dce2487949 GIT binary patch literal 14838 zcmeGjZHya7b?m!dzIwUKhn3ot>Zh{zBJ7#G2`8_Z*%Ki zv%9|BMIyZjiB_sqkN^QfKq%qgAyF07N(iYS5ro=5Kr0XusOk?Ym4G6MREe|-Z^pal zwY|RjNG>$QEA7tio0&In-n^N4^WMJX=Smzxs1I&>xoQd8goYc(g<@EyRV`GmUiNV>EzfC0ugZo_^S9Cm zLPL+5={L>vS}olZj4x_v|C-pX%9p?Vui@{y zzgTu7cM1&%stz?jmyy5fUTp)aAkZJ4Vd}U zsxBEoxPf-g+K8SioFRe%RQJwHifEt_2Xbkn^qkC+Pra&Nwcoge% zwE4vf4CBGpo-l!_4Wr2b58$T^t91>z*f{nv+V#()^T-7w=IaQNjgxg`?vrS%q^GO0 zOq))j&9w21su~6gJM@?qYCk}<9&`d8z!1w1^!3Q_s>#8p{_A%mgRM`x6>DoRC(bqeJAgMt12pI&)kka4e928hMGgCC8c zj(E=mT7LcX`wuWQF2O+e*+#fttTwzm>zHGjhL`C4TDu{!R~aB~h;qQyc;+#MKh@wjiQMbDWZOKo+g6^+LTP^5}!?En56Dy*nNQA+fgSc)hbBa z+70wgPR&fD5;pt+z~9+?IF7aW9M&2efMMyAiBdA=YF%-ycI9|16F(ZuJagm5je2th zAA27~w8IB%@Bn-NjElk#Hr4U7ya!mF5!TEe2HD*p32QI%F+(j&g0&d$eJ=a(zt4Rt z;D(7qTXe%ZxuZUPV4O&aXhZK@y!8HsXColJNp#Nwma7D&*_y}?Mo|P-?!~+ zZ$D3wRqUj)_Xwc2RjCcq^2@(`;^F!0y?}Yr!c6HHXI1RnX{uh6$ctAmMb3@CJW+44 znQ;+{?eSJ@6gw{C$nY}PQ7&5gRV z72aVQbt=REUu(LxT$tRunPuYcWT3emn*6&p%izb3ldjvLUTm^0_g!%`mG<@OXkBrn zCj^hB$=ep|^>Wfd9J_nVy3PyM-eS`M`cS$>2f!7{BdF!Fs@Tr!1*{Vf?9w(3py8`w z_ixoJ+T*NgPd2p8fCkJbTc~YkV5q58O-hRsumS&_OZi5OBRx}{pbvUkCbinv)x*?; z7O8#E=i}SpaTIQ=oJ@)aSDE*7ewvMtCY9zI03GALdUy-e?*w;%wZtALK$B*dMNjQ( z9dw(>oapWh_ZoCJ;kMIllD*1D^ZW4*`5jn4zflwJwsg1P{%E>y4Zm+cPnzj&%M(9$ z(_`5d_geh6SVGhONg^Bg2{0AU@{v=hpO5543gONGXy~PZ!+#5Xe`R0#&&gl>VE<(( zPp8B(+>J7}i9+C7X3y@U%YiNj?n(~aeqUC!7!adDA;d-aBIZIxUzCdo(IOZ0^S%I2 zgit{AfiH`C0GRvHjan78c8Xw&p^BP}8}-5l6%ba%uIY+egx7sQGcDb=Hd~iD@4PFs^zi1qzL)5STgdu0nX^WIeT4b@X5r}iAmGNnR|LV0p2libV9xx zIeRi=EeDPEM;PkWs2^Y&gvoSTHn1e_h?<$84PblCKWdp^bJdDW) zZdMa!`M@c(g%9L)2LDH7_QSz%zxu;JWxjI$owwihKt1MJf~W#Qf)Tz@6#c$vp%B5m zi1|R&HwGaK#7D*g)R23xS=?&P^3$3j8#`)kX9$K+iYztfv4XeXHjQzJ7b1ZOL|>31 z7m9}c9EJc3&R;AD-f+EZ;4t}XQIecQRGcgES>t*dylQW_|oCW zuJ8QID}VgCg%fxl9^*xUCpai#0S5~HU@<5bArVPoOpFxhkK=cWCX zN`%OqI&IW8>==AxcEejQ>a}J*twZBl=;cBXVBqhB30W(GjaI+-?C&>UIx+kzi_q8# zG?p=ooiGsJ8fF7N2jV4*L;!b)s(4os9a*hr#c&wNF}`CMB9V;xAjDWyjDe1}Hs#c7 zJKduY9@T;gcme<523wdw>GrJB?Qx)+u@D|Z9YpwSiO14Y>0~^Xf^af(ghy6!0s+8| grhq#KQCzJ+N5Qc-cA98B!4`;H$", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 8, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 9, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 10, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 11, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0, 1 ], + "rightJoinKeys" : [ 2, 0 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0, 1 ], + "rightJoinKey" : [ 2, 0 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], leftToRight=[Binary], rightToLeft=[Binary], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 12, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "conflictStrategy" : { + "behavior" : "DEDUPLICATE" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1], conflictStrategy=[DEDUPLICATE])" + } ], + "edges" : [ { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-contains-index/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-contains-index/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..a21d39190bfd78cf212de27c0529982b7ed16f94 GIT binary patch literal 14188 zcmeGjZHya7b?m#`<)fEel2afltxn<^QONFk*I(;g5b@a#aqHZjY$qjm3hsD!e7DJZ z*X*t@zWhipAP5qoAOS*1l;%hHQw0PnNJK)S;zujgA5`=Ye^3j5kPwwBid0mJ!kh8# zdHr$0M|!8G@k;jYzS;NY&6_uGX5O2ZeR_#Q2#vsxUV-Kf?LcD%&A4=Y(l#zFiqZu% zDoQ0AAU}T2{@sOFzklcT=NHNEUwZ40F3uXo^N<6>?ByZ)J!cWSt~)j@57Iw`4xDiB zjR5*iMbpV~VFg=4#ne?|*@6@mtWeo1UsfAp^hziquEs7$#fDw3Ss}QVYC=)CST{6> z6UBHm9E&6pk|fD;JS-+eS+Ma6shOJLa1l|KqH#_x#}mqh%a;npWHD1pXA8xA zcIIp`J2zW8GndYw@xpvATWl3%7~c4~WbSOHRLJDB$y|0JlLu%JoinW^9@nt4O!$fp zGUFXw)(LOb4M(dI-dU}Yx})hfpJ_P6GO%7CmWFlh3bCfTK?MWX>}Asd+CymUudFX_ zy@htz7W&KvHPISu{**>^)lOpv7vcS%KfCpbpIm#s6hwg>+FPMzUXksf@!Uncf(6Q! zkkf2ux?!Qm?n>Q5e)(JPO#CSLm1+BvUPD1N47IVg z&aBSjD%7^m*U&S#Hob`Ux#jbwtvSHBgAOkGn4T+NB#HxE4`0#@)j^Xz*roLfQ16rQ z1$?=;=92O%J?yJA2MR&d)7W07)i-L;jmNrM!W5?2jjjNApic$XY#Iu%ejH)E+h0Og zQ2=@@)E6P2BwNIS3uw1y&(?LFR-HpTY2|s-bQ~1x(PH}1#vMfQplRr;g(90}iQ>9! zp#um(tM*dW?XnYs>L23=`Soi(bas#P@BDA4Qo z_dO>5^6$@%GwBWM0P$pZ?0*wz5k5SF)?WSMyN@z50mVUwd_COytl7N>Ta=TQg;(kL zYFzunBR_ZrKR1K|Su||J& z$yx4f%DH4dbvl`U=FXivt?J4l_B@4XgOB>)5_|r1fWnV;;E1g3QC2p|syWOcA8*ry zyBqnWV^%fAGsZ{0UU>YyE1#5LW@hw)U?=yth7a@;St8bfzudU_?zQVOXm1wnSiyRo zz%UzceK%XLmWlPXTi<(R&v)Pa1|?SWv&x=jAnikGcGB95zkTBIOaC4Q%4v@>ZTU1q z@k^%zy{VBm-o7bcnSODmRb!{%WJ>L1H#G`QGjLCg6RgOSG(r+ocxO$C}H<> z4Wo<-+BaKwLE0v37|HHL-JgZ;F^u}v;cc_&f$gyAp1E+PcQ0n*Lnpv(nG{`f=*sVd zF@qmENp6oS`po@69o`vq@q@vQ+y&}`Ql z0M|5LF{@S6@a@;j*d{#8(!Lcy+gsc0-@UHbm}YbJHwH+*&bxR?~CoOGh z*(q4cV&VeYnJC!|{P68#UmhLV{pagzZ~QcWE&B>!(&g@>X)+wblxV;mMkYKRL*C6S9% z#00qeDxC9wS#9ww)M*Xsf&YfOMK`@maoxQi#SQbecpcmg zvu}?QywCP10d%yv@+i@jN6D-a3p=J&!uG16lrCe(8oYv zOHJ$0z_bobtIwYZvC{1{t*-VBT;r`{6`P*reZEMa*%-RdW~#0Bracf={38{j8`z-_ z>IBfXNOBMpp@-@CdxJ2k=CY#VG>Uw`M4lm!Bu=DwoD<6x znTtr}7{nzY<%GylQC6gIG@7VDa~{~pvpWaMT`o&cf=*#bFoO4e-n(XK_Oc&RsF&?}l|no+g}PlM zhDt#rY=pqWuDYS<7}9Z6$k%A`MqP3}Q&QY!2(3>AZ#+^Qn(aWZ0D@!ZVn0GCczI0} z{l@jzci#NM#IM-7{oTN06|)od4vKasUx2HZ2&6$2@Isr0uukaII+}N~-$6Z3{ZBxC ziqq(CO}~dbQfm_Y@?}bWs9Xz#-vqmk=o5{5s!!55AcQ;jf>y!>&|vc5XJC uO6DNR$nD|Db(+8ch@&G7tA{GCBhX&x*&07j^pyE7$Yv;", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 2, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 3, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 4, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 5, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "rightJoinKeys" : [ 2 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 2 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], leftToRight=[Binary], rightToLeft=[Binary], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 6, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "conflictStrategy" : { + "behavior" : "DEDUPLICATE" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1], conflictStrategy=[DEDUPLICATE])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 3, + "target" : 4, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 5, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 4, + "target" : 5, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 5, + "target" : 6, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-equals-index/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-join-key-equals-index/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..1b0472a2713e5da52328dd2cc3f59fadc27004ce GIT binary patch literal 13772 zcmeGjZHya7b?m!bk|yb0l1oHFTAkn;RLFY0>#udLNb&ic#;tRAvYmwH8qDnO*tf}g z*X*t@zDOh&s4tHiQ-F~M6=*^)1+Eg^9!8W)z67ZXCumK#|MA`kLFc@3e7x%pf%Q=ThUO65#BSDDF|%EkQj zsd9d9wsLZACWppK^M!o5Q;=ir#!hDnr*f52u9(jh@@I2JfQHd&)2c;q11pOpQq@6b z5eLgUiC9g;(ds1PEH_Bg(R4eKYdOR+uwEjT1|xfkSd&3e$-oVJ(R2X!5Zd)SHWoMj zh_=`k`dFWyXb*PexJGoBiAVKfZAFOf{*Aq`8-h^pnCX%W~dID=s_;+mxq3zh8j@w z;f9OK>-2!H(*h_2QBPodk#^r|z%U*Oa>5j*J&djZB7mO?tj!twA-IXmr)2t zEV?d4K1p_vh0mggG<&wG>$K|v+D1Dsnx^BRWDkyMrTzm%+W~9nvW4PzN)m1Bl7;pl z1YA8#RWm|0D3UXpb$?;QK|{s4Ga$pmtP<7VNY>PK7%~`2d-9Y~H&v~wF{VJT|26uU z@cJLVIL4$mtOLZ8-I4!|po93pG+O!DC*OUPp$RDt+V30TPPO*%?(JaCSQcKU^Q(3B z=Z}2-C-|E~D3nLTHcWjG7ucwabb5~fKZ$k8&DDuXxF4r8+La9`#}FvKN^Hf_8f~TC z>a+oqf=b|n@JvUXJ`cAVAlEP!)~%X(!|7H{)S`%Xg)YRQ@nEP)~!x=a+Ez!BFgY#A6#V59}Q9Xku_}uR`w_>n_%4>V2}@VX~OM| ze8w^Bn&KJbgP$us{*Ox^7GY&(_`+Z(cXy@_j1#F5YtP%)uD^TrMG3SwkG6nM(Iha< z#ya(8n{}C3FWmUrBRjtOo6l2Z4L_;uSpw9)Chb95dG$w6J$~Vz!+?3l!%R7!Vom(g zS<}3&k=OooUAi>&>U5{awStorB+U8{f4ujr4n?LGO>t_5^qJ{zF6Ux1LgF0S znW(7rojVVJ$fBpDd2hM!sz|AM2Ss;kVP;GS)uDJKXZ6JYx1Rg*MY<{FZ z8Ha$qAf-4lCC0comSdbGiD@obB`Q&*XpF?G2<@ZsO}04_M!{vM-!d2Y45tCH?p3dA zfdlh6_%NsOt~+)=-8i1lc?xZa^Atjd+v{6}u5T5yK`grMRl)YMp;Rtn&8gT9c1XpY zyCF38bncnljJttTer7%gv47acf$}iwy-$+~xKaHbG#az*xMy9;e$cd4wT}`-m7uwLlx&u?Vh1=y!E5stvJ-Vp?hh978D1 z>7-7&eB3_p9#RPV2ncQH1Rfep;JcZ?@5fNaro@8=p^S|PW%$g-(1xrhL0_W3tzY}? zJ-HUpW%xGwWBh@z;_s^n|G*C2s|12gk>ntxN^@}{cN3HjeiaP$aQ^NWi@Om)#0SEP z4|XE{ubqg2FFx2-bhifmS&lR|0|{iCz6K6_aes0CNX3(2|KJ@aN z{~7)D(px|H^1`n^^c`r$_LU%GT8&jjLBX;D@!M!st)?K~S|({BA*-?|ViBtDbgO%P z(WdkYwbRLWw|g!U3~Aye8q8~#Zh8*6L=-WWil@0~Jf%VoIZknM3d^qbz-D| z%6c{nPD3ac*g=j+cJ*=IqLF<^BkVj0$S)j5+s>jfRkIarsab3&1d_ScvuKwKI*#Fh z3A~Sb@0y|6i+)I{Dcj9Dg?MO6O}jx1m4Zgt$r^ar-ZT^)!*L81^7T8k(U@w_BMB}W ze5We!%^BK;X1miXgdpu$SdS1&-nkaq{Mw5*w_X3l_!}&5`yrsQj#);vgA&2ydx+H& zp0tPp+X$w?uOd3W+RV%Bc2Li(?{PSv;B*Ed!*b;vbRyVQFRuM`Pr{*z049(HB#FiX zLZEaPGX*$g;*RjdDo!8(#IfcWP7hUFZ9qq%CpUhY SXqNGI$Yw8{D$VC+XZ{P}g|p`X literal 0 HcmV?d00001 diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json new file mode 100644 index 0000000000000..0ad2e61e1a3ab --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json @@ -0,0 +1,474 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 13, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 14, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 15, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 16, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 17, + "type" : "stream-exec-delta-join_2", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "rightJoinKeys" : [ 2 ], + "lookupRightTableJoinSpec" : null, + "lookupLeftTableJoinSpec" : null, + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "leftAllBinaryInputOrdinals" : [ 0 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 1 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + } ], + "deltaJoinTree" : { + "root" : { + "type" : "JoinNode", + "joinType" : "INNER", + "condition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$AND$1", + "operands" : [ { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT" + } ], + "type" : "BOOLEAN" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } ], + "type" : "BOOLEAN" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 2 ], + "left" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 0, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>" + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 1, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>" + } + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a2 <> b2))], leftToRight=[Binary], rightToLeft=[Binary], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 18, + "type" : "stream-exec-sink_2", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "conflictStrategy" : { + "behavior" : "DEDUPLICATE" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1], conflictStrategy=[DEDUPLICATE])" + } ], + "edges" : [ { + "source" : 13, + "target" : 14, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 14, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 16, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 17, + "target" : 18, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-with-non-equiv-condition/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/delta-join-with-with-non-equiv-condition/savepoint/_metadata new file mode 100644 index 0000000000000000000000000000000000000000..d4e6dc30aebc1c306516abe910fcbe4360ee6df8 GIT binary patch literal 13778 zcmeHOZ*1Gf6+g+2lQebWBwo6tYv5sShPI(d>Ms>{SY%0Q)GD$oDao23RgdJ!W=xS9 zNjY)`qz*bP9R_qjv0~_eZNZQ)15%(^Hf+H5X&<%$`?i0di(=S-p#z3(K!A2Z_U=ep zlqF|QT4zly0ye34ckkZ4d-v|$?~#R<<~W4V2>j72i>%Z9{9lUe|PTgkQuKUpI9qwE1X)w*=Xe zFQ{!XaZ!kiOUVlfv2DvuOMq*x$yfRFEkkp-bUZ31Qt`AD7vhpANQq>MxA7urnwsHo zbuos8Iwz`W31c-CtH+`|R->Y%)&)*Y$PyRFN|KXtQsk1!m=I6MGLFSD$n`h)0}X~y zU%BbnKr1t|`EsT@Q?6C2nQFe4D^#lG!t}{%VWw0&F_X)qvC3?*Q0)}tS-Y`Qnc~TO zt&%SnGR4A~d>N==WSQ1n1UIoVPa<_)GZx^77E#Pb!!#n!Qj@eCO}8U5wn+qA7G8=> zFF9n^(i)nhEs_i+RfwfwUAsuE$vYKR4BWKmO$YiNg1_!-uWuzfsk=rJqec?*jN}(L ze{}58+DpqH9DU^U>pOq?7idhiu?*^vm3m5!#*?C)s3!$UsLNn4sYEIr6^NW7NvQUK z7zs(MVqH)YoT>R#s(;}IdiSlyw|}!GdL#YMH-6xo2;7;i&^x=fdiQ1TKDZ4uFwnpz)WCg@0#r#( z$LcAGi>B)k1&DH#lcc1~CFGPKsv;ptoI>bRG)7@@R2Wrzwr2;+I*C{XV3xXm zbgc8&Wj7HFU;oAa1o+I zC=a9l;a}!1K`wAM=!bUFOp}_*CKIL8) zu%|t~$3$Eg%bd-M`^%Xva8tB9f3*#WIXPX-d-_}79&id<*?DYrD=<_*BRVnW zocYbVmz>I!v&S>#mv7&`9q?O2Xe0KMURSjC9_5;Q$01LyCUiNzGixj%-^0?SrP27N_NnVMR{4PrwEGk_mWu0ih_i60=L(F)`}k4)Ic!dX!|hqj7R%?8S? zcU~I_H=h6E=U@K%AAc@u>@qCr5Gnd)>=s_v6w8>LBJn_fGp#@dI&DA`OzkuduMYkG&|pR zh-F|sa02ZmbQ$2PQ1a*#SL|1Kk3Z*Xh-)~?4%v+bgUkjs95$&wgGQY@n z&{&Z|mOMSa@)QA`0;uBgJ976M|LU9n8vkDSl|~qai)fc(8jJ9jfo2+69oxWox67Eq z%seAa710h4+oobUD0A4TS+2MmyM8rc2t6a8CyE174=-qj>Y#}};?jP3==V9O0X26vJyw3F`vaX8 zVGR-WIJW0$_wDAoQxm|SD$E-jq$Xe{hwWri$8hqwK1l>VI3&G?hgNN1|7!xr_u6nU%Yvc zu?Z;-+83DNPBnLW_jWk57fI;)YG3}{V{iT%e|HE0Iyh{@TP5u#^-&jShG7K8ljyZ> zmJ)NAje83y;@QL`+>g-}-7$_KFnpERilsGOquvv+X%~v3Hf_M7pc;4-o+)ekJlt+V z0w|2db(?0NtQloAiZ0RHSSOprEcF26E&%rr0|v1ULVgEcfEeH(1^yomXxrV>)&Pjm z=}s2d^JIutJ`|uw+4GZ(ewYHFfijUjA4IgxM4SHK&nTbnvV=Q3`HW*WG{twu2VSi_ z@&3inMJZ}e`Gvtx?q=r4@_aB)WR6&S-o1M5=H)9ASZ@JsS;Tsafb$sZ)GM_bGO@mP z{f)H56|8PC-s~?eK-=n zpHLRq_aBk$%Hym>U)pmt!Z1N#6Aq(oXV93c*$TGQEH)GZCvx=+8uduWFnpE*zjyh+ zHAA!KgOo~3wp$HK@!3>bc9R$?C5^Dr!^8HLq39Ui5LE$m>BzPwuH+4a?^NY|!>4WF zxXu(3LU48z*Q11r|2Yh8e)Y=lw_W?v_|L)!4&M)fj1A0QCUsDv7c+v_@CZ-ZL;=Xc zH24)p$5xv8$8!huJ-Uwr$mDb;A;SO<2OaP2s^6%Ca!&&C)x!yN0q9MvM+sE!$co&N zK63juwVsUp_H3p!Qz~RLML-I@8J<|-2?BsRR(-hB#}ro@&{^m^nxIVdsJI; Date: Mon, 23 Mar 2026 19:10:55 +0800 Subject: [PATCH 3/3] address comment --- .../planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java | 1 + .../runtime/operators/join/deltajoin/BinaryLookupHandler.java | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java index 463a37596a652..409ff0c0ea1fc 100644 --- a/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java +++ b/flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/nodes/exec/stream/StreamExecDeltaJoin.java @@ -584,6 +584,7 @@ private static LookupHandlerBase generateLookupHandler( node.lookupTableBinaryInputOrdinal); } + // TODO FLINK-39233 Support cascaded delta join in runtime throw new IllegalStateException("Support later"); } diff --git a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java index 3f366dce14ca1..7bb8239cf77ab 100644 --- a/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java +++ b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/operators/join/deltajoin/BinaryLookupHandler.java @@ -78,10 +78,9 @@ public void setNext(@Nullable DeltaJoinHandlerBase next) { public void asyncHandle() throws Exception { Collection allSourceRowData = handlerContext.getSharedMultiInputRowDataBuffer().getData(ownedSourceOrdinals); - Preconditions.checkState(allSourceRowData.size() == 1); - RowData input = allSourceRowData.stream().findFirst().get(); + RowData input = allSourceRowData.iterator().next(); fetcher.asyncInvoke(input, createLookupResultFuture(input)); }