From 0404af872b8f604eedbd086b36966669386dc4e5 Mon Sep 17 00:00:00 2001 From: xuyang Date: Tue, 14 Oct 2025 18:07:01 +0800 Subject: [PATCH 1/2] [FLINK-38504][table] Add restore tests for delta join --- .../table/test/program/SinkTestStep.java | 85 ++++- .../table/test/program/SourceTestStep.java | 25 +- .../table/test/program/TableTestProgram.java | 20 ++ .../table/test/program/TableTestStep.java | 56 ++- .../factories/TestValuesTableFactory.java | 31 +- .../exec/stream/DeltaJoinRestoreTest.java | 44 +++ .../exec/stream/DeltaJoinTestPrograms.java | 287 +++++++++++++++ .../nodes/exec/testutils/RestoreTestBase.java | 28 +- ...h-cache-and-cdc-source-without-delete.json | 337 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 14834 bytes .../plan/delta-join-with-cache.json | 304 ++++++++++++++++ .../delta-join-with-cache/savepoint/_metadata | Bin 0 -> 13782 bytes ...a-join-with-cdc-source-without-delete.json | 337 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 14834 bytes ...lta-join-with-join-key-contains-index.json | 284 +++++++++++++++ .../savepoint/_metadata | Bin 0 -> 14188 bytes ...delta-join-with-join-key-equals-index.json | 276 ++++++++++++++ .../savepoint/_metadata | Bin 0 -> 13768 bytes ...ta-join-with-with-non-equiv-condition.json | 304 ++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 13839 bytes 20 files changed, 2381 insertions(+), 37 deletions(-) create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java create mode 100644 flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cache/savepoint/_metadata create mode 100644 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 create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-cdc-source-without-delete/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-contains-index/plan/delta-join-with-join-key-contains-index.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-contains-index/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-equals-index/plan/delta-join-with-join-key-equals-index.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-join-key-equals-index/savepoint/_metadata create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-with-non-equiv-condition/plan/delta-join-with-with-non-equiv-condition.json create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-with-non-equiv-condition/savepoint/_metadata 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..3e433979a640a 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}. */ @@ -66,6 +71,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 +92,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 +109,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..48753223bcc95 --- /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,44 @@ +/* + * 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_CACHE, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_CDC_SOURCE_WITHOUT_DELETE, + DeltaJoinTestPrograms.DELTA_JOIN_WITH_CACHE_AND_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..ac576644da023 --- /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,287 @@ +/* + * 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.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_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_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_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(); + + 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..8b35564001edc 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) { @@ -326,6 +326,7 @@ public void generateTestSetupFiles(TableTestProgram program) throws Exception { Files.move(savepointPath, savepointDirPath, StandardCopyOption.ATOMIC_MOVE); } + // @Disabled @ParameterizedTest @MethodSource("createSpecs") @Order(1) @@ -355,7 +356,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 +412,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 +432,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-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..dc46cc9e8bb73 --- /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,337 @@ +{ + "flinkVersion" : "2.2", + "nodes" : [ { + "id" : 33, + "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" : 34, + "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" : 35, + "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" : 36, + "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" : 37, + "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" : 38, + "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" : 39, + "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" : 40, + "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" ], + "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" : 33, + "target" : 34, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 34, + "target" : 35, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 36, + "target" : 37, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 37, + "target" : 38, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 35, + "target" : 39, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 38, + "target" : 39, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 39, + "target" : 40, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_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 0000000000000000000000000000000000000000..d054f347b6a70e396d0524002b45ebd2e87e3a7a GIT binary patch literal 14834 zcmeHOeQX>@6`wo1PMp-W<2We^O=YQWAdtP=+uQq!D>Y~PLaw>k#&=FzTj7}7owGOI z+r94YId+6XLZlX{5=eyzf<(z5p$I8RNL57tK|-j206|pX9|{tIe~>7s5k#uOhr*lL zUEkh^-LsoSseCKl&F*~s=FOWo@6Eoqlm6@+hY;$5AH95bg0`T6tX?rBa%j*{=H{)e zAj~1oDU7g1wj%-QVfN|Q5na0Ar^`X zybvx##E1|VM5y|(r!U&2+pn*aJ?yYMpCVW$M%Ka>p_QQ4D@8>mhk}dP2o`k}3eBJp z2^xICD4dp;LXk7P7+Q{=j)a!XLfPQqnkxr$!BZ7YvA9quk>C>%J{Aqd65*H-6T?9h zFOssZXckup3t}YBMa4ovD)Lx}^Wk75CKgCh409sfaAGu+;NnCGadBBJ#Du6EPT&O4 z^*8td2Ysk1-E?fgm5IsJR5CX)mCxpqxm12Eoy|?9N2YV>iShjLiLn&w%T8v}xoSa* zwd*^P%uJ{9+0;}znMu#2rXbgk4BeOu;4+rxNua1I+5$W%0ZA{FbS+>lmr2D^R5MV( zCJA7}z{`P=Ws6K2N=dPlMUtdhSz;(yRn8D&XsyDmhRf!>Zb82t@T&R=;-WkKg$&G^M;aMk>QLaJrTDYK+ zfKkybr9?c+cS=7!hAr$-vkf_Qy482}ALt>|KnGXpgDQ42pz3pqgw;S=vvQE(h>6dS z(2{}pmYQa4h}!1<8MI9?$1AEz2b4iuX?Ih)ZdoXr0ob5zZ-UI;1lg#AjG~}JQP2ZR z+YK+3(OCnD>m`Y{b%np5jp~t8co7Fwtjz^dTBYPIEL&(xdOVl<6j;WAXa5I9w!Iy1 z=yi1q37quIQq5D%4EfN0+q~tedCTi%Vqi-*^4MI~r2J{DSb39fR`TwgpuS_N=Tl>j z$=0aKl_(FN>qFW4X=wu)uSW<~vt=@n2@v^#gTZC(x7H8wFll1VxIYt06nz{co2 zHfkB@OQSB8Xmi&5gJ(;8_r;c+NKTC&O-_CO&Ye3Rz14y4g*|ChMXTE=TimM(d8j_4 zM)A$8vV>d>OP7+Z??H`%>aE+TG|Q~P-xONBt%ugD$lH1d)>U=bHgPwUY` z?M($WQtOq)CT|5DP2S!=#N8mF-J`l@ zS_X!=TuLkI5Qj8oMgAmuROIv0N)fjR;VYSX;AXn{?Kl4P=A~=5k6n22;*0@6%_B`K zJs>nNX6WVVGPEZn1o|Me;n+#ED;&$qP{37K2?yanDC(X5;SYBGbKo0`*9+I*eex0%Oe_ z-E-@$+j}1U+%Mj42IWLqAExy=gP0Z7GAU=a(LaRxpK{W#JLxSu-8t!`SM79QtCPx3 z`Xf8t^_Y_`-FJU}QNS3BB*Y0JEOTN!Byl(a@mnnM#bO*nz6Bu;{`}tofh=HK&jQ zq*rupRK+F$@34WXD?iAxhBIip6F;D2Sx8uStO4Q-iw4@;NFFmp5OlG{5q0>%`;NA= zv5qWLC1s&r&p?mk^6)I`b;>7o6Jq7MW}%(49-=1-r-)<$)tw89CR=E*3Awc2AoP0- zYCz3L$_|v<=^js~8CZivK8nqG+Wk_wXP4{WCiA7w!7 z40P0K8pY{?@kAP>n}r=MCgaWm3OF(`1nCjFqN~Eu0fH|RQ!m>u5M=Y?P|Kp(Br-{XdeB*#^i?BvOj7qUbRN(@h{LV617{QN!50Ak z?l{I-Prj9*cpNp=?g~7+A40V9eowx@?myw9`3I?!*;5u`_m8tSgG;o=9+vZw8cA4N z$tNwnq)4tZ-uuPufq$QQLZFcmisuJCxr2!xi>1OmkvU@Y|K;+P_s+i*2kA|thhS4x zA)q|^s`bVzr2;Wtyn1op_OIS}nSzzQq_X=sKzo|l-sy9%|M-~$3;*c^%Q)C93MP!Z2P-6~s$>(KW};FNNv(rLc2;>0!WW|)QA&+8BBH(hAXHbumbp*q)HF)lIpA}6p=e;@E zO2Mp@XpRe$t(aw^$uz5rg}~sZyP`=dhGPaYc$KTP(Rb>mj&o-i!D`i@YuRZV`hh;Z zdRNCG z=*UVl*Z*jtroPDm@S^cGYauUb=_CwcmgQynydZ!wpQJbO%=G4m6>g zu@L_HTR5uFJ(?V!7*CHTGY|@KW_WM~69@oy)E(?=B8n>w=qxlHjh80s3+o1-Yj!$2 InHnGaKgdoxegFUf literal 0 HcmV?d00001 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..430f2226bf1a2 --- /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,304 @@ +{ + "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" ] + } ] + } + } + } + }, + "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", + "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" + }, { + "id" : 20, + "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" : 21, + "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" : 22, + "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" : 23, + "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" : 24, + "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" ], + "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" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 21, + "target" : 22, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 22, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 23, + "target" : 24, + "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 0000000000000000000000000000000000000000..915966fd949484149a2da635d6d262e978bd549c GIT binary patch literal 13782 zcmeGjTWlOx_3qlSlcr9b#FQp&BO~01K<2eO`-&?y>-C0gyMAbQlhRmWxifRuo_Ka< znVEIgp$dswks>7cpa|sy2q<5WN=OBXkMhH#fDk|M2_F@zgoHpUqEZnp3g_OL@y>eJ z!Aa}T)H{+rv*+G>?z!ild!93e&n_m`sw`O$G-Ra7v7zJ;}brz%M@XNl|F~Bv9i?!#K*^N zeR0Vx+v-A0I)`?}q^iRjqQj{dpZn#W@BSuxvjg z!IGMhB`ixMDW}w=5?7(ygMN3ou`8e=G6+8-XrDKEiC~S`Wf!}|@AdVXZjj@`3buus zWk91N#1eunDz5zaSF-Q}cMER#;A;^|mC5l<&2 zNtQE$gI7qy0#0(Gl#)|vt|nq79>-WqC9t4KVltiyXR^i9`D!^|DrAd=v-uK0qsX@GMIJY>xM5{cuh`VEvYH0l$Mm5R8vx7CX-C2 z zual6CE?F0*v5Q02xC^q*1V?WE9qn*zbZmn;sQq_%!&=l;Z14rstpLE6lSmYymW{+s z4J~>oI)_Kkq6c(mwrLo2K}EEaPFJ!l*G0)9pcOo~L||@-z*a3ViUgk`A%vEWn`|{A zmuw{8DM@r}ApQH|2%Ex2n}3Hg&_R*jY8z>IJO3RfjsY)nHH1=+pUU^V7a(Sb26n zj|Pk2A3}Q+^>zTWcR+VhcS3}BkUH`g&*KiWB-ax)V=diB&$A5VxYBGGq|KEc$fM}U zZ4lDoJVPRf_GJ()c8!6Ehv2#ndmGH{L2RO0w#JuPbKW3z(y>h&zl~F{zz+P|LW}qH(48vsz8->e)%MWW`NO`QLYwkwLwcz1R8S*zr?S{1 zDj4Vyz1=SwcPs4Aw$S4JVEV3t>Ao)xIhMQ8JAE6t6*jQ%i-$9b9lrQhScb9L>`cBq zpUrg@D9?077C}tGQeXTTOV?&`oj6d#4CRj|Hh_1mByf<>NEs@h$HyIWnbp6Cq5kav zuL z>Fj*VCANu;Pzc%$>ex;{2kBdqPjv}SIID9~Cx+(GFqG=tw|;o@(U-4$ry50(BHB}< zZ9$XcqR}GFSqe0M6=(`{5^@!f-xhl?__eS7bL`vE=j%}vEuy{fO2rDiX`owX&cF^J z-sd5vd1iruCW~mdpWLQoxhQtWiEv1Quh{5NH+f7;LPdayBb4BykQzq2*;=Po=?QYt z-q1p4aAR@-?e*K|EeGoTmg%Ac3n8NCl=DP&f$G6!-PBw(-h*5^uK@Et4LzXeo`#Pq znDk(1(jpupqMgFd5}m%)*mP|I@Kc8QYlGSZ?Bful-T6Gagd!ij>=L>cm5PDV+%v$# zBP{1m<(JP{h5=g!Tj@-kHtUwA*K}5X0l9v6@8i;s|Mt~U#=SuUAOY`={ci+q#0RI) z<==kx?ZXUBM0L?ap&f2_^S1Xu8#8;Cgr2X~mES*h?N|6)0|?TCgATk{(rQp0b%B-` zhG0F3Zs}$e}H&jCMMv1lAh?ca14OpYs67)z2OP<{*X+2xhN{rCL9XNfk)t( zqNdM-tp-$pqFCCrX!a9Z_V1F;!AT~iHn85cCL-=EW|NY6@ zcFr^$$|AI>h$s}KYQxv@;ifoc{;#M>6m0g!rWOm z+>^-bZ{A2>oP1@fJ>z=Dr75-(-PkC2nt^+cc!D(roJJwfH_{`2v5~okQ$`sbvesRY z4r2```9qgyx+gk4$TRf==6=BJo4Wtoix44o-xtTeIDYWN@xzmhy{EP%(%whePW{xXDiS0NBe zItD^%#)b%jxK7#(6UxEYDs=eutAE&e<1=GFi6RJ$9{?Kbn7!%gqC|H-2i}n*0%;Ky z(g~I+tRXtNHY^C8UDPwI9)kps+unpMOGdcpRQFW9Rvo5$0Sv-G-$JcNI0l`c0szJ Le7ZcJpPl|Mu4S-{ literal 0 HcmV?d00001 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..47812cc71c093 --- /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,337 @@ +{ + "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 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" : 20, + "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" : 21, + "type" : "stream-exec-exchange_1", + "inputProperties" : [ { + "requiredDistribution" : { + "type" : "HASH", + "keys" : [ 0 ] + }, + "damBehavior" : "PIPELINED", + "priority" : 0 + } ], + "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", + "description" : "Exchange(distribution=[hash[a1]])" + }, { + "id" : 22, + "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" : 23, + "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" : 24, + "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" : 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" : [ 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" : 26, + "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" ], + "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" : 19, + "target" : 20, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 20, + "target" : 21, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 22, + "target" : 23, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 23, + "target" : 24, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 21, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 24, + "target" : 25, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + }, { + "source" : 25, + "target" : 26, + "shuffle" : { + "type" : "FORWARD" + }, + "shuffleMode" : "PIPELINED" + } ] +} \ No newline at end of file diff --git a/flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_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 0000000000000000000000000000000000000000..0ea956396b686f44ae866804aed77ac5c5a55545 GIT binary patch literal 14834 zcmeHOeP|rV6`wmP@>gWbw(Q1VL2VV&G`^4B`?do1NjlY+oF#F0auN&oI=ee3t+ls% z+1-;R7hIXrx}@MhXd!7y>lXjh;y@u3@`nq>&;;5)3TgUBOADbC`audVO$j6|?wi?N z-QMk;a#AAK!MCtCdv9ib^XAQ)H?!~UWWO{QKnV50O)sC7pe<-{T-T~IRSBD9l$14{ z%<_?QXdfRbNstk!v7CUwvQ~B;*(*Hvw|}4b%&jYb{`4ObHqZ(Ma}epX`|2Mjiiw}l zWObI0okRV6tRzuN;5?8D<+lr8TX_D3Kc4od^v9}ss7P^EC_n??7D|C@h5dGax)SJ0 z;J%cA54}*96>>bZi1koeQzW7rAwCw;!-6hcke0aE#c-5cj$epzONLO@!*I=2L&ebf znkt)tL@JpGN8;gRJerIqqI@_S3mJHkRKXaTfp9Xx#iN0ESV*LBSqk%_6iV^Qaw059 zfmjN2foKGN1R+%pNJ+v+B`G4Nq%cDJ2#f|Zu6|V1^tm9eVsV}X%ZjWn1Wha`B&gR^ zQ?8Jpxm+bRQ&x;%Ub`@kOhf{Lrgyk-J?58S>R|3D8IIy%;-4T^09 z6>Uxyu@cOxW)TRsqWJs>E$JxSQPrFU(OBJo7HyY}$(o|j4&~5R+FV}KOcTX(02{J; z9b&x>v0jH5MAOTC2D-t0vl# zoh)X)kjbOLr~XT&$nHr`tGFr$MXgfNRM#L0*dSDru2E|E*DyjpbkthjN_2TEX;q?Q zQ`1Y>SXRZ-1uUB-!-S=_WHabP15ak2$&8l@#dI-KDn2uvLA|+bp$PeYG;k)Jo5_?4 znS3^#%bo@PDK)(gzH_qSV!VOX{=f7xF}OuC))JbDmX4#PwT>PaYgL8RrP2c2kB+VZ zNUH}KNYphhy=cx;$1rt*D>96&)3!y?iKyvPu)wl23aOBWVQNEf$*5Zjn>m4dx)Ye% zl#NbcTY4dnidZI=8Jkt}H?GydOws0?*>?Y0z`pURnbBOvmS1OUz!|KM&11b$fW9p1 zQ;0fe&fj}$iSMqgC1=w4u~X^%SMJ=olXfxImuPJK3Mp^1!H^{@yga*TRw#^iB zEG(T$mc0iv3Z}Pdqtc4lfPY_D@va?OuOaW+A(&V79eUG#*j}eP+ta%2PC>6Qbb8F=pE~1tvvA%inCmNeMw6vUTp>GIsX zSCyS*XMu1TwM@9@}VKjCj^N)f0Ic}#CVJ& zaSo@VaS@C0cv6f((Fg6~PIFEy%4fOEc3X|-2?qCbf%@}U!MpC8PD-&@LWtu4PL&gZ zXgmtOX-N_Su|zl)6Qju(A5Q^ie}@}<(>~OaZ_61vG&P;cr;Ag0+Y>nsKFEA_bf%b{ znk=238qc7C!gMZMEUji_02SPqEnSqlM+Mq%qOh*x<>2VDNv3tVBAfCeNmG*RMRu*s z*6P8=1EIZ+mMEmml!Qxxl5i-#zr}h2rl=fA-_atB1cc(GtqGISOp0M% z2oQ;YKH^vuzB7p7Ks(Bpd@O%ERR{Y4HL%TrP7a^P~YwDPS4FKM40n?~jh>=EeXonrj zqimUoUw7CKLVk-nI_PDO?Fk|NX{jUX@k1aC?O=T!U8eg56YX+!^fa!H%%WX({?@|@w-G0ncMlSsRtQ)7|=h8@4K-9`+eAj&jbF)6Qg1+`BuB)@|o6~6T<9y7}3H< zU38v3f8Iyw$7sCAmB+E?hgqGWC0gPDqde5032Phqw5e5O(J{sczfpMf{fnRFX^@iQ z`N2-^X6na2Yk+Ygb41_sr)$^WyL=@H+M7iWz^0@|z<3PQ%T3lQ0@1&I8zf z`cCz<{bJSRoV$I3deD+iCPF87!1F1%9dAE%8V9NiT!6C*u~37X&@uMy6Wcn(=B=ja z??>m`2J4(}bh1O4@N3Bg-T>I;$kF<@4tcM^cW2mZ$h!%9{qp`ev76=H7TY8559hlX zyRCkEuvZnntqwM0Z-{SOOIXt46k3>`LJPMzh4$jD?fZ7T|AVOm)3d^B6Z?M$094af zC{UxyQ3|{4&0o^vmvJJKa8^THV2410W2=wTZ~%_zrc9W>9f16PwDl|+kYq!|x-^DW zk$|^KI*a;k(gcRHDe!-n^IukFW8O_E)C8kep%e$EP&2ATl_;r?MIYc{TTK-e42MM| za7fm9qpxEveXe#lLiM5{XHlSa=rId=`4IS3xK4ynaL&Zg>esIPX6yB*2Ve6ea7=9n z9xIrgD=|^b>raI*@=1s+5fR*Zni^UKnpmyoxUNjp(ycWJ4p_532x;aFG|?$}XK z=6e{t!X6}$1$-qO@d$zP9bDx**n)2+LipL!!GV))W9iAM$?RA<2VQi0gojo!fdLRl g(;=Z2s<>K#jzY`cxOt*(lx^UyE6fz8Gn3>010%{d=>Px# literal 0 HcmV?d00001 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..e31b56008e4a7 --- /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,284 @@ +{ + "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" ], + "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 0000000000000000000000000000000000000000..080fd695fe5c0fa1c80c30aeb1cb6586836d6d48 GIT binary patch literal 14188 zcmeGjZHya7b?m#`<)fEel2afltxn<^QHb};U+Y{D@!1Y>>)f4eCna|Z?s#{6x5;|f z?5;1q{75e#2ojoAK^> z{c*uZdZ(uGO7`x)+4ttndvD&%yf>45dMShu8i60Z0?ixRfyN4&aq0M^ZCqLur3+|O zlu9;0e*B#My9=*=|IX{rFOuKC^wu9;oHdH)AqR%p%R}`0kVWje?%1?ENdFKzaKgPe z0_ZyxO()0s6>RYpQ&)*)^HP+zgtAq>tTx2x6(J(7#x6(2hFz{%0$fWqzQ|v!8=4am zBa$FXiMSk&Nij)I2;pdqxA6+8nVR8*uqX?-5-LZ_aUrbAWhst%wJb#=q!JF5<+2cp zBor|uhs9VZj72q}M8&AA$iU$*fI|>T0Cl#3HZg!E=jJo{WN|KEDio8&Oevi$6!Y1c zv&HP(Z0XEgI)law^SNxXRghtL3|zC9O$TTXp|QU*UEF#L z?XWHMnGI^9HQ3xKjp(YK#ttsR`#*nn>k~h@_IxRb0y(s|Ld(1&+d<>Gi+Ba|lr27| z+0JysLXX{*x`+Jox89lfQSd9(APVNtKE*UvKBiAK{O1tv9`{v z&f+T6w$InlGq^Uri1xYV^QNsiz_^1BF8Y|BD_lRem__3}{flkf$6 zxwqz$@+v*-t274&LDbXOUZ&MIYM{nr-7R4T(^R7?01oI=fi;_k0!)u1jCcD>=qd_; z#)Q5I`6SsQ7F$K_|+DR+Vo2KKSSdSLdk2Y2i#e=4ys}_oEmL-bovV{&H z1g+XjO_QS*#AS?n)n8b1&`^HvDUjh&_7SyTE@kREs0>tTPn|WYrm9sm)+n&+_xC*} z{_^k7jx*^E>j3d&ckF)?Xc0aBd!Osn$Ko$+#(DgxF;G-_n?mYtSiS{Ko*CwXmev0;JS2kcALtywSu@y_JHI4d! z&!!8Gr5Q`qrVZ#6R00pfGu1eK9&Xe?u0bsJZJJrbX*W&OpoGQ(SLtosL8IB(V&?Oi zJhRjzjJg2S`Mw&NHtS$*Ju5(71taOXvopDjkG=%-`|A(qu$5ZIR(l1YmJyv8OU`m< zQ_dyxsnf~)Gk5OXX;oJi*z**k4L<6FOYHg60SZ6Xfg`fAM_JhOP<+vP7%{f4On<-D}rnu-+`%v4ZtFfo?Y5 z`fj#fEfec&x4!qtp6|Z-4N9!$XO%t6K-!1WRMOguzkTBIOaC4Q%4v@>ZTU1q@k^%z zy=jp*-o7bcnSODmRb!{%WJ>L1H#G`QGjLCg6RgOSG(r+9Q=zL7-9Et4ZVyC z+BaHvLD~ju=*jLt-Jgf=(T)1G;cfHjf$i|=p1W|RcQ0<@Lnpv(85CW2=*sVdaf2T^ zNp6oR`rQ4%9NkI#X6I-FbEGQ-&YQ{mob_fUIofTGgRx~(>jiIY@y!87&}`Qn0M|55 zF{@S6@U7R&*d`o|(!Lo$+gsb{-#xF`m}YbKy|=}Pg>g2vQw~> z#l!`)Gf}b`_~F~fzC1dz`_I?c-uP+$TJ{ydq|@6q2yThxazavMNr($E0TWS<%Ys_L zBr3@wM9kvw*+cHO8>^(JMIEX+|NoKinrNQ7lDcQYcQM@ld1!;l2cs z!yzdu#6%?)5kysi2K)_vEZoRUziILRehD*ZuES z*w$5>>T3OurHSs~Cm>$BD2f-*m?)N13JF~V(3|g(-|Ttjjn4-D^r@5kPn`b;pfMYv z=$^(IsZsV1*5lp>w}Au(5_kY5u>BrBqDWD!sNs;TM3hh@EI~p?jmn{TMUIEXL|hUo zFkl_J1Il1It|023t!y+K>WbFTxM}<3sKdDi2k{#k7TxqJ#SQm<6c^0f5_NDF%)T{B z@IG6k1klmu%%enS9woCzEbN$83EQiNQo4*ar)1NKWXbia0W|((=6ojYW?+lw=QEJJ zgFGyh2T|`tYjP8lE}C>{O|aU!xtmk<2|IvNJxQ|&xdR=dnZETR#Kn3|CwC^8>zNBn)X0=@sCuBZs3PLs2xDt zBFRBcgl?we?j@__uGiYUc=x(`A>% z#?x0lf{=MmK*G5qRAePq7Q>ZTScoQ+azYg_7KKP8PP54OOXfKdOQ?ykEQf?}IRbeJ zMGBQkJPb!pAcY>ml^B+2a~{~tvpWXLT_H=)2uP&f57jz240V8fcDVyF}} z!g>fi?5Z1zj^Q|t3grGx*l0yw@Ko4n6HauKKzpAK70Y8Uc!?GQ6LI!8vHt;Q|oBn&VC2=-1R>J=Tn@P z5|Zq!go93ZSJkW4e!i#R&`cL4kOgcK4tEKG^3ARD&Gq1$i4gw!nLF%CC1>Ymv#Deb r4jH)`o?NF141hQ~j$!pM#dQR#g`TbP^F()", + "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" ], + "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 0000000000000000000000000000000000000000..7132e3b158200c762894f0d8fe6e97b36562ec85 GIT binary patch literal 13768 zcmeGjZHya7b?m!blIG(s$t5Botxj+aDr7&r{x~~Vr1*SJF&0@)T#QGWwp_PFxTb|I#w15U(rEi zyn|((@K(ccv>M@^qTN|FtC@1H68>N4P3VuO$Trfpk2RXLviDe zXp3#3PxR=C_F(fTG@`5aGcI<|p*m=+3%Rsk0s4IkYCz51 zbr+S_=|Nwoc~A(Vp2YSd?Y>!uVLTe-gegpG7+nGIfS(Gi#Tp8+aU5c_+n+_3Q3ys% zTo)prB-_ZsXVF8NJ=4&2+I1dnqn#H_({WIu3&*rl?*XFifHid4La{p~iMDmgLVFPc zu69$^l$88!#!T1U?MUbj0cNV6zT#4dckVRWoNeovMkN6w$8GWqKQN(AL~cDf>jW zz%=y`!!81LVO>m4n+?#mt{ot+gQ4l!(^L7Z55ENX*Eb*Lv6WfGR%Zt=EJHdms?OqC zR_4=%%*k}&%B@?s+TF<_dmcxW;Uhk{#GXGMqVS_@+C*5{BdlzUb#stGKGdNJw>R=> z$E<0JXN(Vhq4?N8E`3yjg_+?CgPq*do<1;6q)M#4?_9h7-qjb9puIV?1zd^-foV3< zt~b-D$;5j8#@8O+@$zrJNRie3q_XEEp!PLs4bsYMKYH@93;!Gh%+nrb%K0Q~;+M{v z<}Ho9_NVK~OOvlnwR>DExFp4PJcx~gry00wi^o}!CuxL6&-@!?+*hPx7^w4J5bkAJ4(z_co@xBw_rYwrCIdtWBZ_MEP zPLi9WiavASS4Y=rzf&Krua5K!f%iOlpRvAENp20QV}EVAqwfW8ZSna4Lue-81K_&G zD`u@`8ouv!8QX-1RodqPbi8$}{z1Q@H_n##LOfm@BLd}0(}WQfD+jBKs}Mv zL@F-HoP?zc7b6g($FUUVBsqywQALSLi6}z*X?T-ujf7Ef73#Ih1-`;*Jgjrk>sa5w zI`+TIX}If-y-zoc=kuLH8{#{K(2>^iR;J5a*{l#|-;|!7%R=ZMc5$FQjJj{rqyt`5Z~u&jY}?*hhcdZ4G94nxbVbeLYYvFL zwfW=4mr9MgPHu}ljE>v}A#KhxBrzWmgQyy4V>=eXHHiGKZbh{q7Ew$~<-sq6;;c?; zq{GGS0q-J(u#bSyhECvt{sg|83H(8fWNb<}s2|DLh)9OdYz%D3dJ^;{dfWQ7pWfqZ z0Ud^Kqrb%;2rK@93h?*s(7nnZ*c3_jBdRnNhrgScbnvTSpo{Z&zgXOjh#@`@R=mFv z@qcYZ^nLOEuA;j$=udK_u^Gr8+w?WC?~D60d^&vnjaV5*XVRy##kq859cjc~#E6By z_<2)PX-=yRnZ~~O(RdGd2Pdl$@4ZE{0h#8<#&l9~ik7kvIfu4I$~J@E{PW>g-uln* zub1Bb!B-c4^^xyDE4Hl!3Dat{B1IG|D~SpbD{5sTp(JIJio|7AmLS9nRd>48y}oEu zdWBlav?|OSsnC6x8Q_BvSv$PtdF)tC8YT6Dfj^D1ZyKb=YKxm(* zB9>NNWI7TB9f`U+^3%ml!-J^fvWe3j)*Bq5lK8sj}@p~ED}#~I3XuE;3FocaExGC1a2N!9dTWlWI$s* ziv>p^lnd+>2cKDeoU>?L-_Zy=O9Ju>htamPXhhX)1zTzc8w!C;ZuKnM<$_LNI9vkn z!`{1QX!fEXQf$a}qedYfnqtGQ6GNq-A$F<;9=10OMaOU!Lxp7hHf=Pb+Vwnw%ZAXd zDtJ?dwxOx+^a>%!IgYGH2u1Hu3vGVwg`3;1e`@qimbCp4&{)GPo!UY1VCFqU>WM&_ zM1f5N(-2k>omg$=rFA=~Ys&X198PfBgOFy4@(wx~?5Z2re!9ou#6$oS$O1A%qX8jM zy7^VQ`7U%b5yIbklK!`6(lfI&xlB3_CrsQC9$Uo;1b{f!oWbd$imMIiD0Jn", + "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" ], + "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 0000000000000000000000000000000000000000..ee69cff594ab7b5ebdbcc0824e31f2dac0cc93aa GIT binary patch literal 13839 zcmeGjZHya7b?m!blBSotB-fOrG&;mJsBP@^uD{l~BI2_h;@0t9Z6~3LQg?QDe7Cvv zuGw8*d_hVtDk4QlMJiMXBp^chgOo}Y5dsN*@dJ?f$Jg(wLLi|cAwY`M5`{OjyXW<- zFI++slAc$xclOQ9n>TOXysz0UzA_&{2#vsxULog(wxEfs(Kcmrc+yno7woDjpW%~d z(RMyrw^&8_nafk5AAad(;q~wQ{^ic}0k`u?2u&u4fW@?si19d+mKsVTDT)~kHKG%qrS1|klwhVVfL`g8N5QULd>d-ZQ6&ndb6QwGSeF=)(j*1V9ugrD z6w*i~(m2wPq)a5i$E1iD7jZ;@*38`p2ksV}4W^)dJ?(#*#*&W?Q$_ zCh=M5ig7-VZS1qf0mvEy9J=;5w8b*fp>@Wf*58V1#=I(HEn3v=8UWli4}}rxm?*ZP zt|bRWZ};$N^oVMe+nPovR6^V6a23O_ZImhjTF!AwIOddaY}Op3h;u38d}wLE=}s$j z(L{+mC5iU+BzZTR)y{Kx2}d=o&qoV-yXmiN*=TFATq}GY+^30CH=)XL%F2OJ*LS0U zlfG5zdFolA5IX3Xw>mX%b)!X0Y#U}BTg$p!KaW+rZc&f6?yU(Lf1&VFA@BNywU_1! zXt)I4A=HOafA=qY2XvQI%SU(*siS{!+-*Nga(z)V*3xx!{7O%bYweasx?JglJd6(B z1|jVpWk?dfbs0wUfj*XP65Lc_ZN0G_h)HC_RH9W@UC>CA^eodl@VGq8d8!Uhg}aepSU#}~gBmSLitJzl8J zWpjb!$&-P|!iyMMHEP?a)n5(`c+efguQb>KZSp$^jCZ@}NU=PGUS3UIHZmG0A| zSM;|)XyV9>AoxcAiVpw*-|975g@f0_la{`Ch8NGGt-M&b0CMx) zH%7wE7eDuzSHJp)pT$0vhxY8yllG$pdb#J^kJ9!LlUQxdwiuk%qTSEk_at|k#FUtz zC=sm4$w(rRX+$y^l8h*+xQtWrSUT3g2<`3wfso`5p|zybnSIo!+(9u~FO6R4*u>Pa z<_mp;pdK-DbDypSdp01j{%q1wm1v4Z!(XcF|MkwbPriNW`}HsimC()xt@ElZ8;zG} zu9Ty3Ge=Xe(~u*1>bBUu&cFQnKPSEyeytfs;Sw5?4Sfk-Z%_?Ar(p{a?{W~+{5Hox z(WYmf`;be=X{NgYNSUT7tbp zlw;Uhpu=}s8$J^O{FGtt+~78Qe%Y|D(wk_l~sHm%Zo? zC&gHKDnuI}^uYyIeulvh(afu_j%Vcqi1wN6(Eoi5@`)ZvxV@6k+D22AJ!QQA_3G0f zT==v=j|(WiFzCr$O#IjZBdimdC+6<=u3Wuw>9PpYTSQxyu+}D^JjT22%I&5^%&%N~ z>&YG8eD`YGuN=oD5HJWIsoZ0*07U5cKNOQ zqSL+n)*w(H1nRD-`@g+_;ZyemaU6)_gC~w3pCs^owJkm$U}SIt-s86fJ)+Hgl0;)F z(G9HN3lsmC~3 z>U~Q?yI;BdyKPs$F!9qcf&lsvps|VBo2WKQ2J=4f4j Date: Thu, 30 Oct 2025 10:09:22 +0800 Subject: [PATCH 2/2] add restore test for calc on source --- .../table/test/program/SourceTestStep.java | 13 + .../exec/stream/DeltaJoinRestoreTest.java | 7 +- .../exec/stream/DeltaJoinTestPrograms.java | 139 ++++ .../nodes/exec/testutils/RestoreTestBase.java | 1 - ...and-calc-on-cdc-source-without-delete.json | 669 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 14481 bytes ...ta-join-with-cache-and-calc-on-source.json | 636 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 15685 bytes ...h-cache-and-cdc-source-without-delete.json | 45 +- .../savepoint/_metadata | Bin 14834 -> 14691 bytes .../plan/delta-join-with-cache.json | 33 +- .../delta-join-with-cache/savepoint/_metadata | Bin 13782 -> 13843 bytes ...ith-calc-on-cdc-source-without-delete.json | 669 ++++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 14534 bytes ...calc-on-source-and-filter-pushed-down.json | 636 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 15681 bytes .../plan/delta-join-with-calc-on-source.json | 636 +++++++++++++++++ .../savepoint/_metadata | Bin 0 -> 15681 bytes ...a-join-with-cdc-source-without-delete.json | 45 +- .../savepoint/_metadata | Bin 14834 -> 14695 bytes ...lta-join-with-join-key-contains-index.json | 1 + .../savepoint/_metadata | Bin 14188 -> 14253 bytes ...delta-join-with-join-key-equals-index.json | 1 + .../savepoint/_metadata | Bin 13768 -> 13768 bytes ...ta-join-with-with-non-equiv-condition.json | 1 + .../savepoint/_metadata | Bin 13839 -> 13843 bytes 26 files changed, 3470 insertions(+), 62 deletions(-) create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 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 create mode 100644 flink-table/flink-table-planner/src/test/resources/restore-tests/stream-exec-delta-join_1/delta-join-with-calc-on-source/savepoint/_metadata 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 3e433979a640a..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 @@ -57,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() diff --git a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java index 48753223bcc95..38cf2bf787e4b 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinRestoreTest.java @@ -37,8 +37,13 @@ public List programs() { 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_CACHE_AND_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 index ac576644da023..b4eb48932fc1f 100644 --- a/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java +++ b/flink-table/flink-table-planner/src/test/java/org/apache/flink/table/planner/plan/nodes/exec/stream/DeltaJoinTestPrograms.java @@ -27,6 +27,7 @@ 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; @@ -168,6 +169,70 @@ public class DeltaJoinTestPrograms { + "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( @@ -180,6 +245,19 @@ public class DeltaJoinTestPrograms { .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", @@ -259,6 +337,44 @@ public class DeltaJoinTestPrograms { + "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", @@ -274,6 +390,29 @@ public class DeltaJoinTestPrograms { .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), 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 8b35564001edc..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 @@ -326,7 +326,6 @@ public void generateTestSetupFiles(TableTestProgram program) throws Exception { Files.move(savepointPath, savepointDirPath, StandardCopyOption.ATOMIC_MOVE); } - // @Disabled @ParameterizedTest @MethodSource("createSpecs") @Order(1) 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 0000000000000000000000000000000000000000..41d0181f0421e3883cfcac81c0e02c8272a9d8b5 GIT binary patch literal 14481 zcmeHOZ)hCH6`wmP^1mpOZP~8fCJ3UqxbfcY-re4jQWNkncZ`ND=+Oj|7HPl)55c~)TeL}DHoA(d8+y3Uk?59-`~0P z>;?Vj??3+=x9~N&wDuF-!H=4nF(xGrp;jr8Qz?Zz45X7oFK+oD`0nL*Cgv`G_pSFv z7iG8#Qs_0y{u_S{4ct{4*hKEC=3b+*0rb7RrjuRaDQt!Frmhmp4oAeW#mkm_M4cAI zqkK%5i60S#XjwvUgD#CEGmUSCW?sArhkCU%+Uu?*0jb$ zxQLYr63XkEF&T2Otdo#cG90ZyLe5N)lpIaBLnG#qe(c~3u{5k}M~T%{L&+GpXiu09 zuy2B2;M_%Yk8Poyi_}4R+acW?*TCzC1`H<)1XrDbAVSj?;+Ir3U1m{TJUEKhYW84B z*J*>&Xf>^F#55fT#nZqxTyAxkwK~jN9cB`RU5UazwzOQ&bP*l1P;9v((Xwu<&<573 z`wrtN9MZ8d9!eP{I=``tCKesEYG5#%`ci5Hbv*uGpwk&IfCg)7#YHWon1zCA_&Q0( zHldpI^-|SaRSyNw_VVvpj7iMbFU@GYri|@a1Q#^uTd(di#UhGn zsi6!@PU)mTs=8?rdR;;NT3E>e+~f}6a7`o2PJ#`W`2pN-mSCV|8)F#8a7+Kt{@!$| z;Q*;QT(1LkwYRi6-~hJzCa_g)et`khq7!4>nYjJtlE5vQOAaJQ`t~MAj$gfc)#taG z>dgC#&nM-9U2&dW3kg+DY9(f>(M24P=V7TMD*JnIqu_cMEL7?dt7L9?@rD<#vl44~ z@dZm@cOkf}Wo*NXm-ja}qRNIBhwWy=i_^_?V=cbi*dFF?6C;xT)#DjCGACFF^x*$KTVwRTk>I#$rxO-`S&LVb6 zcW6}%=!M&cN9YUOPrLWITMmAA3(=U2;{qq+sK~`+Im&e_@i-@7UXBu3P!o!b(8gg9 z5Ww#MTG$lY(g5ll8cvNQvqK{;%GnS2=Ey+r{_MceU~b<~e+so{hSLMt+MH1gILVV{Pr7p(ENs zZN7%?!^NI4)aIrSo3`eF;11e2<`a5IK1>t`q;8(n4AnuMb;PChh9NTY803JQ_Z3}M zUZq=om8M}V5p^%NCusH4#U;l|fd2xZD~(#1?DkjDF%-D{(5_n&IcXXhdK`4Poy7pC z{f7EXU574%uC%-M8wFF<@)`>VfnEQn?IGc{zdzf~MnSU<6i;_MKb=8^@z!26`>Q8E zet@wFC=S}}>)}eaa`$ema86nlo}uGw`owSVf8ibcViN+mve|~v8U&iaM;)Oaw*}fG z)>=1<(NWZ^56jTGBg=fl| zJ~vMn0nH9#VacXhCyYlKwW4G6)?Nv3F-zUXxbwjMM)oz;6}antAO8sOzd0x}t0^$w z6yLtQQk~4R=PpDuZ}-t7?D-1;O5aHlSznsKo*!UkI;Uxlt&H-SDoePwlTSKkK~p?u zyzOLW$0tW0jL<+T_e^2EM8J8pSMm*( z3No?2aqjv1*PZ&{8Op5a7nMCrz}i=&3=hv<_}Qa7Ca*LD=YEefm7|9h@oQ&6@v=pp zxpY=K+H;||QsY9!NtD~}T5gow&&YL4Ji?MZO(P#F9;kzgFE(AnD5Hw@&DJ%Ps>vEg zvO7^XqT$yS4R^J7BW_};PQbTat~k*<_H%-2_~Yid!P`!gw>+5EFh}@ihps_y42^4; zV`clZTurX>tv(ua%k|h|oiD9gR^&^o+K+T@R2lzpyC#@N#>(%P zzIZP@NM0gByD2)i7k(asrjxuMq^m*gAnQR`?Xt@&Q+`_p?+60`>+*J4&&%V!?_Q$Z z5f@yn$J=8~?;A5_$x_I!PTJm#rHmy2X<4tzPBOr+<4#@2VdRN_VAV5-=G9- z>?|N}L{$?)BpQ+O@dPg@k|<$SNbvcHoR`Hsp}M8A`gP+yyXfTla@`{Y!>6_x`t2g* zrzLLjcet@=B%wxdjEkx`&c(1ADm6|yQez-=kWx~5P6xGMFA(INVh5` zqF9K^k+?_{(j60Gd20K2$aqmiUWvuJc`hPER4yh+aSn?B-SZ;P#|aj&s79bQckOs_ zmv>Ul==FkC@cF(n$<76Y`sN>p;d~ac9F4F*4d^czM5{+pyQjQhO0r!lP>RPUQ?iT1P${W}jS+ZQQ!*4C!}$Rf5b%m@bd6RQ zc)o)qc6VAPiB%33LG; zjz(*gK;;h2%N?pCH&Y?}E#(Ztn!e=V(BME{G7VuHw}(6Dd4d3-js@pN>X_nu0on_7 VTjQ6B;&y8QAI|L045tSB{}1#wG9CZ` literal 0 HcmV?d00001 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 0000000000000000000000000000000000000000..c6827672128da57f43f284a26b76fb017458cfef GIT binary patch literal 15685 zcmeGjZHya7b?jU&Nt?@EE~kd1Z4}gN)I#j_+Uu`eMB?)~#I198vYnJBr0#fke7BeN zuDiRw^W{glfar&iLJ?F@AxH%I86;2?si*=3r4(8OQX_$cB0gFHDzpd*q)1RW-puZv z*Jt0wM{?H&ypq?m^YPxBH}B27c{4Bl_=FE3)D1uSc&tCP5%uI`_0+!IhI;Bmm_La& zg!zI2Ip5#^yPu3+G4~$$@vlGW`Q?|tWw)85?VQG*F7~mR*7xbes4Ax6D(~O#L7y+l z3fad^W1TB$ibQmS3x+v8P}GYvQjHIv4hZ~gWG2kljABI(z%x39Rs-#kA9|dC=~$@l-aEAIlbU`9wZd7)j^y+4S(ye0prO zaAa&Gh5B;inRLEhkYernjwLcjQ-xeAn@(iX$5UCz^`c{%KHtl^oG!tjMQ{KG>XCJDbU4-Og$=z$*Hz^P(=m#`-!uJBrJI zt=HAjBe*hj0`=PEBTl2yq>yIYV;@p6FoVWViq!|zhA|1xYBt>7Vz%cG@?g@KXBXkB*f5#Zq-R*A~5KdC7S3>gfiF?duhYm!`&S*Jj*-|fAhzw-9E zKE}N+1#%qT?Y%L8I^Z3{XzsPguJ2-CJfey2a*c4knl-%J>W~w71#{ z>I9W(6_kQ<;BH9M5vS>{S_Sy(#eCbMnO4n4(L^;0sK;}L{`Hw?LwYoy`e-W4By}f) z9suZETb&%ysvvDG5g-Ra_sH1M;Y`W}KM3&K(}!bNPflXJ5djQKw?fnjb8>N4jwQ0m z!-?!C=I7_@=!yZB9z?XmdtCWJmi~~3=I>q9hG%8FSlModW(UjpK!YSKKglOdtt^Yq zWW4j~+=Kr(eP7TEibFAaVV>MtS05NBGC}nIH!fVfe)g#tNN*agpTy1{+ zMWR1>>A8FF{K9WPLxEM?q_T7jK)ae)gEV*f2M;}X>YrTzdBlNC=@?>7+}v5z+>*!( zf4vwxJ#=}vj&U*LVielJW@t2fgk`sEv7Z$=oJMHik$(Y?t;IjTgb+3_*PvyT(XQ3H zDW_qv22HjX>h@;%7HZV446oUo?pcdD-I*?I?%m8Z@xEi=nkfdZvEDh7Lo?KuTXEZ=Q-bC$E4hGAtT4c1e%4~pt`%=Cd z<4E^ZG5Vs9<4?1`1`aB|u#ZRR)6TbUe>c{%O5wdNr&V>4?Axu7l9|)R6GG@D+9ZSu z2Fvd{`H`1@)t^wqZ@m8Q_zPLfK1FB_JX8T8EQN>|mM|X__y`GwNl*%w#BiV(3kL&0 z1KJEf$5lr$I%(T&x7EU(B^catITM?fD`D+@c12MVN}-VC!#uo%qKJL*P$1$9#DhVR zFGgaL5C`)A1V84pYe~4pn_KI+)0^~aX^q8nC+{DgHOaUxm%(u~P1xq%rNhe=Hs~82 zEcdf9i4_oyD0pXuXaUw-ar>6v|NGBR9C$3&GxhAwcYgjh)TRuWqph4_rfK(cR}vf` zrjBLROsORe#7<`(-|gV>1vi~tEMZgArxSm9<$4Z;E^NvI7&;LIiJcJWAs6STtG;g<{2GKn#WgK_MJ0#RJ6-tncXO z-+az|^~vvEeetCaMzsTujKF`RhHRiY@XWDy=hflBnsVUoD{G3bgKS9-ya$F)gy2vN z$9#M#haXuW43BD*_3df}gA0{#I;NNoyNM`n<7d2brOIi9&o=AEo&}jHCvN^mR z-z0VUuXqP;A7%V6I)m24qh>pUT>EeDZMOgRpgor5QJ|JbL8}m*8qfu7%&KBx2Fqr_ zp!=(WJ@eSlRnH+A3r?}(V{I6V_w)Az5edieC-S?`r$mSA*uxy1H=8X30zAwH|t(ti5^B zCHjsT2Ui?d;g;N58r_{O-!0hkb*z$(Rnll#z&+13(PH=M|82akuEx5z+G}-ZUG1!^ zEpDGT-fZfu44swXyjK8{PG> zXO&o!-`?RMggc6-Sfr*uxsV*!>FJ0m6BZ~6XuMvu@i^*}WJAQdl*Fn?AX-a0j(Y5@ zLl`1R;k(!QmQ~r9bm!!%MWb4#IS!ay)u<3vqFLSSrV}JKRaH^J5N|0#Kwh0TIyJP6 zIkdaM)vI#O@X|IkoQ*yn1aU=tJ50zqfvdFng{Q7=y!f$!7g%7{9YA9lvydqhg);!G zG5itEI|xT=M1-RlP30B<9a?DSgl?IrWmwk$1R$FAK}fJ*I1?RiM%Aj", "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" }, { - "id" : 34, + "id" : 72, "type" : "stream-exec-drop-update-before_1", "inputProperties" : [ { "requiredDistribution" : { @@ -46,7 +46,7 @@ "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", "description" : "DropUpdateBefore" }, { - "id" : 35, + "id" : 73, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -59,7 +59,7 @@ "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[a1]])" }, { - "id" : 36, + "id" : 74, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -92,7 +92,7 @@ "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" : 37, + "id" : 75, "type" : "stream-exec-drop-update-before_1", "inputProperties" : [ { "requiredDistribution" : { @@ -104,7 +104,7 @@ "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", "description" : "DropUpdateBefore" }, { - "id" : 38, + "id" : 76, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -117,7 +117,7 @@ "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", "description" : "Exchange(distribution=[hash[b1]])" }, { - "id" : 39, + "id" : 77, "type" : "stream-exec-delta-join_1", "configuration" : { "table.exec.async-lookup.buffer-capacity" : "100", @@ -230,7 +230,7 @@ "output-mode" : "ORDERED" } }, { - "id" : 40, + "id" : 78, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -273,6 +273,7 @@ } }, "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", "inputUpsertKey" : [ 0, 1, 3, 5 ], "inputProperties" : [ { "requiredDistribution" : { @@ -285,50 +286,50 @@ "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" } ], "edges" : [ { - "source" : 33, - "target" : 34, + "source" : 71, + "target" : 72, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 34, - "target" : 35, + "source" : 72, + "target" : 73, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 36, - "target" : 37, + "source" : 74, + "target" : 75, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 37, - "target" : 38, + "source" : 75, + "target" : 76, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 35, - "target" : 39, + "source" : 73, + "target" : 77, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 38, - "target" : 39, + "source" : 76, + "target" : 77, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 39, - "target" : 40, + "source" : 77, + "target" : 78, "shuffle" : { "type" : "FORWARD" }, 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 index d054f347b6a70e396d0524002b45ebd2e87e3a7a..4f0e8e1dbfa94153864705ea3a32a3b83901fb2d 100644 GIT binary patch delta 1039 zcmb7CTSyd97(P3BNi7rG-6+=DeCWYEIdf*_T!PS|=qU)avTR$=nKM(SxwfuFPi`0L zr6QRw5QGJW1VzMBXucGM5JIm#L`C%wK3Ni2M%o$EF3>~%@bh2z{*V8h{~PaopT4ks z&(iu-SK4vc$J<;lQ=fL4h>-wA#7K6|^j|$5x!rp0-NRdluCyLL*PjIdgp_APATFh2 zID$J^FzF(}Im40>EQmoRCsgmr9NU2qB092di($&Bfla-plOJ(AQZ4Nn=t-9`4jm+2 zZd1?pD6NrPcvKNW*CPm8mghi$Z2?V38lKQC(=v&_DfHdH919l`ngGiP_3yCJ z4i)&ND|P>TLEY~8Z_FIdm|;WC2!WQs7*eV0&~a_U;nc<$832T3Zerz^a=#xv_n~#i zuhmK_2}yv(ge0Zgmf`TA`jcnJar1-G@P>T0eR$&A2^u^im(}tXXc0Hpqb&>tD~frq zh=s|e%}nUoQmx3b0jY&Ns0+iv+^`5W1prY`9kxI@(ELxuiQKuuyhvsAwW`a4W?;eT z=j=^T#6h9oqpez#o$CUi|4wJJx)K%Q%0s|6P&A@tRX%H>>i;A8SX-nf;>s1t4Kv#P z7s>SWES(DBIXS0W_8<7EXDiUNHLK^RgI$H*1FNd;XyeZ7le;QsD1YkNu>>QwPB0eO zv=Fy+LTq(Eu6x`j2r1~IS$I!(d&9i1pj(n*#9e4GGmtiNAUCL|^iLUggv(syZwyWO zld!s4}>fwP$y)QQ8k1RRc+r0Lr{H61;SVa}FgE9Lb eW2Gmzl#HFav~76B$BTN?KY#@X{pYf&$mRA`^P&EG-;X^Qs-(&OWFvz zh;X4}YAR7_+6<_YwDTUO4mZ@75TgW1~t-ywO-p}TWz`=+ZHZY0f zxB@a{L)>Oq#NmQqK-2{gQ>Vb@NC&n@91EEqri{uXp`Rdpj1LsqN%c_*)3g@GG~=x) zpifDUW*hGo$DS^9d>**n{^9U8Fqo;GFMFnZQxML z0Z4G-c!GPJLrshD^1jBtNT&)&)1pWHT%i~+wP$a<{k z2R&7=X5rK1==kcX(v6XxKxX0zc{5%kpBS}=u$9&#*vfc;p9h0^W#B-?@&x(YM%HZPAGWDxxmX(AdG>UnR691^duz+q67I_8`rDI*T>tvCJJhrV zr5t`Y@@jEzHM}ynnuuL_Jujy!T0_!aNoAt#;jWlf5OWR9aAjTNsIzT$} z0I_u!FsiU%E)>F6Bss`diN7sBul!ys{a92^M~}-lv1a)(nvjoUjdCJ7qqIGe#?Gs= zrtOsvqr24Gmpye`*bi{AK|Nlh9&b`VlDRi?Mhn0H)wpnQJXyM^R@tJS8irto%ZY#x zdY&VM?s15yV<2c_OWz*##Dfl`5CWI$0ubgZ;9ZD;%aBe@wYO;rMg5PmZ?07Q2EnK* AApigX 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 index 430f2226bf1a2..25adae9cfabd8 100644 --- 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 @@ -1,7 +1,7 @@ { "flinkVersion" : "2.2", "nodes" : [ { - "id" : 19, + "id" : 37, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -29,7 +29,7 @@ "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" }, { - "id" : 20, + "id" : 38, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -42,7 +42,7 @@ "outputType" : "ROW<`a1` INT, `a0` DOUBLE, `a2` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[a1]])" }, { - "id" : 21, + "id" : 39, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -70,7 +70,7 @@ "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", "description" : "TableSourceScan(table=[[default_catalog, default_database, rightSrc]], fields=[b0, b2, b1])" }, { - "id" : 22, + "id" : 40, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -83,7 +83,7 @@ "outputType" : "ROW<`b0` DOUBLE, `b2` VARCHAR(2147483647), `b1` INT>", "description" : "Exchange(distribution=[hash[b1]])" }, { - "id" : 23, + "id" : 41, "type" : "stream-exec-delta-join_1", "configuration" : { "table.exec.async-lookup.buffer-capacity" : "100", @@ -212,7 +212,7 @@ "output-mode" : "ORDERED" } }, { - "id" : 24, + "id" : 42, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -255,6 +255,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" @@ -266,36 +267,36 @@ "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" } ], "edges" : [ { - "source" : 19, - "target" : 20, + "source" : 37, + "target" : 38, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 21, - "target" : 22, + "source" : 39, + "target" : 40, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 20, - "target" : 23, + "source" : 38, + "target" : 41, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 22, - "target" : 23, + "source" : 40, + "target" : 41, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 23, - "target" : 24, + "source" : 41, + "target" : 42, "shuffle" : { "type" : "FORWARD" }, 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 index 915966fd949484149a2da635d6d262e978bd549c..873636efd9aa0f53182242434c4256cc8fa2a3c1 100644 GIT binary patch delta 827 zcmbVKPe@cj7=P}%D{E%DY2u}xAra)@{CjWSNP&kutP~Ynfr+fMGxKay-rDXItB`+Xn3`F-Eow3)dE1}L6z zdMGZ*YE+MFq7ubmPRvx4h^w{l54Y9M?p+=m`TY37(8~PS^|5~Qta;0BiYA<`6qS^o zi3K=QtwhDEnOk3$W}iNJ1xKft?ls?vO|%_1dRv>jvpbBbaWKLvF1ybviEAwK^+imex@#*{14W4$tb20gdPe2 bg<4rj6$2Rh(DR|Mh{Wba`bXmp{+jv)7l+`g delta 737 zcmb8t&ubGw6bJB4jD`vo8mJ11wTcIYPCIXYPZe4ZDg}jtSWpRJW_B0S#H3_nQCeya zf_e}PD@cz@PgM^Zj}-)epir+~^w2~9fC@c`KhTYM@S1^lm*jB6umwIrKofj$8li5X$FQtAeeN*QWrnVU!NgF zD

Olv5=D7}H~snGfdbT?_`vq(UJ!mQrJU$<3GrEU3Z+`9L*dQe$L|1;jMS0E{zO zxVTmL6@SMAQ=k(p=zmzbHF4_cm9_o(uRrFN!{)?)SSmwIL&l9SeCt4g4I#7=Yb}Ii zH0)w8^p5pI?08@p;+PV|KslrXr;!RY6U<=_Hoj5#-Co*#n9!ovPiV;xs!fAtllMP) zfBdvKc4zID3C=zEw)(EU`l{1Po!!yi*c-2BCSC6wOF)@W5(Lawh*@Wl@*PH!YwmMM zw#u>R4F<_#b|vMdW87EltnbVWj&!eH|M_yQGCmyDDwn&_$tKZ>lFs(#@|VYj*_-3# zH(B=a^=Q0uU}PZucf!kcBaxikwfHVfUh3VfxuVQ!i6mtPF=3Pc#y&>^Q)~mNfV#w- Td(nx87IN`(e5P}dZm0hM?()}7 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 0000000000000000000000000000000000000000..be94da0751fb756e581d8bd7eb3c697af274001a GIT binary patch literal 14534 zcmeHOe{3699e;MxB<(xyn+v&Dsop(Q;lWWd* z=I)%N6>2jfZ3s}i36O@+wh9$7iB=#c5C{n*BsLKI#gGOX8Uv*WWwm@3yO;O`{xA1U(i*!Y5Ie_-w23?I4>`-{HXxPo(#|P z?1Cwli~twD7EMw!z zOguN1$!Bx%TrxkB%H}eu;fY*oY&3s#Y$S>Lvg7Ggu6iR$+w~obrzev6Y%-IIr&E*3 z3{btu(2Z%IP!gmW>?^2>HVZ!#pQIOyy5_S^mT=ipRMRI4CiV%2A)NFLpS19}p%fKM znZt2H%3?ziROJ*l23NY7)r68cqg&8#2mI9@-3dXJrb5S2S12#RrSEcK;m@~SI`{nj zyFY#={;fyg>37t#8xy)_T803Vkkr6NBy3Evplcsw*}SxLi?YXxPSV-bcjJwr`` z2b17!lkmm%er{&{6f^n5JeNHC<2xNhnEN0O#e%V@6pjXBycm>uK`iiv5XbRREb*ZN zJbJyDM5rLe!hu4J;bH;Ia9lxR1WpndHpa>^t{{tI+a$mwB482`s%YOiv-!!d4*z`d zle@;AKXZA{*BrHgGqnwJUVd!Kf=TZmFtu4~3Ef}`CQ!cjjY;`z_IHQYJ+Sk|U*6P7 z)TYk5Y!|wbJkJ=|EUT7D;MZ09*m{S3e#}0f124;k0-P8I-z(!F!$mR2 z#NKv>kh${9OWzABG*yUrB3l=ssL3O#) zdQy!O-SjG5<}Usftuqbuu_iqcFK+q{Dp-}x5y292@cWH#Ufg-2&eL|QM{KWP9(~4;gEf{F`is(+`XU_e-{|WD>ieBVRqh4^jbFj23x|UD{6M(l= zfK@@8_EXYO8r`7k(+aqJ!j^@CY3RpaS&;m6LGn|1`ROS9_Ot$Uwaw5%Y0f}9>e-{i z&gkb38W>n1F4x@MtmQ!b{*#1mMufa}g# zMUyQw(12aiuOIq70#Cq`n@cv7)9H3sr)e-QEFTuk8Pfei38ryh{Ya?7RHo4ufDiDA z!0fSk=sb2&-px;-)5rre7HA2`CCMt7cM@$-%+a!{lCIOJhjgCNb<0BG1|ud9HJ>2T z4u*zK8;DyiOQfyM7TSssjA}DcO&`%9Us5X<-8V}X>d1`U4KmzAA0qneOX#W!QwCFM z4o+xAT~-PT9Te#G+r7KlZ~x_NAC+FG3KU0n``#I#3cP(7E&lBOH*cjd9?3#Cx@Nfg zSY>**Rgr1%MC@zf!C&0+rDui5JCG-ZI!%bAKwRLX=81WC!Fc$V|lVqaYHv8Lr8Ulk3if63Epnuq~@*O0#NJ6D<&+ zP3SZ^^;xJpHJVEvN@l30?x5%aK=-#a$Pv8^+O~2BT_a-?!|9|8KM44(*~2lxNX!UE zZ3i$dT`Jb5t(nSDJWE447SALO$1`WHT)9&1P7KiNK|}_;$3+j)>yLT}ec!S+EWLLt zZ8Jc-*-j}p)o8-XPCjnwMMZL)@s3Yt_r7&%SI`TFLwfXrpWIS4ADAaRjg76pKY!uP z2Oo}t_NLG}2#(7b%&f2a+-SKdV&kEUU%cgpFTC_w0#K^is93*6{9kfEtGn*F5&wnAu%jSjSk z9b{~%*0EQ?_X^k$rPIS1YvtD}u-_Y**j&@o#u2Pq!dlUxK{kKum-c3%LtScvBpGWXiJG7Ryka}q zL2`(z4(aXOs4F1#tVuxXL3=9eTb`_MdA)=UvL~Jw%#)gwpBEG>Z<2zEyluT6)ORF# zZ*s)0I>_BSo`n4rs6C*X)VR1HUKg4+4|+)+yc+V>D3iU-RU?u_Lui_`v_R~Yss$%$ zmn)Z4ydt<4?O6ekH1|;u&c#G0nyzbOS_T%1kWOFP->XmzEa`^qgJ2tOCska;HSY~g z=nbUcbur*sQv>d3CpfN3f}{OL{;uB0+u2IGEAG2W=^>Zf=xE?Jn)~JDxrf2!gc>Kc z7O&iEo#1GD@ugg3Wo~QtKFOwgYkLMRRds>}r~|oPO9^Vz)NL=`_Ttr=BNEj^Imb2M zb835W_lA4T%G37ZZ7<%SAgt}hU90lDO-8h1@pddumj4d4*0K0nyg&4h#*Zem}DR)j|ymVEds|+p11NoR$q+=x}{kjYjR}dl2f!>ySVeP^1*969VDex5T%v0pC=F z@UN8;==F*C=-6l~5l=(Cy*", + "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 0000000000000000000000000000000000000000..9f2a8ab906d5128e027c20a17c31c6bbca7db723 GIT binary patch literal 15681 zcmeGjZEPG@@!i>RoVJb~$0{UkBcr;3mh64(-e=r4IiD}hHJ7uwySOw~dVRP1&R%D4 z_qx01*cJ+os*<)y&_FAcAJ9ldDjz{Ysw%+`Bvc_j6e&NTLgEh~geXD<6^I&CsKUH= zyS_c&onw=nY2a46+uipuGjHC!nR)Yel26ZY2%#SM(aTl6p>pN0(ah@N zG#@#Ew(*g?!OFir)OBrc;??Poz5ScZ+aG)X*H+(U7aGQbKj0IJ;SdfKqs3r>_mi+x z6bn8Q2p6$Lpy^!>R1~9)*452kc2sd^2(;F-CN^mw+UiS1St0wq^H}#5H3cdSFCX;k zzJgvjB`x@aOTK`AF?1^EUoZ+~-3Ql9*_-puRaDvJ_>fNs`Gb*YG!P03(P%Uj^cr}c zlr>p3xq{>mh9%BdB!0pRk#IQ5dwrr5;6+^EdTP>DYiP6O)NdEH{zKXLGS!B0rwY<}%5#W4Yu+I)8LxJb?zXlc{8`R*_)+ z2Bu=EV~Kn=kx9l<$>WI(l)90w=`$W&#^NmT6ct%L2`{oo)Ji2y^_YugQZZ%4@D#8? zJXqK9qGxQ;B$K*al1+J@#As2L=rUI1C8Ce4Rhdfq@CN zsDZrb(?aM3`iKz9iz`*!Jm8VAsp-8tMvn}B;wIsds+5)C`m*ly}kgoKB32Vj7f4OK1ll-gLPPNBD3^AO+4Z`l(!o zf*iy9?STKpMV#OhV zjYV~0*Y`Hl3zz?u>rK`Aj4qVtb+o&YJT~l% zc8ba*TwQL|4cORN#}?DL1lal=13ik%qtmG0s-M&hPzjo9qMg$Yq9+P-L^Ofw?vt`A znP|8PxfHJ#@E(B{&~kg(0%arJEqdR0HT~QQ@I)!>E@{FdLCJHqfF>Ta(f@nV& z8al0`z(z@;eJ!%kAVM&z?Nl^9RDwJSwNi3cmQB=^nRo(txR*7e^6QCfiULyxQ)!GG zQ%jm87iBgmkn4Z-Kj44koi7Y9?sY3rV)Jg_jRDjE?;1nPm!G<}hkD^g_%*+j4Ukhh`e&6|@;IDQeR}ys_V0(aFprcOF<=q4039gH8mQ0Mm{SaNz7H>c} zxomO$m}qMu!ZA}p%8$r+b#Idnd!U3NL`0HB_Q&^AB zV!a*#OiPbK)ERSj^-!i_nfT#Y=CjwYU$3Dn_}KFZq66ORDCgPpLoQmrZ&e>ZtJ}ls zh8db&tmLD0lCZXtkC|FY7Hws``}ypH?=RiYyJ6u_jBe1AJ8H`Z=84P@eeib|E?zrx zHUiR{L|ecmsSsFZ1GRSPN~u8f&s}=uzB^z3?dK`5vXfNy90AadF4aj|e*MkIA3XWb zZh$;)L#A|$vMx^UtmVi7R#~Y|^6=0~kRjZtqW`%X| zZ(qvWF^_al6`_ChvZKS*x1oay7v%g1J!ik`)_Z-sYAyU%BX|N25j=rs5xgIK{I$L3 zpBG;H=7FofPrvDU1RC8if@eVqQYb+N<)JI+C*t(hW%M9F!;Z?wc3>ZskOCoc(d-?%$s$gy;a8}PaTiE zRwg<2!Cte79iB*YQd^UP--Y0bfmR$|`oA zrQODMwH4rp?Y4pKu-ol`Ih2ka zO=Kry@rK;1#zYr8&dmb$o+(Y1Xx4=R=~<4b_ehKKx(Ec_CGNQbfAnK$c5SgA*nQ|A zz0J^XB|br)YS+*~yQTechW#-{wQ+x~=~_?jWk>Dx>$R65I~`D$XJ%PGpx}pmK*4Y2 z1CF~s_|YH!@~gAYKJ&xRe0lf%G%b#O(E*unfq>6nl*9lD5h6t+;fN6Si@vZJ75tbN z1lsgA$p;joVu9y*KNln<$^{~l$l&g{SKe* z7_-ijY^7jSO0>iVldTwKqDr)=hh-|l!?ublDj2fUBuHPX(MDB&(+n+Z7~Wb_uf2R} zADa9_FBgI=4F7tVkhRlKY4;0ff6;sK(?j25X)JdDjU~(yOiUC^0kHaHF*wg6Ua~+$ zIBL^W?+TzpE8Xlw7ZWv2f*FEzEweTWF_t=HqQi}-nvL2?_Xy;WH9!Jhz&GZakvQ0#Br30OXrlWDvM3XMI OL2^&_Savdz9{)GB5$R$8 literal 0 HcmV?d00001 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 0000000000000000000000000000000000000000..021c454cea252586b7b4d1c67635592a6c73bf44 GIT binary patch literal 15681 zcmeGje{369`PoU+bX}UX4P$M$Dxz7{vD)YJ&vP?rbsVSl<|di#W?Q3Tp6{NMYtDD( z?wmM9Dyd)$4T(t;{INf_RZ~%g)M-eihzb4yiGdiZ&_IHJkR}8IrU9Km>clpM?>*ne z=h$hJwvJZnlk9u0u=E8>*jcM@&(`*I>Y z?4EgJpZ1$0S3XEy>PLV0^OIKhMLNVDN6XHuXM`vweq58)DSzlB>hXti60Px^f~rj6 z-ORJIFTD82WA3Q_NGS~iDV#+Lpbp?>S_9W2`)~iXDX^gw*mHS9r)y8PDg|8V#e%Gm zL);wJxq_xhL^n8ph|_&}JwGp1_|R!zkY5PThxm$-FX=wGrb}FwJ5^R?(-R3sc_HGD z`hsD<;17j^As=VpIZ^^)WO{giln?SAoSfiGi^OX#lt|f> z6~mj>=0~uJGenoMBA+JuU_(tt#U*1#GXZ=D{M;A+fi@XBI=IRl6t>}2v}qaSwKu7n zS*Wmx>2f1f(UEUWLsL}{^~2pK&^@v-R#p_cpcLw+)1@`dG*LJOV7cn5Iksw!t(s#P zaTZ3L11udkR4KX6=qR{el;~J%RcHrW)wWZ34to`>PJ0t-x#(;xnP_uzESorzNL$q7 z)->OUR)$s9Q{ZkTVAsg_@!?bg^*{9qkd~@uHyUf8A}+~ZQ7aZT)xqR>Y!J$c4lb{I zs^jp+cV~5&)%|gRU0G2F*@gC34<0#s@W^Q;qGMCjbJ$o=#oRoW&75IkljJPcaiP8^ z63-+?a+z!_o5*FKnMk0{R5FugYww$krH&_ZnM67nOC?Vv=*h*6V(XwtN;VuMsYA8r zublwSj0#~y$ndufkY>oZWv>jI>AL5 z-a32kYK=}rO_#hG)|^mCk<>B3D)mhft+AY|rJ}X7EgV2+P;s|j>1qjwwraW=;;m3k zYtzn?SdY(Oya;m?w`%#xSUP?*mj24p(o&llZ-p7_q<-skaw|>{CmM9r zYNu~dLZG--9Kf4Z)~Tgd;M~@$%O*?v&<#gFYSMt!($=dh%tas#yKchdYa7hBVlcN) zB{g@*dQZoj9jB5zSg*E1@3mT2xAkgUuQu@pw)N`m)2nUGaC-9?RMMvw>QthEX zqm^}$92(H&=@~PlLnem*B--Nl=M2{H+LwQ49(ewJvKN1E^jB}q{2WHI)Cm_F!2U>H z3Pyr{LBK=|hjf~vHA-8mAs_KYqv3EQ7$`v3JA~CjB;pgKK+sbN2nA12z(J1?Ekr$`uqeQb6BY6jeF^S* zRtr;B3o}+*dj8G7{pFXx?(Vp<*?jq(r;pft0ryo|ttSHs5r+m0b#}@hp`ao;18RPF z^rzP@^c;Qr)VBUljub!sdoxg$Y`T~B_vpkZE2iO?YR97+E}n!?)X$PZ@)1D$CCF-q^OpV%dHUD>_ViH}&ibD=HR@F%#D==-HRUc`P zfz_Ez2q7BYQJE-`LA~qrVE3QzUwq*8v#;jd$dy9d3bf5`GECH$qJeCV2C+FB7axKE z)I zJEt5-kLOPj(FCkJXJu6~(LfV)>AW1w`vmlWp4&U%ZV9Q`Djltt; zQIq6?%(A6`uRrd6h`;>)xB3|MIu)p~X?O6(0BV4D4Wq@kzi@3Y1LG1+bf05~Yu&2b zyR!zFojU0IRXO|4gD?F8zubW!X548&=2@i#JKHr+Pk&voo=_uqvlO=Kt8KkNG<}Wo z1Wj5SrEheNIXZyxC1QxWT&gnl4hK(HO%}@2D!dew0(+oLLDO<)r39f5H|E#OnoY9Y zD4-tK8G7rhWq&bF-OZr;0Q#+Dfw2jYJ%Qy=a=`Zk{M+LiS+Nv{Gr2Zh-p9&=h<4uZ zsQ0t-=Uue^APtK<+IUvp%i0Diw8t)1LoJ~iCtP31$4sp#i?%S{{f*4S|2@6WPxA~Z zJU7V6?Tr0cULNcdnI`&&?d)>`aBmWAn!`$&fbi(6^&2Y}^F;sp#qT`0_1o`$ zlL9L_QDtQTKs$z1qlSx@e)jmovmbQ=REV>~|vEILad&J;c#i_%y zT~kMQZr-v(+*>d6t!S-J9TD5?cxR>2H?m_pui~ciDja8kt~)G|zSTkd)}&n?4GCBrKZboYt0{f3& z3lTE*$7gi-x#xb{eg5hGAGr}2(Dwk2Ma(|nGEu0}?*kuL5RO!c2yO>W<(2`BE)TO! zY!fv#zx%;aWY#tz#+()=I@&l@vsF9s9t7`I10", "description" : "TableSourceScan(table=[[default_catalog, default_database, leftSrc]], fields=[a1, a0, a2])" }, { - "id" : 20, + "id" : 53, "type" : "stream-exec-drop-update-before_1", "inputProperties" : [ { "requiredDistribution" : { @@ -46,7 +46,7 @@ "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", "description" : "DropUpdateBefore" }, { - "id" : 21, + "id" : 54, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -59,7 +59,7 @@ "outputType" : "ROW<`a1` INT NOT NULL, `a0` DOUBLE NOT NULL, `a2` VARCHAR(2147483647)>", "description" : "Exchange(distribution=[hash[a1]])" }, { - "id" : 22, + "id" : 55, "type" : "stream-exec-table-source-scan_1", "scanTableSource" : { "table" : { @@ -92,7 +92,7 @@ "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" : 23, + "id" : 56, "type" : "stream-exec-drop-update-before_1", "inputProperties" : [ { "requiredDistribution" : { @@ -104,7 +104,7 @@ "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", "description" : "DropUpdateBefore" }, { - "id" : 24, + "id" : 57, "type" : "stream-exec-exchange_1", "inputProperties" : [ { "requiredDistribution" : { @@ -117,7 +117,7 @@ "outputType" : "ROW<`b0` DOUBLE NOT NULL, `b2` VARCHAR(2147483647), `b1` INT NOT NULL>", "description" : "Exchange(distribution=[hash[b1]])" }, { - "id" : 25, + "id" : 58, "type" : "stream-exec-delta-join_1", "configuration" : { "table.exec.async-lookup.buffer-capacity" : "100", @@ -230,7 +230,7 @@ "output-mode" : "ORDERED" } }, { - "id" : 26, + "id" : 59, "type" : "stream-exec-sink_1", "configuration" : { "table.exec.sink.keyed-shuffle" : "AUTO", @@ -273,6 +273,7 @@ } }, "inputChangelogMode" : [ "INSERT", "UPDATE_AFTER" ], + "upsertMaterializeStrategy" : "MAP", "inputUpsertKey" : [ 0, 1, 3, 5 ], "inputProperties" : [ { "requiredDistribution" : { @@ -285,50 +286,50 @@ "description" : "Sink(table=[default_catalog.default_database.snk], fields=[a1, a0, a2, b0, b2, b1])" } ], "edges" : [ { - "source" : 19, - "target" : 20, + "source" : 52, + "target" : 53, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 20, - "target" : 21, + "source" : 53, + "target" : 54, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 22, - "target" : 23, + "source" : 55, + "target" : 56, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 23, - "target" : 24, + "source" : 56, + "target" : 57, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 21, - "target" : 25, + "source" : 54, + "target" : 58, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 24, - "target" : 25, + "source" : 57, + "target" : 58, "shuffle" : { "type" : "FORWARD" }, "shuffleMode" : "PIPELINED" }, { - "source" : 25, - "target" : 26, + "source" : 58, + "target" : 59, "shuffle" : { "type" : "FORWARD" }, 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 index 0ea956396b686f44ae866804aed77ac5c5a55545..ec2d5e9be7083313a788cb2ef36d3e615924507b 100644 GIT binary patch delta 1018 zcmbtTPe@cj7@yVh4-#$W`iES1ku*q0-@JJA`K<(~2R`lVQ1;POi^5^Am{RumKxE0*dV zIG27>vOGJx*onL2*-(CE+`AiU^c8roLR!8Lc;2<(!3d(3W?({@ftYDxWMa!|PVpg` zi1!Ypp((hAEyyf{rAu^71aEeTtr6Qapu?mI4NZ3;vk`*Wacu19nk_XBK$)0$82(=B zJvZvTIZVBN6_<!$4^=Syk{i-H}5#f0onbJ(X2FCE|6_SLvG z7VLcA@$`1xlwwyk{y3YL7m(vKMZT>t;LM^9r8*R*!J*+I2XdPes7YjUs&OH>1wgqw z!MFDX3d-jJ0DjeMSEl9^+L$Yr@qw_Em=W6*NV9C4arLZ3w=`2qU24+QrFSZLz_+fDGykCw zlNne?NQYE7E;J}&kn6U9rop73j)63XPSn)w@-^nvTA|eH#0OI8B+p51i1+s@k;jw! z9yGjoJ$-fN*i`qn#W(Fol>qie%DUF6wS?ENqbun|2+46K8=QNh&%U&djgGgquI`?n zeEaAR&eSBrA(CUk9n}uPE*7o|LfWQaw%8hHrg4fL!l5m&goZ8<9D**0NAf<;s!Cmf8yw?`TE-Po+v9!2kdN delta 922 zcmbtTT}Tx{6yCi?q1lgm_nKS_Q;1MEXJ_YUR|6^NMh}%_1ZEeT+1X9gytlop2)+0c zftaQ9&>undR?tJ-mmU;C`k^2aF`_8wA*iUJf*=a1x#>d>y+{LR4(H6A;rr%%=WX~k zcebG(+G)*zc9#eF-Yyn1~FA9??+wRZZ{n~g82Y$J8DI8j;)aavmi zahLG>B4!@V^z^p}uiH=do|$;HJLZNB*?sD2RlY6pE0)hDO7tRKo0!nIden6&unBMo zau7ujAl#H=gML=M5ShTC7C|NfoVqL!0D7Xyg-$>L7bbR@Wg=T`oV$T393LVfY|j-y z(+pqJwB^dQn7mb-&UYl28HWp@7H`iCgr47A6ZRbGj@m=76T-As3}IL9&%w15bJ6CJ zkNyUBJ@x2*_dQjjD;w6e8U`sB1(c4%e2-wuCl+%N@t8-5<6${vwde&}2bNCD5g1cW zn91CLnus7pO9*aa3NZGNgCTL{D4ds`Rkq7=6=KU>L`*-o?znkua$WMv&+LUDYW)kb zO#@&D2s1$h_bkWaffaBHT$d1v?BIV82as?CsAU2x5GF#lWip2<>?(<{<8#**`B7M= zH$sxuDk16eGX1$IWQEQSvc4Xrt4uf<6UU!M?`kd=Sq~PkQ&i*gkb*&sg=v)a>|`M!jVGF{o2a)ryNne`moP)r)>;L1YJ)-VH=O#{-a? zzVHlbo+es`#}0RIyfe zsQgTPjchTXd}v^qGVaPaDX)j1Fk|2!Gj@Ku#8~;`>fWlhPJTO;c%E+~l|{b+uiX-y 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 index e31b56008e4a7..3e3a5d5ca434a 100644 --- 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 @@ -235,6 +235,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" 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 index 080fd695fe5c0fa1c80c30aeb1cb6586836d6d48..f9d56a3df402341101dde1cf26215cf730525f1e 100644 GIT binary patch delta 517 zcmbV|KS~2Z6vk&0#YPYkO(aFg1s-`b^LAz)D0qbwB33f9I}3s&f+B(`Ql=F50$#vs z?_im0Xkp<2#F#&N0P{7!2e0_}ejme^(c{%_B35Y}CE}?ZbdJwCCBxPzOi7a^w6cHW z@8b37a(D|!m}3C61&8PyMxuUT6{AN2)M*MNGXxHtf{oTk>2ix0g=pMY?c||R&YI83 zs_69JMQl>eT)~jDCLqlk*ck%lW$-#+aMqV&{Z)M*yw+;1c{ONFllsQHI5Q^AYz9(U zy{K~*n6v_=)ncSl6ip#Sds@ABw%XX8Ej%8sWeTzVcdWR7vc!d$EU`|nJZtuLmhyXI wryR7pd$q~$Yl->1dXq`Asi~;Ynv4jnb_i5t0UT)$5?v8OE?N1a8tc#c4`X_M6#xJL delta 444 zcmbV|JxW7C6h`MIc7nx=Mo8rbZZdQ4{M-Ruf~|$!{DL5fporiOtc14$7hv@)Y|^?4 zmtf%n0{RGUzURix$~SGIntr?#s>U zYIz4iu?!WWiB3}Hh%PL0M59Y3h#VzopA;}D3lvoL$tc5#g=pXUgXXdA&L^MEmUQud z5+)PSSOiy$16>D_FHC?&T9vGpDyMEuoBsXiW7MBapPKRRS~{T#DHCwg2{hrLl?{Wd w>_~ZsvN|EAKmE&W@3@=eW@jyLefGo*lA<0+2_Y6oL&_nh4%G%M{hGeVzXdpV!~g&Q 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 index 8b31ffed1fdeb..ff6f5a9dbb654 100644 --- 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 @@ -227,6 +227,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" 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 index 7132e3b158200c762894f0d8fe6e97b36562ec85..aece6ec4018d8706ea6c7eb5e6e86b43ec21356a 100644 GIT binary patch delta 378 zcmbWz!AV0w5P)HVJs3n=7&5anyE}8X1i}06><&o4h)NnsBZ1dM&Yn_+1$Zzuyaqn_ z|I7ZeKR<0=g^Jc96->!mn;3={R5;|YQF#g delta 378 zcmbWzK}thG490PacThyH5Sqy(naSEC2=0@4DM&#@OE1!k6y7DecGY8e02e;5@OIzo z$H)J&zwFPCn-@jH)~HD#mZn11w_kqDkN4NN11PB(!VcBF4+>1$8c#!PD1;>gy9$_* z12VU%YYL*xpPT2q6?W33CKNzUJK$VcXLF$FZgq@GTIYxRr`r|w@Is9kA%blA-37eW rWd#kZgsHusuI2Z1#_tt88VyXT{~adaA_lIZfRT*moE|t|%g@apej{j+ 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 index ffc4faf8a161a..fad8b5770edc9 100644 --- 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 @@ -255,6 +255,7 @@ } }, "inputChangelogMode" : [ "INSERT" ], + "upsertMaterializeStrategy" : "MAP", "inputProperties" : [ { "requiredDistribution" : { "type" : "UNKNOWN" 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 index ee69cff594ab7b5ebdbcc0824e31f2dac0cc93aa..9d06051f9b3369d6e7ed1b4dac6b7554764f435d 100644 GIT binary patch delta 400 zcmbWzF-iqN490QvVc`L^_5g;LnItnC5Nxe1EVNE`CJ*7Eg5ot)7SExuNAetki$~b* z>wbLyxBaw#|2mjHzAx@lX5=hM2V!EZ8MSPm?X^EW-aov6R?Zd>Ok1WL6m%f*wFiVdgT=6AY0{9KI~CGD0NbQ~2pNdq5-MZ)IIolA6?cbn^@)k3hR zrqw{51DKXCgA@cow=B|&Qp{J!H>dx??TS!jZ44X>bae-=IfJS>l%T}Xu~MDS&wft+ E0IWf9qyPW_ delta 396 zcmbWzPfA2F41jTTaNz-TJr@X*CTY`v3-^KxLHBLb!h=BvM!bf99-c$t9m#VDiZ^gJ z-}VRJynWujd>p;KeqP*hj+TidX-;Ysb4Z(gdFs1|>HZM{_e2S}aAZfWEN$bxyO<0Z z?Kh&vkb?=-5K0~rF}cmt>U+LCem~S8?<0v87_}NmzzUgj0n(_&M@8?nOsngYLjgf1 zCLv&=22w+WnsNeFF{_~!=Du8=-kkpdxBtJS9UaLXinj