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 5 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
5 changes: 4 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,10 @@ 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_stream()
{
Ok(optimized_logical_plan) => optimized_logical_plan,
Err(err) => {
ret.optimizer_error = Some(err.to_string());
Expand Down
27 changes: 10 additions & 17 deletions src/frontend/planner_test/tests/testdata/agg.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -701,18 +701,15 @@
create table t(a int, b int, c int);
select a, count(distinct b) as distinct_b_num, sum(distinct c) filter(where c < 100) as distinct_c_sum from t group by a;
optimized_logical_plan: |
LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 0:Int64)), sum(t.c) filter((count filter((t.c < 100:Int32)) > 0:Int64) AND (flag = 1:Int64))] }
└─LogicalAgg { group_key: [t.a, t.b, t.c, flag], aggs: [count filter((t.c < 100:Int32))] }
└─LogicalExpand { column_subsets: [[t.a, t.b], [t.a, t.c]] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
chenzl25 marked this conversation as resolved.
Show resolved Hide resolved
LogicalAgg { group_key: [t.a], aggs: [count(distinct t.b), sum(distinct t.c) filter((t.c < 100:Int32))] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
- name: single distinct agg and non-disintct agg
sql: |
create table t(a int, b int, c int);
select a, count(distinct b) as distinct_b_num, sum(c) as sum_c from t group by a;
optimized_logical_plan: |
LogicalAgg { group_key: [t.a], aggs: [count(t.b), sum(sum(t.c))] }
└─LogicalAgg { group_key: [t.a, t.b], aggs: [sum(t.c)] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
LogicalAgg { group_key: [t.a], aggs: [count(distinct t.b), sum(t.c)] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
batch_plan: |
BatchExchange { order: [], dist: Single }
└─BatchHashAgg { group_key: [t.a], aggs: [count(t.b), sum(sum(t.c))] }
Expand All @@ -731,10 +728,8 @@
create table t(a int, b int, c int);
select a, count(distinct b) as distinct_b_num, count(distinct c) as distinct_c_sum, sum(c) as sum_c from t group by a;
optimized_logical_plan: |
LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 1:Int64)), count(t.c) filter((flag = 0:Int64)), sum(sum(t.c)) filter((flag = 0:Int64))] }
└─LogicalAgg { group_key: [t.a, t.b, t.c, flag], aggs: [sum(t.c)] }
└─LogicalExpand { column_subsets: [[t.a, t.c], [t.a, t.b]] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
LogicalAgg { group_key: [t.a], aggs: [count(distinct t.b), count(distinct t.c), sum(t.c)] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
batch_plan: |
BatchExchange { order: [], dist: Single }
└─BatchHashAgg { group_key: [t.a], aggs: [count(t.b) filter((flag = 1:Int64)), count(t.c) filter((flag = 0:Int64)), sum(sum(t.c)) filter((flag = 0:Int64))] }
Expand All @@ -754,9 +749,8 @@
create table t(a int, b int, c int);
select a, count(distinct b) filter(where b < 100), sum(c) from t group by a;
optimized_logical_plan: |
LogicalAgg { group_key: [t.a], aggs: [count(t.b) filter((count filter((t.b < 100:Int32)) > 0:Int64)), sum(sum(t.c))] }
└─LogicalAgg { group_key: [t.a, t.b], aggs: [count filter((t.b < 100:Int32)), sum(t.c)] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
LogicalAgg { group_key: [t.a], aggs: [count(distinct t.b) filter((t.b < 100:Int32)), sum(t.c)] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
batch_plan: |
BatchExchange { order: [], dist: Single }
└─BatchHashAgg { group_key: [t.a], aggs: [count(t.b) filter((count filter((t.b < 100:Int32)) > 0:Int64)), sum(sum(t.c))] }
Expand All @@ -775,9 +769,8 @@
create table t(a int, b int, c int);
select a, count(distinct b), sum(c) filter(where b < 100) from t group by a;
optimized_logical_plan: |
LogicalAgg { group_key: [t.a], aggs: [count(t.b), sum(sum(t.c) filter((t.b < 100:Int32)))] }
└─LogicalAgg { group_key: [t.a, t.b], aggs: [sum(t.c) filter((t.b < 100:Int32))] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
LogicalAgg { group_key: [t.a], aggs: [count(distinct t.b), sum(t.c) filter((t.b < 100:Int32))] }
└─LogicalScan { table: t, columns: [t.a, t.b, t.c] }
- name: combined order by & filter clauses
sql: |
create table t(a varchar, b int);
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
Loading