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 @@ -1052,33 +1052,29 @@ class FlinkChangelogModeInferenceProgram extends FlinkOptimizeProgram[StreamOpti
val inputChangelogMode =
ChangelogPlanUtils.getChangelogMode(sink.getInput.asInstanceOf[StreamPhysicalRel]).get
val primaryKeys = sink.contextResolvedTable.getResolvedSchema.getPrimaryKeyIndexes
val upsertMaterialize =
tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE) match {
case UpsertMaterialize.FORCE => primaryKeys.nonEmpty
case UpsertMaterialize.NONE => false
case UpsertMaterialize.AUTO =>
val sinkAcceptInsertOnly = sink.tableSink
.getChangelogMode(inputChangelogMode)
.containsOnly(RowKind.INSERT)
val inputInsertOnly = inputChangelogMode.containsOnly(RowKind.INSERT)

if (!sinkAcceptInsertOnly && !inputInsertOnly && primaryKeys.nonEmpty) {
val pks = ImmutableBitSet.of(primaryKeys: _*)
val fmq = FlinkRelMetadataQuery.reuseOrCreate(sink.getCluster.getMetadataQuery)
val changeLogUpsertKeys = fmq.getUpsertKeys(sink.getInput)
// if input has update and primary key != upsert key (upsert key can be null) we should
// enable upsertMaterialize. An optimize is: do not enable upsertMaterialize when sink
// pk(s) contains input changeLogUpsertKeys
if (changeLogUpsertKeys == null || !changeLogUpsertKeys.exists(pks.contains)) {
true
} else {
false
}
} else {
false
}
}
upsertMaterialize
val sinkChangelogMode = sink.tableSink.getChangelogMode(inputChangelogMode)
val inputIsAppend = inputChangelogMode.containsOnly(RowKind.INSERT)
val sinkIsAppend = sinkChangelogMode.containsOnly(RowKind.INSERT)
val sinkIsRetract = sinkChangelogMode.contains(RowKind.UPDATE_BEFORE)

