Skip to content
Open
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 @@ -177,16 +177,6 @@ private String getDigest(Sink sink) {
List<String> digest = new ArrayList<>();
digest.add(sink.contextResolvedTable().getIdentifier().asSummaryString());

int[][] targetColumns = sink.targetColumns();
if (targetColumns != null && targetColumns.length > 0) {
digest.add(
"targetColumns=["
+ Arrays.stream(targetColumns)
.map(Arrays::toString)
.collect(Collectors.joining(","))
+ "]");
}

String fieldTypes =
sink.getRowType().getFieldList().stream()
.map(f -> f.getType().toString())
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -80,7 +80,9 @@ abstract class Sink(
.getOrElse(Array.empty[Array[Int]])
.map(_.mkString("[", ",", "]"))
.mkString(","),
targetColumns != null
// only print target columns when the sink supports TargetColumnWriting
targetColumns != null && abilitySpecs.exists(
spec => spec.isInstanceOf[TargetColumnWritingSpec])
)
.item("fields", getRowType.getFieldNames.mkString(", "))
.itemIf("hints", RelExplainUtil.hintsToString(hints), !hints.isEmpty)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import org.apache.flink.table.connector.sink.abilities.SupportsDeletePushDown;
import org.apache.flink.table.connector.sink.abilities.SupportsRowLevelDelete;
import org.apache.flink.table.connector.sink.abilities.SupportsRowLevelUpdate;
import org.apache.flink.table.connector.sink.abilities.SupportsTargetColumnWriting;
import org.apache.flink.table.connector.sink.abilities.SupportsTruncate;
import org.apache.flink.table.connector.source.DynamicTableSource;
import org.apache.flink.table.connector.source.ScanTableSource;
Expand Down Expand Up @@ -323,7 +324,7 @@ private static class TestScanContext implements RowLevelModificationScanContext

/** A sink that supports row-level update. */
private static class SupportsRowLevelUpdateSink
implements DynamicTableSink, SupportsRowLevelUpdate {
implements DynamicTableSink, SupportsRowLevelUpdate, SupportsTargetColumnWriting {

protected final ObjectIdentifier tableIdentifier;
protected final ResolvedCatalogTable resolvedCatalogTable;
Expand Down Expand Up @@ -447,6 +448,12 @@ public RowLevelUpdateMode getRowLevelUpdateMode() {
}
};
}

@Override
public boolean applyTargetColumns(int[][] targetColumns) {
// Implement SupportsTargetColumnWriting for the compatibility of existing test cases
return true;
}
}

/** A sink that supports row-level delete/update. */
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@

