diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java index 819a632fa5ffb..721e05e67b988 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SinkTestStep.java @@ -20,12 +20,14 @@ import org.apache.flink.table.catalog.TableDistribution; import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.stream.Collectors; @@ -42,6 +44,7 @@ public final class SinkTestStep extends TableTestStep { public final @Nullable List expectedMaterializedRows; public final @Nullable List expectedMaterializedStrings; public final boolean testChangelogData; + public final @Nullable int[] deduplicatedFieldIndices; SinkTestStep( String name, @@ -55,17 +58,16 @@ public final class SinkTestStep extends TableTestStep { @Nullable List expectedAfterRestoreStrings, @Nullable List expectedMaterializedRows, @Nullable List expectedMaterializedStrings, - boolean testChangelogData) { - super(name, schemaComponents, distribution, partitionKeys, options); - boolean hasRowsSet = - expectedBeforeRestore != null - || expectedAfterRestore != null - || expectedMaterializedRows != null; - boolean hasStringsSet = - expectedBeforeRestoreStrings != null - || expectedAfterRestoreStrings != null - || expectedMaterializedStrings != null; - if (hasRowsSet && hasStringsSet) { + boolean testChangelogData, + @Nullable int[] deduplicatedFieldIndices) { + super( + name, + schemaComponents, + distribution, + partitionKeys, + options, + Collections.emptyList()); + if (hasRowsSet() && hasStringsSet()) { throw new IllegalArgumentException( "You can not mix Row/String representations in restore data."); } @@ -76,6 +78,12 @@ public final class SinkTestStep extends TableTestStep { this.expectedMaterializedRows = expectedMaterializedRows; this.expectedMaterializedStrings = expectedMaterializedStrings; this.testChangelogData = testChangelogData; + this.deduplicatedFieldIndices = deduplicatedFieldIndices; + + if (deduplicatedFieldIndices != null && !hasRowsSet()) { + throw new IllegalArgumentException( + "DeduplicatedFieldIndices can only be used for Row representations in restore data."); + } } /** Builder for creating a {@link SinkTestStep}. */ @@ -108,9 +116,29 @@ public List getExpectedAfterRestoreAsStrings() { } public List getExpectedAsStrings() { - final List data = new ArrayList<>(getExpectedBeforeRestoreAsStrings()); - data.addAll(getExpectedAfterRestoreAsStrings()); - return data; + if (hasStringsSet() || deduplicatedFieldIndices == null) { + final List data = new ArrayList<>(getExpectedBeforeRestoreAsStrings()); + data.addAll(getExpectedAfterRestoreAsStrings()); + return data; + } + Preconditions.checkState(hasRowsSet()); + final List data = new ArrayList<>(); + if (expectedBeforeRestore != null) { + data.addAll(expectedBeforeRestore); + } + if (expectedAfterRestore != null) { + data.addAll(expectedAfterRestore); + } + + Map, Row> deduplicatedMap = new HashMap<>(); + for (Row row : data) { + List key = new ArrayList<>(deduplicatedFieldIndices.length); + for (int i = 0; i < deduplicatedFieldIndices.length; i++) { + key.add(row.getField(deduplicatedFieldIndices[i])); + } + deduplicatedMap.put(key, row); + } + return deduplicatedMap.values().stream().map(Row::toString).collect(Collectors.toList()); } public List getExpectedMaterializedResultsAsStrings() { @@ -118,6 +146,10 @@ public List getExpectedMaterializedResultsAsStrings() { return expectedMaterializedStrings; } if (expectedMaterializedRows != null) { + if (deduplicatedFieldIndices != null) { + throw new UnsupportedOperationException( + "Unsupported to deduplicate data for materialized rows"); + } return expectedMaterializedRows.stream() .map(Row::toString) .collect(Collectors.toList()); @@ -138,6 +170,18 @@ public boolean shouldTestChangelogData() { return testChangelogData; } + private boolean hasRowsSet() { + return expectedBeforeRestore != null + || expectedAfterRestore != null + || expectedMaterializedRows != null; + } + + private boolean hasStringsSet() { + return expectedBeforeRestoreStrings != null + || expectedAfterRestoreStrings != null + || expectedMaterializedStrings != null; + } + /** Builder pattern for {@link SinkTestStep}. */ public static final class Builder extends AbstractBuilder { @@ -151,6 +195,8 @@ public static final class Builder extends AbstractBuilder { private boolean testChangelogData = true; + private @Nullable int[] deduplicatedFieldIndices; + private Builder(String name) { super(name); } @@ -203,6 +249,14 @@ public Builder testMaterializedData() { return this; } + public Builder deduplicatedFieldIndices(int[] deduplicatedFieldIndices) { + // TODO FLINK-38518 use pk to deduplicate data rather than specific fields. + // This task requires refactoring the current `AbstractBuilder` to separate the + // declaration of the primary key from the `List schemaComponents`. + this.deduplicatedFieldIndices = deduplicatedFieldIndices; + return this; + } + public SinkTestStep build() { return new SinkTestStep( name, @@ -216,7 +270,8 @@ public SinkTestStep build() { expectedAfterRestoreStrings, expectedMaterializedBeforeRows, expectedMaterializedBeforeStrings, - testChangelogData); + testChangelogData, + deduplicatedFieldIndices); } } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java index 95ebcfc6dd99f..45c0b86a303fb 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/SourceTestStep.java @@ -34,17 +34,22 @@ public final class SourceTestStep extends TableTestStep { public final List dataBeforeRestore; public final List dataAfterRestore; + public final boolean treatDataBeforeRestoreAsFullStageData; + SourceTestStep( String name, List schemaComponents, @Nullable TableDistribution distribution, List partitionKeys, Map options, + List> indexes, List dataBeforeRestore, - List dataAfterRestore) { - super(name, schemaComponents, distribution, partitionKeys, options); + List dataAfterRestore, + boolean treatDataBeforeRestoreAsFullStageData) { + super(name, schemaComponents, distribution, partitionKeys, options, indexes); this.dataBeforeRestore = dataBeforeRestore; this.dataAfterRestore = dataAfterRestore; + this.treatDataBeforeRestoreAsFullStageData = treatDataBeforeRestoreAsFullStageData; } /** Builder for creating a {@link SourceTestStep}. */ @@ -52,6 +57,19 @@ public static Builder newBuilder(String name) { return new Builder(name); } + public SourceTestStep withNewOptions(Map newOptions) { + return new SourceTestStep( + name, + schemaComponents, + distribution, + partitionKeys, + newOptions, + indexes, + dataBeforeRestore, + dataAfterRestore, + treatDataBeforeRestoreAsFullStageData); + } + @Override public TestKind getKind() { return dataBeforeRestore.isEmpty() @@ -66,6 +84,8 @@ public static final class Builder extends AbstractBuilder { private final List dataBeforeRestore = new ArrayList<>(); private final List dataAfterRestore = new ArrayList<>(); + private final List> indexes = new ArrayList<>(); + private boolean treatDataBeforeRestoreAsFullStageData = false; private Builder(String name) { super(name); @@ -85,6 +105,16 @@ public Builder producedAfterRestore(Row... data) { return this; } + public Builder addIndex(String... index) { + this.indexes.add(Arrays.asList(index)); + return this; + } + + public Builder treatDataBeforeRestoreAsFullStageData() { + this.treatDataBeforeRestoreAsFullStageData = true; + return this; + } + public SourceTestStep build() { return new SourceTestStep( name, @@ -92,8 +122,10 @@ public SourceTestStep build() { distribution, partitionKeys, options, + indexes, dataBeforeRestore, - dataAfterRestore); + dataAfterRestore, + treatDataBeforeRestoreAsFullStageData); } } } diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java index bb8ca296cae5a..935ee187f98eb 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestProgram.java @@ -317,6 +317,16 @@ public Builder setupTableSource(SourceTestStep sourceTestStep) { return this; } + /** + * Setup steps for each table source. + * + *

Use {@link SourceTestStep.Builder} to construct this step. + */ + public Builder setupTableSources(List sourceTestSteps) { + setupSteps.addAll(sourceTestSteps); + return this; + } + /** * Setup step for a table sink. * @@ -327,6 +337,16 @@ public Builder setupTableSink(SinkTestStep sinkTestStep) { return this; } + /** + * Setup steps for each table sink. + * + *

Use {@link SinkTestStep.Builder} to construct this step. + */ + public Builder setupTableSinks(List sinkTestSteps) { + setupSteps.addAll(sinkTestSteps); + return this; + } + /** * Setup step for a model. * diff --git a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java index 2058e9e2dcc76..0f3e375d66a56 100644 --- a/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java +++ b/flink-table/flink-table-api-java/src/test/java/org/apache/flink/table/test/program/TableTestStep.java @@ -20,11 +20,18 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigurationUtils; +import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.TableResult; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; import org.apache.flink.table.catalog.TableDistribution; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; import org.apache.flink.table.connector.ChangelogMode; import org.apache.flink.types.RowKind; +import org.apache.flink.util.Preconditions; import javax.annotation.Nullable; @@ -58,25 +65,28 @@ public abstract class TableTestStep implements TestStep { public final @Nullable TableDistribution distribution; public final List partitionKeys; public final Map options; + public final List> indexes; TableTestStep( String name, List schemaComponents, @Nullable TableDistribution distribution, List partitionKeys, - Map options) { + Map options, + List> indexes) { this.name = name; this.schemaComponents = schemaComponents; this.distribution = distribution; this.partitionKeys = partitionKeys; this.options = options; + this.indexes = indexes; } - public TableResult apply(TableEnvironment env) { - return apply(env, Collections.emptyMap()); + public void apply(TableEnvironment env) { + apply(env, Collections.emptyMap()); } - public TableResult apply(TableEnvironment env, Map extraOptions) { + public void apply(TableEnvironment env, Map extraOptions) { final Map allOptions = new HashMap<>(options); allOptions.putAll(extraOptions); @@ -97,7 +107,41 @@ public TableResult apply(TableEnvironment env, Map extraOptions) .map(e -> String.format("'%s'='%s'", e.getKey(), e.getValue())) .collect(Collectors.joining(",\n"))); - return env.executeSql(createTable); + env.executeSql(createTable); + if (indexes.isEmpty()) { + return; + } + + Optional currentCatalogOp = env.getCatalog(env.getCurrentCatalog()); + Preconditions.checkState(currentCatalogOp.isPresent()); + Catalog catalog = currentCatalogOp.get(); + + String currentDatabaseName = env.getCurrentDatabase(); + ObjectPath tablePath = new ObjectPath(currentDatabaseName, name); + CatalogTable oldTable; + try { + oldTable = (CatalogTable) catalog.getTable(tablePath); + catalog.dropTable(tablePath, false); + } catch (TableNotExistException e) { + throw new IllegalStateException(e); + } + Schema schema = oldTable.getUnresolvedSchema(); + Schema.Builder schemaBuilder = Schema.newBuilder().fromSchema(schema); + indexes.forEach(schemaBuilder::index); + CatalogTable newTable = + CatalogTable.newBuilder() + .schema(schemaBuilder.build()) + .comment(oldTable.getComment()) + .partitionKeys(oldTable.getPartitionKeys()) + .options(oldTable.getOptions()) + .snapshot(oldTable.getSnapshot().orElse(null)) + .distribution(oldTable.getDistribution().orElse(null)) + .build(); + try { + catalog.createTable(tablePath, newTable, false); + } catch (TableAlreadyExistException | DatabaseNotExistException e) { + throw new IllegalStateException(e); + } } /** Builder pattern for {@link SourceTestStep} and {@link SinkTestStep}. */ diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java index 4e12f5f72d870..bbc40493beb86 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/factories/TestValuesTableFactory.java @@ -195,6 +195,10 @@ public final class TestValuesTableFactory private static final AtomicInteger idCounter = new AtomicInteger(0); private static final Map> registeredData = new HashMap<>(); private static final Map> registeredRowData = new HashMap<>(); + // The difference between registeredDataForFullStage and `registeredData` is that + // `registeredData` is used for data delivered from the source to downstream, while the rows in + // `registeredDataForFullStage` will not be sent to downstream and are only used for lookup. + private static final Map> registeredDataForFullStage = new HashMap<>(); /** * Register the given data into the data factory context and return the data id. The data id can @@ -232,6 +236,18 @@ public static String registerRowData(Seq data) { return registerRowData(JavaScalaConversionUtil.toJava(data)); } + /** + * Register the given data for full stage into the data factory context with the specified id + * and return the data id. The data id can be used as a reference to the registered data in data + * connector DDL. + */ + public static void registerFullStageData(Collection data, String id) { + if (registeredDataForFullStage.containsKey(id)) { + throw new IllegalArgumentException("Full stage data already exists"); + } + registeredDataForFullStage.put(id, data); + } + /** * Returns received raw results of the registered table sink. The raw results are encoded with * {@link RowKind}. @@ -299,6 +315,7 @@ public static List getWatermarkOutput(String tableName) { public static void clearAllData() { registeredData.clear(); registeredRowData.clear(); + registeredDataForFullStage.clear(); TestValuesRuntimeFunctions.clearResults(); } @@ -714,6 +731,8 @@ public DynamicTableSource createDynamicTableSource(Context context) { enableAggregatePushDown); } } else { + Collection fullStageData = + registeredDataForFullStage.getOrDefault(dataId, Collections.emptyList()); if (enableCustomShuffle) { return new TestValuesScanLookupTableSourceWithCustomShuffle( context.getCatalogTable().getResolvedSchema().toPhysicalRowDataType(), @@ -724,6 +743,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { runtimeSource, failingSource, partition2Rows, + fullStageData, isAsync, lookupFunctionClass, nestedProjectionSupported, @@ -753,6 +773,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { runtimeSource, failingSource, partition2Rows, + fullStageData, isAsync, lookupFunctionClass, nestedProjectionSupported, @@ -1814,6 +1835,7 @@ private static class TestValuesScanLookupTableSource extends TestValuesScanTable protected final DataType originType; protected final int[] primaryKeyIndices; + protected final Collection fullStageData; private TestValuesScanLookupTableSource( DataType originType, @@ -1824,6 +1846,7 @@ private TestValuesScanLookupTableSource( String runtimeSource, boolean failingSource, Map, Collection> data, + Collection fullStageData, boolean isAsync, @Nullable String lookupFunctionClass, boolean nestedProjectionSupported, @@ -1867,6 +1890,7 @@ private TestValuesScanLookupTableSource( this.reloadTrigger = reloadTrigger; this.lookupThreshold = lookupThreshold; this.primaryKeyIndices = primaryKeyIndices; + this.fullStageData = fullStageData; } @SuppressWarnings({"unchecked", "rawtypes"}) @@ -1902,7 +1926,8 @@ public LookupRuntimeProvider getLookupRuntimeProvider(LookupContext context) { key -> rows.addAll(data.getOrDefault(key, new ArrayList<>()))); } - List data = new ArrayList<>(rows); + List data = new ArrayList<>(fullStageData); + data.addAll(rows); if (numElementToSkip > 0) { if (numElementToSkip >= data.size()) { data = Collections.EMPTY_LIST; @@ -2075,6 +2100,7 @@ public DynamicTableSource copy() { runtimeSource, failingSource, data, + fullStageData, isAsync, lookupFunctionClass, nestedProjectionSupported, @@ -2115,6 +2141,7 @@ private TestValuesScanLookupTableSourceWithCustomShuffle( String runtimeSource, boolean failingSource, Map, Collection> data, + Collection fullStageData, boolean isAsync, @Nullable String lookupFunctionClass, boolean nestedProjectionSupported, @@ -2143,6 +2170,7 @@ private TestValuesScanLookupTableSourceWithCustomShuffle( runtimeSource, failingSource, data, + fullStageData, isAsync, lookupFunctionClass, nestedProjectionSupported, @@ -2175,6 +2203,7 @@ public DynamicTableSource copy() { runtimeSource, failingSource, data, + fullStageData, isAsync, lookupFunctionClass, nestedProjectionSupported, 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 new file mode 100644 index 0000000000000..38cf2bf787e4b --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.planner.plan.nodes.exec.testutils.RestoreTestBase; +import org.apache.flink.table.test.program.TableTestProgram; + +import java.util.Arrays; +import java.util.List; + +/** Restore tests for {@link StreamExecDeltaJoin}. */ +public class DeltaJoinRestoreTest extends RestoreTestBase { + + public DeltaJoinRestoreTest() { + super(StreamExecDeltaJoin.class); + } + + @Override + public List programs() { + return Arrays.asList( + DeltaJoinTestPrograms.DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_JOIN_KEY_CONTAINS_INDEX, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_NON_EQUIV_CONDITION, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_CALC_ON_SOURCE, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_CALC_ON_SOURCE_AND_FILTER_PUSHED_DOWN, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_CACHE, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_CACHE_AND_CALC_ON_SOURCE, + 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); + } +} 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 new file mode 100644 index 0000000000000..b4eb48932fc1f --- /dev/null +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.flink.table.planner.plan.nodes.exec.stream; + +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.flink.table.api.config.OptimizerConfigOptions; +import org.apache.flink.table.test.program.SinkTestStep; +import org.apache.flink.table.test.program.SourceTestStep; +import org.apache.flink.table.test.program.TableTestProgram; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** {@link TableTestProgram} definitions for testing {@link StreamExecDeltaJoin}. */ +public class DeltaJoinTestPrograms { + + static final String[] LEFT_TABLE_BASE_SCHEMA = + new String[] {"a1 int", "a0 double", "a2 string"}; + + static final String[] RIGHT_TABLE_BASE_SCHEMA = + new String[] {"b0 double", "b2 string", "b1 int"}; + + 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 Map TABLE_BASE_OPTIONS = + Map.of("async", "true", "sink-insert-only", "false"); + + public static final TableTestProgram DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX = + TableTestProgram.of( + "delta-join-with-join-key-equals-index", + "validates delta join with join key equals index") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSource( + SourceTestStep.newBuilder("leftSrc") + .addSchema(LEFT_TABLE_BASE_SCHEMA) + .addOptions(TABLE_BASE_OPTIONS) + .addIndex("a1") + .treatDataBeforeRestoreAsFullStageData() + .producedBeforeRestore( + Row.of(1, 1.0, "l-1-1"), + Row.of(1, 1.0, "l-1-2"), + Row.of(5, 5.0, "l-5-1")) + .producedAfterRestore( + Row.of(3, 3.0, "l-3-1"), + Row.of(3, 3.0, "l-3-2"), + Row.of(5, 5.0, "l-5-2")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("rightSrc") + .addSchema(RIGHT_TABLE_BASE_SCHEMA) + .addOptions(TABLE_BASE_OPTIONS) + .addIndex("b1") + .treatDataBeforeRestoreAsFullStageData() + .producedBeforeRestore( + Row.of(5.0, "r-5-1", 5), Row.of(3.0, "r-3-1", 3)) + .producedAfterRestore( + Row.of(3.0, "r-3-2", 3), Row.of(1.0, "r-1-1", 1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("snk") + .addSchema(addPk2Schema(SINK_TABLE_BASE_SCHEMA, "a0", "b0")) + .addOptions(TABLE_BASE_OPTIONS) + .testMaterializedData() + // deduplicate data by pk + .deduplicatedFieldIndices(new int[] {1, 3}) + .consumedBeforeRestore(Row.of(5, 5.0, "l-5-1", 5.0, "r-5-1", 5)) + .consumedAfterRestore( + Row.of(1, 1.0, "l-1-2", 1.0, "r-1-1", 1), + Row.of(3, 3.0, "l-3-2", 3.0, "r-3-2", 3), + Row.of(5, 5.0, "l-5-2", 5.0, "r-5-1", 5)) + .build()) + .runSql( + "insert into snk " + + "select * from leftSrc join rightSrc " + + "on a1 = b1") + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_JOIN_KEY_CONTAINS_INDEX = + TableTestProgram.of( + "delta-join-with-join-key-contains-index", + "validates delta join with join key contains index") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSources( + DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX.getSetupSourceTestSteps()) + .setupTableSinks(DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX.getSetupSinkTestSteps()) + .runSql( + "insert into snk " + + "select * from leftSrc join rightSrc " + + "on a1 = b1 and a0 = b0") + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_NON_EQUIV_CONDITION = + TableTestProgram.of( + "delta-join-with-with-non-equiv-condition", + "validates delta join with non equiv condition") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSource( + SourceTestStep.newBuilder("leftSrc") + .addSchema(LEFT_TABLE_BASE_SCHEMA) + .addOptions(TABLE_BASE_OPTIONS) + .addIndex("a1") + .treatDataBeforeRestoreAsFullStageData() + .producedBeforeRestore( + Row.of(1, 1.0, "Tim"), + Row.of(1, 1.0, "Sandy"), + Row.of(5, 5.0, "Bob")) + .producedAfterRestore( + Row.of(3, 3.0, "Lilith"), + Row.of(3, 3.1, "Lilith"), + Row.of(5, 5.0, "Jim")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("rightSrc") + .addSchema(RIGHT_TABLE_BASE_SCHEMA) + .addOptions(TABLE_BASE_OPTIONS) + .addIndex("b1") + .treatDataBeforeRestoreAsFullStageData() + .producedBeforeRestore( + Row.of(5.0, "Mark", 5), Row.of(3.0, "Lilith", 3)) + .producedAfterRestore( + Row.of(3.2, "Lilith", 3), Row.of(1.0, "Tim", 1)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("snk") + .addSchema(addPk2Schema(SINK_TABLE_BASE_SCHEMA, "a1", "b1")) + .addOptions(TABLE_BASE_OPTIONS) + .testMaterializedData() + // deduplicate data by pk + .deduplicatedFieldIndices(new int[] {0, 5}) + .consumedBeforeRestore(Row.of(5, 5.0, "Bob", 5.0, "Mark", 5)) + .consumedAfterRestore( + Row.of(1, 1.0, "Sandy", 1.0, "Tim", 1), + Row.of(5, 5.0, "Jim", 5.0, "Mark", 5)) + .build()) + .runSql( + "insert into snk " + + "select * from leftSrc join rightSrc " + + "on a1 = b1 and a2 <> b2") + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_CALC_ON_SOURCE = + TableTestProgram.of( + "delta-join-with-calc-on-source", + "validates delta join with calc on source") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSources( + DELTA_JOIN_WITH_JOIN_KEY_EQUALS_INDEX.getSetupSourceTestSteps()) + .setupTableSink( + SinkTestStep.newBuilder("snk") + .addSchema(addPk2Schema(SINK_TABLE_BASE_SCHEMA, "a0", "b0")) + .addOptions(TABLE_BASE_OPTIONS) + .testMaterializedData() + // deduplicate data by pk + .deduplicatedFieldIndices(new int[] {1, 3}) + .consumedBeforeRestore(Row.of(6, 5.0, "l-5-1", 5.0, "r-5-1", 6)) + .consumedAfterRestore(Row.of(6, 5.0, "l-5-2", 5.0, "r-5-1", 6)) + .build()) + .runSql( + "insert into snk " + + "select new_a1, a0, a2, b0, b2, new_b1 from ( " + + " select a0, a1, a2, a1 + 1 as new_a1 from leftSrc " + + " where a1 = 1 or a1 = 5 " + + ") join (" + + " select b0, b1, b1 + 1 as new_b1, b2 from rightSrc " + + " where b0 = cast(3.0 as double) or b0 = cast(5.0 as double) " + + ") " + + "on a1 = b1 and a0 = b0") + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_CALC_ON_SOURCE_AND_FILTER_PUSHED_DOWN = + TableTestProgram.of( + "delta-join-with-calc-on-source-and-filter-pushed-down", + "validates 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( + DELTA_JOIN_WITH_CALC_ON_SOURCE.getSetupSourceTestSteps().stream() + .map( + sourceTestStep -> { + String filterableFields; + if (sourceTestStep.name.equals("leftSrc")) { + filterableFields = "a1"; + } else if (sourceTestStep.name.equals("rightSrc")) { + filterableFields = "b0"; + } else { + throw new IllegalStateException( + "Unknown test table name: " + + sourceTestStep.name); + } + Map oldOptions = + new HashMap<>(sourceTestStep.options); + oldOptions.put( + "filterable-fields", filterableFields); + return sourceTestStep.withNewOptions(oldOptions); + }) + .collect(Collectors.toList())) + .setupTableSinks(DELTA_JOIN_WITH_CALC_ON_SOURCE.getSetupSinkTestSteps()) + .runSql(DELTA_JOIN_WITH_CALC_ON_SOURCE.getRunSqlTestStep().sql) + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_CACHE = + TableTestProgram.of("delta-join-with-cache", "validates delta join with cache") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupConfig(ExecutionConfigOptions.TABLE_EXEC_DELTA_JOIN_CACHE_ENABLED, true) + .setupTableSources( + DELTA_JOIN_WITH_NON_EQUIV_CONDITION.getSetupSourceTestSteps()) + .setupTableSinks(DELTA_JOIN_WITH_NON_EQUIV_CONDITION.getSetupSinkTestSteps()) + .runSql(DELTA_JOIN_WITH_NON_EQUIV_CONDITION.getRunSqlTestStep().sql) + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_CACHE_AND_CALC_ON_SOURCE = + TableTestProgram.of( + "delta-join-with-cache-and-calc-on-source", + "validates 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(DELTA_JOIN_WITH_CALC_ON_SOURCE.getSetupSourceTestSteps()) + .setupTableSinks(DELTA_JOIN_WITH_CALC_ON_SOURCE.getSetupSinkTestSteps()) + .runSql(DELTA_JOIN_WITH_CALC_ON_SOURCE.getRunSqlTestStep().sql) + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_CDC_SOURCE_WITHOUT_DELETE = + TableTestProgram.of( + "delta-join-with-cdc-source-without-delete", + "validates delta join with cdc source without delete") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupTableSource( + SourceTestStep.newBuilder("leftSrc") + .addSchema(addPk2Schema(LEFT_TABLE_BASE_SCHEMA, "a1", "a0")) + .addOptions(TABLE_BASE_OPTIONS) + .addOption("changelog-mode", "I,UA,UB") + .addIndex("a1") + .treatDataBeforeRestoreAsFullStageData() + .producedBeforeRestore( + // insert pk1 + Row.ofKind(RowKind.INSERT, 1, 1.0, "left-pk1-1"), + // update pk1 + Row.ofKind(RowKind.UPDATE_BEFORE, 1, 1.0, "left-pk1-1"), + Row.ofKind(RowKind.UPDATE_AFTER, 1, 1.0, "left-pk1-2"), + // insert pk2 + Row.ofKind(RowKind.INSERT, 1, 2.0, "left-pk2-1")) + .producedAfterRestore( + // update pk2 + Row.ofKind(RowKind.UPDATE_BEFORE, 1, 2.0, "left-pk2-1"), + Row.ofKind(RowKind.UPDATE_AFTER, 1, 2.0, "left-pk2-2"), + // insert pk3 + Row.ofKind(RowKind.INSERT, 3, 3.0, "left-pk3")) + .build()) + .setupTableSource( + SourceTestStep.newBuilder("rightSrc") + .addSchema(addPk2Schema(RIGHT_TABLE_BASE_SCHEMA, "b0", "b1")) + .addOptions(TABLE_BASE_OPTIONS) + .addOption("changelog-mode", "I,UA,UB") + .addIndex("b1") + .treatDataBeforeRestoreAsFullStageData() + .producedBeforeRestore( + // insert pk1 + Row.ofKind(RowKind.INSERT, 2.0, "right-pk1-1", 1), + // insert pk2 + Row.ofKind(RowKind.INSERT, 1.0, "right-pk2", 1)) + .producedAfterRestore( + // update pk1 + Row.ofKind( + RowKind.UPDATE_BEFORE, 2.0, "right-pk1-1", 1), + Row.ofKind(RowKind.UPDATE_AFTER, 2.0, "right-pk1-2", 1), + // insert pk3 + Row.ofKind(RowKind.INSERT, 3.0, "right-pk3-1", 3), + // update pk3 + Row.ofKind( + RowKind.UPDATE_BEFORE, 3.0, "right-pk3-1", 3), + Row.ofKind(RowKind.UPDATE_AFTER, 3.0, "right-pk3-2", 3)) + .build()) + .setupTableSink( + SinkTestStep.newBuilder("snk") + .addSchema( + addPk2Schema( + SINK_TABLE_BASE_SCHEMA, "a0", "b0", "a1", "b1")) + .addOptions(TABLE_BASE_OPTIONS) + .testMaterializedData() + // deduplicate data by pk + .deduplicatedFieldIndices(new int[] {0, 1, 3, 5}) + .consumedBeforeRestore( + Row.of(1, 1.0, "left-pk1-2", 2.0, "right-pk1-1", 1), + Row.of(1, 2.0, "left-pk2-1", 2.0, "right-pk1-1", 1), + Row.of(1, 1.0, "left-pk1-2", 1.0, "right-pk2", 1), + Row.of(1, 2.0, "left-pk2-1", 1.0, "right-pk2", 1)) + .consumedAfterRestore( + Row.of(1, 1.0, "left-pk1-2", 2.0, "right-pk1-2", 1), + Row.of(1, 2.0, "left-pk2-2", 2.0, "right-pk1-2", 1), + Row.of(1, 2.0, "left-pk2-2", 1.0, "right-pk2", 1), + Row.of(3, 3.0, "left-pk3", 3.0, "right-pk3-2", 3)) + .build()) + .runSql( + "insert into snk " + + "select * from leftSrc join rightSrc " + + "on a1 = b1") + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_CALC_ON_CDC_SOURCE_WITHOUT_DELETE = + TableTestProgram.of( + "delta-join-with-calc-on-cdc-source-without-delete", + "validates delta join with calc on cdc source without delete ") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupConfig(ExecutionConfigOptions.TABLE_EXEC_DELTA_JOIN_CACHE_ENABLED, true) + .setupTableSources( + DELTA_JOIN_WITH_CDC_SOURCE_WITHOUT_DELETE.getSetupSourceTestSteps()) + .setupTableSink( + SinkTestStep.newBuilder("snk") + .addSchema( + addPk2Schema( + SINK_TABLE_BASE_SCHEMA, "a0", "b0", "a1", "b1")) + .addOptions(TABLE_BASE_OPTIONS) + .testMaterializedData() + // deduplicate data by pk + .deduplicatedFieldIndices(new int[] {0, 1, 3, 5}) + .consumedBeforeRestore( + Row.of(1, 1.0, "left-pk1-2-s", 2.0, "right-pk1-1-s", 1), + Row.of(1, 2.0, "left-pk2-1-s", 2.0, "right-pk1-1-s", 1)) + .consumedAfterRestore( + Row.of(1, 1.0, "left-pk1-2-s", 2.0, "right-pk1-2-s", 1), + Row.of(1, 2.0, "left-pk2-2-s", 2.0, "right-pk1-2-s", 1)) + .build()) + .runSql( + "insert into snk " + + "select a1, a0, new_a2, b0, new_b2, b1 from ( " + + " select a0, a1, a2, concat(a2, '-s') as new_a2 from leftSrc " + + " where a0 = cast(1.0 as double) or a0 = cast(2.0 as double) " + + ") join (" + + " select b0, b1, concat(b2, '-s') as new_b2, b2 from rightSrc " + + " where b0 = cast(2.0 as double) or b0 = cast(3.0 as double) " + + ") " + + "on a1 = b1") + .build(); + + public static final TableTestProgram DELTA_JOIN_WITH_CACHE_AND_CDC_SOURCE_WITHOUT_DELETE = + TableTestProgram.of( + "delta-join-with-cache-and-cdc-source-without-delete", + "validates delta join with cache and cdc source without delete") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupConfig(ExecutionConfigOptions.TABLE_EXEC_DELTA_JOIN_CACHE_ENABLED, true) + .setupTableSources( + DELTA_JOIN_WITH_CDC_SOURCE_WITHOUT_DELETE.getSetupSourceTestSteps()) + .setupTableSinks( + DELTA_JOIN_WITH_CDC_SOURCE_WITHOUT_DELETE.getSetupSinkTestSteps()) + .runSql(DELTA_JOIN_WITH_CDC_SOURCE_WITHOUT_DELETE.getRunSqlTestStep().sql) + .build(); + + public static final TableTestProgram + DELTA_JOIN_WITH_CACHE_AND_CALC_ON_CDC_SOURCE_WITHOUT_DELETE = + TableTestProgram.of( + "delta-join-with-cache-and-calc-on-cdc-source-without-delete", + "validates delta join with cache and calc on cdc source without delete") + .setupConfig( + OptimizerConfigOptions.TABLE_OPTIMIZER_DELTA_JOIN_STRATEGY, + OptimizerConfigOptions.DeltaJoinStrategy.FORCE) + .setupConfig( + ExecutionConfigOptions.TABLE_EXEC_DELTA_JOIN_CACHE_ENABLED, + true) + .setupTableSources( + DELTA_JOIN_WITH_CALC_ON_CDC_SOURCE_WITHOUT_DELETE + .getSetupSourceTestSteps()) + .setupTableSinks( + DELTA_JOIN_WITH_CALC_ON_CDC_SOURCE_WITHOUT_DELETE + .getSetupSinkTestSteps()) + .runSql( + DELTA_JOIN_WITH_CALC_ON_CDC_SOURCE_WITHOUT_DELETE + .getRunSqlTestStep() + .sql) + .build(); + + private static String[] addPk2Schema(String[] originalSchema, String... pkCols) { + return Stream.concat( + Arrays.stream(originalSchema), + Stream.of( + Arrays.stream(pkCols) + .collect( + Collectors.joining( + ", ", "primary key (", ") not enforced")))) + .toArray(String[]::new); + } +} diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java index d1b7525ba3122..e550fc039b8ff 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/testutils/RestoreTestBase.java @@ -244,7 +244,7 @@ private void registerSinkObserver( if (!ignoreAfter) { expected.addAll(sinkTestStep.getExpectedAfterRestoreAsStrings()); } - final List actual = getExpectedResults(sinkTestStep, tableName); + final List actual = getActualResults(sinkTestStep, tableName); final boolean shouldComplete = CollectionUtils.isEqualCollection(actual, expected); if (shouldComplete) { @@ -355,7 +355,17 @@ void testRestore(TableTestProgram program, Path planPath, String savepointPath) afterRestoreSource == AfterRestoreSource.NO_RESTORE ? sourceTestStep.dataBeforeRestore : sourceTestStep.dataAfterRestore; + final String id = TestValuesTableFactory.registerData(data); + + if (sourceTestStep.treatDataBeforeRestoreAsFullStageData) { + final Collection fullStageData = + afterRestoreSource == AfterRestoreSource.NO_RESTORE + ? Collections.emptyList() + : sourceTestStep.dataBeforeRestore; + TestValuesTableFactory.registerFullStageData(fullStageData, id); + } + final Map options = new HashMap<>(); options.put("connector", "values"); options.put("data-id", id); @@ -401,18 +411,11 @@ void testRestore(TableTestProgram program, Path planPath, String savepointPath) } else { compiledPlan.execute().await(); for (SinkTestStep sinkTestStep : program.getSetupSinkTestSteps()) { - List expectedResults = getExpectedResults(sinkTestStep, sinkTestStep.name); - assertThat(expectedResults) + List actualResults = getActualResults(sinkTestStep, sinkTestStep.name); + assertThat(actualResults) .as("%s", program.id) .containsExactlyInAnyOrder( - Stream.concat( - sinkTestStep - .getExpectedBeforeRestoreAsStrings() - .stream(), - sinkTestStep - .getExpectedAfterRestoreAsStrings() - .stream()) - .toArray(String[]::new)); + sinkTestStep.getExpectedAsStrings().toArray(new String[0])); } } } @@ -428,7 +431,7 @@ private String getTestResourceDirectory(TableTestProgram program, ExecNodeMetada System.getProperty("user.dir"), metadata.name(), metadata.version(), program.id); } - private static List getExpectedResults(SinkTestStep sinkTestStep, String tableName) { + private static List getActualResults(SinkTestStep sinkTestStep, String tableName) { if (sinkTestStep.shouldTestChangelogData()) { return TestValuesTableFactory.getRawResultsAsStrings(tableName); } else { diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-cdc-source-without-delete/plan/delta-join-with-cache-and-calc-on-cdc-source-without-delete.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-cdc-source-without-delete/plan/delta-join-with-cache-and-calc-on-cdc-source-without-delete.json new file mode 100644 index 0000000000000..beed3bd5cdae3 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-cdc-source-without-delete/plan/delta-join-with-cache-and-calc-on-cdc-source-without-delete.json @@ -0,0 +1,669 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 79, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 80, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 81, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Calc(select=[a0, a1, CONCAT(a2, '-s') AS new_a2], where=[SEARCH(a0, Sarg[1.0, 2.0])])" + }, { + "id" : 82, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 83, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 84, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 85, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, CONCAT(b2, '-s') AS new_b2], where=[SEARCH(b0, Sarg[2.0, 3.0])])" + }, { + "id" : 86, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 87, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1 ], + "leftUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "rightJoinKeys" : [ 1 ], + "rightUpsertKey" : [ 0, 1 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], select=[a0, a1, new_a2, b0, b1, new_b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 88, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Calc(select=[a1, a0, new_a2, b0, new_b2, b1])" + }, { + "id" : 89, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, new_a2, b0, new_b2, b1])" + } ], + "edges" : [ { + "source" : 79, + "target" : 80, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 80, + "target" : 81, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 81, + "target" : 82, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 83, + "target" : 84, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 84, + "target" : 85, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 85, + "target" : 86, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 82, + "target" : 87, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 86, + "target" : 87, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 87, + "target" : 88, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 88, + "target" : 89, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000..41d0181f0421e Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-cdc-source-without-delete/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-source/plan/delta-join-with-cache-and-calc-on-source.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-source/plan/delta-join-with-cache-and-calc-on-source.json new file mode 100644 index 0000000000000..7fb0b5b96dfbc --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-calc-on-source/plan/delta-join-with-cache-and-calc-on-source.json @@ -0,0 +1,636 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 43, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 44, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Calc(select=[a0, a1, a2, (a1 + 1) AS new_a1], where=[SEARCH(a1, Sarg[1, 5])])" + }, { + "id" : 45, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 46, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 47, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, (b1 + 1) AS new_b1, b2], where=[SEARCH(b0, Sarg[3.0, 5.0])])" + }, { + "id" : 48, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 49, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1, 0 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "rightJoinKeys" : [ 1, 0 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT, `b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], select=[a0, a1, a2, new_a1, b0, b1, new_b1, b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 50, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Calc(select=[new_a1, a0, a2, b0, b2, new_b1])" + }, { + "id" : 51, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[new_a1, a0, a2, b0, b2, new_b1])" + } ], + "edges" : [ { + "source" : 43, + "target" : 44, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 44, + "target" : 45, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 46, + "target" : 47, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 47, + "target" : 48, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 45, + "target" : 49, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 48, + "target" : 49, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 49, + "target" : 50, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 50, + "target" : 51, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/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_1/delta-join-with-cache-and-calc-on-source/savepoint/_metadata new file mode 100644 index 0000000000000..c6827672128da Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/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_1/delta-join-with-cache-and-cdc-source-without-delete/plan/delta-join-with-cache-and-cdc-source-without-delete.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-cdc-source-without-delete/plan/delta-join-with-cache-and-cdc-source-without-delete.json new file mode 100644 index 0000000000000..b31aedde057fe --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-cdc-source-without-delete/plan/delta-join-with-cache-and-cdc-source-without-delete.json @@ -0,0 +1,338 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 71, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 72, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 73, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 74, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 75, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 76, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 77, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "leftUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "rightJoinKeys" : [ 2 ], + "rightUpsertKey" : [ 0, 2 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : null + }, + "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)], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 78, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 71, + "target" : 72, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 72, + "target" : 73, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 74, + "target" : 75, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 75, + "target" : 76, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 73, + "target" : 77, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 76, + "target" : 77, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 77, + "target" : 78, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000..4f0e8e1dbfa94 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache-and-cdc-source-without-delete/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache/plan/delta-join-with-cache.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache/plan/delta-join-with-cache.json new file mode 100644 index 0000000000000..25adae9cfabd8 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache/plan/delta-join-with-cache.json @@ -0,0 +1,305 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 37, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 38, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 39, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 40, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 41, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "rightJoinKeys" : [ 2 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a2 <> b2))], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 42, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 37, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 39, + "target" : 40, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 38, + "target" : 41, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 40, + "target" : 41, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 41, + "target" : 42, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache/savepoint/_metadata new file mode 100644 index 0000000000000..873636efd9aa0 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-cdc-source-without-delete/plan/delta-join-with-calc-on-cdc-source-without-delete.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-cdc-source-without-delete/plan/delta-join-with-calc-on-cdc-source-without-delete.json new file mode 100644 index 0000000000000..710506117ad5f --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-cdc-source-without-delete/plan/delta-join-with-calc-on-cdc-source-without-delete.json @@ -0,0 +1,669 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 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-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Calc(select=[a0, a1, CONCAT(a2, '-s') AS new_a2], where=[SEARCH(a0, Sarg[1.0, 2.0])])" + }, { + "id" : 4, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 5, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 6, + "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" : 7, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, CONCAT(b2, '-s') AS new_b2], where=[SEARCH(b0, Sarg[2.0, 3.0])])" + }, { + "id" : 8, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 9, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1 ], + "leftUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "rightJoinKeys" : [ 1 ], + "rightUpsertKey" : [ 0, 1 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT NOT NULL" + }, { + "kind" : "CALL", + "internalName" : "$CONCAT$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "LITERAL", + "value" : "-s", + "type" : "CHAR(2) NOT NULL" + } ], + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "1.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "1.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "2.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "2.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN NOT NULL" + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE NOT NULL, `a1` INT NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b1` INT NOT NULL, `new_b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], select=[a0, a1, new_a2, b0, b1, new_b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 10, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "INT NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "DOUBLE NOT NULL" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 5, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "INT NOT NULL" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Calc(select=[a1, a0, new_a2, b0, new_b2, b1])" + }, { + "id" : 11, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `new_a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `new_b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, new_a2, b0, new_b2, b1])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 3, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 3, + "target" : 4, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 5, + "target" : 6, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 6, + "target" : 7, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 4, + "target" : 9, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 9, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "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_1/delta-join-with-calc-on-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000..be94da0751fb7 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-cdc-source-without-delete/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source-and-filter-pushed-down/plan/delta-join-with-calc-on-source-and-filter-pushed-down.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source-and-filter-pushed-down/plan/delta-join-with-calc-on-source-and-filter-pushed-down.json new file mode 100644 index 0000000000000..2a210a97b51d5 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source-and-filter-pushed-down/plan/delta-join-with-calc-on-source-and-filter-pushed-down.json @@ -0,0 +1,636 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 28, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 29, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Calc(select=[a0, a1, a2, (a1 + 1) AS new_a1], where=[SEARCH(a1, Sarg[1, 5])])" + }, { + "id" : 30, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 31, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 32, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, (b1 + 1) AS new_b1, b2], where=[SEARCH(b0, Sarg[3.0, 5.0])])" + }, { + "id" : 33, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 34, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1, 0 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "rightJoinKeys" : [ 1, 0 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT, `b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], select=[a0, a1, a2, new_a1, b0, b1, new_b1, b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 35, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Calc(select=[new_a1, a0, a2, b0, b2, new_b1])" + }, { + "id" : 36, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[new_a1, a0, a2, b0, b2, new_b1])" + } ], + "edges" : [ { + "source" : 28, + "target" : 29, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 29, + "target" : 30, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 31, + "target" : 32, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 32, + "target" : 33, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 30, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 33, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 34, + "target" : 35, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 35, + "target" : 36, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/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_1/delta-join-with-calc-on-source-and-filter-pushed-down/savepoint/_metadata new file mode 100644 index 0000000000000..9f2a8ab906d51 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/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_1/delta-join-with-calc-on-source/plan/delta-join-with-calc-on-source.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source/plan/delta-join-with-calc-on-source.json new file mode 100644 index 0000000000000..3eee97dab63e3 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source/plan/delta-join-with-calc-on-source.json @@ -0,0 +1,636 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 19, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc, filter=[]]], fields=[a1, a0, a2])" + }, { + "id" : 20, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Calc(select=[a0, a1, a2, (a1 + 1) AS new_a1], where=[SEARCH(a1, Sarg[1, 5])])" + }, { + "id" : 21, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 22, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc, filter=[]]], fields=[b0, b2, b1])" + }, { + "id" : 23, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "condition" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Calc(select=[b0, b1, (b1 + 1) AS new_b1, b2], where=[SEARCH(b0, Sarg[3.0, 5.0])])" + }, { + "id" : 24, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 1, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 25, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 1, 0 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 0 + }, + "2" : { + "type" : "FieldRef", + "index" : 1 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "VARCHAR(2147483647)" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : "3.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "3.0", + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : "5.0", + "boundType" : "CLOSED" + }, + "upper" : { + "value" : "5.0", + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "DOUBLE NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "rightJoinKeys" : [ 1, 0 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + }, + "abilities" : [ { + "type" : "FilterPushDown", + "predicates" : [ ] + } ] + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null, + "projectionOnTemporalTable" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 1, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$+$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "value" : 1, + "type" : "INT NOT NULL" + } ], + "type" : "INT" + } ], + "filterOnTemporalTable" : { + "kind" : "CALL", + "syntax" : "INTERNAL", + "internalName" : "$SEARCH$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "INT" + }, { + "kind" : "LITERAL", + "sarg" : { + "ranges" : [ { + "lower" : { + "value" : 1, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 1, + "boundType" : "CLOSED" + } + }, { + "lower" : { + "value" : 5, + "boundType" : "CLOSED" + }, + "upper" : { + "value" : 5, + "boundType" : "CLOSED" + } + } ], + "nullAs" : "UNKNOWN" + }, + "type" : "INT NOT NULL" + } ], + "type" : "BOOLEAN" + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a0` DOUBLE, `a1` INT, `a2` VARCHAR(2147483647), `new_a1` INT, `b0` DOUBLE, `b1` INT, `new_b1` INT, `b2` VARCHAR(2147483647)>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], select=[a0, a1, a2, new_a1, b0, b1, new_b1, b2])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 26, + "type" : "stream-exec-calc_1", + "projection" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 3, + "type" : "INT" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 0, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "DOUBLE" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 7, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 6, + "type" : "INT" + } ], + "condition" : null, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Calc(select=[new_a1, a0, a2, b0, b2, new_b1])" + }, { + "id" : 27, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`new_a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `new_b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[new_a1, a0, a2, b0, b2, new_b1])" + } ], + "edges" : [ { + "source" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 21, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 22, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 23, + "target" : 24, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 21, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 24, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 25, + "target" : 26, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 26, + "target" : 27, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source/savepoint/_metadata new file mode 100644 index 0000000000000..021c454cea252 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/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_1/delta-join-with-cdc-source-without-delete/plan/delta-join-with-cdc-source-without-delete.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cdc-source-without-delete/plan/delta-join-with-cdc-source-without-delete.json new file mode 100644 index 0000000000000..4290d15687017 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cdc-source-without-delete/plan/delta-join-with-cdc-source-without-delete.json @@ -0,0 +1,338 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 52, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 53, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "DropUpdateBefore" + }, { + "id" : 54, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 55, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 56, + "type" : "stream-exec-drop-update-before_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "DropUpdateBefore" + }, { + "id" : 57, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 58, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "leftUpsertKey" : [ 0, 1 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_b0_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "b0", "b1" ] + }, + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "rightJoinKeys" : [ 2 ], + "rightUpsertKey" : [ 0, 2 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "primaryKey" : { + "name" : "PK_a1_a0", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "a0" ] + }, + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : null + }, + "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)], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 59, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a0_b0_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0", "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", + "inputUpsertKey" : [ 0, 1, 3, 5 ], + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647), `b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 52, + "target" : 53, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 53, + "target" : 54, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 55, + "target" : 56, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 56, + "target" : 57, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 54, + "target" : 58, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 57, + "target" : 58, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 58, + "target" : 59, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cdc-source-without-delete/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cdc-source-without-delete/savepoint/_metadata new file mode 100644 index 0000000000000..ec2d5e9be7083 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cdc-source-without-delete/savepoint/_metadata differ diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-contains-index/plan/delta-join-with-join-key-contains-index.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-contains-index/plan/delta-join-with-join-key-contains-index.json new file mode 100644 index 0000000000000..3e3a5d5ca434a --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-contains-index/plan/delta-join-with-join-key-contains-index.json @@ -0,0 +1,285 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 7, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 8, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0, 1 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1, a0]])" + }, { + "id" : 9, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 10, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2, 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1, b0]])" + }, { + "id" : 11, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0, 1 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 1 + }, + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "rightJoinKeys" : [ 2, 0 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + }, + "1" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a0 = b0))], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 12, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 7, + "target" : 8, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 9, + "target" : 10, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 8, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 10, + "target" : 11, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 11, + "target" : 12, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-contains-index/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-contains-index/savepoint/_metadata new file mode 100644 index 0000000000000..f9d56a3df4023 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/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_1/delta-join-with-join-key-equals-index/plan/delta-join-with-join-key-equals-index.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-equals-index/plan/delta-join-with-join-key-equals-index.json new file mode 100644 index 0000000000000..ff6f5a9dbb654 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-equals-index/plan/delta-join-with-join-key-equals-index.json @@ -0,0 +1,277 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 1, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 2, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 3, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 4, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 5, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : null + }, + "rightJoinKeys" : [ 2 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : null + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[(a1 = b1)], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 6, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE NOT NULL" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "primaryKey" : { + "name" : "PK_a0_b0", + "type" : "PRIMARY_KEY", + "columns" : [ "a0", "b0" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 1, + "target" : 2, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 3, + "target" : 4, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 2, + "target" : 5, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 4, + "target" : 5, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 5, + "target" : 6, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-equals-index/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-equals-index/savepoint/_metadata new file mode 100644 index 0000000000000..aece6ec4018d8 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/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_1/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json new file mode 100644 index 0000000000000..fad8b5770edc9 --- /dev/null +++ b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json @@ -0,0 +1,305 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 13, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 14, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 15, + "type" : "stream-exec-table-source-scan_1", + "scanTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" + }, { + "id" : 16, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 2 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Exchange(distribution=[hash[b1]])" + }, { + "id" : 17, + "type" : "stream-exec-delta-join_1", + "configuration" : { + "table.exec.async-lookup.buffer-capacity" : "100", + "table.exec.async-lookup.timeout" : "3 min" + }, + "joinType" : "INNER", + "leftJoinKeys" : [ 0 ], + "lookupRightTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`rightSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT" + } ], + "indexes" : [ { + "name" : "INDEX_b1", + "columns" : [ "b1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT> NOT NULL" + }, + "lookupKeys" : { + "2" : { + "type" : "FieldRef", + "index" : 0 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "rightJoinKeys" : [ 2 ], + "lookupLeftTableJoinSpec" : { + "lookupTable" : { + "lookupTableSource" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`leftSrc`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + } ], + "indexes" : [ { + "name" : "INDEX_a1", + "columns" : [ "a1" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)> NOT NULL" + }, + "lookupKeys" : { + "0" : { + "type" : "FieldRef", + "index" : 2 + } + }, + "remainingCondition" : { + "kind" : "CALL", + "syntax" : "BINARY", + "internalName" : "$<>$1", + "operands" : [ { + "kind" : "INPUT_REF", + "inputIndex" : 2, + "type" : "VARCHAR(2147483647)" + }, { + "kind" : "INPUT_REF", + "inputIndex" : 4, + "type" : "VARCHAR(2147483647)" + } ], + "type" : "BOOLEAN" + } + }, + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + }, { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "DeltaJoin(joinType=[InnerJoin], where=[((a1 = b1) AND (a2 <> b2))], select=[a1, a0, a2, b0, b2, b1])", + "asyncOptions" : { + "capacity " : 100, + "timeout" : 180000, + "is-key-ordered" : true, + "output-mode" : "ORDERED" + } + }, { + "id" : 18, + "type" : "stream-exec-sink_1", + "configuration" : { + "table.exec.sink.keyed-shuffle" : "AUTO", + "table.exec.sink.not-null-enforcer" : "ERROR", + "table.exec.sink.rowtime-inserter" : "ENABLED", + "table.exec.sink.type-length-enforcer" : "IGNORE", + "table.exec.sink.upsert-materialize" : "AUTO" + }, + "dynamicTableSink" : { + "table" : { + "identifier" : "`default_catalog`.`default_database`.`snk`", + "resolvedTable" : { + "schema" : { + "columns" : [ { + "name" : "a1", + "dataType" : "INT NOT NULL" + }, { + "name" : "a0", + "dataType" : "DOUBLE" + }, { + "name" : "a2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b0", + "dataType" : "DOUBLE" + }, { + "name" : "b2", + "dataType" : "VARCHAR(2147483647)" + }, { + "name" : "b1", + "dataType" : "INT NOT NULL" + } ], + "primaryKey" : { + "name" : "PK_a1_b1", + "type" : "PRIMARY_KEY", + "columns" : [ "a1", "b1" ] + } + } + } + } + }, + "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "UNKNOWN" + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647), `b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", + "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" + } ], + "edges" : [ { + "source" : 13, + "target" : 14, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 15, + "target" : 16, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 14, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 16, + "target" : 17, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 17, + "target" : 18, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-with-non-equiv-condition/savepoint/_metadata b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-with-non-equiv-condition/savepoint/_metadata new file mode 100644 index 0000000000000..9d06051f9b336 Binary files /dev/null and b/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-with-non-equiv-condition/savepoint/_metadata differ