Skip to content

[FLINK-37720][table-planner] Enable sink reuse table optimizer by default and apply plan changes to existing test cases #26503

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
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 @@ -85,7 +85,7 @@
</tr>
<tr>
<td><h5>table.optimizer.reuse-sink-enabled</h5><br> <span class="label label-primary">Batch</span> <span class="label label-primary">Streaming</span></td>
<td style="word-wrap: break-word;">false</td>
<td style="word-wrap: break-word;">true</td>
<td>Boolean</td>
<td>When it is true, the optimizer will try to find out duplicated table sinks and reuse them. This works only when table.optimizer.reuse-sub-plan-enabled is true.</td>
</tr>
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -105,19 +105,15 @@ LogicalSink(table=[default_catalog.default_database.StreamingTable], fields=[EXP

== Optimized Physical Plan ==
Sink(table=[default_catalog.default_database.StreamingTable], fields=[EXPR$0, EXPR$1])
+- Values(type=[RecordType(INTEGER EXPR$0, VARCHAR(11) EXPR$1)], tuples=[[{ 1, _UTF-16LE'Hello World' }, { 2, _UTF-16LE'Hi' }, { 2, _UTF-16LE'Hi' }, { 3, _UTF-16LE'Hello' }, { 3, _UTF-16LE'World' }, { 4, _UTF-16LE'ADD' }, { 5, _UTF-16LE'LINE' }]])

Sink(table=[default_catalog.default_database.StreamingTable], fields=[EXPR$0, EXPR$1])
+- Values(type=[RecordType(INTEGER EXPR$0, VARCHAR(11) EXPR$1)], tuples=[[{ 1, _UTF-16LE'Hello World' }, { 2, _UTF-16LE'Hi' }, { 2, _UTF-16LE'Hi' }, { 3, _UTF-16LE'Hello' }, { 3, _UTF-16LE'World' }, { 4, _UTF-16LE'ADD' }, { 5, _UTF-16LE'LINE' }]])
+- Union(all=[true], union=[EXPR$0, EXPR$1])
:- Values(type=[RecordType(INTEGER EXPR$0, VARCHAR(11) EXPR$1)], tuples=[[{ 1, _UTF-16LE'Hello World' }, { 2, _UTF-16LE'Hi' }, { 2, _UTF-16LE'Hi' }, { 3, _UTF-16LE'Hello' }, { 3, _UTF-16LE'World' }, { 4, _UTF-16LE'ADD' }, { 5, _UTF-16LE'LINE' }]])
+- Values(type=[RecordType(INTEGER EXPR$0, VARCHAR(11) EXPR$1)], tuples=[[{ 1, _UTF-16LE'Hello World' }, { 2, _UTF-16LE'Hi' }, { 2, _UTF-16LE'Hi' }, { 3, _UTF-16LE'Hello' }, { 3, _UTF-16LE'World' }, { 4, _UTF-16LE'ADD' }, { 5, _UTF-16LE'LINE' }]])

== Optimized Execution Plan ==
Values(tuples=[[{ 1, _UTF-16LE'Hello World' }, { 2, _UTF-16LE'Hi' }, { 2, _UTF-16LE'Hi' }, { 3, _UTF-16LE'Hello' }, { 3, _UTF-16LE'World' }, { 4, _UTF-16LE'ADD' }, { 5, _UTF-16LE'LINE' }]])(reuse_id=[1])

Sink(table=[default_catalog.default_database.StreamingTable], fields=[EXPR$0, EXPR$1])
+- Reused(reference_id=[1])

Sink(table=[default_catalog.default_database.StreamingTable], fields=[EXPR$0, EXPR$1])
+- Reused(reference_id=[1])
+- Union(all=[true], union=[EXPR$0, EXPR$1])
:- Values(tuples=[[{ 1, _UTF-16LE'Hello World' }, { 2, _UTF-16LE'Hi' }, { 2, _UTF-16LE'Hi' }, { 3, _UTF-16LE'Hello' }, { 3, _UTF-16LE'World' }, { 4, _UTF-16LE'ADD' }, { 5, _UTF-16LE'LINE' }]])(reuse_id=[1])
+- Reused(reference_id=[1])
!ok

EXECUTE STATEMENT SET BEGIN
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ public class OptimizerConfigOptions {
public static final ConfigOption<Boolean> TABLE_OPTIMIZER_REUSE_SINK_ENABLED =
key("table.optimizer.reuse-sink-enabled")
.booleanType()
.defaultValue(false)
.defaultValue(true)
.withDescription(
"When it is true, the optimizer will try to find out duplicated table sinks and "
+ "reuse them. This works only when "
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -9,17 +9,15 @@ LogicalSink(table=[default_catalog.default_database.MySink], fields=[first])

== Optimized Physical Plan ==
Sink(table=[default_catalog.default_database.MySink], fields=[last])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: last)]]], fields=[last])

