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

refactor(optimizer): divide logical optimizer into one for batch and one for streaming. #8192

Merged
merged 8 commits into from
Feb 28, 2023
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
1 change: 1 addition & 0 deletions Cargo.lock

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

1 change: 1 addition & 0 deletions src/frontend/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -31,6 +31,7 @@ fixedbitset = "0.4.1"
futures = { version = "0.3", default-features = false, features = ["alloc"] }
futures-async-stream = "0.2"
itertools = "0.10"
lazy_static = "1"
maplit = "1"
md5 = "0.7.0"
num-integer = "0.1"
Expand Down
4 changes: 3 additions & 1 deletion src/frontend/planner_test/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -502,7 +502,9 @@ impl TestCase {
};

if self.optimized_logical_plan.is_some() || self.optimizer_error.is_some() {
let optimized_logical_plan = match logical_plan.gen_optimized_logical_plan() {
// TODO: separate `optimized_logical_plan` into `optimized_logical_plan_for_batch` and
// `optimized_logical_plan_for_stream`
let optimized_logical_plan = match logical_plan.gen_optimized_logical_plan_for_batch() {
Ok(optimized_logical_plan) => optimized_logical_plan,
Err(err) => {
ret.optimizer_error = Some(err.to_string());
Expand Down
8 changes: 4 additions & 4 deletions src/frontend/planner_test/tests/testdata/join.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -200,14 +200,14 @@
└─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr1, i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x] }
└─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] }
├─StreamProject { exprs: [i.x, i.t._row_id, i.t._row_id, i.x] }
| └─StreamShare { id = 513 }
| └─StreamShare { id = 503 }
| └─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] }
| ├─StreamExchange { dist: HashShard(i.x) }
| | └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
| └─StreamExchange { dist: HashShard(i.x) }
| └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
└─StreamProject { exprs: [i.x, i.t._row_id, i.t._row_id, i.x] }
└─StreamShare { id = 513 }
└─StreamShare { id = 503 }
└─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] }
├─StreamExchange { dist: HashShard(i.x) }
| └─StreamTableScan { table: i, columns: [i.x, i.t._row_id], pk: [i.t._row_id], dist: UpstreamHashShard(i.x) }
Expand Down Expand Up @@ -503,7 +503,7 @@
└─BatchExchange { order: [], dist: HashShard(b.x) }
└─BatchScan { table: b, columns: [b.x], distribution: SomeShard }
stream_plan: |
StreamMaterialize { columns: [y, z, $expr159(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr159, a._row_id, b._row_id, a.x, b.x], pk_conflict: "no check" }
StreamMaterialize { columns: [y, z, $expr153(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr153, a._row_id, b._row_id, a.x, b.x], pk_conflict: "no check" }
└─StreamExchange { dist: HashShard(a._row_id, b._row_id, a.x, b.x) }
└─StreamProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr1, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr2, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr3, a._row_id, b._row_id, a.x, b.x] }
└─StreamFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) }
Expand Down Expand Up @@ -602,7 +602,7 @@
└─BatchExchange { order: [], dist: HashShard(t2.v2) }
└─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard }
stream_plan: |
StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr29(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr29, v2], pk_conflict: "no check" }
StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr28(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr28, v2], pk_conflict: "no check" }
└─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] }
├─StreamExchange { dist: HashShard($expr1) }
| └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] }
Expand Down
98 changes: 49 additions & 49 deletions src/frontend/planner_test/tests/testdata/nexmark.yaml

Large diffs are not rendered by default.

106 changes: 53 additions & 53 deletions src/frontend/planner_test/tests/testdata/nexmark_source.yaml

Large diffs are not rendered by default.

