Skip to content
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

fix(optimizer): reduce expr tree depth when merge logical operations #17342

Merged
merged 11 commits into from
Jun 21, 2024
Original file line number Diff line number Diff line change
Expand Up @@ -332,7 +332,7 @@
select * from t1 where a = 1 or b = 2 or c = 3 or p = 4 or a = 5
batch_plan: |-
BatchExchange { order: [], dist: Single }
└─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR (t1.c = 3:Int32)) OR (t1.p = 4:Int32)) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 }
└─BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR ((t1.c = 3:Int32) OR (t1.p = 4:Int32))) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 }
└─BatchExchange { order: [], dist: UpstreamHashShard(idx1.t1._row_id) }
└─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] }
└─BatchExchange { order: [], dist: HashShard(idx1.t1._row_id) }
Expand All @@ -346,7 +346,7 @@
└─BatchExchange { order: [], dist: Single }
└─BatchScan { table: idx4, columns: [idx4.t1._row_id], scan_ranges: [idx4.p = Int32(4)], distribution: SomeShard }
batch_local_plan: |-
BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND (((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR (t1.c = 3:Int32)) OR (t1.p = 4:Int32)) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 }
BatchLookupJoin { type: Inner, predicate: idx1.t1._row_id IS NOT DISTINCT FROM t1._row_id AND ((((t1.a = 1:Int32) OR (t1.b = 2:Decimal)) OR ((t1.c = 3:Int32) OR (t1.p = 4:Int32))) OR (t1.a = 5:Int32)), output: [t1.a, t1.b, t1.c, t1.p], lookup table: t1 }
└─BatchHashAgg { group_key: [idx1.t1._row_id], aggs: [] }
└─BatchUnion { all: true }
├─BatchExchange { order: [], dist: Single }
Expand Down
2 changes: 1 addition & 1 deletion src/frontend/planner_test/tests/testdata/output/join.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -204,7 +204,7 @@
StreamMaterialize { columns: [x, i.t._row_id(hidden), i.t._row_id#1(hidden), i.x(hidden), i.t._row_id#2(hidden), i.t._row_id#3(hidden), i.x#1(hidden)], stream_key: [i.t._row_id, i.t._row_id#1, i.x, i.t._row_id#2, i.t._row_id#3, i.x#1], pk_columns: [i.t._row_id, i.t._row_id#1, i.x, i.t._row_id#2, i.t._row_id#3, i.x#1], pk_conflict: NoCheck }
└─StreamExchange { dist: HashShard(i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x) }
└─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr1, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] }
└─StreamFilter { predicate: (((((IsNotNull(i.t._row_id) OR IsNotNull(i.t._row_id)) OR IsNotNull(i.x)) OR IsNotNull(i.t._row_id)) OR IsNotNull(i.t._row_id)) OR IsNotNull(i.x)) }
└─StreamFilter { predicate: (((IsNotNull(i.t._row_id) OR IsNotNull(i.t._row_id)) OR (IsNotNull(i.x) OR IsNotNull(i.t._row_id))) OR (IsNotNull(i.t._row_id) OR IsNotNull(i.x))) }
└─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.t._row_id, i.t._row_id] }
├─StreamShare { id: 4 }
│ └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id] }
Expand Down
8 changes: 4 additions & 4 deletions src/frontend/planner_test/tests/testdata/output/nexmark.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -107,20 +107,20 @@
sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
batch_plan: |-
BatchExchange { order: [], dist: Single }
└─BatchFilter { predicate: (((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR (bid.auction = 2001:Int32)) OR (bid.auction = 2019:Int32)) OR (bid.auction = 2087:Int32)) }
└─BatchFilter { predicate: ((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR ((bid.auction = 2001:Int32) OR (bid.auction = 2019:Int32))) OR (bid.auction = 2087:Int32)) }
└─BatchScan { table: bid, columns: [bid.auction, bid.price], distribution: SomeShard }
sink_plan: |-
StreamSink { type: append-only, columns: [auction, price, bid._row_id(hidden)] }
└─StreamFilter { predicate: (((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR (bid.auction = 2001:Int32)) OR (bid.auction = 2019:Int32)) OR (bid.auction = 2087:Int32)) }
└─StreamFilter { predicate: ((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR ((bid.auction = 2001:Int32) OR (bid.auction = 2019:Int32))) OR (bid.auction = 2087:Int32)) }
└─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) }
stream_plan: |-
StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck }
└─StreamFilter { predicate: (((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR (bid.auction = 2001:Int32)) OR (bid.auction = 2019:Int32)) OR (bid.auction = 2087:Int32)) }
└─StreamFilter { predicate: ((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR ((bid.auction = 2001:Int32) OR (bid.auction = 2019:Int32))) OR (bid.auction = 2087:Int32)) }
└─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction, price, bid._row_id(hidden)], stream_key: [bid._row_id], pk_columns: [bid._row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] }
└── StreamFilter { predicate: (((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR (bid.auction = 2001:Int32)) OR (bid.auction = 2019:Int32)) OR (bid.auction = 2087:Int32)) }
└── StreamFilter { predicate: ((((bid.auction = 1007:Int32) OR (bid.auction = 1020:Int32)) OR ((bid.auction = 2001:Int32) OR (bid.auction = 2019:Int32))) OR (bid.auction = 2087:Int32)) }
└── StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) }
├── tables: [ StreamScan: 0 ]
├── Upstream
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,21 +116,21 @@
sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
batch_plan: |-
BatchExchange { order: [], dist: Single }
└─BatchFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) }
└─BatchFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) }
└─BatchProject { exprs: [auction, price] }
└─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
stream_plan: |-
StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck }
└─StreamProject { exprs: [auction, price, _row_id] }
└─StreamFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) }
└─StreamFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) }
└─StreamRowIdGen { row_id_index: 7 }
└─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] }
stream_dist_plan: |+
Fragment 0
StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck }
├── tables: [ Materialize: 4294967294 ]
└── StreamProject { exprs: [auction, price, _row_id] }
└── StreamFilter { predicate: (((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR (auction = 2001:Int32)) OR (auction = 2019:Int32)) OR (auction = 2087:Int32)) }
└── StreamFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) }
└── StreamRowIdGen { row_id_index: 7 }
└── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 0 ] }

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -116,7 +116,7 @@
StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck }
└─StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, _row_id] }
└─StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, person, auction, bid, $expr1, _row_id], cleaned_by_watermark: true }
├─StreamFilter { predicate: (((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR (Field(bid, 0:Int32) = 2001:Int32)) OR (Field(bid, 0:Int32) = 2019:Int32)) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
├─StreamFilter { predicate: ((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR ((Field(bid, 0:Int32) = 2001:Int32) OR (Field(bid, 0:Int32) = 2019:Int32))) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
│ └─StreamRowIdGen { row_id_index: 5 }
│ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
│ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
Expand All @@ -128,7 +128,7 @@
StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] }
└── StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 2:Int32) as $expr4, _row_id] }
└── StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, person, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { tables: [ DynamicFilterLeft: 0, DynamicFilterRight: 1 ] }
├── StreamFilter { predicate: (((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR (Field(bid, 0:Int32) = 2001:Int32)) OR (Field(bid, 0:Int32) = 2019:Int32)) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
├── StreamFilter { predicate: ((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR ((Field(bid, 0:Int32) = 2001:Int32) OR (Field(bid, 0:Int32) = 2019:Int32))) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
│ └── StreamRowIdGen { row_id_index: 5 }
│ └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] }
│ └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 2 ] }
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -77,13 +77,13 @@
batch_plan: |-
BatchExchange { order: [], dist: Single }
└─BatchProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 2:Int32) as $expr3] }
└─BatchFilter { predicate: (((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR (Field(bid, 0:Int32) = 2001:Int32)) OR (Field(bid, 0:Int32) = 2019:Int32)) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
└─BatchFilter { predicate: ((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR ((Field(bid, 0:Int32) = 2001:Int32) OR (Field(bid, 0:Int32) = 2019:Int32))) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
└─BatchProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
└─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] }
stream_plan: |-
StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck }
└─StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, _row_id] }
└─StreamFilter { predicate: (((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR (Field(bid, 0:Int32) = 2001:Int32)) OR (Field(bid, 0:Int32) = 2019:Int32)) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
└─StreamFilter { predicate: ((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR ((Field(bid, 0:Int32) = 2001:Int32) OR (Field(bid, 0:Int32) = 2019:Int32))) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
└─StreamRowIdGen { row_id_index: 5 }
└─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] }
└─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
Expand All @@ -92,7 +92,7 @@
Fragment 0
StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] }
└── StreamProject { exprs: [Field(bid, 0:Int32) as $expr2, Field(bid, 2:Int32) as $expr3, _row_id] }
└── StreamFilter { predicate: (((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR (Field(bid, 0:Int32) = 2001:Int32)) OR (Field(bid, 0:Int32) = 2019:Int32)) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
└── StreamFilter { predicate: ((((Field(bid, 0:Int32) = 1007:Int32) OR (Field(bid, 0:Int32) = 1020:Int32)) OR ((Field(bid, 0:Int32) = 2001:Int32) OR (Field(bid, 0:Int32) = 2019:Int32))) OR (Field(bid, 0:Int32) = 2087:Int32)) AND (event_type = 2:Int32) }
└── StreamRowIdGen { row_id_index: 5 }
└── StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [$expr1] } { tables: [ WatermarkFilter: 0 ] }
└── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] }
Expand Down
Loading
Loading