Sink(table=[default_catalog.default_database.MySink], fields=[first])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])
+- Union(all=[true], union=[last])
:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: last)]]], fields=[last])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])

== Optimized Execution Plan ==
Sink(table=[default_catalog.default_database.MySink], fields=[last])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: last)]]], fields=[last])

Sink(table=[default_catalog.default_database.MySink], fields=[first])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])
+- Union(all=[true], union=[last])
:- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: last)]]], fields=[last])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CsvTableSource(read fields: first)]]], fields=[first])

== Physical Execution Plan ==
{
Expand Down Expand Up @@ -51,17 +49,6 @@ Sink(table=[default_catalog.default_database.MySink], fields=[first])
"ship_strategy" : "FORWARD",
"side" : "second"
} ]
}, {
"id" : ,
"type" : "StreamingFileWriter",
"pact" : "Operator",
"contents" : "StreamingFileWriter",
"parallelism" : 1,
"predecessors" : [ {
"id" : ,
"ship_strategy" : "FORWARD",
"side" : "second"
} ]
}, {
"id" : ,
"type" : "Source: Custom File source",
Expand Down Expand Up @@ -100,14 +87,7 @@ Sink(table=[default_catalog.default_database.MySink], fields=[first])
"id" : ,
"ship_strategy" : "FORWARD",
"side" : "second"
} ]
}, {
"id" : ,
"type" : "end: Writer",
"pact" : "Operator",
"contents" : "end: Writer",
"parallelism" : 1,
"predecessors" : [ {
}, {
"id" : ,
"ship_strategy" : "FORWARD",
"side" : "second"
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -157,11 +157,10 @@ LogicalSink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
<![CDATA[
Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
+- Union(all=[true], union=[a, b, c])
:- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=0}]]])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=1}]]])

Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=2}]]])
:- Union(all=[true], union=[a, b, c])
: :- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=0}]]])
: +- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=1}]]])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=2}]]])
]]>
</Resource>
</TestCase>
Expand Down Expand Up @@ -196,13 +195,10 @@ LogicalSink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=1}]]])(reuse_id=[1])

Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
+- Reused(reference_id=[1])

Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
+- Reused(reference_id=[1])
+- Union(all=[true], union=[a, b, c])
:- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=1}]]])(reuse_id=[1])
+- Reused(reference_id=[1])
]]>
</Resource>
</TestCase>
Expand All @@ -221,10 +217,9 @@ LogicalSink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
<Resource name="optimized exec plan">
<![CDATA[
Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=1}]]])

