From 853722f865783bff489f1cd90d16d75512613c40 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Tue, 6 Feb 2024 13:49:19 +0800 Subject: [PATCH 1/4] support PullUpCorrelatedPredicateAggRule --- .../testdata/output/lateral_subquery.yaml | 55 +- .../tests/testdata/output/subquery.yaml | 40 +- .../output/subquery_expr_correlated.yaml | 160 ++--- .../tests/testdata/output/tpch.yaml | 490 ++++++-------- .../tests/testdata/output/tpch_variant.yaml | 629 +++++++----------- .../src/optimizer/logical_optimization.rs | 1 + src/frontend/src/optimizer/rule/mod.rs | 3 + .../pull_up_correlated_predicate_agg_rule.rs | 236 +++++++ 8 files changed, 770 insertions(+), 844 deletions(-) create mode 100644 src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs diff --git a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml index 85bfb1a6cda3..790227117865 100644 --- a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml @@ -26,39 +26,30 @@ AS max_sale_customer; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name] } - ├─BatchHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount] } - │ ├─BatchExchange { order: [], dist: HashShard(salesperson.id) } - │ │ └─BatchScan { table: salesperson, columns: [salesperson.id, salesperson.name], distribution: SomeShard } - │ └─BatchExchange { order: [], dist: HashShard(all_sales.salesperson_id) } - │ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } - └─BatchHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount)] } - └─BatchHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount] } - ├─BatchHashAgg { group_key: [salesperson.id], aggs: [] } - │ └─BatchExchange { order: [], dist: HashShard(salesperson.id) } - │ └─BatchScan { table: salesperson, columns: [salesperson.id], distribution: SomeShard } - └─BatchExchange { order: [], dist: HashShard(all_sales.salesperson_id) } - └─BatchFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount], distribution: SomeShard } + └─BatchHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id AND max(all_sales.amount) = all_sales.amount, output: [salesperson.name, max(all_sales.amount), all_sales.customer_name] } + ├─BatchExchange { order: [], dist: HashShard(salesperson.id, max(all_sales.amount)) } + │ └─BatchHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, max(all_sales.amount)] } + │ ├─BatchExchange { order: [], dist: HashShard(salesperson.id) } + │ │ └─BatchScan { table: salesperson, columns: [salesperson.id, salesperson.name], distribution: SomeShard } + │ └─BatchHashAgg { group_key: [all_sales.salesperson_id], aggs: [max(all_sales.amount)] } + │ └─BatchExchange { order: [], dist: HashShard(all_sales.salesperson_id) } + │ └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(all_sales.salesperson_id, all_sales.amount) } + └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), all_sales._row_id(hidden), salesperson.id(hidden), all_sales.amount(hidden), salesperson.id#1(hidden)], stream_key: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_columns: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount) } - └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] } - ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] } - │ ├─StreamExchange { dist: HashShard(salesperson.id) } - │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - └─StreamProject { exprs: [salesperson.id, max(all_sales.amount)] } - └─StreamHashAgg { group_key: [salesperson.id], aggs: [max(all_sales.amount), count] } - └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.id, all_sales.amount, all_sales._row_id] } - ├─StreamProject { exprs: [salesperson.id] } - │ └─StreamHashAgg { group_key: [salesperson.id], aggs: [count] } - │ └─StreamExchange { dist: HashShard(salesperson.id) } - │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } - └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } - └─StreamFilter { predicate: IsNotNull(all_sales.salesperson_id) } - └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales._row_id(hidden)], stream_key: [salesperson._row_id, salesperson.id, all_sales._row_id, amount], pk_columns: [salesperson._row_id, salesperson.id, all_sales._row_id, amount], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(max(all_sales.amount), salesperson._row_id, salesperson.id, all_sales._row_id) } + └─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id AND max(all_sales.amount) = all_sales.amount, output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales._row_id] } + ├─StreamExchange { dist: HashShard(salesperson.id, max(all_sales.amount)) } + │ └─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, max(all_sales.amount), salesperson._row_id, all_sales.salesperson_id] } + │ ├─StreamExchange { dist: HashShard(salesperson.id) } + │ │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } + │ └─StreamProject { exprs: [all_sales.salesperson_id, max(all_sales.amount)] } + │ └─StreamHashAgg { group_key: [all_sales.salesperson_id], aggs: [max(all_sales.amount), count] } + │ └─StreamExchange { dist: HashShard(all_sales.salesperson_id) } + │ └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } + └─StreamExchange { dist: HashShard(all_sales.salesperson_id, all_sales.amount) } + └─StreamTableScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount, all_sales._row_id], pk: [all_sales._row_id], dist: UpstreamHashShard(all_sales._row_id) } - name: lateral join 2 sql: | create table all_sales (salesperson_id int, customer_name varchar, amount int ); diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 6dda70d6f98e..73e7cffbc89b 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -804,42 +804,28 @@ select * from integers where 2 in (select count(distinct k) + count(distinct v) from rows where correlated_col = integers.correlated_col); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col_expanded AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col] } + └─BatchHashJoin { type: LeftSemi, predicate: integers.correlated_col = rows.correlated_col_expanded AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col] } ├─BatchExchange { order: [], dist: HashShard(integers.correlated_col) } │ └─BatchProject { exprs: [integers.i, integers.correlated_col, 2:Int64] } │ └─BatchScan { table: integers, columns: [integers.i, integers.correlated_col], distribution: SomeShard } - └─BatchProject { exprs: [integers.correlated_col_expanded, (count(rows.k_expanded) filter((flag = 0:Int64)) + count(rows.v_expanded) filter((flag = 1:Int64))) as $expr1] } - └─BatchHashAgg { group_key: [integers.correlated_col_expanded], aggs: [count(rows.k_expanded) filter((flag = 0:Int64)), count(rows.v_expanded) filter((flag = 1:Int64))] } - └─BatchExchange { order: [], dist: HashShard(integers.correlated_col_expanded) } - └─BatchHashAgg { group_key: [integers.correlated_col_expanded, rows.k_expanded, rows.v_expanded, flag], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(integers.correlated_col_expanded, rows.k_expanded, rows.v_expanded, flag) } - └─BatchExpand { column_subsets: [[integers.correlated_col, rows.k], [integers.correlated_col, rows.v]] } - └─BatchHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, 1:Int32] } - ├─BatchHashAgg { group_key: [integers.correlated_col], aggs: [] } - │ └─BatchExchange { order: [], dist: HashShard(integers.correlated_col) } - │ └─BatchScan { table: integers, columns: [integers.correlated_col], distribution: SomeShard } - └─BatchExchange { order: [], dist: HashShard(rows.correlated_col) } - └─BatchProject { exprs: [rows.correlated_col, rows.k, rows.v, 1:Int32] } - └─BatchFilter { predicate: IsNotNull(rows.correlated_col) } - └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } + └─BatchProject { exprs: [(count(rows.k_expanded) filter((flag = 0:Int64)) + count(rows.v_expanded) filter((flag = 1:Int64))) as $expr1, rows.correlated_col_expanded] } + └─BatchHashAgg { group_key: [rows.correlated_col_expanded], aggs: [count(rows.k_expanded) filter((flag = 0:Int64)), count(rows.v_expanded) filter((flag = 1:Int64))] } + └─BatchExchange { order: [], dist: HashShard(rows.correlated_col_expanded) } + └─BatchHashAgg { group_key: [rows.k_expanded, rows.v_expanded, rows.correlated_col_expanded, flag], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(rows.k_expanded, rows.v_expanded, rows.correlated_col_expanded, flag) } + └─BatchExpand { column_subsets: [[rows.k, rows.correlated_col], [rows.v, rows.correlated_col]] } + └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, correlated_col, integers._row_id(hidden), 2:Int64(hidden)], stream_key: [integers._row_id, correlated_col, 2:Int64], pk_columns: [integers._row_id, correlated_col, 2:Int64], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(integers.correlated_col, integers._row_id, 2:Int64) } - └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } + └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col = rows.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } ├─StreamExchange { dist: HashShard(integers.correlated_col) } │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] } │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] } - └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } - └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] } - ├─StreamProject { exprs: [integers.correlated_col] } - │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] } - │ └─StreamExchange { dist: HashShard(integers.correlated_col) } - │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamExchange { dist: HashShard(rows.correlated_col) } - └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } - └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } - └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } + └─StreamProject { exprs: [(count(distinct rows.k) + count(distinct rows.v)) as $expr1, rows.correlated_col] } + └─StreamHashAgg { group_key: [rows.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } + └─StreamExchange { dist: HashShard(rows.correlated_col) } + └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } - name: test hop window subquery 1 sql: | create table t1 (k int primary key, ts timestamp); diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index dba3854b060b..069e110b65ae 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -14,16 +14,12 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) AND (t2.y = 1000:Int32) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan_for_batch: |- - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } + LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr1] } │ └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t1.y, (1.5:Decimal * min(t2.x)::Decimal) as $expr2] } - └─LogicalAgg { group_key: [t1.y], aggs: [min(t2.x)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, t2.x] } - ├─LogicalAgg { group_key: [t1.y], aggs: [] } - │ └─LogicalScan { table: t1, columns: [t1.y] } - └─LogicalProject { exprs: [t2.y, t2.x] } - └─LogicalScan { table: t2, columns: [t2.x, t2.y], predicate: IsNotNull(t2.y) AND (t2.y = 1000:Int32) } + └─LogicalProject { exprs: [(1.5:Decimal * min(t2.x)::Decimal) as $expr2, t2.y] } + └─LogicalAgg { group_key: [t2.y], aggs: [min(t2.x)] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y], predicate: (t2.y = 1000:Int32) } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -95,16 +91,12 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan_for_batch: |- - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } + LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr1] } │ └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t1.y, (1.5:Decimal * min(t2.x)::Decimal) as $expr2] } - └─LogicalAgg { group_key: [t1.y], aggs: [min(t2.x)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, t2.x] } - ├─LogicalAgg { group_key: [t1.y], aggs: [] } - │ └─LogicalScan { table: t1, columns: [t1.y] } - └─LogicalProject { exprs: [t2.y, t2.x] } - └─LogicalScan { table: t2, columns: [t2.x, t2.y], predicate: IsNotNull(t2.y) } + └─LogicalProject { exprs: [(1.5:Decimal * min(t2.x)::Decimal) as $expr2, t2.y] } + └─LogicalAgg { group_key: [t2.y], aggs: [min(t2.x)] } + └─LogicalScan { table: t2, columns: [t2.x, t2.y] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -120,15 +112,11 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan_for_batch: |- - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > count(1:Int32)), output: [t1.x, t1.y] } + LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > count), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } │ └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalAgg { group_key: [t1.y], aggs: [count(1:Int32)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, 1:Int32] } - ├─LogicalAgg { group_key: [t1.y], aggs: [] } - │ └─LogicalScan { table: t1, columns: [t1.y] } - └─LogicalProject { exprs: [t2.y, 1:Int32] } - └─LogicalScan { table: t2, columns: [t2.y], predicate: IsNotNull(t2.y) } + └─LogicalAgg { group_key: [t2.y], aggs: [count] } + └─LogicalScan { table: t2, columns: [t2.y] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -144,16 +132,12 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan_for_batch: |- - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } + LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } │ └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t1.y, (count(1:Int32) + count(1:Int32)) as $expr2] } - └─LogicalAgg { group_key: [t1.y], aggs: [count(1:Int32)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, 1:Int32] } - ├─LogicalAgg { group_key: [t1.y], aggs: [] } - │ └─LogicalScan { table: t1, columns: [t1.y] } - └─LogicalProject { exprs: [t2.y, 1:Int32] } - └─LogicalScan { table: t2, columns: [t2.y], predicate: IsNotNull(t2.y) } + └─LogicalProject { exprs: [(count + count) as $expr2, t2.y] } + └─LogicalAgg { group_key: [t2.y], aggs: [count] } + └─LogicalScan { table: t2, columns: [t2.y] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -466,19 +450,13 @@ └─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, internal_last_seen_value(a.a3)) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } + └─LogicalJoin { type: Inner, on: (b.b2 = c.c2) AND (a.a3 = c.c3), output: [] } ├─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } │ ├─LogicalScan { table: a, columns: [a.a3] } │ └─LogicalScan { table: b, columns: [b.b2] } - └─LogicalFilter { predicate: (3:Int32 = count(1:Int32)) } - └─LogicalAgg { group_key: [internal_last_seen_value(a.a3), b.b2], aggs: [count(1:Int32)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(a.a3), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [internal_last_seen_value(a.a3), b.b2, 1:Int32] } - ├─LogicalAgg { group_key: [b.b2], aggs: [internal_last_seen_value(a.a3)] } - │ └─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } - │ ├─LogicalScan { table: a, columns: [a.a3] } - │ └─LogicalScan { table: b, columns: [b.b2] } - └─LogicalProject { exprs: [c.c3, c.c2, 1:Int32] } - └─LogicalScan { table: c, columns: [c.c2, c.c3], predicate: (c.c3 = c.c2) AND IsNotNull(c.c3) AND IsNotNull(c.c2) } + └─LogicalFilter { predicate: (3:Int32 = count) } + └─LogicalAgg { group_key: [c.c2, c.c3], aggs: [count] } + └─LogicalScan { table: c, columns: [c.c2, c.c3], predicate: (c.c3 = c.c2) } - sql: | create table a(x int, y int, z int); create table b(x int, y int, z int); @@ -503,60 +481,45 @@ select count(*) from a where a.x=3 and a.y = (select count(*) from b where b.z = a.z); optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(1:Int32)), output: [] } + └─LogicalJoin { type: Inner, on: (a.z = b.z) AND ($expr1 = count), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } │ └─LogicalScan { table: a, output_columns: [a.y, a.z], required_columns: [a.y, a.z, a.x], predicate: (a.x = 3:Int32) } - └─LogicalAgg { group_key: [a.z], aggs: [count(1:Int32)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, 1:Int32] } - ├─LogicalAgg { group_key: [a.z], aggs: [] } - │ └─LogicalScan { table: a, output_columns: [a.z], required_columns: [a.z, a.x], predicate: (a.x = 3:Int32) } - └─LogicalProject { exprs: [b.z, 1:Int32] } - └─LogicalScan { table: b, columns: [b.z], predicate: IsNotNull(b.z) } + └─LogicalAgg { group_key: [b.z], aggs: [count] } + └─LogicalScan { table: b, columns: [b.z] } - sql: | create table a(x int, y varchar, z int); create table b(x varchar, y int, z int); select count(*) from a where a.y = (select string_agg(x, ',' order by x) from b where b.z = a.z); optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND (a.y = string_agg(b.x, ',':Varchar order_by(b.x ASC))), output: [] } + └─LogicalJoin { type: Inner, on: (a.z = b.z) AND (a.y = string_agg(b.x, ',':Varchar order_by(b.x ASC))), output: [] } ├─LogicalScan { table: a, columns: [a.y, a.z] } - └─LogicalAgg { group_key: [a.z], aggs: [string_agg(b.x, ',':Varchar order_by(b.x ASC))] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x, ',':Varchar] } - ├─LogicalAgg { group_key: [a.z], aggs: [] } - │ └─LogicalScan { table: a, columns: [a.z] } - └─LogicalProject { exprs: [b.z, b.x, ',':Varchar] } - └─LogicalScan { table: b, columns: [b.x, b.z], predicate: IsNotNull(b.z) } + └─LogicalAgg { group_key: [b.z], aggs: [string_agg(b.x, ',':Varchar order_by(b.x ASC))] } + └─LogicalProject { exprs: [b.x, ',':Varchar, b.z] } + └─LogicalScan { table: b, columns: [b.x, b.z] } - sql: | create table a(x int, y int, z int); create table b(x int, y int, z int); select count(*) from a where a.y = (select count(distinct x) from b where b.z = a.z); optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(b.x)), output: [] } + └─LogicalJoin { type: Inner, on: (a.z = b.z) AND ($expr1 = count(b.x)), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } │ └─LogicalScan { table: a, columns: [a.y, a.z] } - └─LogicalAgg { group_key: [a.z], aggs: [count(b.x)] } - └─LogicalAgg { group_key: [a.z, b.x], aggs: [] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x] } - ├─LogicalAgg { group_key: [a.z], aggs: [] } - │ └─LogicalScan { table: a, columns: [a.z] } - └─LogicalProject { exprs: [b.z, b.x] } - └─LogicalScan { table: b, columns: [b.x, b.z], predicate: IsNotNull(b.z) } + └─LogicalAgg { group_key: [b.z], aggs: [count(b.x)] } + └─LogicalAgg { group_key: [b.x, b.z], aggs: [] } + └─LogicalScan { table: b, columns: [b.x, b.z] } - sql: | create table a(x int, y int, z int); create table b(x int, y int, z int); select count(*) from a where a.y = (select count(x) filter(where x < 100) from b where b.z = a.z); optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(b.x) filter((b.x < 100:Int32))), output: [] } + └─LogicalJoin { type: Inner, on: (a.z = b.z) AND ($expr1 = count(b.x) filter((b.x < 100:Int32))), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } │ └─LogicalScan { table: a, columns: [a.y, a.z] } - └─LogicalAgg { group_key: [a.z], aggs: [count(b.x) filter((b.x < 100:Int32))] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x] } - ├─LogicalAgg { group_key: [a.z], aggs: [] } - │ └─LogicalScan { table: a, columns: [a.z] } - └─LogicalProject { exprs: [b.z, b.x] } - └─LogicalScan { table: b, columns: [b.x, b.z], predicate: IsNotNull(b.z) } + └─LogicalAgg { group_key: [b.z], aggs: [count(b.x) filter((b.x < 100:Int32))] } + └─LogicalScan { table: b, columns: [b.x, b.z] } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -978,35 +941,23 @@ select Array(select c from t2 where b = d) arr from t1; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: LeftOuter, predicate: t1.b IS NOT DISTINCT FROM t1.b, output: [array_agg(t2.c)] } + └─BatchHashJoin { type: LeftOuter, predicate: t1.b = t2.d, output: [array_agg(t2.c)] } ├─BatchExchange { order: [], dist: HashShard(t1.b) } │ └─BatchScan { table: t1, columns: [t1.b], distribution: SomeShard } - └─BatchHashAgg { group_key: [t1.b], aggs: [array_agg(t2.c)] } - └─BatchHashJoin { type: LeftOuter, predicate: t1.b IS NOT DISTINCT FROM t2.d, output: [t1.b, t2.c] } - ├─BatchHashAgg { group_key: [t1.b], aggs: [] } - │ └─BatchExchange { order: [], dist: HashShard(t1.b) } - │ └─BatchScan { table: t1, columns: [t1.b], distribution: SomeShard } + └─BatchProject { exprs: [array_agg(t2.c), t2.d] } + └─BatchHashAgg { group_key: [t2.d], aggs: [array_agg(t2.c)] } └─BatchExchange { order: [], dist: HashShard(t2.d) } - └─BatchProject { exprs: [t2.d, t2.c] } - └─BatchFilter { predicate: IsNotNull(t2.d) } - └─BatchScan { table: t2, columns: [t2.c, t2.d], distribution: SomeShard } + └─BatchScan { table: t2, columns: [t2.c, t2.d], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [arr, t1._row_id(hidden), t1.b(hidden), t1.b#1(hidden)], stream_key: [t1._row_id, t1.b], pk_columns: [t1._row_id, t1.b], pk_conflict: NoCheck } + StreamMaterialize { columns: [arr, t1._row_id(hidden), t1.b(hidden), t2.d(hidden)], stream_key: [t1._row_id, t1.b], pk_columns: [t1._row_id, t1.b], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(t1._row_id, t1.b) } - └─StreamHashJoin { type: LeftOuter, predicate: t1.b IS NOT DISTINCT FROM t1.b, output: [array_agg(t2.c), t1._row_id, t1.b, t1.b] } + └─StreamHashJoin { type: LeftOuter, predicate: t1.b = t2.d, output: [array_agg(t2.c), t1._row_id, t1.b, t2.d] } ├─StreamExchange { dist: HashShard(t1.b) } │ └─StreamTableScan { table: t1, columns: [t1.b, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamProject { exprs: [t1.b, array_agg(t2.c)] } - └─StreamHashAgg { group_key: [t1.b], aggs: [array_agg(t2.c), count] } - └─StreamHashJoin { type: LeftOuter, predicate: t1.b IS NOT DISTINCT FROM t2.d, output: [t1.b, t2.c, t2._row_id] } - ├─StreamProject { exprs: [t1.b] } - │ └─StreamHashAgg { group_key: [t1.b], aggs: [count] } - │ └─StreamExchange { dist: HashShard(t1.b) } - │ └─StreamTableScan { table: t1, columns: [t1.b, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(t2.d) } - └─StreamProject { exprs: [t2.d, t2.c, t2._row_id] } - └─StreamFilter { predicate: IsNotNull(t2.d) } - └─StreamTableScan { table: t2, columns: [t2.c, t2.d, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + └─StreamProject { exprs: [array_agg(t2.c), t2.d] } + └─StreamHashAgg { group_key: [t2.d], aggs: [array_agg(t2.c), count] } + └─StreamExchange { dist: HashShard(t2.d) } + └─StreamTableScan { table: t2, columns: [t2.c, t2.d, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: correlated array subquery \du sql: | SELECT r.rolname, r.rolsuper, r.rolinherit, @@ -1025,29 +976,16 @@ BatchExchange { order: [rw_users.name ASC], dist: Single } └─BatchProject { exprs: [rw_users.name, rw_users.is_super, true:Boolean, rw_users.create_user, rw_users.create_db, rw_users.can_login, -1:Int32, null:Timestamptz, array_agg(rw_users.name), true:Boolean, true:Boolean] } └─BatchSort { order: [rw_users.name ASC] } - └─BatchHashJoin { type: LeftOuter, predicate: rw_users.id IS NOT DISTINCT FROM rw_users.id, output: all } + └─BatchHashJoin { type: LeftOuter, predicate: rw_users.id = null:Int32, output: all } ├─BatchExchange { order: [], dist: HashShard(rw_users.id) } │ └─BatchFilter { predicate: Not(RegexpEq(rw_users.name, '^pg_':Varchar)) } │ └─BatchScan { table: rw_users, columns: [rw_users.id, rw_users.name, rw_users.is_super, rw_users.create_db, rw_users.create_user, rw_users.can_login], distribution: Single } - └─BatchHashAgg { group_key: [rw_users.id], aggs: [array_agg(rw_users.name)] } - └─BatchHashJoin { type: LeftOuter, predicate: rw_users.id IS NOT DISTINCT FROM rw_users.id, output: [rw_users.id, rw_users.name] } - ├─BatchHashAgg { group_key: [rw_users.id], aggs: [] } - │ └─BatchExchange { order: [], dist: HashShard(rw_users.id) } - │ └─BatchProject { exprs: [rw_users.id] } - │ └─BatchFilter { predicate: Not(RegexpEq(rw_users.name, '^pg_':Varchar)) } - │ └─BatchScan { table: rw_users, columns: [rw_users.id, rw_users.name], distribution: Single } - └─BatchExchange { order: [], dist: HashShard(rw_users.id) } - └─BatchHashJoin { type: Inner, predicate: null:Int32 = rw_users.id, output: [rw_users.id, rw_users.name] } + └─BatchProject { exprs: [array_agg(rw_users.name), null:Int32] } + └─BatchHashAgg { group_key: [null:Int32], aggs: [array_agg(rw_users.name)] } + └─BatchExchange { order: [], dist: HashShard(null:Int32) } + └─BatchHashJoin { type: Inner, predicate: null:Int32 = rw_users.id, output: [rw_users.name, null:Int32] } ├─BatchExchange { order: [], dist: HashShard(null:Int32) } - │ └─BatchProject { exprs: [rw_users.id, null:Int32] } - │ └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } - │ ├─BatchExchange { order: [], dist: Single } - │ │ └─BatchHashAgg { group_key: [rw_users.id], aggs: [] } - │ │ └─BatchExchange { order: [], dist: HashShard(rw_users.id) } - │ │ └─BatchProject { exprs: [rw_users.id] } - │ │ └─BatchFilter { predicate: (null:Int32 = rw_users.id) AND Not(RegexpEq(rw_users.name, '^pg_':Varchar)) } - │ │ └─BatchScan { table: rw_users, columns: [rw_users.id, rw_users.name], distribution: Single } - │ └─BatchValues { rows: [] } + │ └─BatchValues { rows: [] } └─BatchExchange { order: [], dist: HashShard(rw_users.id) } └─BatchScan { table: rw_users, columns: [rw_users.id, rw_users.name], distribution: Single } - name: correlated array subquery (issue 14423) diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 0cc7a3ba1f39..89ae1972bfd7 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -265,25 +265,22 @@ optimized_logical_plan_for_batch: |- LogicalTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment] } - ├─LogicalJoin { type: Inner, on: IsNotDistinctFrom(part.p_partkey, part.p_partkey) AND (partsupp.ps_supplycost = min(partsupp.ps_supplycost)), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey] } - │ ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey] } - │ │ ├─LogicalJoin { type: Inner, on: (supplier.s_suppkey = partsupp.ps_suppkey), output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } + ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey] } + │ ├─LogicalJoin { type: Inner, on: (supplier.s_suppkey = partsupp.ps_suppkey), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } + │ │ ├─LogicalJoin { type: Inner, on: (part.p_partkey = partsupp.ps_partkey) AND (partsupp.ps_supplycost = min(partsupp.ps_supplycost)), output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (part.p_partkey = partsupp.ps_partkey), output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost] } │ │ │ │ ├─LogicalScan { table: part, output_columns: [part.p_partkey, part.p_mfgr], required_columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } │ │ │ │ └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost] } - │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } - │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey] } - │ └─LogicalAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost)] } - │ └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(part.p_partkey, partsupp.ps_partkey), output: [part.p_partkey, partsupp.ps_supplycost] } - │ ├─LogicalAgg { group_key: [part.p_partkey], aggs: [] } - │ │ └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_type, part.p_size], predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - │ └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [partsupp.ps_partkey, partsupp.ps_supplycost] } - │ ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey] } - │ │ ├─LogicalJoin { type: Inner, on: (supplier.s_suppkey = partsupp.ps_suppkey), output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey] } - │ │ │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], predicate: IsNotNull(partsupp.ps_partkey) } - │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey] } - │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey] } - │ └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'AFRICA':Varchar) } + │ │ │ └─LogicalAgg { group_key: [partsupp.ps_partkey], aggs: [min(partsupp.ps_supplycost)] } + │ │ │ └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [partsupp.ps_supplycost, partsupp.ps_partkey] } + │ │ │ ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey] } + │ │ │ │ ├─LogicalJoin { type: Inner, on: (supplier.s_suppkey = partsupp.ps_suppkey), output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey] } + │ │ │ │ │ ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost] } + │ │ │ │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey] } + │ │ │ │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey] } + │ │ │ └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'AFRICA':Varchar) } + │ │ └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } + │ └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey] } └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'AFRICA':Varchar) } batch_plan: |- BatchTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } @@ -291,305 +288,271 @@ └─BatchTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'AFRICA':Varchar), output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment], lookup table: region } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey] } - ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } - │ └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey], lookup table: nation } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - │ └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_supplycost, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], lookup table: supplier } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } - │ └─BatchHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost] } - │ ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } - │ │ └─BatchProject { exprs: [part.p_partkey, part.p_mfgr] } - │ │ └─BatchFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - │ │ └─BatchScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], distribution: UpstreamHashShard(part.p_partkey) } - │ └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } - │ └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - └─BatchHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost)] } - └─BatchHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost] } - ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } - │ └─BatchSortAgg { group_key: [part.p_partkey], aggs: [] } - │ └─BatchProject { exprs: [part.p_partkey] } - │ └─BatchFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - │ └─BatchScan { table: part, columns: [part.p_partkey, part.p_type, part.p_size], distribution: UpstreamHashShard(part.p_partkey) } - └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } - └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'AFRICA':Varchar), output: [partsupp.ps_partkey, partsupp.ps_supplycost], lookup table: region } - └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } - └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey], lookup table: nation } - └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } - └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey], lookup table: supplier } - └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } - └─BatchFilter { predicate: IsNotNull(partsupp.ps_partkey) } - └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey], lookup table: nation } + └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } + └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], lookup table: supplier } + └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } + └─BatchHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey] } + ├─BatchHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost] } + │ ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } + │ │ └─BatchProject { exprs: [part.p_partkey, part.p_mfgr] } + │ │ └─BatchFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } + │ │ └─BatchScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], distribution: UpstreamHashShard(part.p_partkey) } + │ └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } + │ └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + └─BatchHashAgg { group_key: [partsupp.ps_partkey], aggs: [min(partsupp.ps_supplycost)] } + └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } + └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'AFRICA':Varchar), output: [partsupp.ps_supplycost, partsupp.ps_partkey], lookup table: region } + └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } + └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, nation.n_regionkey], lookup table: nation } + └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } + └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey], lookup table: supplier } + └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } + └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_plan: |- - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), part.p_partkey(hidden), partsupp.ps_suppkey(hidden), min(partsupp.ps_supplycost)(hidden)], stream_key: [region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost)(hidden), partsupp.ps_partkey(hidden), region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), partsupp.ps_partkey#1(hidden)], stream_key: [p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_conflict: NoCheck } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } └─StreamTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost), Vnode(supplier.s_suppkey) as $expr1] } - └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } - │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost)] } - ├─StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } - │ └─StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } - │ └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, region.r_regionkey, supplier.s_nationkey] } - │ ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ │ └─StreamProject { exprs: [part.p_partkey] } - │ │ └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } - │ │ └─StreamProject { exprs: [part.p_partkey] } - │ │ └─StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } - │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } - │ │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ │ │ └─StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } - │ │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } - │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost, partsupp.ps_partkey] } - ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ └─StreamProject { exprs: [part.p_partkey, part.p_mfgr] } - │ └─StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey, Vnode(partsupp.ps_partkey, min(partsupp.ps_supplycost)) as $expr1] } + └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost AND partsupp.ps_partkey = partsupp.ps_partkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } + ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, min(partsupp.ps_supplycost)) } + │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } + │ ├─StreamExchange { dist: HashShard(part.p_partkey) } + │ │ └─StreamProject { exprs: [part.p_partkey, part.p_mfgr] } + │ │ └─StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } + │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } + │ └─StreamProject { exprs: [partsupp.ps_partkey, min(partsupp.ps_supplycost)] } + │ └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [min(partsupp.ps_supplycost), count] } + │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } + │ │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } + │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } + │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_supplycost) } + └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [nation.n_name, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_suppkey] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } + │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + └─StreamFilter { predicate: (partsupp.ps_partkey = partsupp.ps_partkey) } └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), part.p_partkey(hidden), partsupp.ps_suppkey(hidden), min(partsupp.ps_supplycost)(hidden)], stream_key: [region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], pk_conflict: NoCheck } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost)(hidden), partsupp.ps_partkey(hidden), region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), partsupp.ps_partkey#1(hidden)], stream_key: [p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } + └── StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } └── StreamTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } └── StreamExchange Single from 1 Fragment 1 StreamGroupTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } - └── StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost), Vnode(supplier.s_suppkey) as $expr1] } - └── StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } { tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] } - ├── StreamExchange Hash([1]) from 2 - └── StreamExchange Hash([2]) from 7 + └── StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey, Vnode(partsupp.ps_partkey, min(partsupp.ps_supplycost)) as $expr1] } + └── StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost AND partsupp.ps_partkey = partsupp.ps_partkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } { tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] } + ├── StreamExchange Hash([2, 3]) from 2 + └── StreamExchange Hash([6, 7]) from 11 Fragment 2 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] } + StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } { tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] } ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([3]) from 6 + └── StreamProject { exprs: [partsupp.ps_partkey, min(partsupp.ps_supplycost)] } + └── StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [min(partsupp.ps_supplycost), count] } { tables: [ HashAggState: 12, HashAggCall0: 11 ] } + └── StreamExchange Hash([1]) from 4 Fragment 3 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } { tables: [ HashJoinLeft: 10, HashJoinDegreeLeft: 11, HashJoinRight: 12, HashJoinDegreeRight: 13 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([2]) from 5 - - Fragment 4 - StreamProject { exprs: [region.r_regionkey] } - └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 14 ] } + StreamProject { exprs: [part.p_partkey, part.p_mfgr] } + └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 10 ] } ├── Upstream └── BatchPlanNode + Fragment 4 + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] } + ├── StreamExchange Hash([2]) from 5 + └── StreamExchange Hash([0]) from 8 + Fragment 5 - StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 15 ] } - ├── Upstream - └── BatchPlanNode + StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } { tables: [ HashJoinLeft: 17, HashJoinDegreeLeft: 18, HashJoinRight: 19, HashJoinDegreeRight: 20 ] } + ├── StreamExchange Hash([1]) from 6 + └── StreamExchange Hash([0]) from 7 Fragment 6 - StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 16 ] } + StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 21 ] } ├── Upstream └── BatchPlanNode Fragment 7 - StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost)] } { tables: [ HashJoinLeft: 17, HashJoinDegreeLeft: 18, HashJoinRight: 19, HashJoinDegreeRight: 20 ] } - ├── StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } - │ └── StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } { tables: [ HashAggState: 22, HashAggCall0: 21 ] } - │ └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, region.r_regionkey, supplier.s_nationkey] } { tables: [ HashJoinLeft: 23, HashJoinDegreeLeft: 24, HashJoinRight: 25, HashJoinDegreeRight: 26 ] } - │ ├── StreamExchange Hash([0]) from 8 - │ └── StreamExchange Hash([0]) from 9 - └── StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost, partsupp.ps_partkey] } { tables: [ HashJoinLeft: 45, HashJoinDegreeLeft: 46, HashJoinRight: 47, HashJoinDegreeRight: 48 ] } - ├── StreamExchange Hash([0]) from 16 - └── StreamExchange Hash([0]) from 17 + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 22 ] } + ├── Upstream + └── BatchPlanNode Fragment 8 - StreamProject { exprs: [part.p_partkey] } - └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { tables: [ HashAggState: 27 ] } - └── StreamProject { exprs: [part.p_partkey] } - └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 28 ] } - ├── Upstream - └── BatchPlanNode + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { tables: [ HashJoinLeft: 23, HashJoinDegreeLeft: 24, HashJoinRight: 25, HashJoinDegreeRight: 26 ] } + ├── StreamExchange Hash([0]) from 9 + └── StreamExchange Hash([1]) from 10 Fragment 9 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 29, HashJoinDegreeLeft: 30, HashJoinRight: 31, HashJoinDegreeRight: 32 ] } - ├── StreamExchange Hash([2]) from 10 - └── StreamExchange Hash([0]) from 13 + StreamProject { exprs: [region.r_regionkey] } + └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 27 ] } + ├── Upstream + └── BatchPlanNode Fragment 10 - StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } { tables: [ HashJoinLeft: 33, HashJoinDegreeLeft: 34, HashJoinRight: 35, HashJoinDegreeRight: 36 ] } - ├── StreamExchange Hash([1]) from 11 - └── StreamExchange Hash([0]) from 12 + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 28 ] } + ├── Upstream + └── BatchPlanNode Fragment 11 - StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } - └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 37 ] } - ├── Upstream - └── BatchPlanNode + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [nation.n_name, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_suppkey] } { tables: [ HashJoinLeft: 29, HashJoinDegreeLeft: 30, HashJoinRight: 31, HashJoinDegreeRight: 32 ] } + ├── StreamExchange Hash([1]) from 12 + └── StreamExchange Hash([1]) from 17 Fragment 12 - StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 38 ] } - ├── Upstream - └── BatchPlanNode + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 33, HashJoinDegreeLeft: 34, HashJoinRight: 35, HashJoinDegreeRight: 36 ] } + ├── StreamExchange Hash([0]) from 13 + └── StreamExchange Hash([3]) from 16 Fragment 13 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { tables: [ HashJoinLeft: 39, HashJoinDegreeLeft: 40, HashJoinRight: 41, HashJoinDegreeRight: 42 ] } + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } { tables: [ HashJoinLeft: 37, HashJoinDegreeLeft: 38, HashJoinRight: 39, HashJoinDegreeRight: 40 ] } ├── StreamExchange Hash([0]) from 14 - └── StreamExchange Hash([1]) from 15 + └── StreamExchange Hash([2]) from 15 Fragment 14 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 43 ] } + └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], pk: [region.r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 41 ] } ├── Upstream └── BatchPlanNode Fragment 15 - StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 44 ] } + StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 42 ] } ├── Upstream └── BatchPlanNode Fragment 16 - StreamProject { exprs: [part.p_partkey, part.p_mfgr] } - └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 49 ] } - ├── Upstream - └── BatchPlanNode - - Fragment 17 - StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 50 ] } + StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 43 ] } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey_0, partsupp_ps_suppkey, min(partsupp_ps_supplycost), $expr1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [], read pk prefix len hint: 0 } - - Table 1 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey_0, partsupp_ps_suppkey, min(partsupp_ps_supplycost), $expr1 ], primary key: [ $14 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 10 ], read pk prefix len hint: 1, vnode column idx: 14 } - - Table 2 { columns: [ nation_n_name, supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, region_r_regionkey, nation_n_nationkey ], primary key: [ $1 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - - Table 3 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 4 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, part_p_partkey_0, min(partsupp_ps_supplycost) ], primary key: [ $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - - Table 5 { columns: [ partsupp_ps_suppkey, part_p_partkey, min(partsupp_ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Fragment 17 + StreamFilter { predicate: (partsupp.ps_partkey = partsupp.ps_partkey) } + └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 44 ] } + ├── Upstream + └── BatchPlanNode - Table 6 { columns: [ nation_n_nationkey, nation_n_name, region_r_regionkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, min(partsupp_ps_supplycost), partsupp_ps_partkey, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_partkey_0, $expr1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [], read pk prefix len hint: 0 } - Table 7 { columns: [ nation_n_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, min(partsupp_ps_supplycost), partsupp_ps_partkey, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_partkey_0, $expr1 ], primary key: [ $14 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 9, 8 ], read pk prefix len hint: 1, vnode column idx: 14 } - Table 8 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, supplier_s_phone, supplier_s_acctbal, supplier_s_comment ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 2 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_partkey, min(partsupp_ps_supplycost) ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } - Table 9 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 10 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ nation_n_name, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_supplycost, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, partsupp_ps_suppkey ], primary key: [ $6 ASC, $7 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ], distribution key: [ 6, 7 ], read pk prefix len hint: 3 } - Table 11 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_partkey_0, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 12 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 6 { columns: [ part_p_partkey, part_p_mfgr ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ vnode, r_regionkey, region_backfill_finished, region_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 8 { columns: [ partsupp_ps_partkey, min(partsupp_ps_supplycost) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 9 { columns: [ partsupp_ps_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ vnode, s_suppkey, supplier_backfill_finished, supplier_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 10 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 17 { columns: [ part_p_partkey, min(partsupp_ps_supplycost) ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 11 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 12 { columns: [ partsupp_ps_partkey, min(partsupp_ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 19 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, partsupp_ps_supplycost, partsupp_ps_partkey ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 13 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 20 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 14 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ nation_n_nationkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ nation_n_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 24 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, supplier_s_nationkey, region_r_regionkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 19 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, region_r_regionkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 27 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished, partsupp_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 28 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 22 { columns: [ vnode, s_suppkey, supplier_backfill_finished, supplier_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 29 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 23 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 30 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 31 { columns: [ nation_n_nationkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 25 { columns: [ nation_n_nationkey, nation_n_regionkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 32 { columns: [ nation_n_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 26 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 33 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 27 { columns: [ vnode, r_regionkey, region_backfill_finished, region_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 34 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 28 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 35 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ nation_n_name, supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, region_r_regionkey, nation_n_nationkey ], primary key: [ $1 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 36 { columns: [ supplier_s_suppkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 30 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 37 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished, partsupp_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 31 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 38 { columns: [ vnode, s_suppkey, supplier_backfill_finished, supplier_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 32 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 39 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ nation_n_nationkey, nation_n_name, region_r_regionkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 40 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 34 { columns: [ nation_n_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 41 { columns: [ nation_n_nationkey, nation_n_regionkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 35 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, supplier_s_phone, supplier_s_acctbal, supplier_s_comment ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 42 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 36 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 43 { columns: [ vnode, r_regionkey, region_backfill_finished, region_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 37 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 44 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 38 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 45 { columns: [ part_p_partkey, part_p_mfgr ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 39 { columns: [ nation_n_nationkey, nation_n_name, nation_n_regionkey ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 46 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 40 { columns: [ nation_n_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 47 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_supplycost ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 41 { columns: [ vnode, r_regionkey, region_backfill_finished, region_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 48 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 42 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 49 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 43 { columns: [ vnode, s_suppkey, supplier_backfill_finished, supplier_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 50 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished, partsupp_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 44 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished, partsupp_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost) ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [], read pk prefix len hint: 9 } - id: tpch_q3 before: @@ -3389,46 +3352,36 @@ optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(sum(lineitem.l_extendedprice) / 7.0:Decimal) as $expr2] } └─LogicalAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(part.p_partkey, part.p_partkey) AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } + └─LogicalJoin { type: Inner, on: (part.p_partkey = lineitem.l_partkey) AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } ├─LogicalJoin { type: Inner, on: (part.p_partkey = lineitem.l_partkey), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey] } │ ├─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice] } │ └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_brand, part.p_container], predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └─LogicalProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } - └─LogicalAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(part.p_partkey, lineitem.l_partkey), output: [part.p_partkey, lineitem.l_quantity] } - ├─LogicalAgg { group_key: [part.p_partkey], aggs: [] } - │ └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_brand, part.p_container], predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], predicate: IsNotNull(lineitem.l_partkey) } + └─LogicalProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } + └─LogicalAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } + └─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity] } batch_plan: |- BatchProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2] } └─BatchSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─BatchHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } + └─BatchHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } │ └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey], lookup table: part } │ └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } │ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice], distribution: SomeShard } - └─BatchProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } - └─BatchHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } - └─BatchHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity] } - ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } - │ └─BatchSortAgg { group_key: [part.p_partkey], aggs: [] } - │ └─BatchProject { exprs: [part.p_partkey] } - │ └─BatchFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - │ └─BatchScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], distribution: UpstreamHashShard(part.p_partkey) } - └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey) } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) } - └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], distribution: SomeShard } + └─BatchProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } + └─BatchHashAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } + └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey) } + └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2] } └─StreamSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, part.p_partkey] } + └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, lineitem.l_partkey] } └─StreamFilter { predicate: (lineitem.l_quantity < $expr1) } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } + └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: all } ├─StreamExchange { dist: HashShard(part.p_partkey) } │ └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } │ ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } @@ -3437,18 +3390,10 @@ │ └─StreamProject { exprs: [part.p_partkey] } │ └─StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } - └─StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } - └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ └─StreamProject { exprs: [part.p_partkey] } - │ └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } - │ └─StreamProject { exprs: [part.p_partkey] } - │ └─StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamExchange { dist: HashShard(lineitem.l_partkey) } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } + └─StreamHashAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } + └─StreamExchange { dist: HashShard(lineitem.l_partkey) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } @@ -3460,16 +3405,15 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └── StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, part.p_partkey] } + └── StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, lineitem.l_partkey] } └── StreamFilter { predicate: (lineitem.l_quantity < $expr1) } - └── StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } { tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] } + └── StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: all } + ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] ├── StreamExchange Hash([2]) from 2 - └── StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } - └── StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } { tables: [ HashAggState: 11 ] } - └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - ├── tables: [ HashJoinLeft: 12, HashJoinDegreeLeft: 13, HashJoinRight: 14, HashJoinDegreeRight: 15 ] - ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([0]) from 6 + └── StreamProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } + └── StreamHashAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } + ├── tables: [ HashAggState: 11 ] + └── StreamExchange Hash([0]) from 5 Fragment 2 StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } @@ -3491,20 +3435,10 @@ └── BatchPlanNode Fragment 5 - StreamProject { exprs: [part.p_partkey] } - └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { tables: [ HashAggState: 16 ] } - └── StreamProject { exprs: [part.p_partkey] } - └── StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 17 ] } - ├── Upstream - └── BatchPlanNode - - Fragment 6 - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } - └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── tables: [ StreamScan: 18 ] - ├── Upstream - └── BatchPlanNode + StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + ├── tables: [ StreamScan: 12 ] + ├── Upstream + └── BatchPlanNode Table 0 { columns: [ sum(sum(lineitem_l_extendedprice)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } @@ -3517,9 +3451,9 @@ Table 2 { columns: [ part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ part_p_partkey, $expr1 ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr1, lineitem_l_partkey ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 4 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ lineitem_l_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -3533,21 +3467,9 @@ Table 10 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 11 { columns: [ part_p_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 12 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 13 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 14 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ lineitem_l_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 16 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 17 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - - Table 18 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished, lineitem_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished, lineitem_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } Table 4294967294 { columns: [ avg_yearly ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml index a3a5593262da..00f96df8f7aa 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -154,397 +154,304 @@ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } optimized_logical_plan_for_batch: |- LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment] } - ├─LogicalJoin { type: Inner, on: IsNotDistinctFrom(p_partkey, p_partkey) AND (ps_supplycost = min(ps_supplycost)), output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } - │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } - │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] } + ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } + │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [p_partkey, p_mfgr, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] } + │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = ps_partkey) AND (ps_supplycost = min(ps_supplycost)), output: [p_partkey, p_mfgr, ps_suppkey] } │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = ps_partkey), output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost] } │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } │ │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalAgg { group_key: [p_partkey], aggs: [min(ps_supplycost)] } - │ └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(p_partkey, ps_partkey), output: [p_partkey, ps_supplycost] } - │ ├─LogicalAgg { group_key: [p_partkey], aggs: [] } - │ │ └─LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [p_partkey] } - │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [p_partkey, n_regionkey] } - │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [p_partkey, s_nationkey] } - │ │ │ │ ├─LogicalJoin { type: Inner, on: (p_partkey = ps_partkey), output: [p_partkey, ps_suppkey] } - │ │ │ │ │ ├─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [ps_partkey, ps_supplycost] } - │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [ps_partkey, ps_supplycost, n_regionkey] } - │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [ps_partkey, ps_supplycost, s_nationkey] } - │ │ │ ├─LogicalFilter { predicate: IsNotNull(ps_partkey) } - │ │ │ │ └─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ └─LogicalAgg { group_key: [ps_partkey], aggs: [min(ps_supplycost)] } + │ │ │ └─LogicalJoin { type: Inner, on: (n_regionkey = r_regionkey), output: [ps_supplycost, ps_partkey] } + │ │ │ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [ps_partkey, ps_supplycost, n_regionkey] } + │ │ │ │ ├─LogicalJoin { type: Inner, on: (s_suppkey = ps_suppkey), output: [ps_partkey, ps_supplycost, s_nationkey] } + │ │ │ │ │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ │ └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ │ └─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } + │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } └─LogicalSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], time_range: (Unbounded, Unbounded) } batch_plan: |- BatchExchange { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], dist: Single } └─BatchSort { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC] } └─BatchHashJoin { type: Inner, predicate: n_regionkey = r_regionkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment] } ├─BatchExchange { order: [], dist: HashShard(n_regionkey) } - │ └─BatchHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } - │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ └─BatchHashJoin { type: Inner, predicate: s_nationkey = n_nationkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } - │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } - │ │ │ └─BatchHashJoin { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] } - │ │ │ ├─BatchExchange { order: [], dist: HashShard(ps_suppkey) } - │ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost] } - │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } - │ │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } - │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } - │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } - │ └─BatchHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost)] } - │ └─BatchHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost] } - │ ├─BatchHashAgg { group_key: [p_partkey], aggs: [] } - │ │ └─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ └─BatchHashJoin { type: Inner, predicate: n_regionkey = r_regionkey, output: [p_partkey] } - │ │ ├─BatchExchange { order: [], dist: HashShard(n_regionkey) } - │ │ │ └─BatchHashJoin { type: Inner, predicate: s_nationkey = n_nationkey, output: [p_partkey, n_regionkey] } - │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } - │ │ │ │ └─BatchHashJoin { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, s_nationkey] } - │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(ps_suppkey) } - │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, ps_suppkey] } - │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } - │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } - │ │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } - │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } - │ │ └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - │ │ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } - │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ └─BatchHashJoin { type: Inner, predicate: n_regionkey = r_regionkey, output: [ps_partkey, ps_supplycost] } - │ ├─BatchExchange { order: [], dist: HashShard(n_regionkey) } - │ │ └─BatchHashJoin { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, n_regionkey] } - │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } - │ │ │ └─BatchHashJoin { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey] } - │ │ │ ├─BatchExchange { order: [], dist: HashShard(ps_suppkey) } - │ │ │ │ └─BatchFilter { predicate: IsNotNull(ps_partkey) } - │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } - │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } - │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } - │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } - │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } - │ └─BatchExchange { order: [], dist: HashShard(r_regionkey) } - │ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + │ └─BatchHashJoin { type: Inner, predicate: s_nationkey = n_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, n_name, n_regionkey] } + │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } + │ │ └─BatchHashJoin { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment] } + │ │ ├─BatchExchange { order: [], dist: HashShard(ps_suppkey) } + │ │ │ └─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND ps_supplycost = min(ps_supplycost), output: [p_partkey, p_mfgr, ps_suppkey] } + │ │ │ ├─BatchHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(p_partkey) } + │ │ │ │ │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } + │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchHashAgg { group_key: [ps_partkey], aggs: [min(ps_supplycost)] } + │ │ │ └─BatchExchange { order: [], dist: HashShard(ps_partkey) } + │ │ │ └─BatchHashJoin { type: Inner, predicate: n_regionkey = r_regionkey, output: [ps_supplycost, ps_partkey] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(n_regionkey) } + │ │ │ │ └─BatchHashJoin { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, n_regionkey] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(s_nationkey) } + │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(ps_suppkey) } + │ │ │ │ │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } + │ │ │ │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } + │ │ │ │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } + │ │ │ └─BatchExchange { order: [], dist: HashShard(r_regionkey) } + │ │ │ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } + │ │ └─BatchExchange { order: [], dist: HashShard(s_suppkey) } + │ │ └─BatchSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], filter: (None, None) } + │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } + │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(r_regionkey) } └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(p_partkey, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost) } - └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } - ├─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamShare { id: 26 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 7 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamProject { exprs: [p_partkey, min(ps_supplycost)] } - └─StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } - └─StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } - ├─StreamAppendOnlyDedup { dedup_cols: [p_partkey] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey] } - │ └─StreamShare { id: 26 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 7 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamExchange { dist: HashShard(ps_partkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } - ├─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamFilter { predicate: IsNotNull(ps_partkey) } - │ │ └─StreamShare { id: 15 } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamShare { id: 21 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └─StreamExchange { dist: HashShard(n_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } - ├─StreamExchange { dist: HashShard(r_regionkey) } - │ └─StreamShare { id: 3 } - │ └─StreamProject { exprs: [r_regionkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 3 } - │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - └─StreamExchange { dist: HashShard(n_regionkey) } - └─StreamShare { id: 7 } - └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - └─StreamRowIdGen { row_id_index: 4 } - └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(p_partkey, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey) } + └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_partkey, min(ps_supplycost)) } + │ └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + │ └─StreamProject { exprs: [ps_partkey, min(ps_supplycost)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [min(ps_supplycost), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_supplycost, ps_partkey, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamShare { id: 5 } + │ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamShare { id: 9 } + │ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamShare { id: 15 } + │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamShare { id: 19 } + │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 4 } + │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─StreamExchange { dist: HashShard(ps_partkey, ps_supplycost) } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = ps_suppkey, output: [n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, s_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamShare { id: 15 } + │ │ │ └─StreamProject { exprs: [r_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 19 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 4 } + │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamFilter { predicate: (ps_partkey = ps_partkey) } + └─StreamShare { id: 5 } + └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([3, 8, 9, 10, 11, 12, 13, 14, 15, 16]) from 1 + └── StreamExchange Hash([3, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17]) from 1 Fragment 1 - StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } - ├── StreamExchange Hash([0]) from 2 - └── StreamProject { exprs: [p_partkey, min(ps_supplycost)] } - └── StreamHashAgg { group_key: [p_partkey], aggs: [min(ps_supplycost), count] } { tables: [ HashAggState: 26, HashAggCall0: 25 ] } - └── StreamHashJoin { type: LeftOuter, predicate: p_partkey IS NOT DISTINCT FROM ps_partkey, output: [p_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, _row_id, _row_id, r_regionkey, s_nationkey] } { tables: [ HashJoinLeft: 27, HashJoinDegreeLeft: 28, HashJoinRight: 29, HashJoinDegreeRight: 30 ] } - ├── StreamAppendOnlyDedup { dedup_cols: [p_partkey] } { tables: [ AppendOnlyDedup: 31 ] } - │ └── StreamExchange Hash([0]) from 16 - └── StreamExchange Hash([0]) from 17 + StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } + ├── StreamExchange Hash([2, 3]) from 2 + └── StreamExchange Hash([6, 7]) from 15 Fragment 2 - StreamNoOp - └── StreamExchange NoShuffle from 3 + StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id] } { tables: [ HashJoinLeft: 4, HashJoinDegreeLeft: 5, HashJoinRight: 6, HashJoinDegreeRight: 7 ] } + ├── StreamExchange Hash([0]) from 3 + └── StreamProject { exprs: [ps_partkey, min(ps_supplycost)] } + └── StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [min(ps_supplycost), count] } { tables: [ HashAggState: 9 ] } + └── StreamExchange Hash([1]) from 4 Fragment 3 - StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, _row_id, ps_suppkey] } { tables: [ HashJoinLeft: 4, HashJoinDegreeLeft: 5, HashJoinRight: 6, HashJoinDegreeRight: 7 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([5]) from 9 + StreamRowIdGen { row_id_index: 9 } + └── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { tables: [ Source: 8 ] } Fragment 4 - StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { tables: [ HashJoinLeft: 8, HashJoinDegreeLeft: 9, HashJoinRight: 10, HashJoinDegreeRight: 11 ] } - ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([2]) from 7 + StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_supplycost, ps_partkey, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } { tables: [ HashJoinLeft: 10, HashJoinDegreeLeft: 11, HashJoinRight: 12, HashJoinDegreeRight: 13 ] } + ├── StreamExchange Hash([2]) from 5 + └── StreamExchange Hash([0]) from 10 Fragment 5 - StreamNoOp - └── StreamExchange NoShuffle from 6 + StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } { tables: [ HashJoinLeft: 14, HashJoinDegreeLeft: 15, HashJoinRight: 16, HashJoinDegreeRight: 17 ] } + ├── StreamExchange Hash([1]) from 6 + └── StreamExchange Hash([0]) from 8 Fragment 6 - StreamProject { exprs: [r_regionkey, _row_id] } - └── StreamRowIdGen { row_id_index: 3 } - └── StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } { tables: [ Source: 12 ] } + StreamNoOp + └── StreamExchange NoShuffle from 7 Fragment 7 - StreamNoOp - └── StreamExchange NoShuffle from 8 + StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + └── StreamRowIdGen { row_id_index: 5 } + └── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { tables: [ Source: 18 ] } Fragment 8 - StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - └── StreamRowIdGen { row_id_index: 4 } - └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { tables: [ Source: 13 ] } + StreamNoOp + └── StreamExchange NoShuffle from 9 Fragment 9 - StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id, ps_suppkey, _row_id] } { tables: [ HashJoinLeft: 14, HashJoinDegreeLeft: 15, HashJoinRight: 16, HashJoinDegreeRight: 17 ] } - ├── StreamExchange Hash([2]) from 10 - └── StreamExchange Hash([0]) from 14 + StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └── StreamRowIdGen { row_id_index: 7 } + └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { tables: [ Source: 19 ] } Fragment 10 - StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id] } { tables: [ HashJoinLeft: 18, HashJoinDegreeLeft: 19, HashJoinRight: 20, HashJoinDegreeRight: 21 ] } + StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } { tables: [ HashJoinLeft: 20, HashJoinDegreeLeft: 21, HashJoinRight: 22, HashJoinDegreeRight: 23 ] } ├── StreamExchange Hash([0]) from 11 - └── StreamExchange Hash([0]) from 12 + └── StreamExchange Hash([2]) from 13 Fragment 11 - StreamRowIdGen { row_id_index: 9 } - └── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { tables: [ Source: 22 ] } + StreamNoOp + └── StreamExchange NoShuffle from 12 Fragment 12 - StreamNoOp - └── StreamExchange NoShuffle from 13 + StreamProject { exprs: [r_regionkey, _row_id] } + └── StreamRowIdGen { row_id_index: 3 } + └── StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } { tables: [ Source: 24 ] } Fragment 13 - StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - └── StreamRowIdGen { row_id_index: 5 } - └── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { tables: [ Source: 23 ] } + StreamNoOp + └── StreamExchange NoShuffle from 14 Fragment 14 - StreamNoOp - └── StreamExchange NoShuffle from 15 + StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + └── StreamRowIdGen { row_id_index: 4 } + └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { tables: [ Source: 25 ] } Fragment 15 - StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - └── StreamRowIdGen { row_id_index: 7 } - └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { tables: [ Source: 24 ] } + StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = ps_suppkey, output: [n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, s_suppkey, _row_id] } { tables: [ HashJoinLeft: 26, HashJoinDegreeLeft: 27, HashJoinRight: 28, HashJoinDegreeRight: 29 ] } + ├── StreamExchange Hash([1]) from 16 + └── StreamExchange Hash([1]) from 21 Fragment 16 - StreamProject { exprs: [p_partkey] } - └── StreamExchange NoShuffle from 3 + StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } { tables: [ HashJoinLeft: 30, HashJoinDegreeLeft: 31, HashJoinRight: 32, HashJoinDegreeRight: 33 ] } + ├── StreamExchange Hash([0]) from 17 + └── StreamExchange Hash([3]) from 20 Fragment 17 - StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } { tables: [ HashJoinLeft: 32, HashJoinDegreeLeft: 33, HashJoinRight: 34, HashJoinDegreeRight: 35 ] } - ├── StreamExchange Hash([2]) from 18 - └── StreamExchange Hash([0]) from 21 + StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { tables: [ HashJoinLeft: 34, HashJoinDegreeLeft: 35, HashJoinRight: 36, HashJoinDegreeRight: 37 ] } + ├── StreamExchange Hash([0]) from 18 + └── StreamExchange Hash([2]) from 19 Fragment 18 - StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } { tables: [ HashJoinLeft: 36, HashJoinDegreeLeft: 37, HashJoinRight: 38, HashJoinDegreeRight: 39 ] } - ├── StreamExchange Hash([1]) from 19 - └── StreamExchange Hash([0]) from 20 + StreamNoOp + └── StreamExchange NoShuffle from 12 Fragment 19 - StreamFilter { predicate: IsNotNull(ps_partkey) } - └── StreamExchange NoShuffle from 13 + StreamNoOp + └── StreamExchange NoShuffle from 14 Fragment 20 StreamNoOp - └── StreamExchange NoShuffle from 15 + └── StreamExchange NoShuffle from 9 Fragment 21 - StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } { tables: [ HashJoinLeft: 40, HashJoinDegreeLeft: 41, HashJoinRight: 42, HashJoinDegreeRight: 43 ] } - ├── StreamExchange Hash([0]) from 22 - └── StreamExchange Hash([2]) from 23 - - Fragment 22 - StreamNoOp - └── StreamExchange NoShuffle from 6 - - Fragment 23 - StreamNoOp - └── StreamExchange NoShuffle from 8 + StreamFilter { predicate: (ps_partkey = ps_partkey) } + └── StreamExchange NoShuffle from 7 - Table 0 { columns: [ p_partkey, p_mfgr, s_name, s_address, s_phone, s_acctbal, s_comment, ps_supplycost, n_name, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1, _row_id_2, _row_id_3, ps_suppkey ], primary key: [ $0 ASC, $7 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 0 { columns: [ p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id ], primary key: [ $0 ASC, $3 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } - Table 1 { columns: [ p_partkey, ps_supplycost, _row_id, _row_id_0, r_regionkey, _row_id_1, _row_id_2, _row_id_3, ps_suppkey, n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 1 { columns: [ p_partkey, min(ps_supplycost), ps_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 2 { columns: [ p_partkey, min(ps_supplycost) ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 2 { columns: [ n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, s_suppkey, _row_id_2 ], primary key: [ $6 ASC, $7 ASC, $6 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $14 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 6, 7 ], read pk prefix len hint: 3 } - Table 3 { columns: [ p_partkey, min(ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 3 { columns: [ ps_partkey, ps_supplycost, ps_partkey_0, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, _row_id_2, s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } - Table 4 { columns: [ n_nationkey, n_name, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ p_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ p_partkey, p_mfgr, ps_supplycost, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id, _row_id_0, ps_suppkey, _row_id_1 ], primary key: [ $5 ASC, $9 ASC, $10 ASC, $0 ASC, $12 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 6 { columns: [ ps_partkey, min(ps_supplycost) ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ s_nationkey, _row_id, _row_id_0, p_partkey, _row_id_1, ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ ps_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ r_regionkey, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 9 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 9 { columns: [ ps_partkey, min(ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 10 { columns: [ ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id_0 ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 11 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ s_nationkey, _row_id, _row_id_0, ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 12 { columns: [ n_nationkey, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 13 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 14 { columns: [ p_partkey, p_mfgr, ps_suppkey, ps_supplycost, _row_id, _row_id_0 ], primary key: [ $2 ASC, $4 ASC, $5 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 14 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id ], primary key: [ $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 15 { columns: [ ps_suppkey, _row_id, _row_id_0, p_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 15 { columns: [ ps_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 16 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 17 { columns: [ s_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 18 { columns: [ p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id ], primary key: [ $0 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 18 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 19 { columns: [ p_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 19 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 20 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 20 { columns: [ r_regionkey, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 21 { columns: [ ps_partkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 22 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 22 { columns: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 23 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 23 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 24 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 25 { columns: [ p_partkey, ps_supplycost, _row_id, _row_id_0, ps_suppkey, _row_id_1, _row_id_2, r_regionkey, s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 26 { columns: [ p_partkey, min(ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 27 { columns: [ p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 28 { columns: [ p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 29 { columns: [ ps_partkey, ps_supplycost, _row_id, _row_id_0, ps_suppkey, s_nationkey, _row_id_1, _row_id_2, r_regionkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $6 ASC, $7 ASC, $8 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 30 { columns: [ ps_partkey, _row_id, _row_id_0, ps_suppkey, _row_id_1, _row_id_2, r_regionkey, s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - - Table 31 { columns: [ p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 25 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 32 { columns: [ ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id_0 ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 26 { columns: [ n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id_0, r_regionkey, n_nationkey, _row_id_1 ], primary key: [ $1 ASC, $7 ASC, $8 ASC, $9 ASC, $11 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 33 { columns: [ s_nationkey, _row_id, _row_id_0, ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 27 { columns: [ s_suppkey, _row_id, _row_id_0, r_regionkey, _row_id_1, n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { columns: [ n_nationkey, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 28 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id ], primary key: [ $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 35 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 29 { columns: [ ps_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 36 { columns: [ ps_partkey, ps_suppkey, ps_supplycost, _row_id ], primary key: [ $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 30 { columns: [ n_nationkey, n_name, _row_id, r_regionkey, _row_id_0 ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 37 { columns: [ ps_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 31 { columns: [ n_nationkey, _row_id, _row_id_0, r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 38 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 32 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 39 { columns: [ s_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 33 { columns: [ s_nationkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 40 { columns: [ r_regionkey, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 34 { columns: [ r_regionkey, _row_id ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 41 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 35 { columns: [ r_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 42 { columns: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 36 { columns: [ n_nationkey, n_name, n_regionkey, _row_id ], primary key: [ $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 43 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 37 { columns: [ n_regionkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16 ], read pk prefix len hint: 13 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], read pk prefix len hint: 14 } - id: tpch_q5 before: @@ -1719,23 +1626,15 @@ ├─LogicalJoin { type: Inner, on: (s_nationkey = n_nationkey), output: [s_suppkey, s_name, s_address] } │ ├─LogicalSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id], time_range: (Unbounded, Unbounded) } │ └─LogicalSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(ps_partkey, ps_partkey) AND IsNotDistinctFrom(ps_suppkey, ps_suppkey) AND ($expr1 > $expr2), output: [ps_suppkey] } + └─LogicalJoin { type: Inner, on: (ps_partkey = l_partkey) AND (ps_suppkey = l_suppkey) AND ($expr1 > $expr2), output: [ps_suppkey] } ├─LogicalProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1] } │ └─LogicalJoin { type: LeftSemi, on: (ps_partkey = p_partkey), output: [ps_partkey, ps_suppkey, ps_availqty] } │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } │ └─LogicalProject { exprs: [p_partkey] } │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - └─LogicalAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity)] } - └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(ps_partkey, l_partkey) AND IsNotDistinctFrom(ps_suppkey, l_suppkey), output: [ps_partkey, ps_suppkey, l_quantity] } - ├─LogicalAgg { group_key: [ps_partkey, ps_suppkey], aggs: [] } - │ └─LogicalJoin { type: LeftSemi, on: (ps_partkey = p_partkey), output: [ps_partkey, ps_suppkey] } - │ ├─LogicalSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], time_range: (Unbounded, Unbounded) } - │ └─LogicalProject { exprs: [p_partkey] } - │ └─LogicalSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], time_range: (Unbounded, Unbounded) } - └─LogicalProject { exprs: [l_partkey, l_suppkey, l_quantity] } - └─LogicalFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } - └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } + └─LogicalAgg { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity)] } + └─LogicalSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], time_range: (Unbounded, Unbounded) } batch_plan: |- BatchExchange { order: [s_name ASC], dist: Single } └─BatchSort { order: [s_name ASC] } @@ -1747,7 +1646,7 @@ │ └─BatchExchange { order: [], dist: HashShard(n_nationkey) } │ └─BatchSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(ps_suppkey) } - └─BatchHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey AND ($expr1 > $expr2), output: [ps_suppkey] } + └─BatchHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey AND ($expr1 > $expr2), output: [ps_suppkey] } ├─BatchExchange { order: [], dist: HashShard(ps_partkey, ps_suppkey) } │ └─BatchProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1] } │ └─BatchHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty] } @@ -1756,21 +1655,10 @@ │ └─BatchExchange { order: [], dist: HashShard(p_partkey) } │ └─BatchProject { exprs: [p_partkey] } │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } - └─BatchProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - └─BatchHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity)] } - └─BatchHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity] } - ├─BatchExchange { order: [], dist: HashShard(ps_partkey, ps_suppkey) } - │ └─BatchHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [] } - │ └─BatchHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey] } - │ ├─BatchExchange { order: [], dist: HashShard(ps_partkey) } - │ │ └─BatchSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id], filter: (None, None) } - │ └─BatchExchange { order: [], dist: HashShard(p_partkey) } - │ └─BatchProject { exprs: [p_partkey] } - │ └─BatchSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id], filter: (None, None) } - └─BatchExchange { order: [], dist: HashShard(l_partkey, l_suppkey) } - └─BatchProject { exprs: [l_partkey, l_suppkey, l_quantity] } - └─BatchFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } - └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } + └─BatchProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } + └─BatchHashAgg { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity)] } + └─BatchExchange { order: [], dist: HashShard(l_partkey, l_suppkey) } + └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(_row_id, _row_id, s_nationkey, s_suppkey) } @@ -1784,40 +1672,24 @@ │ └─StreamRowIdGen { row_id_index: 4 } │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } └─StreamExchange { dist: HashShard(ps_suppkey) } - └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } + └─StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, l_partkey, l_suppkey] } └─StreamFilter { predicate: ($expr1 > $expr2) } - └─StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } + └─StreamHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey, output: all } ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } - │ └─StreamShare { id: 13 } - │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 9 } - │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - └─StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } - └─StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } - ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } - │ └─StreamProject { exprs: [ps_partkey, ps_suppkey] } - │ └─StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } - │ └─StreamShare { id: 13 } - │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 5 } - │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamProject { exprs: [p_partkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 9 } - │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } - └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } - └─StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } - └─StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } - └─StreamRowIdGen { row_id_index: 16 } - └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 5 } + │ │ └─StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamProject { exprs: [p_partkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 9 } + │ └─StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } + └─StreamProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } + └─StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } + └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } + └─StreamRowIdGen { row_id_index: 16 } + └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } @@ -1843,52 +1715,39 @@ └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { tables: [ Source: 9 ] } Fragment 5 - StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, ps_partkey, ps_suppkey] } + StreamProject { exprs: [ps_suppkey, _row_id, ps_partkey, l_partkey, l_suppkey] } └── StreamFilter { predicate: ($expr1 > $expr2) } - └── StreamHashJoin { type: Inner, predicate: ps_partkey IS NOT DISTINCT FROM ps_partkey AND ps_suppkey IS NOT DISTINCT FROM ps_suppkey, output: all } { tables: [ HashJoinLeft: 10, HashJoinDegreeLeft: 11, HashJoinRight: 12, HashJoinDegreeRight: 13 ] } + └── StreamHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey, output: all } { tables: [ HashJoinLeft: 10, HashJoinDegreeLeft: 11, HashJoinRight: 12, HashJoinDegreeRight: 13 ] } ├── StreamExchange Hash([0, 1]) from 6 - └── StreamProject { exprs: [ps_partkey, ps_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - └── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [sum(l_quantity), count] } { tables: [ HashAggState: 20 ] } - └── StreamHashJoin { type: LeftOuter, predicate: ps_partkey IS NOT DISTINCT FROM l_partkey AND ps_suppkey IS NOT DISTINCT FROM l_suppkey, output: [ps_partkey, ps_suppkey, l_quantity, _row_id] } - ├── tables: [ HashJoinLeft: 21, HashJoinDegreeLeft: 22, HashJoinRight: 23, HashJoinDegreeRight: 24 ] - ├── StreamExchange Hash([0, 1]) from 10 - └── StreamExchange Hash([0, 1]) from 11 + └── StreamProject { exprs: [(0.5:Decimal * sum(l_quantity)) as $expr2, l_partkey, l_suppkey] } + └── StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } { tables: [ HashAggState: 20 ] } + └── StreamExchange Hash([1, 2]) from 9 Fragment 6 StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } - └── StreamExchange NoShuffle from 7 + └── StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } { tables: [ HashJoinLeft: 14, HashJoinDegreeLeft: 15, HashJoinRight: 16, HashJoinDegreeRight: 17 ] } + ├── StreamExchange Hash([0]) from 7 + └── StreamExchange Hash([0]) from 8 Fragment 7 - StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_partkey, ps_suppkey, ps_availqty, _row_id] } { tables: [ HashJoinLeft: 14, HashJoinDegreeLeft: 15, HashJoinRight: 16, HashJoinDegreeRight: 17 ] } - ├── StreamExchange Hash([0]) from 8 - └── StreamExchange Hash([0]) from 9 - - Fragment 8 StreamRowIdGen { row_id_index: 5 } └── StreamSource { source: partsupp, columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id] } { tables: [ Source: 18 ] } - Fragment 9 + Fragment 8 StreamProject { exprs: [p_partkey, _row_id] } └── StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: part, columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _row_id] } { tables: [ Source: 19 ] } - Fragment 10 - StreamProject { exprs: [ps_partkey, ps_suppkey] } - └── StreamHashAgg { group_key: [ps_partkey, ps_suppkey], aggs: [count] } { tables: [ HashAggState: 25 ] } - └── StreamExchange NoShuffle from 7 - - Fragment 11 - StreamProject { exprs: [l_partkey, l_suppkey, l_quantity, _row_id] } - └── StreamFilter { predicate: IsNotNull(l_partkey) AND IsNotNull(l_suppkey) } - └── StreamRowIdGen { row_id_index: 16 } - └── StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } - └── tables: [ Source: 26 ] + Fragment 9 + StreamRowIdGen { row_id_index: 16 } + └── StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + └── tables: [ Source: 21 ] Table 0 { columns: [ s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id_0 ], primary key: [ $0 ASC, $3 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 { columns: [ s_suppkey, _row_id, _row_id_0, s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ ps_suppkey, _row_id, ps_partkey, ps_partkey_0, ps_suppkey_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ ps_suppkey, _row_id, ps_partkey, l_partkey, l_suppkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 3 { columns: [ ps_suppkey, _row_id, ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1908,9 +1767,9 @@ Table 11 { columns: [ ps_partkey, ps_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 12 { columns: [ ps_partkey, ps_suppkey, $expr2 ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 12 { columns: [ $expr2, l_partkey, l_suppkey ], primary key: [ $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } - Table 13 { columns: [ ps_partkey, ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 13 { columns: [ l_partkey, l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 14 { columns: [ ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _row_id ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1924,19 +1783,9 @@ Table 19 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 20 { columns: [ ps_partkey, ps_suppkey, sum(l_quantity), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 21 { columns: [ ps_partkey, ps_suppkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 20 { columns: [ l_partkey, l_suppkey, sum(l_quantity), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 22 { columns: [ ps_partkey, ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 23 { columns: [ l_partkey, l_suppkey, l_quantity, _row_id ], primary key: [ $0 ASC, $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 24 { columns: [ l_partkey, l_suppkey, _row_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - - Table 25 { columns: [ ps_partkey, ps_suppkey, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - - Table 26 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 21 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3, 4, 5 ], read pk prefix len hint: 5 } diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 6a1ceac758e4..334370293672 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -144,6 +144,7 @@ static SIMPLE_UNNESTING: LazyLock = LazyLock::new(|| { ApplyToJoinRule::create(), // Pull correlated predicates up the algebra tree to unnest simple subquery. PullUpCorrelatedPredicateRule::create(), + PullUpCorrelatedPredicateAggRule::create(), ], ApplyOrder::BottomUp, ) diff --git a/src/frontend/src/optimizer/rule/mod.rs b/src/frontend/src/optimizer/rule/mod.rs index acde2f7b72eb..c77d4f24f155 100644 --- a/src/frontend/src/optimizer/rule/mod.rs +++ b/src/frontend/src/optimizer/rule/mod.rs @@ -154,8 +154,10 @@ mod apply_hop_window_transpose_rule; pub use apply_hop_window_transpose_rule::*; mod agg_call_merge_rule; pub use agg_call_merge_rule::*; +mod pull_up_correlated_predicate_agg_rule; mod values_extract_project_rule; pub use batch::batch_push_limit_to_scan_rule::*; +pub use pull_up_correlated_predicate_agg_rule::*; pub use values_extract_project_rule::*; #[macro_export] @@ -227,6 +229,7 @@ macro_rules! for_all_rules { , { AggCallMergeRule } , { ValuesExtractProjectRule } , { BatchPushLimitToScanRule } + , { PullUpCorrelatedPredicateAggRule } } }; } diff --git a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs new file mode 100644 index 000000000000..b40508255fbf --- /dev/null +++ b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs @@ -0,0 +1,236 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use itertools::{Either, Itertools}; +use risingwave_common::types::DataType; +use risingwave_common::util::column_index_mapping::ColIndexMapping; + +use super::super::plan_node::*; +use super::{BoxedRule, Rule}; +use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, FunctionCall, InputRef}; +use crate::optimizer::plan_node::generic::{Agg, GenericPlanNode, GenericPlanRef}; +use crate::optimizer::plan_visitor::{PlanCorrelatedIdFinder, PlanVisitor}; +use crate::optimizer::PlanRef; +use crate::utils::{Condition, IndexSet}; + +/// Pull up correlated predicates from the right agg side of Apply to the `on` clause of Join. +/// +/// Before: +/// +/// ```text +/// LogicalApply +/// / \ +/// LHS Project +/// | +/// Agg [group by nothing] +/// | +/// Project +/// | +/// Filter [correlated_input_ref(yyy) = xxx] +/// ``` +/// +/// After: +/// +/// ```text +/// LogicalApply [yyy = xxx] +/// / \ +/// LHS Project +/// | +/// Agg [group by xxx] +/// | +/// Project +/// | +/// Filter +/// ``` + +pub struct PullUpCorrelatedPredicateAggRule {} +impl Rule for PullUpCorrelatedPredicateAggRule { + fn apply(&self, plan: PlanRef) -> Option { + let apply = plan.as_logical_apply()?; + let (apply_left, apply_right, apply_on, join_type, correlated_id, _, max_one_row) = + apply.clone().decompose(); + + if max_one_row { + return None; + } + + let top_project = if let Some(project) = apply_right.as_logical_project() { + project.clone() + } else { + LogicalProject::with_mapping( + apply_right.clone(), + ColIndexMapping::identity(apply_right.schema().len()), + ) + }; + let (top_proj_exprs, _) = top_project.clone().decompose(); + + let input = top_project.input(); + let agg: &LogicalAgg = input.as_logical_agg()?; + let (agg_calls, group_key, grouping_sets, input, _enable_two_phase) = + agg.clone().decompose(); + // It could be too restrictive to require the group key to be empty. We can relax this in the future if necessary. + if !group_key.is_empty() { + return None; + } + assert!(grouping_sets.is_empty()); + let bottom_project = if let Some(project) = input.as_logical_project() { + project.clone() + } else { + LogicalProject::with_mapping( + input.clone(), + ColIndexMapping::identity(input.schema().len()), + ) + }; + let (mut bottom_proj_exprs, _) = bottom_project.clone().decompose(); + let bottom_project_input = bottom_project.input(); + let filter: &LogicalFilter = bottom_project_input.as_logical_filter()?; + + // Split predicates in LogicalFilter into correlated expressions and uncorrelated + // expressions. + let (cor_exprs, uncor_exprs): (Vec, Vec) = filter + .predicate() + .clone() + .into_iter() + .partition_map(|expr| { + if expr.has_correlated_input_ref_by_correlated_id(correlated_id) { + Either::Left(expr) + } else { + Either::Right(expr) + } + }); + + // Ensure all correlated expressions look like [correlated_input_ref = input_ref] + let mut cor_eq_exprs = vec![]; + for cor_expr in &cor_exprs { + if let Some((input_ref, cor_input_ref)) = cor_expr.as_eq_correlated_input_ref() { + if cor_input_ref.correlated_id() == correlated_id { + cor_eq_exprs.push((input_ref, cor_input_ref)); + } else { + return None; + } + } else { + return None; + } + } + let cor_eq_exprs_len = cor_eq_exprs.len(); + + let filter = LogicalFilter::create( + filter.input(), + Condition { + conjunctions: uncor_exprs, + }, + ); + + // Append `InputRef`s in the predicate expression to be pulled to the project, so that they + // are accessible by the expression after it is pulled. + bottom_proj_exprs.extend( + cor_eq_exprs + .iter() + .map(|(input_ref, _)| ExprImpl::InputRef(input_ref.clone().into())), + ); + + let new_bottom_proj: PlanRef = LogicalProject::new(filter, bottom_proj_exprs).into(); + + // New agg with group key extracted from the cor_eq_exprs. + let new_agg = Agg::new( + agg_calls, // TODO: use count0 to replace count. + IndexSet::from_iter( + new_bottom_proj.schema().len() - cor_eq_exprs_len..new_bottom_proj.schema().len(), + ), + new_bottom_proj, + ); + + let mut shift_input_ref_rewriter = ShiftInputRefRewriter { + offset: cor_eq_exprs_len, + }; + // Shift the top project expressions to the right by cor_eq_exprs_len. + let mut top_proj_exprs = top_proj_exprs + .into_iter() + .map(|expr| shift_input_ref_rewriter.rewrite_expr(expr)) + .collect_vec(); + top_proj_exprs.extend((0..new_agg.group_key.len()).map(|i| { + ExprImpl::InputRef( + InputRef::new(i, new_agg.schema().fields[i].data_type.clone()).into(), + ) + })); + + let new_top_proj: PlanRef = LogicalProject::new(new_agg.into(), top_proj_exprs).into(); + + let cor_eq_exprs = cor_eq_exprs + .into_iter() + .enumerate() + .map(|(i, (input_ref, correlated_input_ref))| { + ExprImpl::FunctionCall( + FunctionCall::new_unchecked( + ExprType::Equal, + vec![ + InputRef::new( + correlated_input_ref.index(), + correlated_input_ref.return_type(), + ) + .into(), + InputRef::new( + new_top_proj.schema().len() - cor_eq_exprs_len + + i + + apply_left.schema().len(), + input_ref.return_type(), + ) + .into(), + ], + DataType::Boolean, + ) + .into(), + ) + }) + .collect_vec(); + + // Check whether correlated_input_ref with same correlated_id exists for the join right + // side. If yes, bail out and leave for general subquery unnesting to deal with + let mut plan_correlated_id_finder = PlanCorrelatedIdFinder::default(); + plan_correlated_id_finder.visit(new_top_proj.clone()); + if plan_correlated_id_finder.contains(&correlated_id) { + return None; + } + + // Merge these expressions with LogicalApply into LogicalJoin. + let on = apply_on.and(Condition { + conjunctions: cor_eq_exprs, + }); + Some( + LogicalJoin::with_output_indices( + apply_left, + new_top_proj, + join_type, + on, + (0..apply.schema().len()).collect(), + ) + .into(), + ) + } +} + +struct ShiftInputRefRewriter { + offset: usize, +} +impl ExprRewriter for ShiftInputRefRewriter { + fn rewrite_input_ref(&mut self, input_ref: InputRef) -> ExprImpl { + InputRef::new(input_ref.index() + self.offset, input_ref.return_type()).into() + } +} + +impl PullUpCorrelatedPredicateAggRule { + pub fn create() -> BoxedRule { + Box::new(PullUpCorrelatedPredicateAggRule {}) + } +} From f79245735b71422db6611dd648c2114ad3564382 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Tue, 6 Feb 2024 15:26:10 +0800 Subject: [PATCH 2/4] forbid count agg in PullUpCorrelatedPredicateAggRule --- .../tests/testdata/output/subquery.yaml | 40 ++++--- .../output/subquery_expr_correlated.yaml | 68 ++++++++---- .../tests/testdata/output/tpch.yaml | 101 ++++++++++++------ .../pull_up_correlated_predicate_agg_rule.rs | 8 +- 4 files changed, 152 insertions(+), 65 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 73e7cffbc89b..6dda70d6f98e 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -804,28 +804,42 @@ select * from integers where 2 in (select count(distinct k) + count(distinct v) from rows where correlated_col = integers.correlated_col); batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: LeftSemi, predicate: integers.correlated_col = rows.correlated_col_expanded AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col] } + └─BatchHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col_expanded AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col] } ├─BatchExchange { order: [], dist: HashShard(integers.correlated_col) } │ └─BatchProject { exprs: [integers.i, integers.correlated_col, 2:Int64] } │ └─BatchScan { table: integers, columns: [integers.i, integers.correlated_col], distribution: SomeShard } - └─BatchProject { exprs: [(count(rows.k_expanded) filter((flag = 0:Int64)) + count(rows.v_expanded) filter((flag = 1:Int64))) as $expr1, rows.correlated_col_expanded] } - └─BatchHashAgg { group_key: [rows.correlated_col_expanded], aggs: [count(rows.k_expanded) filter((flag = 0:Int64)), count(rows.v_expanded) filter((flag = 1:Int64))] } - └─BatchExchange { order: [], dist: HashShard(rows.correlated_col_expanded) } - └─BatchHashAgg { group_key: [rows.k_expanded, rows.v_expanded, rows.correlated_col_expanded, flag], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(rows.k_expanded, rows.v_expanded, rows.correlated_col_expanded, flag) } - └─BatchExpand { column_subsets: [[rows.k, rows.correlated_col], [rows.v, rows.correlated_col]] } - └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } + └─BatchProject { exprs: [integers.correlated_col_expanded, (count(rows.k_expanded) filter((flag = 0:Int64)) + count(rows.v_expanded) filter((flag = 1:Int64))) as $expr1] } + └─BatchHashAgg { group_key: [integers.correlated_col_expanded], aggs: [count(rows.k_expanded) filter((flag = 0:Int64)), count(rows.v_expanded) filter((flag = 1:Int64))] } + └─BatchExchange { order: [], dist: HashShard(integers.correlated_col_expanded) } + └─BatchHashAgg { group_key: [integers.correlated_col_expanded, rows.k_expanded, rows.v_expanded, flag], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(integers.correlated_col_expanded, rows.k_expanded, rows.v_expanded, flag) } + └─BatchExpand { column_subsets: [[integers.correlated_col, rows.k], [integers.correlated_col, rows.v]] } + └─BatchHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, 1:Int32] } + ├─BatchHashAgg { group_key: [integers.correlated_col], aggs: [] } + │ └─BatchExchange { order: [], dist: HashShard(integers.correlated_col) } + │ └─BatchScan { table: integers, columns: [integers.correlated_col], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(rows.correlated_col) } + └─BatchProject { exprs: [rows.correlated_col, rows.k, rows.v, 1:Int32] } + └─BatchFilter { predicate: IsNotNull(rows.correlated_col) } + └─BatchScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, correlated_col, integers._row_id(hidden), 2:Int64(hidden)], stream_key: [integers._row_id, correlated_col, 2:Int64], pk_columns: [integers._row_id, correlated_col, 2:Int64], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(integers.correlated_col, integers._row_id, 2:Int64) } - └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col = rows.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } + └─StreamHashJoin { type: LeftSemi, predicate: integers.correlated_col IS NOT DISTINCT FROM integers.correlated_col AND 2:Int64 = $expr1, output: [integers.i, integers.correlated_col, integers._row_id, 2:Int64] } ├─StreamExchange { dist: HashShard(integers.correlated_col) } │ └─StreamProject { exprs: [integers.i, integers.correlated_col, 2:Int64, integers._row_id] } │ └─StreamTableScan { table: integers, columns: [integers.i, integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamProject { exprs: [(count(distinct rows.k) + count(distinct rows.v)) as $expr1, rows.correlated_col] } - └─StreamHashAgg { group_key: [rows.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } - └─StreamExchange { dist: HashShard(rows.correlated_col) } - └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } + └─StreamProject { exprs: [integers.correlated_col, (count(distinct rows.k) + count(distinct rows.v)) as $expr1] } + └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count(distinct rows.k), count(distinct rows.v), count] } + └─StreamHashJoin { type: LeftOuter, predicate: integers.correlated_col IS NOT DISTINCT FROM rows.correlated_col, output: [integers.correlated_col, rows.k, rows.v, rows._row_id] } + ├─StreamProject { exprs: [integers.correlated_col] } + │ └─StreamHashAgg { group_key: [integers.correlated_col], aggs: [count] } + │ └─StreamExchange { dist: HashShard(integers.correlated_col) } + │ └─StreamTableScan { table: integers, columns: [integers.correlated_col, integers._row_id], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamExchange { dist: HashShard(rows.correlated_col) } + └─StreamProject { exprs: [rows.correlated_col, rows.k, rows.v, rows._row_id] } + └─StreamFilter { predicate: IsNotNull(rows.correlated_col) } + └─StreamTableScan { table: rows, columns: [rows.k, rows.v, rows.correlated_col, rows._row_id], pk: [rows._row_id], dist: UpstreamHashShard(rows._row_id) } - name: test hop window subquery 1 sql: | create table t1 (k int primary key, ts timestamp); diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 069e110b65ae..347ab1551943 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -112,11 +112,15 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan_for_batch: |- - LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > count), output: [t1.x, t1.y] } + LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > count(1:Int32)), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } │ └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalAgg { group_key: [t2.y], aggs: [count] } - └─LogicalScan { table: t2, columns: [t2.y] } + └─LogicalAgg { group_key: [t1.y], aggs: [count(1:Int32)] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, 1:Int32] } + ├─LogicalAgg { group_key: [t1.y], aggs: [] } + │ └─LogicalScan { table: t1, columns: [t1.y] } + └─LogicalProject { exprs: [t2.y, 1:Int32] } + └─LogicalScan { table: t2, columns: [t2.y], predicate: IsNotNull(t2.y) } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -132,12 +136,16 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan_for_batch: |- - LogicalJoin { type: Inner, on: (t1.y = t2.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } + LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr1 > $expr2), output: [t1.x, t1.y] } ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr1] } │ └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [(count + count) as $expr2, t2.y] } - └─LogicalAgg { group_key: [t2.y], aggs: [count] } - └─LogicalScan { table: t2, columns: [t2.y] } + └─LogicalProject { exprs: [t1.y, (count(1:Int32) + count(1:Int32)) as $expr2] } + └─LogicalAgg { group_key: [t1.y], aggs: [count(1:Int32)] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, 1:Int32] } + ├─LogicalAgg { group_key: [t1.y], aggs: [] } + │ └─LogicalScan { table: t1, columns: [t1.y] } + └─LogicalProject { exprs: [t2.y, 1:Int32] } + └─LogicalScan { table: t2, columns: [t2.y], predicate: IsNotNull(t2.y) } - sql: | create table t1(x int, y int); create table t2(x int, y int); @@ -450,13 +458,19 @@ └─LogicalScan { table: c, columns: [c.c1, c.c2, c.c3, c._row_id] } optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: (b.b2 = c.c2) AND (a.a3 = c.c3), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.a3, internal_last_seen_value(a.a3)) AND IsNotDistinctFrom(b.b2, b.b2), output: [] } ├─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } │ ├─LogicalScan { table: a, columns: [a.a3] } │ └─LogicalScan { table: b, columns: [b.b2] } - └─LogicalFilter { predicate: (3:Int32 = count) } - └─LogicalAgg { group_key: [c.c2, c.c3], aggs: [count] } - └─LogicalScan { table: c, columns: [c.c2, c.c3], predicate: (c.c3 = c.c2) } + └─LogicalFilter { predicate: (3:Int32 = count(1:Int32)) } + └─LogicalAgg { group_key: [internal_last_seen_value(a.a3), b.b2], aggs: [count(1:Int32)] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(internal_last_seen_value(a.a3), c.c3) AND IsNotDistinctFrom(b.b2, c.c2), output: [internal_last_seen_value(a.a3), b.b2, 1:Int32] } + ├─LogicalAgg { group_key: [b.b2], aggs: [internal_last_seen_value(a.a3)] } + │ └─LogicalJoin { type: Inner, on: (a.a3 = b.b2), output: all } + │ ├─LogicalScan { table: a, columns: [a.a3] } + │ └─LogicalScan { table: b, columns: [b.b2] } + └─LogicalProject { exprs: [c.c3, c.c2, 1:Int32] } + └─LogicalScan { table: c, columns: [c.c2, c.c3], predicate: (c.c3 = c.c2) AND IsNotNull(c.c3) AND IsNotNull(c.c2) } - sql: | create table a(x int, y int, z int); create table b(x int, y int, z int); @@ -481,11 +495,15 @@ select count(*) from a where a.x=3 and a.y = (select count(*) from b where b.z = a.z); optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: (a.z = b.z) AND ($expr1 = count), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(1:Int32)), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } │ └─LogicalScan { table: a, output_columns: [a.y, a.z], required_columns: [a.y, a.z, a.x], predicate: (a.x = 3:Int32) } - └─LogicalAgg { group_key: [b.z], aggs: [count] } - └─LogicalScan { table: b, columns: [b.z] } + └─LogicalAgg { group_key: [a.z], aggs: [count(1:Int32)] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, 1:Int32] } + ├─LogicalAgg { group_key: [a.z], aggs: [] } + │ └─LogicalScan { table: a, output_columns: [a.z], required_columns: [a.z, a.x], predicate: (a.x = 3:Int32) } + └─LogicalProject { exprs: [b.z, 1:Int32] } + └─LogicalScan { table: b, columns: [b.z], predicate: IsNotNull(b.z) } - sql: | create table a(x int, y varchar, z int); create table b(x varchar, y int, z int); @@ -503,23 +521,31 @@ select count(*) from a where a.y = (select count(distinct x) from b where b.z = a.z); optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: (a.z = b.z) AND ($expr1 = count(b.x)), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(b.x)), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } │ └─LogicalScan { table: a, columns: [a.y, a.z] } - └─LogicalAgg { group_key: [b.z], aggs: [count(b.x)] } - └─LogicalAgg { group_key: [b.x, b.z], aggs: [] } - └─LogicalScan { table: b, columns: [b.x, b.z] } + └─LogicalAgg { group_key: [a.z], aggs: [count(b.x)] } + └─LogicalAgg { group_key: [a.z, b.x], aggs: [] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x] } + ├─LogicalAgg { group_key: [a.z], aggs: [] } + │ └─LogicalScan { table: a, columns: [a.z] } + └─LogicalProject { exprs: [b.z, b.x] } + └─LogicalScan { table: b, columns: [b.x, b.z], predicate: IsNotNull(b.z) } - sql: | create table a(x int, y int, z int); create table b(x int, y int, z int); select count(*) from a where a.y = (select count(x) filter(where x < 100) from b where b.z = a.z); optimized_logical_plan_for_batch: |- LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: (a.z = b.z) AND ($expr1 = count(b.x) filter((b.x < 100:Int32))), output: [] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr1 = count(b.x) filter((b.x < 100:Int32))), output: [] } ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr1] } │ └─LogicalScan { table: a, columns: [a.y, a.z] } - └─LogicalAgg { group_key: [b.z], aggs: [count(b.x) filter((b.x < 100:Int32))] } - └─LogicalScan { table: b, columns: [b.x, b.z] } + └─LogicalAgg { group_key: [a.z], aggs: [count(b.x) filter((b.x < 100:Int32))] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x] } + ├─LogicalAgg { group_key: [a.z], aggs: [] } + │ └─LogicalScan { table: a, columns: [a.z] } + └─LogicalProject { exprs: [b.z, b.x] } + └─LogicalScan { table: b, columns: [b.x, b.z], predicate: IsNotNull(b.z) } - sql: | create table t1(x int, y int); create table t2(x int, y int); diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 89ae1972bfd7..672c4433397f 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -3352,36 +3352,46 @@ optimized_logical_plan_for_batch: |- LogicalProject { exprs: [(sum(lineitem.l_extendedprice) / 7.0:Decimal) as $expr2] } └─LogicalAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─LogicalJoin { type: Inner, on: (part.p_partkey = lineitem.l_partkey) AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(part.p_partkey, part.p_partkey) AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } ├─LogicalJoin { type: Inner, on: (part.p_partkey = lineitem.l_partkey), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey] } │ ├─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice] } │ └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_brand, part.p_container], predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └─LogicalProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } - └─LogicalAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } - └─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity] } + └─LogicalProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } + └─LogicalAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } + └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(part.p_partkey, lineitem.l_partkey), output: [part.p_partkey, lineitem.l_quantity] } + ├─LogicalAgg { group_key: [part.p_partkey], aggs: [] } + │ └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_brand, part.p_container], predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } + └─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], predicate: IsNotNull(lineitem.l_partkey) } batch_plan: |- BatchProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2] } └─BatchSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─BatchHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } + └─BatchHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND (lineitem.l_quantity < $expr1), output: [lineitem.l_extendedprice] } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } │ └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey], lookup table: part } │ └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } │ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice], distribution: SomeShard } - └─BatchProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } - └─BatchHashAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } - └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey) } - └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], distribution: SomeShard } + └─BatchProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } + └─BatchHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } + └─BatchHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity] } + ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } + │ └─BatchSortAgg { group_key: [part.p_partkey], aggs: [] } + │ └─BatchProject { exprs: [part.p_partkey] } + │ └─BatchFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } + │ └─BatchScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], distribution: UpstreamHashShard(part.p_partkey) } + └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) } + └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [(sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal) as $expr2] } └─StreamSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, lineitem.l_partkey] } + └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, part.p_partkey] } └─StreamFilter { predicate: (lineitem.l_quantity < $expr1) } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: all } + └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } ├─StreamExchange { dist: HashShard(part.p_partkey) } │ └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } │ ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } @@ -3390,10 +3400,18 @@ │ └─StreamProject { exprs: [part.p_partkey] } │ └─StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } - └─StreamHashAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } - └─StreamExchange { dist: HashShard(lineitem.l_partkey) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } + └─StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } + └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(part.p_partkey) } + │ └─StreamProject { exprs: [part.p_partkey] } + │ └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } + │ └─StreamProject { exprs: [part.p_partkey] } + │ └─StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } + │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } + └─StreamExchange { dist: HashShard(lineitem.l_partkey) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } @@ -3405,15 +3423,16 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └── StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, lineitem.l_partkey] } + └── StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, part.p_partkey] } └── StreamFilter { predicate: (lineitem.l_quantity < $expr1) } - └── StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: all } - ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] + └── StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } { tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] } ├── StreamExchange Hash([2]) from 2 - └── StreamProject { exprs: [(0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1, lineitem.l_partkey] } - └── StreamHashAgg { group_key: [lineitem.l_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } - ├── tables: [ HashAggState: 11 ] - └── StreamExchange Hash([0]) from 5 + └── StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal)) as $expr1] } + └── StreamHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity), count] } { tables: [ HashAggState: 11 ] } + └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } + ├── tables: [ HashJoinLeft: 12, HashJoinDegreeLeft: 13, HashJoinRight: 14, HashJoinDegreeRight: 15 ] + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([0]) from 6 Fragment 2 StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } @@ -3435,10 +3454,20 @@ └── BatchPlanNode Fragment 5 - StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - ├── tables: [ StreamScan: 12 ] - ├── Upstream - └── BatchPlanNode + StreamProject { exprs: [part.p_partkey] } + └── StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } { tables: [ HashAggState: 16 ] } + └── StreamProject { exprs: [part.p_partkey] } + └── StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } + └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 17 ] } + ├── Upstream + └── BatchPlanNode + + Fragment 6 + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } + └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + ├── tables: [ StreamScan: 18 ] + ├── Upstream + └── BatchPlanNode Table 0 { columns: [ sum(sum(lineitem_l_extendedprice)), count ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } @@ -3451,9 +3480,9 @@ Table 2 { columns: [ part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ $expr1, lineitem_l_partkey ], primary key: [ $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 3 { columns: [ part_p_partkey, $expr1 ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ lineitem_l_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -3467,9 +3496,21 @@ Table 10 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 11 { columns: [ lineitem_l_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ part_p_partkey, sum(lineitem_l_quantity), count(lineitem_l_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 12 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished, lineitem_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 12 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 13 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 14 { columns: [ lineitem_l_partkey, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 15 { columns: [ lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 16 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 17 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 18 { columns: [ vnode, l_orderkey, l_linenumber, lineitem_backfill_finished, lineitem_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } Table 4294967294 { columns: [ avg_yearly ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } diff --git a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs index b40508255fbf..4f1f2ea11002 100644 --- a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs +++ b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs @@ -15,6 +15,7 @@ use itertools::{Either, Itertools}; use risingwave_common::types::DataType; use risingwave_common::util::column_index_mapping::ColIndexMapping; +use risingwave_expr::aggregate::AggKind; use super::super::plan_node::*; use super::{BoxedRule, Rule}; @@ -142,9 +143,14 @@ impl Rule for PullUpCorrelatedPredicateAggRule { let new_bottom_proj: PlanRef = LogicalProject::new(filter, bottom_proj_exprs).into(); + // If there is a count aggregate, bail out and leave for general subquery unnesting to deal. + if agg_calls.iter().any(|agg_call| agg_call.agg_kind == AggKind::Count) { + return None; + }; + // New agg with group key extracted from the cor_eq_exprs. let new_agg = Agg::new( - agg_calls, // TODO: use count0 to replace count. + agg_calls, IndexSet::from_iter( new_bottom_proj.schema().len() - cor_eq_exprs_len..new_bottom_proj.schema().len(), ), From 1fa68b0692b2c04feecbebac27db15f2cbc576ec Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Tue, 6 Feb 2024 15:46:27 +0800 Subject: [PATCH 3/4] fmt --- .../optimizer/rule/pull_up_correlated_predicate_agg_rule.rs | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs index 4f1f2ea11002..6a34dd8621e2 100644 --- a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs +++ b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs @@ -144,7 +144,10 @@ impl Rule for PullUpCorrelatedPredicateAggRule { let new_bottom_proj: PlanRef = LogicalProject::new(filter, bottom_proj_exprs).into(); // If there is a count aggregate, bail out and leave for general subquery unnesting to deal. - if agg_calls.iter().any(|agg_call| agg_call.agg_kind == AggKind::Count) { + if agg_calls + .iter() + .any(|agg_call| agg_call.agg_kind == AggKind::Count) + { return None; }; From c709cb49245fa824478edfc5283d90195527a45e Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Wed, 7 Feb 2024 11:58:03 +0800 Subject: [PATCH 4/4] add comments --- .../optimizer/rule/pull_up_correlated_predicate_agg_rule.rs | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs index 6a34dd8621e2..d4c5403bba14 100644 --- a/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs +++ b/src/frontend/src/optimizer/rule/pull_up_correlated_predicate_agg_rule.rs @@ -44,7 +44,7 @@ use crate::utils::{Condition, IndexSet}; /// After: /// /// ```text -/// LogicalApply [yyy = xxx] +/// LogicalJoin [yyy = xxx] /// / \ /// LHS Project /// | @@ -144,6 +144,7 @@ impl Rule for PullUpCorrelatedPredicateAggRule { let new_bottom_proj: PlanRef = LogicalProject::new(filter, bottom_proj_exprs).into(); // If there is a count aggregate, bail out and leave for general subquery unnesting to deal. + // When group by is empty, count would return 0 instead of null. if agg_calls .iter() .any(|agg_call| agg_call.agg_kind == AggKind::Count)