diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java index 38cf2bf787e4b..5962ed629a8e3 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java @@ -23,6 +23,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Map; /** Restore tests for {@link StreamExecDeltaJoin}. */ public class DeltaJoinRestoreTest extends RestoreTestBase { @@ -44,6 +45,30 @@ public List programs() { DeltaJoinTestPrograms.DELTA_JOIN_WITH_CDC_SOURCE_WITHOUT_DELETE, DeltaJoinTestPrograms.DELTA_JOIN_WITH_CALC_ON_CDC_SOURCE_WITHOUT_DELETE, DeltaJoinTestPrograms.DELTA_JOIN_WITH_CACHE_AND_CDC_SOURCE_WITHOUT_DELETE, - DeltaJoinTestPrograms.DELTA_JOIN_WITH_CACHE_AND_CALC_ON_CDC_SOURCE_WITHOUT_DELETE); + DeltaJoinTestPrograms.DELTA_JOIN_WITH_CACHE_AND_CALC_ON_CDC_SOURCE_WITHOUT_DELETE, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_JOIN_KEY_CONTAINS_INDEX, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_NON_EQUIV_CONDITION, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE, + DeltaJoinTestPrograms + .CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE_AND_FILTER_PUSHED_DOWN, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_CACHE, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_CACHE_AND_CALC_ON_SOURCE); + } + + @Override + protected Map> programsToIgnore() { + // Cascaded delta join programs are only supported starting from version 2 + return Map.of( + 1, + List.of( + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_JOIN_KEY_CONTAINS_INDEX, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_NON_EQUIV_CONDITION, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE, + DeltaJoinTestPrograms + .CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE_AND_FILTER_PUSHED_DOWN, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_CACHE, + DeltaJoinTestPrograms.CASCADED_DELTA_JOIN_WITH_CACHE_AND_CALC_ON_SOURCE)); } } 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 04b319c26a96c..2281aa270f7d3 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 @@ -41,12 +41,20 @@ public class DeltaJoinTestPrograms { static final String[] RIGHT_TABLE_BASE_SCHEMA = new String[] {"b0 double", "b2 string", "b1 int"}; + static final String[] SRC_C_TABLE_BASE_SCHEMA = + new String[] {"c1 int", "c0 double", "c2 string"}; + static final String[] SINK_TABLE_BASE_SCHEMA = Stream.concat( Arrays.stream(LEFT_TABLE_BASE_SCHEMA), Arrays.stream(RIGHT_TABLE_BASE_SCHEMA)) .toArray(String[]::new); + static final String[] CASCADED_SINK_TABLE_BASE_SCHEMA = + Stream.of(LEFT_TABLE_BASE_SCHEMA, RIGHT_TABLE_BASE_SCHEMA, SRC_C_TABLE_BASE_SCHEMA) + .flatMap(Arrays::stream) + .toArray(String[]::new); + static final Map TABLE_BASE_OPTIONS = Map.of("async", "true", "sink-insert-only", "false"); @@ -417,6 +425,240 @@ public class DeltaJoinTestPrograms { .sql) .build(); + // --- Cascaded Delta Join Test Programs (3-table join) --- + // All cascaded programs use CDC+PK sources (changelog-mode = I,UA,UB) because PK propagation + // through the inner join is required for the downstream join to be optimized into a delta join. + + public static final TableTestProgram CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX = + TableTestProgram.of( + "cascaded-delta-join-with-join-key-equals-index", + "validates cascaded delta join with join key equals index") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSource( + SourceTestStep.newBuilder("srcA") + .addSchema(addPk2Schema(LEFT_TABLE_BASE_SCHEMA, "a1", "a0")) + .addOptions(TABLE_BASE_OPTIONS) + .addOption("changelog-mode", "I,UA,UB") + .addIndex("a1") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1, 1.0, "a-1"), + Row.ofKind(RowKind.INSERT, 2, 2.0, "a-2"), + Row.ofKind(RowKind.INSERT, 5, 5.0, "a-5-1"), + Row.ofKind(RowKind.UPDATE_BEFORE, 5, 5.0, "a-5-1"), + Row.ofKind(RowKind.UPDATE_AFTER, 5, 5.0, "a-5-2")) + .treatDataBeforeRestoreAsConsumedData() + .producedAfterRestore( + Row.ofKind(RowKind.INSERT, 3, 3.0, "a-3"), + Row.ofKind(RowKind.UPDATE_BEFORE, 2, 2.0, "a-2"), + Row.ofKind(RowKind.UPDATE_AFTER, 2, 2.0, "a-2-u")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("srcB") + .addSchema(addPk2Schema(RIGHT_TABLE_BASE_SCHEMA, "b0", "b1")) + .addOptions(TABLE_BASE_OPTIONS) + .addOption("changelog-mode", "I,UA,UB") + .addIndex("b1") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1.0, "b-1", 1), + Row.ofKind(RowKind.INSERT, 2.0, "b-2", 2), + Row.ofKind(RowKind.INSERT, 5.0, "b-5", 5)) + .treatDataBeforeRestoreAsConsumedData() + .producedAfterRestore(Row.ofKind(RowKind.INSERT, 3.0, "b-3", 3)) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("srcC") + .addSchema(addPk2Schema(SRC_C_TABLE_BASE_SCHEMA, "c1", "c0")) + .addOptions(TABLE_BASE_OPTIONS) + .addOption("changelog-mode", "I,UA,UB") + .addIndex("c1") + .producedBeforeRestore( + Row.ofKind(RowKind.INSERT, 1, 1.0, "c-1"), + Row.ofKind(RowKind.INSERT, 2, 2.0, "c-2"), + Row.ofKind(RowKind.INSERT, 5, 5.0, "c-5")) + .treatDataBeforeRestoreAsConsumedData() + .producedAfterRestore( + Row.ofKind(RowKind.UPDATE_BEFORE, 5, 5.0, "c-5"), + Row.ofKind(RowKind.UPDATE_AFTER, 5, 5.0, "c-5-u"), + Row.ofKind(RowKind.INSERT, 3, 3.0, "c-3")) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("snk") + .addSchema( + addPk2Schema( + CASCADED_SINK_TABLE_BASE_SCHEMA, + "a0", + "b0", + "c0", + "a1", + "b1", + "c1")) + .addOptions(TABLE_BASE_OPTIONS) + .testMaterializedData() + .deduplicatedFieldIndices(new int[] {0, 1, 3, 5, 6, 7}) + .consumedBeforeRestore( + Row.of(1, 1.0, "a-1", 1.0, "b-1", 1, 1, 1.0, "c-1"), + Row.of(2, 2.0, "a-2", 2.0, "b-2", 2, 2, 2.0, "c-2"), + Row.of(5, 5.0, "a-5-2", 5.0, "b-5", 5, 5, 5.0, "c-5")) + .consumedAfterRestore( + Row.of(1, 1.0, "a-1", 1.0, "b-1", 1, 1, 1.0, "c-1"), + Row.of(2, 2.0, "a-2-u", 2.0, "b-2", 2, 2, 2.0, "c-2"), + Row.of(3, 3.0, "a-3", 3.0, "b-3", 3, 3, 3.0, "c-3"), + Row.of(5, 5.0, "a-5-2", 5.0, "b-5", 5, 5, 5.0, "c-5-u")) + .build()) + .runSql( + "insert into snk " + + "select * from srcA join srcB " + + "on a1 = b1 " + + "join srcC on a1 = c1") + .build(); + + public static final TableTestProgram CASCADED_DELTA_JOIN_WITH_JOIN_KEY_CONTAINS_INDEX = + TableTestProgram.of( + "cascaded-delta-join-with-join-key-contains-index", + "validates cascaded delta join with join key contains index") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSources( + CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX + .getSetupSourceTestSteps()) + .setupTableSinks( + CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX.getSetupSinkTestSteps()) + .runSql( + "insert into snk " + + "select * from srcA join srcB " + + "on a1 = b1 and a0 = b0 " + + "join srcC on a1 = c1") + .build(); + + public static final TableTestProgram CASCADED_DELTA_JOIN_WITH_NON_EQUIV_CONDITION = + TableTestProgram.of( + "cascaded-delta-join-with-non-equiv-condition", + "validates cascaded delta join with non equiv condition") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSources( + CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX + .getSetupSourceTestSteps()) + .setupTableSinks( + CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX.getSetupSinkTestSteps()) + .runSql( + "insert into snk " + + "select * from srcA join srcB " + + "on a1 = b1 " + + "join srcC on a1 = c1 and a2 <> c2") + .build(); + + public static final TableTestProgram CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE = + TableTestProgram.of( + "cascaded-delta-join-with-calc-on-source", + "validates cascaded delta join with calc on source") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSources( + CASCADED_DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX + .getSetupSourceTestSteps()) + .setupTableSink( + SinkTestStep.newBuilder("snk") + .addSchema( + addPk2Schema( + CASCADED_SINK_TABLE_BASE_SCHEMA, + "a0", + "b0", + "c0", + "a1", + "b1", + "c1")) + .addOptions(TABLE_BASE_OPTIONS) + .testMaterializedData() + .deduplicatedFieldIndices(new int[] {0, 1, 3, 5, 6, 7}) + .consumedBeforeRestore( + Row.of(2, 2.0, "a-2", 2.0, "b-2", 2, 2, 2.0, "c-2-s"), + Row.of(5, 5.0, "a-5-2", 5.0, "b-5", 5, 5, 5.0, "c-5-s")) + .consumedAfterRestore( + Row.of(2, 2.0, "a-2-u", 2.0, "b-2", 2, 2, 2.0, "c-2-s"), + Row.of( + 5, 5.0, "a-5-2", 5.0, "b-5", 5, 5, 5.0, + "c-5-u-s")) + .build()) + .runSql( + "insert into snk " + + "select a1, a0, a2, b0, b2, b1, c1, c0, new_c2 from " + + "srcA join srcB " + + "on a1 = b1 " + + "join (" + + " select c1, c0, concat(c2, '-s') as new_c2 from srcC " + + " where c1 = 2 or c1 = 5 " + + ") on a1 = c1") + .build(); + + public static final TableTestProgram + CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE_AND_FILTER_PUSHED_DOWN = + TableTestProgram.of( + "cascaded-delta-join-with-calc-on-source-and-filter-pushed-down", + "validates cascaded delta join with calc on source and filter pushed down") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupConfig( + ExecutionConfigOptions.TABLE_EXEC_DELTA_JOIN_CACHE_ENABLED, + true) + .setupTableSources( + CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE + .getSetupSourceTestSteps() + .stream() + .map( + sourceTestStep -> { + if (!sourceTestStep.name.equals("srcC")) { + return sourceTestStep; + } + Map oldOptions = + new HashMap<>( + sourceTestStep.options); + oldOptions.put("filterable-fields", "c1"); + return sourceTestStep.withNewOptions( + oldOptions); + }) + .collect(Collectors.toList())) + .setupTableSinks( + CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE.getSetupSinkTestSteps()) + .runSql(CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE.getRunSqlTestStep().sql) + .build(); + + public static final TableTestProgram CASCADED_DELTA_JOIN_WITH_CACHE = + TableTestProgram.of( + "cascaded-delta-join-with-cache", + "validates cascaded delta join with cache") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupConfig(ExecutionConfigOptions.TABLE_EXEC_DELTA_JOIN_CACHE_ENABLED, true) + .setupTableSources( + CASCADED_DELTA_JOIN_WITH_NON_EQUIV_CONDITION.getSetupSourceTestSteps()) + .setupTableSinks( + CASCADED_DELTA_JOIN_WITH_NON_EQUIV_CONDITION.getSetupSinkTestSteps()) + .runSql(CASCADED_DELTA_JOIN_WITH_NON_EQUIV_CONDITION.getRunSqlTestStep().sql) + .build(); + + public static final TableTestProgram CASCADED_DELTA_JOIN_WITH_CACHE_AND_CALC_ON_SOURCE = + TableTestProgram.of( + "cascaded-delta-join-with-cache-and-calc-on-source", + "validates cascaded delta join with cache and calc on source") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupConfig(ExecutionConfigOptions.TABLE_EXEC_DELTA_JOIN_CACHE_ENABLED, true) + .setupTableSources( + CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE.getSetupSourceTestSteps()) + .setupTableSinks( + CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE.getSetupSinkTestSteps()) + .runSql(CASCADED_DELTA_JOIN_WITH_CALC_ON_SOURCE.getRunSqlTestStep().sql) + .build(); + private static String[] addPk2Schema(String[] originalSchema, String... pkCols) { return Stream.concat( Arrays.stream(originalSchema), diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache-and-calc-on-source/plan/cascaded-delta-join-with-cache-and-calc-on-source.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache-and-calc-on-source/plan/cascaded-delta-join-with-cache-and-calc-on-source.json new file mode 100644 index 0000000000000..581755110d82c --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache-and-calc-on-source/plan/cascaded-delta-join-with-cache-and-calc-on-source.json @@ -0,0 +1,1147 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 55, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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, srcA]], fields=[a1, a0, a2])" + }, { + "id" : 56, + "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" : 57, + "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" : 58, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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, srcB]], fields=[b0, b2, b1])" + }, { + "id" : 59, + "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" : 60, + "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" : 61, + "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`.`srcB`", + "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`.`srcA`", + "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`.`srcA`", + "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`.`srcB`", + "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" : 62, + "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), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 63, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, srcC, filter=[]]], fields=[c1, c0, c2])" + }, { + "id" : 64, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 65, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Calc(select=[c1, c0, CONCAT(c2, '-s') AS new_c2], where=[SEARCH(c1, Sarg[2, 5])])" + }, { + "id" : 66, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[c1]])" + }, { + "id" : 67, + "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, 3, 5 ], + "rightJoinKeys" : [ 0 ], + "rightUpsertKey" : [ 0, 1 ], + "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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0, 1 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0, 1 ], + "lookupTableBinaryInputOrdinal" : 2, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 2 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + }, { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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`.`srcB`", + "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" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> 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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 0 ], + "left" : { + "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>" + } + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 2, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "rowTypeAfterCalc" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "rowType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>" + } + } + }, + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = c1)], leftToRight=[Binary], rightToLeft=[[{round=[1], sourceTables=[default_catalog.default_database.srcC], lookupTable=[default_catalog.default_database.srcA], lookupKeys=[a1=c1]}, {round=[2], sourceTables=[default_catalog.default_database.srcA], lookupTable=[default_catalog.default_database.srcB], lookupKeys=[b1=a1]}]], select=[a1, a0, a2, b0, b2, b1, c1, c0, new_c2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 68, + "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" + }, { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_c0_a1_b1_c1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "c0", "a1", "b1", "c1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "VALUE", + "inputUpsertKey" : [ 0, 1, 3, 5, 6, 7 ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1, c1, c0, new_c2])" + } ], + "edges" : [ { + "source" : 55, + "target" : 56, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 56, + "target" : 57, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 58, + "target" : 59, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 59, + "target" : 60, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 57, + "target" : 61, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 60, + "target" : 61, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 61, + "target" : 62, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 63, + "target" : 64, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 64, + "target" : 65, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 65, + "target" : 66, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 62, + "target" : 67, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 66, + "target" : 67, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 67, + "target" : 68, + "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/cascaded-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/cascaded-delta-join-with-cache-and-calc-on-source/savepoint/_metadata new file mode 100644 index 0000000000000..d6dd41527c889 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache-and-calc-on-source/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache/plan/cascaded-delta-join-with-cache.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache/plan/cascaded-delta-join-with-cache.json new file mode 100644 index 0000000000000..a8bfbd71538b8 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache/plan/cascaded-delta-join-with-cache.json @@ -0,0 +1,638 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 68, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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, srcA]], fields=[a1, a0, a2])" + }, { + "id" : 69, + "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" : 70, + "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" : 71, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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, srcB]], fields=[b0, b2, b1])" + }, { + "id" : 72, + "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" : 73, + "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" : 74, + "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`.`srcB`", + "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`.`srcA`", + "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`.`srcA`", + "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`.`srcB`", + "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" : 75, + "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), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 76, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, srcC]], fields=[c1, c0, c2])" + }, { + "id" : 77, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 78, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[c1]])" + }, { + "id" : 79, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "INNER", + "leftKeys" : [ 0 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 8, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "leftUpsertKeys" : [ [ 0, 1, 3, 5 ] ], + "rightUpsertKeys" : [ [ 0, 1 ] ], + "state" : [ { + "index" : 0, + "ttl" : "0 ms", + "name" : "leftState" + }, { + "index" : 1, + "ttl" : "0 ms", + "name" : "rightState" + } ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Join(joinType=[InnerJoin], where=[((a1 = c1) AND (a2 <> c2))], select=[a1, a0, a2, b0, b2, b1, c1, c0, c2], leftInputSpec=[HasUniqueKey], rightInputSpec=[HasUniqueKey])" + }, { + "id" : 80, + "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" + }, { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_c0_a1_b1_c1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "c0", "a1", "b1", "c1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "inputUpsertKey" : [ 0, 1, 3, 5, 6, 7 ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1, c1, c0, c2])" + } ], + "edges" : [ { + "source" : 68, + "target" : 69, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 69, + "target" : 70, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 71, + "target" : 72, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 72, + "target" : 73, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 70, + "target" : 74, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 73, + "target" : 74, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 74, + "target" : 75, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 76, + "target" : 77, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 77, + "target" : 78, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 75, + "target" : 79, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 78, + "target" : 79, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 79, + "target" : 80, + "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/cascaded-delta-join-with-cache/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache/savepoint/_metadata new file mode 100644 index 0000000000000..764a4d7d4ceb3 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-cache/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source-and-filter-pushed-down/plan/cascaded-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/cascaded-delta-join-with-calc-on-source-and-filter-pushed-down/plan/cascaded-delta-join-with-calc-on-source-and-filter-pushed-down.json new file mode 100644 index 0000000000000..8d8b5ceaee5d3 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source-and-filter-pushed-down/plan/cascaded-delta-join-with-calc-on-source-and-filter-pushed-down.json @@ -0,0 +1,1146 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 41, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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, srcA]], fields=[a1, a0, a2])" + }, { + "id" : 42, + "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" : 43, + "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" : 44, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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, srcB]], fields=[b0, b2, b1])" + }, { + "id" : 45, + "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" : 46, + "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" : 47, + "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`.`srcB`", + "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`.`srcA`", + "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`.`srcA`", + "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`.`srcB`", + "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" : 48, + "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), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 49, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, srcC, filter=[]]], fields=[c1, c0, c2])" + }, { + "id" : 50, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 51, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Calc(select=[c1, c0, CONCAT(c2, '-s') AS new_c2], where=[SEARCH(c1, Sarg[2, 5])])" + }, { + "id" : 52, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[c1]])" + }, { + "id" : 53, + "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, 3, 5 ], + "rightJoinKeys" : [ 0 ], + "rightUpsertKey" : [ 0, 1 ], + "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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0, 1 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0, 1 ], + "lookupTableBinaryInputOrdinal" : 2, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 2 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + }, { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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`.`srcB`", + "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" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> 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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 0 ], + "left" : { + "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>" + } + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 2, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "rowTypeAfterCalc" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "rowType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>" + } + } + }, + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = c1)], leftToRight=[Binary], rightToLeft=[[{round=[1], sourceTables=[default_catalog.default_database.srcC], lookupTable=[default_catalog.default_database.srcA], lookupKeys=[a1=c1]}, {round=[2], sourceTables=[default_catalog.default_database.srcA], lookupTable=[default_catalog.default_database.srcB], lookupKeys=[b1=a1]}]], select=[a1, a0, a2, b0, b2, b1, c1, c0, new_c2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 54, + "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" + }, { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_c0_a1_b1_c1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "c0", "a1", "b1", "c1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "inputUpsertKey" : [ 0, 1, 3, 5, 6, 7 ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1, c1, c0, new_c2])" + } ], + "edges" : [ { + "source" : 41, + "target" : 42, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 42, + "target" : 43, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 44, + "target" : 45, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 45, + "target" : 46, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 43, + "target" : 47, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 46, + "target" : 47, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 47, + "target" : 48, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 49, + "target" : 50, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 50, + "target" : 51, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 51, + "target" : 52, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 48, + "target" : 53, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 52, + "target" : 53, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 53, + "target" : 54, + "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/cascaded-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/cascaded-delta-join-with-calc-on-source-and-filter-pushed-down/savepoint/_metadata new file mode 100644 index 0000000000000..657193b4cb674 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source-and-filter-pushed-down/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source/plan/cascaded-delta-join-with-calc-on-source.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source/plan/cascaded-delta-join-with-calc-on-source.json new file mode 100644 index 0000000000000..275d3d8c71671 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source/plan/cascaded-delta-join-with-calc-on-source.json @@ -0,0 +1,1147 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 27, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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, srcA]], fields=[a1, a0, a2])" + }, { + "id" : 28, + "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" : 29, + "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" : 30, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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, srcB]], fields=[b0, b2, b1])" + }, { + "id" : 31, + "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" : 32, + "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" : 33, + "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`.`srcB`", + "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`.`srcA`", + "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`.`srcA`", + "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`.`srcB`", + "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" : 34, + "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), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 35, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, srcC, filter=[]]], fields=[c1, c0, c2])" + }, { + "id" : 36, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 37, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Calc(select=[c1, c0, CONCAT(c2, '-s') AS new_c2], where=[SEARCH(c1, Sarg[2, 5])])" + }, { + "id" : 38, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[c1]])" + }, { + "id" : 39, + "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, 3, 5 ], + "rightJoinKeys" : [ 0 ], + "rightUpsertKey" : [ 0, 1 ], + "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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0, 1 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0, 1 ], + "lookupTableBinaryInputOrdinal" : 2, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 2 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + }, { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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`.`srcB`", + "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" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> 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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 0 ], + "left" : { + "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>" + } + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 2, + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE 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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 2, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 2, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "rowTypeAfterCalc" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "rowType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>" + } + } + }, + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = c1)], leftToRight=[Binary], rightToLeft=[[{round=[1], sourceTables=[default_catalog.default_database.srcC], lookupTable=[default_catalog.default_database.srcA], lookupKeys=[a1=c1]}, {round=[2], sourceTables=[default_catalog.default_database.srcA], lookupTable=[default_catalog.default_database.srcB], lookupKeys=[b1=a1]}]], select=[a1, a0, a2, b0, b2, b1, c1, c0, new_c2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 40, + "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" + }, { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_c0_a1_b1_c1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "c0", "a1", "b1", "c1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5, 6, 7 ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `new_c2` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1, c1, c0, new_c2])" + } ], + "edges" : [ { + "source" : 27, + "target" : 28, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 28, + "target" : 29, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 30, + "target" : 31, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 31, + "target" : 32, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 29, + "target" : 33, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 32, + "target" : 33, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 33, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 35, + "target" : 36, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 36, + "target" : 37, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 37, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 34, + "target" : 39, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 38, + "target" : 39, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 39, + "target" : 40, + "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/cascaded-delta-join-with-calc-on-source/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source/savepoint/_metadata new file mode 100644 index 0000000000000..8c3b6be457a00 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-calc-on-source/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-contains-index/plan/cascaded-delta-join-with-join-key-contains-index.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-contains-index/plan/cascaded-delta-join-with-join-key-contains-index.json new file mode 100644 index 0000000000000..f30a4e0a74867 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-contains-index/plan/cascaded-delta-join-with-join-key-contains-index.json @@ -0,0 +1,1015 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 14, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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, srcA]], fields=[a1, a0, a2])" + }, { + "id" : 15, + "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" : 16, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 17, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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, srcB]], fields=[b0, b2, b1])" + }, { + "id" : 18, + "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" : 19, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 20, + "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" : [ 2, 0 ], + "rightUpsertKey" : [ 0, 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 NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "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`.`srcB`", + "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" : { + "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`.`srcA`", + "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 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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`.`srcB`", + "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" : "$AND$1", + "operands" : [ { + "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" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "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 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) 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" : 21, + "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), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 22, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, srcC]], fields=[c1, c0, c2])" + }, { + "id" : 23, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 24, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[c1]])" + }, { + "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" : [ 0 ], + "leftUpsertKey" : [ 0, 1 ], + "rightJoinKeys" : [ 0 ], + "rightUpsertKey" : [ 0, 1 ], + "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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0, 1 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0, 1 ], + "lookupTableBinaryInputOrdinal" : 2, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 2 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + }, { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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`.`srcB`", + "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" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> 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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 0 ], + "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" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$=$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "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 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>" + } + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 2, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>" + } + } + }, + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = c1)], leftToRight=[Binary], rightToLeft=[[{round=[1], sourceTables=[default_catalog.default_database.srcC], lookupTable=[default_catalog.default_database.srcA], lookupKeys=[a1=c1]}, {round=[2], sourceTables=[default_catalog.default_database.srcA], lookupTable=[default_catalog.default_database.srcB], lookupKeys=[b1=a1, b0=a0]}]], select=[a1, a0, a2, b0, b2, b1, c1, c0, c2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 26, + "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" + }, { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_c0_a1_b1_c1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "c0", "a1", "b1", "c1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "inputUpsertKey" : [ 0, 1, 6, 7 ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1, c1, c0, c2])" + } ], + "edges" : [ { + "source" : 14, + "target" : 15, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 17, + "target" : 18, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 18, + "target" : 19, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 16, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "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" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-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/cascaded-delta-join-with-join-key-contains-index/savepoint/_metadata new file mode 100644 index 0000000000000..0d37a63b87e2d Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-contains-index/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-equals-index/plan/cascaded-delta-join-with-join-key-equals-index.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-equals-index/plan/cascaded-delta-join-with-join-key-equals-index.json new file mode 100644 index 0000000000000..06c52b1804050 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-equals-index/plan/cascaded-delta-join-with-join-key-equals-index.json @@ -0,0 +1,944 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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, srcA]], fields=[a1, a0, a2])" + }, { + "id" : 2, + "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" : 3, + "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" : 4, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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, srcB]], fields=[b0, b2, b1])" + }, { + "id" : 5, + "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" : 6, + "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" : 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" : [ 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`.`srcB`", + "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`.`srcA`", + "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`.`srcA`", + "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`.`srcB`", + "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" : 8, + "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), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 9, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, srcC]], fields=[c1, c0, c2])" + }, { + "id" : 10, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 11, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[c1]])" + }, { + "id" : 12, + "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, 3, 5 ], + "rightJoinKeys" : [ 0 ], + "rightUpsertKey" : [ 0, 1 ], + "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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "leftAllBinaryInputOrdinals" : [ 0, 1 ], + "rightAllBinaryInputOrdinals" : [ 0 ], + "left2RightLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 0, 1 ], + "lookupTableBinaryInputOrdinal" : 2, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + } ] + }, + "right2LeftLookupChain" : { + "nodes" : [ { + "inputTableBinaryInputOrdinals" : [ 2 ], + "lookupTableBinaryInputOrdinal" : 0, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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" : 0 + } + }, + "remainingCondition" : null + }, + "joinType" : "INNER" + }, { + "inputTableBinaryInputOrdinals" : [ 0 ], + "lookupTableBinaryInputOrdinal" : 1, + "deltaJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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" + } ] + }, + "allBinaryInputTables" : [ { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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`.`srcB`", + "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" + }, { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)> 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" : 6, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "leftJoinKey" : [ 0 ], + "rightJoinKey" : [ 0 ], + "left" : { + "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>" + } + }, + "right" : { + "type" : "BinaryInputNode", + "inputOrdinal" : 2, + "projection" : null, + "filter" : null, + "rowTypeAfterCalc" : null, + "rowType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>" + } + } + }, + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = c1)], leftToRight=[Binary], rightToLeft=[[{round=[1], sourceTables=[default_catalog.default_database.srcC], lookupTable=[default_catalog.default_database.srcA], lookupKeys=[a1=c1]}, {round=[2], sourceTables=[default_catalog.default_database.srcA], lookupTable=[default_catalog.default_database.srcB], lookupKeys=[b1=a1]}]], select=[a1, a0, a2, b0, b2, b1, c1, c0, c2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 13, + "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" + }, { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_c0_a1_b1_c1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "c0", "a1", "b1", "c1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "ADAPTIVE", + "inputUpsertKey" : [ 0, 1, 3, 5, 6, 7 ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1, c1, c0, c2])" + } ], + "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" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 12, + "target" : 13, + "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/cascaded-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/cascaded-delta-join-with-join-key-equals-index/savepoint/_metadata new file mode 100644 index 0000000000000..8c5972a2f8e5e Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-join-key-equals-index/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-non-equiv-condition/plan/cascaded-delta-join-with-non-equiv-condition.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-non-equiv-condition/plan/cascaded-delta-join-with-non-equiv-condition.json new file mode 100644 index 0000000000000..8d322fe17315b --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-non-equiv-condition/plan/cascaded-delta-join-with-non-equiv-condition.json @@ -0,0 +1,639 @@ +{ + "flinkVersion" : "2.3", + "nodes" : [ { + "id" : 27, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcA`", + "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, srcA]], fields=[a1, a0, a2])" + }, { + "id" : 28, + "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" : 29, + "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" : 30, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcB`", + "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, srcB]], fields=[b0, b2, b1])" + }, { + "id" : 31, + "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" : 32, + "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" : 33, + "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`.`srcB`", + "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`.`srcA`", + "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`.`srcA`", + "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`.`srcB`", + "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" : 34, + "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), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 35, + "type" : "stream-exec-table-source-scan_2", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`srcC`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_c1_c0", + "type" : "PRIMARY_KEY", + "columns" : [ "c1", "c0" ] + }, + "indexes" : [ { + "name" : "INDEX_c1", + "columns" : [ "c1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, srcC]], fields=[c1, c0, c2])" + }, { + "id" : 36, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 37, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[c1]])" + }, { + "id" : 38, + "type" : "stream-exec-join_1", + "joinSpec" : { + "joinType" : "INNER", + "leftKeys" : [ 0 ], + "rightKeys" : [ 0 ], + "filterNulls" : [ true ], + "nonEquiCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 8, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "leftUpsertKeys" : [ [ 0, 1, 3, 5 ] ], + "rightUpsertKeys" : [ [ 0, 1 ] ], + "state" : [ { + "index" : 0, + "ttl" : "0 ms", + "name" : "leftState" + }, { + "index" : 1, + "ttl" : "0 ms", + "name" : "rightState" + } ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Join(joinType=[InnerJoin], where=[((a1 = c1) AND (a2 <> c2))], select=[a1, a0, a2, b0, b2, b1, c1, c0, c2], leftInputSpec=[HasUniqueKey], rightInputSpec=[HasUniqueKey])" + }, { + "id" : 39, + "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" + }, { + "name" : "c1", + "dataType" : "INT NOT NULL" + }, { + "name" : "c0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "c2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_c0_a1_b1_c1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "c0", "a1", "b1", "c1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "VALUE", + "inputUpsertKey" : [ 0, 1, 3, 5, 6, 7 ], + "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, `c1` INT NOT NULL, `c0` DOUBLE NOT NULL, `c2` VARCHAR(2147483647)>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1, c1, c0, c2])" + } ], + "edges" : [ { + "source" : 27, + "target" : 28, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 28, + "target" : 29, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 30, + "target" : 31, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 31, + "target" : 32, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 29, + "target" : 33, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 32, + "target" : 33, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 33, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 35, + "target" : 36, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 36, + "target" : 37, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 34, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 37, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 38, + "target" : 39, + "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/cascaded-delta-join-with-non-equiv-condition/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-non-equiv-condition/savepoint/_metadata new file mode 100644 index 0000000000000..8abd813ee5127 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_2/cascaded-delta-join-with-non-equiv-condition/savepoint/_metadata differ