Sink(table=[default_catalog.default_database.MySink], fields=[a, b, c])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=2}]]])
+- Union(all=[true], union=[a, b, c])
:- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=1}]]])
+- TableSourceScan(table=[[default_catalog, default_database, MyTable]], fields=[a, b, c], hints=[[[OPTIONS options:{source.num-element-to-skip=2}]]])
]]>
</Resource>
</TestCase>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -367,32 +367,31 @@ LogicalSink(table=[default_catalog.default_database.sink2], fields=[number, word
<Resource name="optimized rel plan">
<![CDATA[
Sink(table=[default_catalog.default_database.sink1], fields=[number, word], changelogMode=[NONE])
+- Calc(select=[number, word], where=[>(word, 'a')], changelogMode=[I,UB,UA])
+- Union(all=[true], union=[number, word], changelogMode=[I,UB,UA])
:- Calc(select=[+(number, 1) AS number, word], changelogMode=[I,UB,UA])
: +- Calc(select=[number, word], changelogMode=[I,UB,UA])
: +- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
: +- Exchange(distribution=[hash[word]], changelogMode=[I])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])
+- Calc(select=[-(number, 1) AS number, word], changelogMode=[I,UB,UA])
+- Calc(select=[number, word], changelogMode=[I,UB,UA])
+- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
+- Exchange(distribution=[hash[word]], changelogMode=[I])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])

Sink(table=[default_catalog.default_database.sink1], fields=[number, word], changelogMode=[NONE])
+- Calc(select=[number, word], where=[<(word, 'a')], changelogMode=[I,UB,UA])
+- Union(all=[true], union=[number, word], changelogMode=[I,UB,UA])
:- Calc(select=[+(number, 1) AS number, word], changelogMode=[I,UB,UA])
: +- Calc(select=[number, word], changelogMode=[I,UB,UA])
: +- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
: +- Exchange(distribution=[hash[word]], changelogMode=[I])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])
+- Calc(select=[-(number, 1) AS number, word], changelogMode=[I,UB,UA])
+- Calc(select=[number, word], changelogMode=[I,UB,UA])
+- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
+- Exchange(distribution=[hash[word]], changelogMode=[I])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])
+- Union(all=[true], union=[number, word], changelogMode=[I,UB,UA])
:- Calc(select=[number, word], where=[>(word, 'a')], changelogMode=[I,UB,UA])
: +- Union(all=[true], union=[number, word], changelogMode=[I,UB,UA])
: :- Calc(select=[+(number, 1) AS number, word], changelogMode=[I,UB,UA])
: : +- Calc(select=[number, word], changelogMode=[I,UB,UA])
: : +- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
: : +- Exchange(distribution=[hash[word]], changelogMode=[I])
: : +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])
: +- Calc(select=[-(number, 1) AS number, word], changelogMode=[I,UB,UA])
: +- Calc(select=[number, word], changelogMode=[I,UB,UA])
: +- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
: +- Exchange(distribution=[hash[word]], changelogMode=[I])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])
+- Calc(select=[number, word], where=[<(word, 'a')], changelogMode=[I,UB,UA])
+- Union(all=[true], union=[number, word], changelogMode=[I,UB,UA])
:- Calc(select=[+(number, 1) AS number, word], changelogMode=[I,UB,UA])
: +- Calc(select=[number, word], changelogMode=[I,UB,UA])
: +- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
: +- Exchange(distribution=[hash[word]], changelogMode=[I])
: +- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])
+- Calc(select=[-(number, 1) AS number, word], changelogMode=[I,UB,UA])
+- Calc(select=[number, word], changelogMode=[I,UB,UA])
+- GroupAggregate(groupBy=[word], select=[word, SUM(number) AS number], changelogMode=[I,UB,UA])
+- Exchange(distribution=[hash[word]], changelogMode=[I])
+- LegacyTableSourceScan(table=[[default_catalog, default_database, MyTable, source: [CollectionTableSource(word, number)]]], fields=[word, number], changelogMode=[I])

Sink(table=[default_catalog.default_database.sink2], fields=[number, word], changelogMode=[NONE])
+- Calc(select=[number, word], changelogMode=[I,UB,UA])
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -129,16 +129,13 @@ LogicalSink(table=[default_catalog.default_database.t_keyed_sink], fields=[name,
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Exchange(distribution=[hash[name]])(reuse_id=[1])
+- Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])

Sink(table=[default_catalog.default_database.t_keyed_sink], fields=[name, out])
+- ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'a')], uid=[a], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])
+- Reused(reference_id=[1])