tableConfig.get(ExecutionConfigOptions.TABLE_EXEC_SINK_UPSERT_MATERIALIZE) match {
case UpsertMaterialize.FORCE => primaryKeys.nonEmpty && !sinkIsRetract
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Regarding the two changes

  • Auto mode: makes sense we don't add SUM if the sinkIsRetract
  • Force: Do we know where UpsertMaterialize.FORCE is used? Is it safe to "not respect" the force here?

case UpsertMaterialize.NONE => false
case UpsertMaterialize.AUTO =>
if (inputIsAppend || sinkIsAppend || sinkIsRetract) {
return false
}
if (primaryKeys.isEmpty) {
return false
}
val pks = ImmutableBitSet.of(primaryKeys: _*)
val fmq = FlinkRelMetadataQuery.reuseOrCreate(sink.getCluster.getMetadataQuery)
val changeLogUpsertKeys = fmq.getUpsertKeys(sink.getInput)
// if input has updates and primary key != upsert key (upsert key can be null) we should
// enable upsertMaterialize. An optimize is: do not enable upsertMaterialize when sink
// pk(s) contains input changeLogUpsertKeys
changeLogUpsertKeys == null || !changeLogUpsertKeys.exists(pks.contains)
}
}
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,35 @@
/*
* 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.SemanticTestBase;
import org.apache.flink.table.test.program.TableTestProgram;

import java.util.List;

/** Semantic tests for {@link StreamExecSink}. */
public class SinkSemanticTests extends SemanticTestBase {

@Override
public List<TableTestProgram> programs() {
return List.of(
SinkTestPrograms.INSERT_RETRACT_WITHOUT_PK,
SinkTestPrograms.INSERT_RETRACT_WITH_PK);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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.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;

/** Tests for verifying sink semantics. */
public class SinkTestPrograms {

public static final TableTestProgram INSERT_RETRACT_WITHOUT_PK =
TableTestProgram.of(
"insert-retract-without-pk",
"The sink accepts retract input. Retract is directly passed through.")
.setupTableSource(
SourceTestStep.newBuilder("source_t")
.addSchema("name STRING", "score INT")
.addOption("changelog-mode", "I")
.producedValues(
Row.ofKind(RowKind.INSERT, "Alice", 3),
Row.ofKind(RowKind.INSERT, "Bob", 5),
Row.ofKind(RowKind.INSERT, "Bob", 6),
Row.ofKind(RowKind.INSERT, "Charly", 33))
.build())
.setupTableSink(
SinkTestStep.newBuilder("sink_t")
.addSchema("name STRING", "score BIGINT")
.addOption("sink-changelog-mode-enforced", "I,UB,UA,D")
.consumedValues(
"+I[Alice, 3]",
"+I[Bob, 5]",
"-U[Bob, 5]",
"+U[Bob, 11]",
"+I[Charly, 33]")
.build())
.runSql(
"INSERT INTO sink_t SELECT name, SUM(score) FROM source_t GROUP BY name")
.build();

public static final TableTestProgram INSERT_RETRACT_WITH_PK =
TableTestProgram.of(
"insert-retract-with-pk",
"The sink accepts retract input. Although upsert keys (name) and primary keys (UPPER(name))"
+ "don't match, the retract changelog is passed through.")
.setupTableSource(
SourceTestStep.newBuilder("source_t")
.addSchema("name STRING", "score INT")
.addOption("changelog-mode", "I")
.producedValues(
Row.ofKind(RowKind.INSERT, "Alice", 3),
Row.ofKind(RowKind.INSERT, "Bob", 5),
Row.ofKind(RowKind.INSERT, "Bob", 6),
Row.ofKind(RowKind.INSERT, "Charly", 33))
.build())
.setupTableSink(
SinkTestStep.newBuilder("sink_t")
.addSchema(
"name STRING PRIMARY KEY NOT ENFORCED", "score BIGINT")
.addOption("sink-changelog-mode-enforced", "I,UB,UA,D")
Copy link
Contributor

@gustavodemorais gustavodemorais Aug 7, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd probably just have added a plan test to check that SUM is not added anymore. Do we usually prefer semantic tests over to plan tests or is it just a matter of preference?

.consumedValues(
"+I[ALICE, 3]",
"+I[BOB, 5]",
"-U[BOB, 5]",
"+U[BOB, 11]",
"+I[CHARLY, 33]")
.build())
.runSql(
"INSERT INTO sink_t SELECT UPPER(name), SUM(score) FROM source_t GROUP BY name")
.build();
}
Original file line number Diff line number Diff line change
Expand Up @@ -753,10 +753,10 @@ LogicalSink(table=[default_catalog.default_database.another_pk_snk], fields=[a,
</Resource>
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.another_pk_snk], fields=[a, b, c], upsertMaterialize=[true], duplicateChanges=[NONE])
+- Calc(select=[a, b, c], duplicateChanges=[DISALLOW])
+- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[DISALLOW])
+- TableSourceScan(table=[[default_catalog, default_database, retract_src]], fields=[a, b, c, rt], duplicateChanges=[DISALLOW])
Sink(table=[default_catalog.default_database.another_pk_snk], fields=[a, b, c], duplicateChanges=[NONE])
+- Calc(select=[a, b, c], duplicateChanges=[ALLOW])
+- WatermarkAssigner(rowtime=[rt], watermark=[-(rt, 1000:INTERVAL SECOND)], duplicateChanges=[ALLOW])
+- TableSourceScan(table=[[default_catalog, default_database, retract_src]], fields=[a, b, c, rt], duplicateChanges=[ALLOW])
]]>
</Resource>
</TestCase>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ LogicalSink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3
</Resource>
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3, b0, b2, b1], upsertMaterialize=[true])
Sink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3, b0, b2, b1])
+- Join(joinType=[InnerJoin], where=[AND(=(a1, b1), =(a2, b2))], select=[a0, a1, a2, a3, b0, b2, b1], leftInputSpec=[NoUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a1, a2]])
: +- TableSourceScan(table=[[default_catalog, default_database, src1]], fields=[a0, a1, a2, a3])
Expand Down Expand Up @@ -602,7 +602,7 @@ LogicalSink(table=[default_catalog.default_database.snk], fields=[a0, EXPR$1, EX
</Resource>
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.snk], fields=[a0, EXPR$1, EXPR$2, EXPR$3, EXPR$4, EXPR$5, b1], upsertMaterialize=[true])
Sink(table=[default_catalog.default_database.snk], fields=[a0, EXPR$1, EXPR$2, EXPR$3, EXPR$4, EXPR$5, b1])
+- Calc(select=[a0, EXPR$1, EXPR$2, EXPR$3, EXPR$4, EXPR$5, b1])
+- GroupAggregate(groupBy=[a0, b1], select=[a0, b1, MAX(a1) AS EXPR$1, MAX(a2) AS EXPR$2, MAX(a3) AS EXPR$3, MAX(b0) AS EXPR$4, MAX(b2) AS EXPR$5])
+- Exchange(distribution=[hash[a0, b1]])
Expand Down Expand Up @@ -662,7 +662,7 @@ LogicalSink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3
</Resource>
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3, b0, b2, b1], upsertMaterialize=[true])
Sink(table=[default_catalog.default_database.snk], fields=[a0, a1, a2, a3, b0, b2, b1])
+- Join(joinType=[InnerJoin], where=[AND(=(a1, b1), =(a2, b2))], select=[a0, a1, a2, a3, b0, b2, b1], leftInputSpec=[JoinKeyContainsUniqueKey], rightInputSpec=[NoUniqueKey])
:- Exchange(distribution=[hash[a1, a2]])
: +- Calc(select=[a0, a1, a2, a3])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -646,6 +646,29 @@ LogicalSink(table=[default_catalog.default_database.retractSink], fields=[cnt, a
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.retractSink], fields=[cnt, a], changelogMode=[NONE])
+- GroupAggregate(groupBy=[cnt], select=[cnt, COUNT_RETRACT(a) AS a], changelogMode=[I,UB,UA,D])
+- Exchange(distribution=[hash[cnt]], changelogMode=[I,UB,UA])
+- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], changelogMode=[I,UB,UA])
+- Exchange(distribution=[hash[a]], changelogMode=[I])
+- Calc(select=[a], changelogMode=[I])
+- DataStreamScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], changelogMode=[I])
]]>
</Resource>
</TestCase>
<TestCase name="testRetractSinkWithPrimaryKey">
<Resource name="ast">
<![CDATA[
LogicalSink(table=[default_catalog.default_database.retractSink], fields=[cnt, a])
+- LogicalAggregate(group=[{0}], a=[COUNT($1)])
+- LogicalProject(cnt=[$1], a=[$0])
+- LogicalAggregate(group=[{0}], cnt=[COUNT()])
+- LogicalProject(a=[$0])
+- LogicalTableScan(table=[[default_catalog, default_database, MyTable]])
]]>
</Resource>
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.retractSink], fields=[cnt, a], changelogMode=[NONE])
+- GroupAggregate(groupBy=[cnt], select=[cnt, COUNT_RETRACT(a) AS a], changelogMode=[I,UB,UA,D])
+- Exchange(distribution=[hash[cnt]], changelogMode=[I,UB,UA])
+- GroupAggregate(groupBy=[a], select=[a, COUNT(*) AS cnt], changelogMode=[I,UB,UA])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -207,6 +207,30 @@ class TableSinkTest extends TableTestBase {
util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
}

@Test
def testRetractSinkWithPrimaryKey(): Unit = {
util.addTable(s"""
|CREATE TABLE retractSink (
| `cnt` BIGINT,
| `a` BIGINT,
| PRIMARY KEY (a) NOT ENFORCED
|) WITH (
| 'connector' = 'values',
| 'sink-changelog-mode-enforced' = 'I,UB,UA,D'
|)
|""".stripMargin)
val dml =
"""
|INSERT INTO retractSink
|SELECT cnt, COUNT(a) AS a FROM (
| SELECT a, COUNT(*) AS cnt FROM MyTable GROUP BY a) t
|GROUP BY cnt
""".stripMargin
val stmtSet = util.tableEnv.createStatementSet()
stmtSet.addInsertSql(dml)
util.verifyRelPlan(stmtSet, ExplainDetail.CHANGELOG_MODE)
}

@Test
def testUpsertSink(): Unit = {
util.addTable(s"""
Expand Down