2 changes: 1 addition & 1 deletion src/frontend/planner_test/tests/testdata/order_by.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,7 @@
└─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z] }
└─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard }
stream_plan: |
StreamMaterialize { columns: [x, y, $expr72(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr72, t.z, t._row_id], pk_conflict: "no check" }
StreamMaterialize { columns: [x, y, $expr71(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr71, t.z, t._row_id], pk_conflict: "no check" }
└─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] }
└─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- name: order by the number of an output column
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/planner_test/tests/testdata/pk_derive.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -56,13 +56,13 @@
| └─StreamHashAgg { group_key: [t.id], aggs: [count, max(t.v)] }
| └─StreamExchange { dist: HashShard(t.id) }
| └─StreamProject { exprs: [t.id, t.v, t._row_id] }
| └─StreamShare { id = 333 }
| └─StreamShare { id = 325 }
| └─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
└─StreamProject { exprs: [min(t.v), t.id] }
└─StreamHashAgg { group_key: [t.id], aggs: [count, min(t.v)] }
└─StreamExchange { dist: HashShard(t.id) }
└─StreamProject { exprs: [t.id, t.v, t._row_id] }
└─StreamShare { id = 333 }
└─StreamShare { id = 325 }
└─StreamTableScan { table: t, columns: [t.id, t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
- sql: |
create table t (v1 varchar, v2 varchar, v3 varchar);
Expand Down
4 changes: 2 additions & 2 deletions src/frontend/planner_test/tests/testdata/project_set.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -157,13 +157,13 @@
└─StreamHashJoin { type: Inner, predicate: Unnest($0) = Unnest($0), output: [Unnest($0), t._row_id, projected_row_id, t._row_id, projected_row_id, Unnest($0)] }
├─StreamExchange { dist: HashShard(Unnest($0)) }
| └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
| └─StreamShare { id = 488 }
| └─StreamShare { id = 477 }
| └─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
| └─StreamProjectSet { select_list: [Unnest($0), $1] }
| └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
└─StreamExchange { dist: HashShard(Unnest($0)) }
└─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
└─StreamShare { id = 488 }
└─StreamShare { id = 477 }
└─StreamProject { exprs: [Unnest($0), t._row_id, projected_row_id] }
└─StreamProjectSet { select_list: [Unnest($0), $1] }
└─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
16 changes: 8 additions & 8 deletions src/frontend/planner_test/tests/testdata/share.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -44,7 +44,7 @@
| └─StreamProject { exprs: [id, _row_id] }
| └─StreamFilter { predicate: (initial_bid = 1:Int32) }
| └─StreamProject { exprs: [id, initial_bid, _row_id] }
| └─StreamShare { id = 661 }
| └─StreamShare { id = 650 }
| └─StreamProject { exprs: [id, initial_bid, _row_id] }
| └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }
| └─StreamRowIdGen { row_id_index: 10 }
Expand All @@ -53,7 +53,7 @@
└─StreamProject { exprs: [id, _row_id] }
└─StreamFilter { predicate: (initial_bid = 2:Int32) }
└─StreamProject { exprs: [id, initial_bid, _row_id] }
└─StreamShare { id = 661 }
└─StreamShare { id = 650 }
└─StreamProject { exprs: [id, initial_bid, _row_id] }
└─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) }
└─StreamRowIdGen { row_id_index: 10 }
Expand Down Expand Up @@ -117,7 +117,7 @@
└─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all }
├─StreamExchange { dist: HashShard(window_start) }
| └─StreamProject { exprs: [auction, count, window_start] }
| └─StreamShare { id = 1091 }
| └─StreamShare { id = 1064 }
| └─StreamProject { exprs: [auction, window_start, count] }
| └─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count, count] }
| └─StreamExchange { dist: HashShard(auction, window_start) }
Expand All @@ -130,7 +130,7 @@
└─StreamHashAgg { group_key: [window_start], aggs: [count, max(count)] }
└─StreamExchange { dist: HashShard(window_start) }
└─StreamProject { exprs: [auction, window_start, count] }
└─StreamShare { id = 1091 }
└─StreamShare { id = 1064 }
└─StreamProject { exprs: [auction, window_start, count] }
└─StreamAppendOnlyHashAgg { group_key: [auction, window_start], aggs: [count, count] }
└─StreamExchange { dist: HashShard(auction, window_start) }
Expand All @@ -148,15 +148,15 @@
└─StreamUnion { all: true }
├─StreamExchange { dist: HashShard(0:Int32) }
| └─StreamProject { exprs: [sum0(count), 0:Int32] }
| └─StreamShare { id = 334 }
| └─StreamShare { id = 325 }
| └─StreamProject { exprs: [sum0(count)] }
| └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] }
| └─StreamExchange { dist: Single }
| └─StreamStatelessLocalSimpleAgg { aggs: [count, count] }
| └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) }
└─StreamExchange { dist: HashShard(1:Int32) }
└─StreamProject { exprs: [sum0(count), 1:Int32] }
└─StreamShare { id = 334 }
└─StreamShare { id = 325 }
└─StreamProject { exprs: [sum0(count)] }
└─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] }
└─StreamExchange { dist: Single }
Expand Down Expand Up @@ -196,13 +196,13 @@
└─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] }
├─StreamExchange { dist: HashShard(id) }
| └─StreamProject { exprs: [id, _row_id] }
| └─StreamShare { id = 251 }
| └─StreamShare { id = 247 }
| └─StreamProject { exprs: [id, _row_id] }
| └─StreamRowIdGen { row_id_index: 10 }
| └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] }
└─StreamExchange { dist: HashShard(id) }
└─StreamProject { exprs: [id, _row_id] }
└─StreamShare { id = 251 }
└─StreamShare { id = 247 }
└─StreamProject { exprs: [id, _row_id] }
└─StreamRowIdGen { row_id_index: 10 }
└─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] }
6 changes: 3 additions & 3 deletions src/frontend/planner_test/tests/testdata/shared_views.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,11 @@
└─LogicalFilter { predicate: (t1.y > 0:Int32) }
└─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] }
stream_plan: |
StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr93(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr93, z, a], pk_conflict: "no check" }
StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr89(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr89, z, a], pk_conflict: "no check" }
└─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr1] }
├─StreamExchange { dist: HashShard($expr1) }
| └─StreamProject { exprs: [$expr1, t1._row_id] }
| └─StreamShare { id = 365 }
| └─StreamShare { id = 355 }
| └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] }
| └─StreamFilter { predicate: (t1.y > 0:Int32) }
| └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
Expand All @@ -37,7 +37,7 @@
| └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
└─StreamExchange { dist: HashShard($expr1) }
└─StreamProject { exprs: [$expr1, t1._row_id] }
└─StreamShare { id = 365 }
└─StreamShare { id = 355 }
└─StreamProject { exprs: [(t1.x + t1.y) as $expr1, t1._row_id] }
└─StreamFilter { predicate: (t1.y > 0:Int32) }
└─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) }
12 changes: 5 additions & 7 deletions src/frontend/planner_test/tests/testdata/subquery.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -218,16 +218,14 @@
└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }
optimized_logical_plan: |
LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(auction.date_time, auction.date_time), output: all }
├─LogicalShare { id = 248 }
| └─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all }
| └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) }
├─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all }
| └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) }
└─LogicalProject { exprs: [auction.date_time] }
└─LogicalAgg { group_key: [auction.date_time, auction.date_time], aggs: [] }
└─LogicalJoin { type: Inner, on: true, output: [auction.date_time, auction.date_time] }
├─LogicalAgg { group_key: [auction.date_time], aggs: [] }
| └─LogicalShare { id = 248 }
| └─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all }
| └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) }
| └─LogicalHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] }
| └─LogicalScan { table: auction, columns: [auction.date_time], predicate: IsNotNull(auction.date_time) }
└─LogicalValues { rows: [[]], schema: Schema { fields: [] } }
batch_plan: |
BatchExchange { order: [], dist: Single }
Expand All @@ -243,7 +241,7 @@
└─BatchNestedLoopJoin { type: Inner, predicate: true, output: [auction.date_time, auction.date_time] }
├─BatchExchange { order: [], dist: Single }
| └─BatchHashAgg { group_key: [auction.date_time], aggs: [] }
| └─BatchHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: all }
| └─BatchHopWindow { time_col: auction.date_time, slide: 00:00:01, size: 01:00:00, output: [auction.date_time] }
| └─BatchExchange { order: [], dist: HashShard(auction.date_time) }
| └─BatchFilter { predicate: IsNotNull(auction.date_time) }
| └─BatchScan { table: auction, columns: [auction.date_time], distribution: SomeShard }
Expand Down
Loading