Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.flink.table.planner.plan.optimize;

import org.apache.flink.table.api.config.ExecutionConfigOptions;
import org.apache.flink.table.catalog.Column;
import org.apache.flink.table.connector.source.abilities.SupportsReadingMetadata;
import org.apache.flink.table.planner.connectors.DynamicSourceUtils;
Expand All @@ -31,6 +32,7 @@
import org.apache.flink.table.planner.plan.trait.UpdateKindTrait$;
import org.apache.flink.table.planner.plan.trait.UpdateKindTraitDef$;
import org.apache.flink.table.planner.plan.utils.FlinkRelUtil;
import org.apache.flink.table.planner.utils.ShortcutUtils;

import org.apache.calcite.rel.RelNode;
import org.apache.calcite.rel.SingleRel;
Expand Down Expand Up @@ -68,6 +70,12 @@ public static boolean isRequired(StreamPhysicalChangelogNormalize normalize) {
return true;
}

// the changelog normalize is requested to perform deduplication on a retract stream
if (ShortcutUtils.unwrapTableConfig(normalize)
.get(ExecutionConfigOptions.TABLE_EXEC_SOURCE_CDC_EVENTS_DUPLICATE)) {
return true;
}

// check if metadata columns are accessed
final RelNode input = normalize.getInput();

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,10 @@

import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
Expand Down Expand Up @@ -94,7 +96,9 @@ static List<TestSpec> getTests() {
SinkTable.UPSERT_SINK_METADATA),
TestSpec.selectWithoutMetadata(
SourceTable.UPSERT_SOURCE_PARTIAL_DELETES_METADATA_NO_PUSHDOWN,
SinkTable.UPSERT_SINK));
SinkTable.UPSERT_SINK),
TestSpec.select(SourceTable.RETRACT_SOURCE_PARTIAL_DELETES, SinkTable.UPSERT_SINK)
.withSessionOption("table.exec.source.cdc-events-duplicate", "true"));
}

@AfterEach
Expand All @@ -106,6 +110,7 @@ void tearDown() {
@ParameterizedTest()
@MethodSource("getTests")
void testChangelogNormalizePlan(TestSpec spec) {
spec.sessionOptions.forEach((key, value) -> util.tableEnv().getConfig().set(key, value));
for (TableProperties tableProperties : spec.tablesToCreate) {
final String additionalColumns =
String.join(",\n", tableProperties.getAdditionalColumns());
Expand Down Expand Up @@ -149,6 +154,10 @@ public enum SourceTable implements TableProperties {
"'changelog-mode' = 'UA,D'",
"'source.produces-delete-by-key'='true'",
"'readable-metadata' = 'offset:BIGINT'"),
RETRACT_SOURCE_PARTIAL_DELETES(
"retract_table_partial_deletes",
"'changelog-mode' = 'I,UA,UB,D'",
"'source.produces-delete-by-key'='true'"),
UPSERT_SOURCE_PARTIAL_DELETES_METADATA_NO_PUSHDOWN(
"upsert_table_partial_deletes_metadata_no_pushdown",
List.of("`offset` BIGINT METADATA"),
Expand Down Expand Up @@ -254,6 +263,7 @@ public List<String> getAdditionalColumns() {
private static class TestSpec {

private final Set<TableProperties> tablesToCreate;
private final Map<String, String> sessionOptions = new HashMap<>();
private final String query;
private final String description;

Expand Down Expand Up @@ -312,6 +322,11 @@ public static TestSpec join(
rightTable.getTableName()));
}

public TestSpec withSessionOption(String key, String value) {
this.sessionOptions.put(key, value);
return this;
}

@Override
public String toString() {
return description;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -293,6 +293,27 @@ LogicalSink(table=[default_catalog.default_database.upsert_sink_table], fields=[
Sink(table=[default_catalog.default_database.upsert_sink_table], fields=[id, col1, col2], changelogMode=[NONE])
+- Calc(select=[id, col1, col2], changelogMode=[UA,PD])
+- TableSourceScan(table=[[default_catalog, default_database, upsert_table_partial_deletes_metadata_no_pushdown, metadata=[offset]]], fields=[id, col1, col2, offset], changelogMode=[UA,PD])
]]>
</Resource>
</TestCase>
<TestCase name="testChangelogNormalizePlan[[21] select_retract_table_partial_deletes_into_upsert_sink_table]">
<Resource name="sql">
<![CDATA[INSERT INTO upsert_sink_table SELECT * FROM retract_table_partial_deletes]]>
</Resource>
<Resource name="ast">
<![CDATA[
LogicalSink(table=[default_catalog.default_database.upsert_sink_table], fields=[id, col1, col2])
+- LogicalProject(id=[$0], col1=[$1], col2=[$2])
+- LogicalTableScan(table=[[default_catalog, default_database, retract_table_partial_deletes]])
]]>
</Resource>
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.upsert_sink_table], fields=[id, col1, col2], changelogMode=[NONE])
+- ChangelogNormalize(key=[id], changelogMode=[I,UA,D])
+- Exchange(distribution=[hash[id]], changelogMode=[I,UA,PD])
+- DropUpdateBefore(changelogMode=[I,UA,PD])
+- TableSourceScan(table=[[default_catalog, default_database, retract_table_partial_deletes]], fields=[id, col1, col2], changelogMode=[I,UB,UA,PD])
]]>
</Resource>
</TestCase>
Expand Down