import org.apache.flink.table.api.StatementSet;
import org.apache.flink.table.api.TableConfig;
import org.apache.flink.table.api.config.OptimizerConfigOptions;
import org.apache.flink.table.planner.plan.reuse.SinkReuser;
import org.apache.flink.table.planner.utils.TableTestBase;
import org.apache.flink.table.planner.utils.TableTestUtil;
Expand All @@ -35,8 +34,6 @@ public abstract class SinkReuseTestBase extends TableTestBase {
@BeforeEach
protected void setup() {
TableConfig tableConfig = TableConfig.getDefault();
tableConfig.set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SUB_PLAN_ENABLED, true);
tableConfig.set(OptimizerConfigOptions.TABLE_OPTIMIZER_REUSE_SINK_ENABLED, true);
util = getTableTestUtil(tableConfig);

util.tableEnv()
Expand Down Expand Up @@ -153,14 +150,25 @@ public void testSinkReuseFromSameSource() {
}

@Test
public void testSinkReuseWithPartialColumns() {
public void testSinkReuseWithPartialColumnsNotSupportsTargetColumnWriting() {
StatementSet statementSet = util.tableEnv().createStatementSet();
// sink1 has not implemented the SupportsTargetColumnWriting sink ability
statementSet.addInsertSql("INSERT INTO sink1(`x`) (SELECT x FROM source1)");
statementSet.addInsertSql("INSERT INTO sink1(`y`) (SELECT y FROM source1)");
statementSet.addInsertSql("INSERT INTO sink1(`x`) (SELECT x FROM source3)");
util.verifyExecPlan(statementSet);
}

@Test
public void testSinkReuseWithPartialColumnsAndSupportsTargetColumnWriting() {
StatementSet statementSet = util.tableEnv().createStatementSet();
// sink2 has implemented the SupportsTargetColumnWriting sink ability
statementSet.addInsertSql("INSERT INTO sink2(`x`) (SELECT x FROM source1)");
statementSet.addInsertSql("INSERT INTO sink2(`y`) (SELECT y FROM source1)");
statementSet.addInsertSql("INSERT INTO sink2(`x`) (SELECT x FROM source3)");
util.verifyExecPlan(statementSet);
}

@Test
public void testSinkReuseWithOverwrite() {
StatementSet statementSet = util.tableEnv().createStatementSet();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,18 +148,47 @@ Sink(table=[default_catalog.default_database.sink1], fields=[x, y], hints=[[[OPT
]]>
</Resource>
</TestCase>
<TestCase name="testSinkReuseWithPartialColumns">
<TestCase name="testSinkReuseWithPartialColumnsNotSupportsTargetColumnWriting">
<Resource name="ast">
<![CDATA[
LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[0]], fields=[x, EXPR$1])
LogicalSink(table=[default_catalog.default_database.sink1], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[1]], fields=[EXPR$0, y])
LogicalSink(table=[default_catalog.default_database.sink1], fields=[EXPR$0, y])
+- LogicalProject(EXPR$0=[null:BIGINT], y=[$1])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[0]], fields=[x, EXPR$1])
LogicalSink(table=[default_catalog.default_database.sink1], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source3]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Sink(table=[default_catalog.default_database.sink1], fields=[x, EXPR$1])
+- Union(all=[true], union=[x, EXPR$1])
:- Calc(select=[x, null:BIGINT AS EXPR$1])
: +- TableSourceScan(table=[[default_catalog, default_database, source1, project=[x, y], metadata=[]]], fields=[x, y])(reuse_id=[1])
:- Calc(select=[null:BIGINT AS EXPR$0, y])
: +- Reused(reference_id=[1])
+- Calc(select=[x, null:BIGINT AS EXPR$1])
+- TableSourceScan(table=[[default_catalog, default_database, source3, project=[x], metadata=[]]], fields=[x])
]]>
</Resource>
</TestCase>
<TestCase name="testSinkReuseWithPartialColumnsAndSupportsTargetColumnWriting">
<Resource name="ast">
<![CDATA[
LogicalSink(table=[default_catalog.default_database.sink2], targetColumns=[[0]], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink2], targetColumns=[[1]], fields=[EXPR$0, y])
+- LogicalProject(EXPR$0=[null:BIGINT], y=[$1])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink2], targetColumns=[[0]], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source3]])
]]>
Expand All @@ -168,14 +197,14 @@ LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[0]],
<![CDATA[
TableSourceScan(table=[[default_catalog, default_database, source1, project=[x, y], metadata=[]]], fields=[x, y])(reuse_id=[1])

Sink(table=[default_catalog.default_database.sink1], targetColumns=[[0]], fields=[x, EXPR$1])
Sink(table=[default_catalog.default_database.sink2], targetColumns=[[0]], fields=[x, EXPR$1])
+- Union(all=[true], union=[x, EXPR$1])
:- Calc(select=[x, null:BIGINT AS EXPR$1])
: +- Reused(reference_id=[1])
+- Calc(select=[x, null:BIGINT AS EXPR$1])
+- TableSourceScan(table=[[default_catalog, default_database, source3, project=[x], metadata=[]]], fields=[x])

Sink(table=[default_catalog.default_database.sink1], targetColumns=[[1]], fields=[EXPR$0, y])
Sink(table=[default_catalog.default_database.sink2], targetColumns=[[1]], fields=[EXPR$0, y])
+- Calc(select=[null:BIGINT AS EXPR$0, y])
+- Reused(reference_id=[1])
]]>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,18 +148,47 @@ Sink(table=[default_catalog.default_database.sink1], fields=[x, y])
]]>
</Resource>
</TestCase>
<TestCase name="testSinkReuseWithPartialColumns">
<TestCase name="testSinkReuseWithPartialColumnsNotSupportsTargetColumnWriting">
<Resource name="ast">
<![CDATA[
LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[0]], fields=[x, EXPR$1])
LogicalSink(table=[default_catalog.default_database.sink1], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[1]], fields=[EXPR$0, y])
LogicalSink(table=[default_catalog.default_database.sink1], fields=[EXPR$0, y])
+- LogicalProject(EXPR$0=[null:BIGINT], y=[$1])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[0]], fields=[x, EXPR$1])
LogicalSink(table=[default_catalog.default_database.sink1], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source3]])
]]>
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Sink(table=[default_catalog.default_database.sink1], fields=[x, EXPR$1])
+- Union(all=[true], union=[x, EXPR$1])
:- Calc(select=[x, null:BIGINT AS EXPR$1])
: +- TableSourceScan(table=[[default_catalog, default_database, source1, project=[x, y], metadata=[]]], fields=[x, y])(reuse_id=[1])
:- Calc(select=[null:BIGINT AS EXPR$0, y])
: +- Reused(reference_id=[1])
+- Calc(select=[x, null:BIGINT AS EXPR$1])
+- TableSourceScan(table=[[default_catalog, default_database, source3, project=[x], metadata=[]]], fields=[x])
]]>
</Resource>
</TestCase>
<TestCase name="testSinkReuseWithPartialColumnsAndSupportsTargetColumnWriting">
<Resource name="ast">
<![CDATA[
LogicalSink(table=[default_catalog.default_database.sink2], targetColumns=[[0]], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink2], targetColumns=[[1]], fields=[EXPR$0, y])
+- LogicalProject(EXPR$0=[null:BIGINT], y=[$1])
+- LogicalTableScan(table=[[default_catalog, default_database, source1]])

LogicalSink(table=[default_catalog.default_database.sink2], targetColumns=[[0]], fields=[x, EXPR$1])
+- LogicalProject(x=[$0], EXPR$1=[null:BIGINT])
+- LogicalTableScan(table=[[default_catalog, default_database, source3]])
]]>
Expand All @@ -168,14 +197,14 @@ LogicalSink(table=[default_catalog.default_database.sink1], targetColumns=[[0]],
<![CDATA[
TableSourceScan(table=[[default_catalog, default_database, source1, project=[x, y], metadata=[]]], fields=[x, y])(reuse_id=[1])

Sink(table=[default_catalog.default_database.sink1], targetColumns=[[0]], fields=[x, EXPR$1])
Sink(table=[default_catalog.default_database.sink2], targetColumns=[[0]], fields=[x, EXPR$1])
+- Union(all=[true], union=[x, EXPR$1])
:- Calc(select=[x, null:BIGINT AS EXPR$1])
: +- Reused(reference_id=[1])
+- Calc(select=[x, null:BIGINT AS EXPR$1])
+- TableSourceScan(table=[[default_catalog, default_database, source3, project=[x], metadata=[]]], fields=[x])

Sink(table=[default_catalog.default_database.sink1], targetColumns=[[1]], fields=[EXPR$0, y])
Sink(table=[default_catalog.default_database.sink2], targetColumns=[[1]], fields=[EXPR$0, y])
+- Calc(select=[null:BIGINT AS EXPR$0, y])
+- Reused(reference_id=[1])
]]>
Expand Down