Sink(table=[default_catalog.default_database.t_keyed_sink], fields=[name, out])
+- ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'b')], uid=[b], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])
+- Reused(reference_id=[1])
+- Union(all=[true], union=[name, out])
:- ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'a')], uid=[a], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])
: +- Exchange(distribution=[hash[name]])(reuse_id=[1])
: +- Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])
+- ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'b')], uid=[b], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])
+- Reused(reference_id=[1])
]]>
</Resource>
</TestCase>
Expand Down Expand Up @@ -179,15 +176,12 @@ LogicalSink(table=[default_catalog.default_database.t_sink], fields=[out])
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])(reuse_id=[1])

Sink(table=[default_catalog.default_database.t_sink], fields=[out])
+- ProcessTableFunction(invocation=[f(TABLE(#0), 1, DEFAULT(), DEFAULT())], uid=[null], select=[out], rowType=[RecordType(VARCHAR(2147483647) out)])
+- Reused(reference_id=[1])

Sink(table=[default_catalog.default_database.t_sink], fields=[out])
+- ProcessTableFunction(invocation=[f(TABLE(#0), 42, DEFAULT(), DEFAULT())], uid=[null], select=[out], rowType=[RecordType(VARCHAR(2147483647) out)])
+- Reused(reference_id=[1])
+- Union(all=[true], union=[out])
:- ProcessTableFunction(invocation=[f(TABLE(#0), 1, DEFAULT(), DEFAULT())], uid=[null], select=[out], rowType=[RecordType(VARCHAR(2147483647) out)])
: +- Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])(reuse_id=[1])
+- ProcessTableFunction(invocation=[f(TABLE(#0), 42, DEFAULT(), DEFAULT())], uid=[null], select=[out], rowType=[RecordType(VARCHAR(2147483647) out)])
+- Reused(reference_id=[1])
]]>
</Resource>
</TestCase>
Expand Down Expand Up @@ -252,15 +246,12 @@ LogicalSink(table=[default_catalog.default_database.t_keyed_sink], fields=[name,
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'same')], uid=[same], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])(reuse_id=[1])
+- Exchange(distribution=[hash[name]])
+- Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])

Sink(table=[default_catalog.default_database.t_keyed_sink], fields=[name, out])
+- Reused(reference_id=[1])

Sink(table=[default_catalog.default_database.t_keyed_sink], fields=[name, out])
+- Reused(reference_id=[1])
+- Union(all=[true], union=[name, out])
:- ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'same')], uid=[same], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])(reuse_id=[1])
: +- Exchange(distribution=[hash[name]])
: +- Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])
+- Reused(reference_id=[1])
]]>
</Resource>
</TestCase>
Expand All @@ -286,17 +277,14 @@ LogicalSink(table=[default_catalog.default_database.t_keyed_sink], fields=[name,
</Resource>
<Resource name="optimized exec plan">
<![CDATA[
ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'same')], uid=[same], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])(reuse_id=[1])
+- Exchange(distribution=[hash[name]])
+- Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])

Sink(table=[default_catalog.default_database.t_keyed_sink], fields=[name, out])
+- Calc(select=['Bob' AS name, out], where=[(name = 'Bob')])
+- Reused(reference_id=[1])

Sink(table=[default_catalog.default_database.t_keyed_sink], fields=[name, out])
+- Calc(select=['Alice' AS name, out], where=[(name = 'Alice')])
+- Reused(reference_id=[1])
+- Union(all=[true], union=[name, out])
:- Calc(select=['Bob' AS name, out], where=[(name = 'Bob')])
: +- ProcessTableFunction(invocation=[f(TABLE(#0) PARTITION BY($0), 1, DEFAULT(), _UTF-16LE'same')], uid=[same], select=[name,out], rowType=[RecordType(VARCHAR(5) name, VARCHAR(2147483647) out)])(reuse_id=[1])
: +- Exchange(distribution=[hash[name]])
: +- Values(tuples=[[{ _UTF-16LE'Bob', 12 }, { _UTF-16LE'Alice', 42 }]])
+- Calc(select=['Alice' AS name, out], where=[(name = 'Alice')])
+- Reused(reference_id=[1])
]]>
</Resource>
</TestCase>
Expand Down
Loading