From d7dfa6292ca15d64847cc7a7c9b92f3eefd68485 Mon Sep 17 00:00:00 2001 From: st1page <1245835950@qq.com> Date: Wed, 15 Jan 2025 17:06:41 +0800 Subject: [PATCH 1/4] feat: use order key as mv's dist key --- .../tests/testdata/output/agg.yaml | 6 +- .../tests/testdata/output/append_only.yaml | 7 +- .../tests/testdata/output/batch_dist_agg.yaml | 9 +- .../tests/testdata/output/ch_benchmark.yaml | 563 +++--- .../output/functional_dependency.yaml | 31 +- .../testdata/output/index_selection.yaml | 13 +- .../tests/testdata/output/join.yaml | 2 +- .../tests/testdata/output/limit.yaml | 10 +- .../tests/testdata/output/nexmark.yaml | 42 +- .../tests/testdata/output/nexmark_source.yaml | 46 +- .../testdata/output/nexmark_source_kafka.yaml | 46 +- .../output/nexmark_temporal_filter.yaml | 88 +- .../testdata/output/nexmark_watermark.yaml | 80 +- .../tests/testdata/output/order_by.yaml | 49 +- .../tests/testdata/output/project_set.yaml | 15 +- .../tests/testdata/output/range_scan.yaml | 34 +- .../tests/testdata/output/row_filter.yaml | 2 +- .../tests/testdata/output/singleton.yaml | 30 +- .../testdata/output/stream_dist_agg.yaml | 193 +- .../tests/testdata/output/subquery.yaml | 42 +- .../tests/testdata/output/topn.yaml | 2 +- .../tests/testdata/output/tpch.yaml | 1701 +++++++++-------- .../tests/testdata/output/tpch_kafka.yaml | 1206 ++++++------ .../tests/testdata/output/tpch_variant.yaml | 258 +-- src/frontend/src/handler/create_mv.rs | 1 + src/frontend/src/optimizer/mod.rs | 12 + .../optimizer/plan_node/stream_materialize.rs | 43 +- 27 files changed, 2365 insertions(+), 2166 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 80a4de55b6f42..c86f405d78ccd 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -312,8 +312,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchSortAgg { group_key: [mv.v1], aggs: [max(mv.v2)] } - └─BatchExchange { order: [mv.v1 DESC], dist: HashShard(mv.v1) } - └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: UpstreamHashShard(mv.v1) } - sql: | create table t(v1 int, v2 int); select v1, max(v2) from t group by v1 order by v1 desc; @@ -367,8 +366,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [mv.v1], aggs: [max(mv.v2)] } - └─BatchExchange { order: [], dist: HashShard(mv.v1) } - └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: UpstreamHashShard(mv.v1) } with_config_map: RW_BATCH_ENABLE_SORT_AGG: 'false' - name: Not use BatchSortAgg, when output requires order diff --git a/src/frontend/planner_test/tests/testdata/output/append_only.yaml b/src/frontend/planner_test/tests/testdata/output/append_only.yaml index d0701675c3617..5c326ecfcbad3 100644 --- a/src/frontend/planner_test/tests/testdata/output/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/output/append_only.yaml @@ -25,9 +25,10 @@ select v1 from t1 order by v1 limit 3 offset 3; stream_plan: |- StreamMaterialize { columns: [v1, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [v1, t1._row_id], pk_conflict: NoCheck } - └─StreamTopN [append_only] { order: [t1.v1 ASC], limit: 3, offset: 3 } - └─StreamExchange { dist: Single } - └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t1.v1) } + └─StreamTopN [append_only] { order: [t1.v1 ASC], limit: 3, offset: 3 } + └─StreamExchange { dist: Single } + └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | create table t1 (v1 int, v2 int) append only; select max(v1) as max_v1 from t1; diff --git a/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml index 0cef2142c6f9f..7bb8e7fefe867 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml @@ -11,9 +11,10 @@ sql: | select max(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [max(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [max(max(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [max(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } batch_local_plan: |- BatchSimpleAgg { aggs: [max(s.v)] } └─BatchExchange { order: [], dist: Single } @@ -160,7 +161,7 @@ └─BatchProject { exprs: [max(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [max(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } batch_local_plan: |- BatchProject { exprs: [max(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [max(s.v)] } diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index c80253fd16ef3..835a1f9d0d345 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -134,7 +134,7 @@ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey) } + └─StreamExchange { dist: HashShard(nation.n_name, supplier.s_name, item.i_id) } └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard($expr2) } │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } @@ -182,7 +182,7 @@ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([8, 9, 10, 11, 12, 13]) from 1 + └── StreamExchange Hash([2, 1, 3]) from 1 Fragment 1 StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } @@ -363,7 +363,7 @@ Table 42 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 8, 9, 10, 11, 12, 13 ], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 2, 1, 3 ], read pk prefix len hint: 9 } - id: ch_q3 before: @@ -413,67 +413,71 @@ └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d], stream_key: [o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_columns: [revenue, o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } - └─StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } - └─StreamExchange { dist: HashShard(orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND new_order.no_w_id = orders.o_w_id AND new_order.no_d_id = orders.o_d_id AND new_order.no_o_id = orders.o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND new_order.no_o_id = order_line.ol_o_id AND new_order.no_d_id = order_line.ol_d_id AND new_order.no_w_id = order_line.ol_w_id, output: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, customer.c_w_id, customer.c_d_id, customer.c_id, new_order.no_o_id, new_order.no_w_id, new_order.no_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } - ├─StreamExchange { dist: HashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } - │ └─StreamHashJoin { type: Inner, predicate: customer.c_d_id = new_order.no_d_id AND customer.c_w_id = new_order.no_w_id, output: all } - │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id) } - │ │ └─StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } - │ │ └─StreamFilter { predicate: IsNotNull(customer.c_w_id) AND IsNotNull(customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } - │ └─StreamExchange { dist: HashShard(new_order.no_d_id, new_order.no_w_id) } - │ └─StreamFilter { predicate: IsNotNull(new_order.no_w_id) AND IsNotNull(new_order.no_d_id) AND IsNotNull(new_order.no_o_id) } - │ └─StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], stream_scan_type: ArrangementBackfill, stream_key: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], pk: [no_w_id, no_d_id, no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } - └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } - ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─StreamFilter { predicate: IsNotNull(orders.o_d_id) AND IsNotNull(orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } - └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - └─StreamFilter { predicate: IsNotNull(order_line.ol_d_id) AND IsNotNull(order_line.ol_w_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(sum(order_line.ol_amount), orders.o_entry_d) } + └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } + └─StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } + └─StreamExchange { dist: HashShard(orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND new_order.no_w_id = orders.o_w_id AND new_order.no_d_id = orders.o_d_id AND new_order.no_o_id = orders.o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND new_order.no_o_id = order_line.ol_o_id AND new_order.no_d_id = order_line.ol_d_id AND new_order.no_w_id = order_line.ol_w_id, output: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, customer.c_w_id, customer.c_d_id, customer.c_id, new_order.no_o_id, new_order.no_w_id, new_order.no_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } + ├─StreamExchange { dist: HashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } + │ └─StreamHashJoin { type: Inner, predicate: customer.c_d_id = new_order.no_d_id AND customer.c_w_id = new_order.no_w_id, output: all } + │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id) } + │ │ └─StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } + │ │ └─StreamFilter { predicate: IsNotNull(customer.c_w_id) AND IsNotNull(customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ └─StreamExchange { dist: HashShard(new_order.no_d_id, new_order.no_w_id) } + │ └─StreamFilter { predicate: IsNotNull(new_order.no_w_id) AND IsNotNull(new_order.no_d_id) AND IsNotNull(new_order.no_o_id) } + │ └─StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], stream_scan_type: ArrangementBackfill, stream_key: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], pk: [no_w_id, no_d_id, no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } + └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } + ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamFilter { predicate: IsNotNull(orders.o_d_id) AND IsNotNull(orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + └─StreamFilter { predicate: IsNotNull(order_line.ol_d_id) AND IsNotNull(order_line.ol_w_id) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d], stream_key: [o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_columns: [revenue, o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } - └── StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0, 1, 2, 3]) from 1 + └── StreamExchange Hash([3, 4]) from 1 Fragment 1 + StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } + └── StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0, 1, 2, 3]) from 2 + + Fragment 2 StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND new_order.no_w_id = orders.o_w_id AND new_order.no_d_id = orders.o_d_id AND new_order.no_o_id = orders.o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND new_order.no_o_id = order_line.ol_o_id AND new_order.no_d_id = order_line.ol_d_id AND new_order.no_w_id = order_line.ol_w_id, output: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, customer.c_w_id, customer.c_d_id, customer.c_id, new_order.no_o_id, new_order.no_w_id, new_order.no_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([3, 4, 5]) from 2 + ├── StreamExchange Hash([3, 4, 5]) from 3 └── StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } - ├── StreamExchange Hash([0, 1, 2]) from 5 - └── StreamExchange Hash([0, 1, 2]) from 6 + ├── StreamExchange Hash([0, 1, 2]) from 6 + └── StreamExchange Hash([0, 1, 2]) from 7 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: customer.c_d_id = new_order.no_d_id AND customer.c_w_id = new_order.no_w_id, output: all } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([1, 2]) from 3 - └── StreamExchange Hash([1, 2]) from 4 + ├── StreamExchange Hash([1, 2]) from 4 + └── StreamExchange Hash([1, 2]) from 5 - Fragment 3 + Fragment 4 StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } └── StreamFilter { predicate: IsNotNull(customer.c_w_id) AND IsNotNull(customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } └── StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { tables: [ StreamScan: 9 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamFilter { predicate: IsNotNull(new_order.no_w_id) AND IsNotNull(new_order.no_d_id) AND IsNotNull(new_order.no_o_id) } └── StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], stream_scan_type: ArrangementBackfill, stream_key: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], pk: [no_w_id, no_d_id, no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } { tables: [ StreamScan: 10 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: IsNotNull(orders.o_d_id) AND IsNotNull(orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamFilter { predicate: IsNotNull(order_line.ol_d_id) AND IsNotNull(order_line.ol_w_id) } └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { tables: [ StreamScan: 16 ] } ├── Upstream @@ -513,7 +517,7 @@ Table 16 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d, _rw_timestamp ], primary key: [ $3 DESC, $4 ASC, $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4, 0, 2, 1 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d, _rw_timestamp ], primary key: [ $3 DESC, $4 ASC, $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3, 4 ], read pk prefix len hint: 5 } - id: ch_q4 before: @@ -1743,72 +1747,76 @@ └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [c_id, c_last, revenue, c_city, c_phone, n_name], stream_key: [c_id, c_last, c_city, c_phone, n_name], pk_columns: [revenue, c_id, c_last, c_city, c_phone, n_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } - └─StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } - └─StreamExchange { dist: HashShard(customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name) } - └─StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, order_line.ol_amount, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, nation.n_nationkey] } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [order_line.ol_amount, customer.c_id, customer.c_last, customer.c_city, customer.c_phone, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ └─StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } - │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND order_line.ol_d_id = customer.c_d_id AND order_line.ol_w_id = customer.c_w_id, output: all } - │ ├─StreamExchange { dist: HashShard(order_line.ol_d_id, order_line.ol_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - │ │ └─StreamFilter { predicate: IsNotNull(order_line.ol_w_id) AND IsNotNull(order_line.ol_d_id) } - │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } - │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } - │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } - │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } - └─StreamExchange { dist: HashShard(nation.n_nationkey) } - └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(sum(order_line.ol_amount)) } + └─StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } + └─StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } + └─StreamExchange { dist: HashShard(customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name) } + └─StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, order_line.ol_amount, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, nation.n_nationkey] } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [order_line.ol_amount, customer.c_id, customer.c_last, customer.c_city, customer.c_phone, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ └─StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } + │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND order_line.ol_d_id = customer.c_d_id AND order_line.ol_w_id = customer.c_w_id, output: all } + │ ├─StreamExchange { dist: HashShard(order_line.ol_d_id, order_line.ol_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + │ │ └─StreamFilter { predicate: IsNotNull(order_line.ol_w_id) AND IsNotNull(order_line.ol_d_id) } + │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } + │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } + │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamExchange { dist: HashShard(nation.n_nationkey) } + └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_id, c_last, revenue, c_city, c_phone, n_name], stream_key: [c_id, c_last, c_city, c_phone, n_name], pk_columns: [revenue, c_id, c_last, c_city, c_phone, n_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } - └── StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0, 1, 2, 3, 5]) from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } + └── StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0, 1, 2, 3, 5]) from 2 + + Fragment 2 StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, order_line.ol_amount, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, nation.n_nationkey] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([5]) from 2 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([5]) from 3 + └── StreamExchange Hash([0]) from 8 - Fragment 2 + Fragment 3 StreamProject { exprs: [order_line.ol_amount, customer.c_id, customer.c_last, customer.c_city, customer.c_phone, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } └── StreamHashJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND order_line.ol_d_id = customer.c_d_id AND order_line.ol_w_id = customer.c_w_id, output: all } ├── tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] - ├── StreamExchange Hash([1, 2, 0, 1, 2]) from 3 - └── StreamExchange Hash([1, 2, 7, 8, 9]) from 4 + ├── StreamExchange Hash([1, 2, 0, 1, 2]) from 4 + └── StreamExchange Hash([1, 2, 7, 8, 9]) from 5 - Fragment 3 + Fragment 4 StreamFilter { predicate: IsNotNull(order_line.ol_w_id) AND IsNotNull(order_line.ol_d_id) } └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 9 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } { tables: [ HashJoinLeft: 10, HashJoinDegreeLeft: 11, HashJoinRight: 12, HashJoinDegreeRight: 13 ] } - ├── StreamExchange Hash([1, 2, 0]) from 5 - └── StreamExchange Hash([1, 2, 3]) from 6 + ├── StreamExchange Hash([1, 2, 0]) from 6 + └── StreamExchange Hash([1, 2, 3]) from 7 - Fragment 5 + Fragment 6 StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { tables: [ StreamScan: 14 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 16 ] } ├── Upstream └── BatchPlanNode @@ -1847,7 +1855,7 @@ Table 16 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_id, c_last, revenue, c_city, c_phone, n_name, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 3, 4, 5 ], read pk prefix len hint: 6 } + Table 4294967294 { columns: [ c_id, c_last, revenue, c_city, c_phone, n_name, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 6 } - id: ch_q11 before: @@ -1921,78 +1929,81 @@ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_plan: |- StreamMaterialize { columns: [s_i_id, ordercount], stream_key: [s_i_id], pk_columns: [ordercount, s_i_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } - └─StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } - ├─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } - │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } - │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ └─StreamShare { id: 10 } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } - │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamProject { exprs: [nation.n_nationkey] } - │ │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } - └─StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } - └─StreamShare { id: 10 } - └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └─StreamExchange { dist: HashShard(sum(stock.s_order_cnt)) } + └─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } + └─StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } + ├─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } + │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } + │ └─StreamExchange { dist: HashShard(stock.s_i_id) } + │ └─StreamShare { id: 10 } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + │ │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamProject { exprs: [nation.n_nationkey] } + │ │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } + └─StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } + └─StreamShare { id: 10 } + └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } + └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_i_id, ordercount], stream_key: [s_i_id], pk_columns: [ordercount, s_i_id], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } - └── StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } - ├── tables: [ DynamicFilterLeft: 0, DynamicFilterRight: 1 ] - ├── StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } - │ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } - │ ├── tables: [ HashAggState: 2 ] - │ └── StreamExchange Hash([0]) from 1 - └── StreamExchange Broadcast from 7 + └── StreamExchange Hash([1]) from 1 Fragment 1 - StreamNoOp - └── StreamExchange NoShuffle from 2 + StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } + └── StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } + ├── tables: [ DynamicFilterLeft: 0, DynamicFilterRight: 1 ] + ├── StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } + │ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } { tables: [ HashAggState: 2 ] } + │ └── StreamExchange Hash([0]) from 2 + └── StreamExchange Broadcast from 8 Fragment 2 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Fragment 3 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([2]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - ├── StreamExchange Hash([1]) from 4 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([1]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 4 + Fragment 5 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 11 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -2000,21 +2011,21 @@ ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } ├── tables: [ StreamScan: 13 ] ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } └── StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } { tables: [ SimpleAggState: 14 ] } - └── StreamExchange Single from 8 + └── StreamExchange Single from 9 - Fragment 8 + Fragment 9 StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ stock_s_i_id, sum(stock_s_order_cnt), $expr2, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2046,7 +2057,7 @@ Table 14 { columns: [ sum(sum(stock_s_order_cnt)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ s_i_id, ordercount, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_i_id, ordercount, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - id: ch_q12 before: @@ -2171,43 +2182,47 @@ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } stream_plan: |- StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck } - └─StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } - └─StreamExchange { dist: HashShard(count(orders.o_id)) } - └─StreamProject { exprs: [customer.c_id, count(orders.o_id)] } - └─StreamHashAgg { group_key: [customer.c_id], aggs: [count(orders.o_id), count] } - └─StreamExchange { dist: HashShard(customer.c_id) } - └─StreamHashJoin { type: LeftOuter, predicate: customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND customer.c_id = orders.o_c_id, output: [customer.c_id, orders.o_id, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id] } - ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } - └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - └─StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } - └─StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamExchange { dist: HashShard(count, count(orders.o_id)) } + └─StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(orders.o_id)) } + └─StreamProject { exprs: [customer.c_id, count(orders.o_id)] } + └─StreamHashAgg { group_key: [customer.c_id], aggs: [count(orders.o_id), count] } + └─StreamExchange { dist: HashShard(customer.c_id) } + └─StreamHashJoin { type: LeftOuter, predicate: customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND customer.c_id = orders.o_c_id, output: [customer.c_id, orders.o_id, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id] } + ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } + │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } + └─StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } + └─StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } + └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([1]) from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([1]) from 2 + + Fragment 2 StreamProject { exprs: [customer.c_id, count(orders.o_id)] } └── StreamHashAgg { group_key: [customer.c_id], aggs: [count(orders.o_id), count] } { tables: [ HashAggState: 1 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamHashJoin { type: LeftOuter, predicate: customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND customer.c_id = orders.o_c_id, output: [customer.c_id, orders.o_id, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id] } ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] - ├── StreamExchange Hash([1, 2, 0]) from 3 - └── StreamExchange Hash([1, 2, 3]) from 4 + ├── StreamExchange Hash([1, 2, 0]) from 4 + └── StreamExchange Hash([1, 2, 3]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } ├── tables: [ StreamScan: 6 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └── StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } @@ -2231,7 +2246,7 @@ Table 7 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: ch_q14 before: @@ -2348,7 +2363,7 @@ └─LogicalScan { table: revenue1, columns: [revenue1.total_revenue] } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(supplier.s_suppkey, revenue1.total_revenue, revenue1.supplier_no) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } ├─StreamExchange { dist: HashShard(revenue1.total_revenue) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } @@ -2368,7 +2383,7 @@ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([0, 4, 5]) from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } @@ -2443,7 +2458,7 @@ Table 14 { columns: [ vnode, supplier_no, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - id: ch_q16 before: @@ -2487,58 +2502,62 @@ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } stream_plan: |- StreamMaterialize { columns: [i_name, brand, i_price, supplier_cnt], stream_key: [i_name, brand, i_price], pk_columns: [supplier_cnt, i_name, brand, i_price], pk_conflict: NoCheck } - └─StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } - └─StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } - └─StreamExchange { dist: HashShard(item.i_name, $expr2, item.i_price) } - └─StreamProject { exprs: [item.i_name, Substr(item.i_data, 1:Int32, 3:Int32) as $expr2, item.i_price, ((stock.s_w_id * stock.s_i_id) % 10000:Int32) as $expr3, stock.s_w_id, stock.s_i_id, $expr1] } - └─StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: all } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id] } - │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } - │ └─StreamExchange { dist: HashShard(item.i_id) } - │ └─StreamFilter { predicate: Not(Like(item.i_data, 'zz%':Varchar)) } - │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_price, item.i_data], stream_scan_type: ArrangementBackfill, stream_key: [item.i_id], pk: [i_id], dist: UpstreamHashShard(item.i_id) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamProject { exprs: [supplier.s_suppkey] } - └─StreamFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } - └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(count(distinct $expr3)) } + └─StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } + └─StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } + └─StreamExchange { dist: HashShard(item.i_name, $expr2, item.i_price) } + └─StreamProject { exprs: [item.i_name, Substr(item.i_data, 1:Int32, 3:Int32) as $expr2, item.i_price, ((stock.s_w_id * stock.s_i_id) % 10000:Int32) as $expr3, stock.s_w_id, stock.s_i_id, $expr1] } + └─StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: all } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } + │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id] } + │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─StreamExchange { dist: HashShard(item.i_id) } + │ └─StreamFilter { predicate: Not(Like(item.i_data, 'zz%':Varchar)) } + │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_price, item.i_data], stream_scan_type: ArrangementBackfill, stream_key: [item.i_id], pk: [i_id], dist: UpstreamHashShard(item.i_id) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamProject { exprs: [supplier.s_suppkey] } + └─StreamFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } + └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [i_name, brand, i_price, supplier_cnt], stream_key: [i_name, brand, i_price], pk_columns: [supplier_cnt, i_name, brand, i_price], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } - └── StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } { tables: [ HashAggState: 0, HashAggDedupForCol3: 1 ] } - └── StreamExchange Hash([0, 1, 2]) from 1 + └── StreamExchange Hash([3]) from 1 Fragment 1 + StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } + └── StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } { tables: [ HashAggState: 0, HashAggDedupForCol3: 1 ] } + └── StreamExchange Hash([0, 1, 2]) from 2 + + Fragment 2 StreamProject { exprs: [item.i_name, Substr(item.i_data, 1:Int32, 3:Int32) as $expr2, item.i_price, ((stock.s_w_id * stock.s_i_id) % 10000:Int32) as $expr3, stock.s_w_id, stock.s_i_id, $expr1] } └── StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: all } { tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] } - ├── StreamExchange Hash([5]) from 2 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([5]) from 3 + └── StreamExchange Hash([0]) from 6 - Fragment 2 + Fragment 3 StreamProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └── StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id] } ├── tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } ├── tables: [ StreamScan: 10 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamFilter { predicate: Not(Like(item.i_data, 'zz%':Varchar)) } └── StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_price, item.i_data], stream_scan_type: ArrangementBackfill, stream_key: [item.i_id], pk: [i_id], dist: UpstreamHashShard(item.i_id) } ├── tables: [ StreamScan: 11 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamProject { exprs: [supplier.s_suppkey] } └── StreamFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } └── StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -2582,7 +2601,7 @@ Table 12 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ i_name, brand, i_price, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ i_name, brand, i_price, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3 ], read pk prefix len hint: 4 } - id: ch_q17 before: @@ -2765,48 +2784,52 @@ └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount)(hidden), orders.o_id(hidden), orders.o_d_id(hidden), orders.o_w_id(hidden)], stream_key: [o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_entry_d, o_ol_cnt], pk_columns: [sum(order_line.ol_amount), o_entry_d, o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_ol_cnt], pk_conflict: NoCheck } - └─StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } - └─StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } - └─StreamProject { exprs: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount)] } - └─StreamHashAgg { group_key: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], aggs: [sum(order_line.ol_amount), count] } - └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, order_line.ol_amount, customer.c_w_id, customer.c_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, customer.c_w_id, customer.c_d_id] } - │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } - │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } - └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(sum(order_line.ol_amount), orders.o_entry_d) } + └─StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } + └─StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } + └─StreamProject { exprs: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount)] } + └─StreamHashAgg { group_key: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], aggs: [sum(order_line.ol_amount), count] } + └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, order_line.ol_amount, customer.c_w_id, customer.c_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, customer.c_w_id, customer.c_d_id] } + │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount)(hidden), orders.o_id(hidden), orders.o_d_id(hidden), orders.o_w_id(hidden)], stream_key: [o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_entry_d, o_ol_cnt], pk_columns: [sum(order_line.ol_amount), o_entry_d, o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_ol_cnt], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } - └── StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } - └── StreamProject { exprs: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount)] } - └── StreamHashAgg { group_key: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } - └── StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, order_line.ol_amount, customer.c_w_id, customer.c_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([2, 3, 4]) from 1 - └── StreamExchange Hash([0, 1, 2]) from 4 + └── StreamExchange Hash([5, 2]) from 1 Fragment 1 - StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, customer.c_w_id, customer.c_d_id] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([1, 2, 0]) from 2 - └── StreamExchange Hash([1, 2, 3]) from 3 + StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } + └── StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } + └── StreamProject { exprs: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount)] } + └── StreamHashAgg { group_key: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } + └── StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, order_line.ol_amount, customer.c_w_id, customer.c_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] + ├── StreamExchange Hash([2, 3, 4]) from 2 + └── StreamExchange Hash([0, 1, 2]) from 5 Fragment 2 + StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, customer.c_w_id, customer.c_d_id] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } + ├── StreamExchange Hash([1, 2, 0]) from 3 + └── StreamExchange Hash([1, 2, 3]) from 4 + + Fragment 3 StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { tables: [ StreamScan: 9 ] } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 10 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 11 ] ├── Upstream @@ -2836,7 +2859,7 @@ Table 11 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id, _rw_timestamp ], primary key: [ $5 DESC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6, 7, 8 ], read pk prefix len hint: 8 } + Table 4294967294 { columns: [ c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id, _rw_timestamp ], primary key: [ $5 DESC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 5, 2 ], read pk prefix len hint: 8 } - id: ch_q19 before: @@ -2988,7 +3011,7 @@ └─BatchScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_delivery_d], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) } + └─StreamExchange { dist: HashShard(supplier.s_name) } └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } @@ -3019,7 +3042,7 @@ Fragment 0 StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([2, 3]) from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } @@ -3122,7 +3145,7 @@ Table 21 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - id: ch_q21 before: @@ -3183,102 +3206,106 @@ └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(supplier.s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND order_line.ol_w_id = order_line.ol_w_id AND order_line.ol_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d > order_line.ol_delivery_d), output: [supplier.s_name, stock.s_w_id, stock.s_i_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_nationkey, $expr1, order_line.ol_w_id] } - ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } - │ └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [supplier.s_name, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, stock.s_w_id, stock.s_i_id, order_line.ol_number, $expr1, supplier.s_suppkey, supplier.s_nationkey] } - │ ├─StreamExchange { dist: HashShard($expr1) } - │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id, stock.s_i_id, order_line.ol_number] } - │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_w_id AND stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = orders.o_w_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number] } - │ │ ├─StreamExchange { dist: HashShard(stock.s_w_id, stock.s_i_id, stock.s_w_id) } - │ │ │ └─StreamFilter { predicate: IsNotNull(stock.s_w_id) } - │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } - │ │ └─StreamExchange { dist: HashShard(order_line.ol_w_id, order_line.ol_i_id, orders.o_w_id) } - │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, orders.o_w_id, order_line.ol_number, orders.o_d_id, orders.o_id] } - │ │ └─StreamFilter { predicate: (order_line.ol_delivery_d > orders.o_entry_d) } - │ │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_o_id = orders.o_id AND order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id, output: all } - │ │ ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } - │ │ └─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } - │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } - └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(count, supplier.s_name) } + └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(supplier.s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND order_line.ol_w_id = order_line.ol_w_id AND order_line.ol_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d > order_line.ol_delivery_d), output: [supplier.s_name, stock.s_w_id, stock.s_i_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_nationkey, $expr1, order_line.ol_w_id] } + ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } + │ └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [supplier.s_name, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, stock.s_w_id, stock.s_i_id, order_line.ol_number, $expr1, supplier.s_suppkey, supplier.s_nationkey] } + │ ├─StreamExchange { dist: HashShard($expr1) } + │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id, stock.s_i_id, order_line.ol_number] } + │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_w_id AND stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = orders.o_w_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number] } + │ │ ├─StreamExchange { dist: HashShard(stock.s_w_id, stock.s_i_id, stock.s_w_id) } + │ │ │ └─StreamFilter { predicate: IsNotNull(stock.s_w_id) } + │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ └─StreamExchange { dist: HashShard(order_line.ol_w_id, order_line.ol_i_id, orders.o_w_id) } + │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, orders.o_w_id, order_line.ol_number, orders.o_d_id, orders.o_id] } + │ │ └─StreamFilter { predicate: (order_line.ol_delivery_d > orders.o_entry_d) } + │ │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_o_id = orders.o_id AND order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id, output: all } + │ │ ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ │ └─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } + └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0]) from 2 + + Fragment 2 StreamHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND order_line.ol_w_id = order_line.ol_w_id AND order_line.ol_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d > order_line.ol_delivery_d), output: [supplier.s_name, stock.s_w_id, stock.s_i_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_nationkey, $expr1, order_line.ol_w_id] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([1, 3, 2]) from 2 - └── StreamExchange Hash([0, 1, 2]) from 11 + ├── StreamExchange Hash([1, 3, 2]) from 3 + └── StreamExchange Hash([0, 1, 2]) from 12 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [supplier.s_name, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, stock.s_w_id, stock.s_i_id, order_line.ol_number, $expr1, supplier.s_suppkey, supplier.s_nationkey] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([4]) from 3 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([4]) from 4 + └── StreamExchange Hash([0]) from 9 - Fragment 3 + Fragment 4 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id, stock.s_i_id, order_line.ol_number] } └── StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_w_id AND stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = orders.o_w_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number] } { tables: [ HashJoinLeft: 9, HashJoinDegreeLeft: 10, HashJoinRight: 11, HashJoinDegreeRight: 12 ] } - ├── StreamExchange Hash([1, 0, 1]) from 4 - └── StreamExchange Hash([2, 3, 5]) from 5 + ├── StreamExchange Hash([1, 0, 1]) from 5 + └── StreamExchange Hash([2, 3, 5]) from 6 - Fragment 4 + Fragment 5 StreamFilter { predicate: IsNotNull(stock.s_w_id) } └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { tables: [ StreamScan: 13 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, orders.o_w_id, order_line.ol_number, orders.o_d_id, orders.o_id] } └── StreamFilter { predicate: (order_line.ol_delivery_d > orders.o_entry_d) } └── StreamHashJoin { type: Inner, predicate: order_line.ol_o_id = orders.o_id AND order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id, output: all } { tables: [ HashJoinLeft: 14, HashJoinDegreeLeft: 15, HashJoinRight: 16, HashJoinDegreeRight: 17 ] } - ├── StreamExchange Hash([0, 1, 2]) from 6 - └── StreamExchange Hash([0, 1, 2]) from 7 + ├── StreamExchange Hash([0, 1, 2]) from 7 + └── StreamExchange Hash([0, 1, 2]) from 8 - Fragment 6 + Fragment 7 StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 18 ] ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 19 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { tables: [ HashJoinLeft: 20, HashJoinDegreeLeft: 21, HashJoinRight: 22, HashJoinDegreeRight: 23 ] } - ├── StreamExchange Hash([2]) from 9 - └── StreamExchange Hash([0]) from 10 + ├── StreamExchange Hash([2]) from 10 + └── StreamExchange Hash([0]) from 11 - Fragment 9 + Fragment 10 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 24 ] } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 25 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 26 ] @@ -3339,7 +3366,7 @@ Table 26 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: ch_q22 before: diff --git a/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml b/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml index 2b0c10cf89b3d..1d74ed20b82f5 100644 --- a/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml +++ b/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml @@ -14,12 +14,13 @@ └─BatchScan { table: t1, columns: [t1.id, t1.i], limit: 2, distribution: UpstreamHashShard(t1.id) } stream_plan: |- StreamMaterialize { columns: [id, i], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.id, t1.i] } - └─StreamTopN { order: [t1.id ASC], limit: 2, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } - └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } + └─StreamExchange { dist: HashShard(t1.id) } + └─StreamProject { exprs: [t1.id, t1.i] } + └─StreamTopN { order: [t1.id ASC], limit: 2, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } + └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } - name: test functional dependency for order key pruning (order by - suffix fd) sql: | create table t1 (id int primary key, i int); @@ -35,12 +36,13 @@ └─BatchScan { table: t1, columns: [t1.id, t1.i], distribution: UpstreamHashShard(t1.id) } stream_plan: |- StreamMaterialize { columns: [id, i], stream_key: [id], pk_columns: [i, id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.id, t1.i] } - └─StreamTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } - └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } + └─StreamExchange { dist: HashShard(t1.i, t1.id) } + └─StreamProject { exprs: [t1.id, t1.i] } + └─StreamTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } + └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } - name: test functional dependency for order key pruning on singleton sql: | create table t1 (id int primary key, i int); @@ -54,8 +56,9 @@ └─BatchSort { order: [v.cnt ASC] } └─BatchScan { table: v, columns: [v.cnt], distribution: Single } stream_plan: |- - StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamTableScan { table: v, columns: [v.cnt], stream_scan_type: ArrangementBackfill, stream_key: [], pk: [], dist: Single } + StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [cnt], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(v.cnt) } + └─StreamTableScan { table: v, columns: [v.cnt], stream_scan_type: ArrangementBackfill, stream_key: [], pk: [], dist: Single } - name: test functional dependency for order key pruning (index) sql: | create table t1 (v1 int, v2 int); diff --git a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml index 349c5f7d89012..209457976b58d 100644 --- a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml @@ -616,12 +616,13 @@ select * from t1 order by a limit 1; stream_plan: |- StreamMaterialize { columns: [a], stream_key: [], pk_columns: [a], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a] } - └─StreamTopN { order: [t1.a ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t1.a ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t1.a, Vnode(t1.a) as _vnode] } - └─StreamTableScan { table: t1, columns: [t1.a], stream_scan_type: ArrangementBackfill, stream_key: [t1.a], pk: [a], dist: UpstreamHashShard(t1.a) } + └─StreamExchange { dist: HashShard(t1.a) } + └─StreamProject { exprs: [t1.a] } + └─StreamTopN { order: [t1.a ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t1.a ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t1.a, Vnode(t1.a) as _vnode] } + └─StreamTableScan { table: t1, columns: [t1.a], stream_scan_type: ArrangementBackfill, stream_key: [t1.a], pk: [a], dist: UpstreamHashShard(t1.a) } - sql: | create table t1 (a varchar, b int, c int, d int); create index idx on t1(a); diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index aaa401e469d16..8696857df85c1 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -530,7 +530,7 @@ └─BatchScan { table: b, columns: [b.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [y, z, $expr2(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], stream_key: [a._row_id, b._row_id, a.x, b.x], pk_columns: [$expr2, a._row_id, b._row_id, a.x, b.x], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(a._row_id, b._row_id, a.x, b.x) } + └─StreamExchange { dist: HashShard($expr2) } └─StreamProject { exprs: [(2:Int32 * $expr1) as $expr3, $expr2, $expr2, a._row_id, b._row_id, a.x, b.x] } └─StreamProject { exprs: [a.x, b.x, $expr1, ($expr1 + $expr1) as $expr2, a._row_id, b._row_id] } └─StreamProject { exprs: [a.x, b.x, Coalesce(a.x, b.x) as $expr1, a._row_id, b._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/limit.yaml b/src/frontend/planner_test/tests/testdata/output/limit.yaml index a7b5c0d7a7f48..29ab9e6a12f7a 100644 --- a/src/frontend/planner_test/tests/testdata/output/limit.yaml +++ b/src/frontend/planner_test/tests/testdata/output/limit.yaml @@ -99,8 +99,9 @@ └─BatchValues { rows: [[1:Int32]] } stream_plan: |- StreamMaterialize { columns: [c, _row_id(hidden)], stream_key: [], pk_columns: [c], pk_conflict: NoCheck } - └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 1, offset: 0 } - └─StreamValues { rows: [[1:Int32, 0:Int64]] } + └─StreamExchange { dist: HashShard(1:Int32) } + └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 1, offset: 0 } + └─StreamValues { rows: [[1:Int32, 0:Int64]] } - sql: | select 1 c union all select 1 c limit 10 batch_plan: |- @@ -117,8 +118,9 @@ └─BatchValues { rows: [[1:Int32], [1:Int32]] } stream_plan: |- StreamMaterialize { columns: [c, _row_id(hidden)], stream_key: [_row_id], pk_columns: [c, _row_id], pk_conflict: NoCheck } - └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 10, offset: 0 } - └─StreamValues { rows: [[1:Int32, 0:Int64], [1:Int32, 1:Int64]] } + └─StreamExchange { dist: HashShard(1:Int32) } + └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 10, offset: 0 } + └─StreamValues { rows: [[1:Int32, 0:Int64], [1:Int32, 1:Int64]] } - sql: | create table t (a int); select count(*) from t limit 1; diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 36e8c13b2c19f..6c19f66eba1a0 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -2271,42 +2271,46 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } - └─StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } - └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } - └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], stream_scan_type: ArrangementBackfill, stream_key: [auction.id], pk: [id], dist: UpstreamHashShard(auction.id) } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(count(bid.auction)) } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + └─StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } + └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } + └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], stream_scan_type: ArrangementBackfill, stream_key: [auction.id], pk: [id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } - └── StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + └── StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } ├── tables: [ HashAggState: 2 ] └── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 - Fragment 2 + Fragment 3 StreamTableScan { table: auction, columns: [auction.id, auction.item_name], stream_scan_type: ArrangementBackfill, stream_key: [auction.id], pk: [id], dist: UpstreamHashShard(auction.id) } ├── tables: [ StreamScan: 7 ] ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } ├── tables: [ StreamScan: 8 ] ├── Upstream @@ -2362,7 +2366,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 2 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 472bb56b2c12e..fefc517062486 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -2228,41 +2228,45 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [id, item_name, count(auction)] } - └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } - └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } - └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(count(auction)) } + └─StreamProject { exprs: [id, item_name, count(auction)] } + └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [id, item_name, count(auction)] } - └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [id, item_name, count(auction)] } + └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { tables: [ Source: 7 ] } - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 8 ] } @@ -2309,7 +2313,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 3 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml index 469d94ca87ff3..8ab8fdee21d1e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml @@ -2254,43 +2254,47 @@ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [id, item_name, count(auction)] } - └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } - └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } - └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamRowIdGen { row_id_index: 13 } - │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count(auction)) } + └─StreamProject { exprs: [id, item_name, count(auction)] } + └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [id, item_name, count(auction)] } - └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [id, item_name, count(auction)] } + └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 13 } └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } ├── tables: [ SourceBackfill: 7 ] └── Upstream - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 10 } └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 8 ] } └── Upstream @@ -2338,7 +2342,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 3 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml index 0b15386274989..8ec6f1f3920db 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml @@ -1960,73 +1960,77 @@ LIMIT 1000; stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, $expr3, count($expr5)] } - └─StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count($expr5) DESC], limit: 1000, offset: 0, group_key: [$expr6] } - └─StreamProject { exprs: [$expr2, $expr3, count($expr5), Vnode($expr2) as $expr6] } - └─StreamHashAgg { group_key: [$expr2, $expr3], aggs: [count($expr5), count] } - └─StreamHashJoin { type: Inner, predicate: $expr2 = $expr5, output: [$expr2, $expr3, $expr5, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [[$expr1]], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [[$expr4]] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(count($expr5)) } + └─StreamProject { exprs: [$expr2, $expr3, count($expr5)] } + └─StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count($expr5) DESC], limit: 1000, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [$expr2, $expr3, count($expr5), Vnode($expr2) as $expr6] } + └─StreamHashAgg { group_key: [$expr2, $expr3], aggs: [count($expr5), count] } + └─StreamHashJoin { type: Inner, predicate: $expr2 = $expr5, output: [$expr2, $expr3, $expr5, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [[$expr1]], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [[$expr4]] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [$expr2, $expr3, count($expr5)] } - └── StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [$expr2, $expr3, count($expr5)] } + └── StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count($expr5) DESC], limit: 1000, offset: 0, group_key: [$expr6] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [$expr2, $expr3, count($expr5), Vnode($expr2) as $expr6] } └── StreamHashAgg { group_key: [$expr2, $expr3], aggs: [count($expr5), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin { type: Inner, predicate: $expr2 = $expr5, output: [$expr2, $expr3, $expr5, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 5 - Fragment 2 + Fragment 3 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 7 ] } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [[$expr1]], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } ├── tables: [ DynamicFilterLeft: 8, DynamicFilterRight: 9 ] ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 3 - └── StreamExchange Broadcast from 5 + │ └── StreamExchange NoShuffle from 4 + └── StreamExchange Broadcast from 6 - Fragment 5 + Fragment 6 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [[$expr4]] } └── StreamNow { output: [now] } { tables: [ Now: 10 ] } @@ -2072,7 +2076,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 3 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index b655102926a77..ecec45533601e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -2417,56 +2417,60 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, $expr3, count($expr4)] } - └─StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count($expr4) DESC], limit: 1000, offset: 0, group_key: [$expr5] } - └─StreamProject { exprs: [$expr2, $expr3, count($expr4), Vnode($expr2) as $expr5] } - └─StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [count($expr4), count] } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr4, output: [$expr2, $expr3, $expr4, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard(count($expr4)) } + └─StreamProject { exprs: [$expr2, $expr3, count($expr4)] } + └─StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count($expr4) DESC], limit: 1000, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr2, $expr3, count($expr4), Vnode($expr2) as $expr5] } + └─StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [count($expr4), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr4, output: [$expr2, $expr3, $expr4, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [$expr2, $expr3, count($expr4)] } - └── StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [$expr2, $expr3, count($expr4)] } + └── StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count($expr4) DESC], limit: 1000, offset: 0, group_key: [$expr5] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [$expr2, $expr3, count($expr4), Vnode($expr2) as $expr5] } └── StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [count($expr4), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr4, output: [$expr2, $expr3, $expr4, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 5 - Fragment 2 + Fragment 3 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -2474,10 +2478,10 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 8 ] } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ $expr2, $expr3, count($expr4), $expr5, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } @@ -2503,7 +2507,7 @@ Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 3 } eowc_stream_error: |- Not supported: Emit-On-Window-Close mode requires a watermark column in GROUP BY. diff --git a/src/frontend/planner_test/tests/testdata/output/order_by.yaml b/src/frontend/planner_test/tests/testdata/output/order_by.yaml index 16f55c5fd981b..a04e18dd46a39 100644 --- a/src/frontend/planner_test/tests/testdata/output/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/output/order_by.yaml @@ -9,7 +9,8 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [v1, t._row_id], pk_conflict: NoCheck } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: output names are not qualified after table names sql: | create table t (v1 bigint, v2 double precision); @@ -65,8 +66,9 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [2:Int32, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(2:Int32) } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); select * from t order by v; @@ -85,12 +87,13 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [v1, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } - └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t.v1 DESC], limit: 5, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } + └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t.v1 DESC], limit: 5, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); select * from t limit 3 offset 4; @@ -117,12 +120,13 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [v1, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } - └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 7 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t.v1 DESC], limit: 12, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } + └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 7 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t.v1 DESC], limit: 12, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: order by expression that would be valid in select list sql: | create table t (x int, y int, z int); @@ -138,8 +142,9 @@ └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, $expr1(hidden), t.z(hidden), t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [$expr1, t.z, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } - └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard($expr1, t.z) } + └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } + └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: order by the number of an output column sql: | create table t (x int, y int); @@ -175,7 +180,7 @@ select * from mv order by v asc; batch_plan: |- BatchExchange { order: [mv.v ASC], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - name: BatchSort needed, when input is sorted in wrong order sql: | create table t(v int); @@ -184,7 +189,7 @@ batch_plan: |- BatchExchange { order: [mv.v DESC], dist: Single } └─BatchSort { order: [mv.v DESC] } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - name: No BatchSort needed, when input is already sorted sql: | create table t(v int); @@ -192,7 +197,7 @@ select * from mv order by v desc; batch_plan: |- BatchExchange { order: [mv.v DESC], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - name: BatchSort needed, because our BatchScan can not get a ordered result when scan from ranges sql: | create table t(v int); @@ -201,7 +206,7 @@ batch_plan: |- BatchExchange { order: [mv.v ASC], dist: Single } └─BatchSort { order: [mv.v ASC] } - └─BatchScan { table: mv, columns: [mv.v], scan_ranges: [mv.v = Int32(1), mv.v = Int32(2)], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], scan_ranges: [mv.v = Int32(1), mv.v = Int32(2)], distribution: UpstreamHashShard(mv.v) } - name: BatchSort needed, when input is sorted in wrong order sql: | create table t(v int); @@ -210,7 +215,7 @@ batch_plan: |- BatchExchange { order: [mv.v ASC], dist: Single } └─BatchSort { order: [mv.v ASC] } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - sql: | CREATE TABLE test (a INTEGER, b INTEGER); SELECT b % 2 AS f, SUM(a) FROM test GROUP BY b % 2 ORDER BY f; diff --git a/src/frontend/planner_test/tests/testdata/output/project_set.yaml b/src/frontend/planner_test/tests/testdata/output/project_set.yaml index c4102e164d7e8..d54b3d8cff947 100644 --- a/src/frontend/planner_test/tests/testdata/output/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/output/project_set.yaml @@ -98,13 +98,14 @@ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [projected_row_id(hidden), unnest, t._row_id(hidden)], stream_key: [], pk_columns: [projected_row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id] } - └─StreamTopN { order: [projected_row_id ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [projected_row_id ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as _vnode] } - └─StreamProjectSet { select_list: [Unnest($0), $1] } - └─StreamTableScan { table: t, columns: [t.x, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(projected_row_id) } + └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id] } + └─StreamTopN { order: [projected_row_id ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [projected_row_id ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as _vnode] } + └─StreamProjectSet { select_list: [Unnest($0), $1] } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: issue-7812 sql: | -- projected_row_id should be excluded from distinct diff --git a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml index e175558038c81..8e79bb6c62d0f 100644 --- a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml @@ -240,21 +240,21 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (orders_count_by_user_ordered.user_id = 42:Int32) } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - before: - create_table_and_mv_ordered sql: | SELECT * FROM orders_count_by_user_ordered WHERE user_id > 42 AND orders_count = 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10) AND orders_count_by_user_ordered.user_id > Int32(42)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10) AND orders_count_by_user_ordered.user_id > Int32(42)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - before: - create_table_and_mv_ordered sql: | SELECT * FROM orders_count_by_user_ordered WHERE orders_count = 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge mutiple upper bound before: - create_table_and_mv_ordered @@ -262,7 +262,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count < 10 and orders_count < 30 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge include and exclude upper bound of same value before: - create_table_and_mv_ordered @@ -270,7 +270,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count < 10 and orders_count <= 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge mutiple lower bound before: - create_table_and_mv_ordered @@ -278,7 +278,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count > 10 and orders_count > 30 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge include and exclude lower bound of same value before: - create_table_and_mv_ordered @@ -286,7 +286,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count > 10 and orders_count >= 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: invalid range before: - create_table_and_mv_ordered @@ -300,7 +300,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count > 20 and orders_count < 30 and orders_count = 25 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(25)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(25)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: invalid range of merging cmp and eq condition before: - create_table_and_mv_ordered @@ -314,7 +314,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count in (10,20,30,40) and orders_count <30 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10), orders_count_by_user_ordered.orders_count = Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10), orders_count_by_user_ordered.orders_count = Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: invalid range of merging cmp and const-in condition before: - create_table_and_mv_ordered @@ -521,7 +521,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id > 20:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -537,7 +537,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id >= 10:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id >= Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id >= Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -553,7 +553,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id > 20:Int32)) OR (orders_count_by_user_desc.user_id = 15:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -569,7 +569,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((((orders_count_by_user_desc.user_id < 10:Int32) AND (orders_count_by_user_desc.user_id > 1:Int32)) OR ((orders_count_by_user_desc.user_id > 20:Int32) AND (orders_count_by_user_desc.user_id < 30:Int32))) OR ((orders_count_by_user_desc.user_id >= 30:Int32) AND (orders_count_by_user_desc.user_id < 40:Int32))) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20) AND orders_count_by_user_desc.user_id < Int64(30), orders_count_by_user_desc.user_id >= Int64(30) AND orders_count_by_user_desc.user_id < Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20) AND orders_count_by_user_desc.user_id < Int64(30), orders_count_by_user_desc.user_id >= Int64(30) AND orders_count_by_user_desc.user_id < Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - name: When OR clauses contain overlapping conditions, we can merge serveral scan_range and pushdown. before: - create_table_and_mv @@ -586,7 +586,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((((orders_count_by_user_desc.user_id < 10:Int32) AND (orders_count_by_user_desc.user_id > 1:Int32)) OR ((orders_count_by_user_desc.user_id > 20:Int32) AND (orders_count_by_user_desc.user_id <= 30:Int32))) OR (((orders_count_by_user_desc.user_id >= 30:Int32) AND (orders_count_by_user_desc.user_id < 40:Int32)) OR (orders_count_by_user_desc.user_id = 15:Int32))) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id >= Int64(20) AND orders_count_by_user_desc.user_id <= Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id >= Int64(20) AND orders_count_by_user_desc.user_id <= Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -602,7 +602,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id > 30:Int32)) OR ((orders_count_by_user_desc.user_id > 5:Int32) AND (orders_count_by_user_desc.user_id < 15:Int32))) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id <= Int64(15), orders_count_by_user_desc.user_id > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id <= Int64(15), orders_count_by_user_desc.user_id > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - name: When OR clauses contain overlapping conditions, we cannot push down if it results in a full table scan. before: - create_table_and_mv @@ -619,7 +619,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 20:Int32) OR (orders_count_by_user_desc.user_id > 10:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -635,4 +635,4 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 20:Int32) OR (orders_count_by_user_desc.user_id <> 10:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/row_filter.yaml b/src/frontend/planner_test/tests/testdata/output/row_filter.yaml index 5de2e79815476..3820983f004d6 100644 --- a/src/frontend/planner_test/tests/testdata/output/row_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/row_filter.yaml @@ -39,4 +39,4 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (Row(mv1.v1, mv1.v2, mv1.v3) > '(1,3,1)':Struct(StructType { field_names: [], field_types: [Int32, Int32, Int32] })) } - └─BatchScan { table: mv1, columns: [mv1.v1, mv1.v2, mv1.v3], scan_ranges: [(mv1.v1, mv1.v2) >= (Int32(1), Int32(3))], distribution: SomeShard } + └─BatchScan { table: mv1, columns: [mv1.v1, mv1.v2, mv1.v3], scan_ranges: [(mv1.v1, mv1.v2) >= (Int32(1), Int32(3))], distribution: UpstreamHashShard(mv1.v1, mv1.v2, mv1.v3) } diff --git a/src/frontend/planner_test/tests/testdata/output/singleton.yaml b/src/frontend/planner_test/tests/testdata/output/singleton.yaml index b9ff208e484e6..0710b0d88f0d7 100644 --- a/src/frontend/planner_test/tests/testdata/output/singleton.yaml +++ b/src/frontend/planner_test/tests/testdata/output/singleton.yaml @@ -10,10 +10,10 @@ select v from mv; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: Single } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } batch_local_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - id: select_from_singleton_mv_join before: - create_singleton_mv @@ -21,17 +21,12 @@ select mv1.v from mv mv1, mv mv2 where mv1.v = mv2.v; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: HashShard(mv.v) } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: Single } - └─BatchExchange { order: [], dist: HashShard(mv.v) } - └─BatchScan { table: mv, columns: [mv.v], distribution: Single } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } batch_local_plan: |- - BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: Single } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - id: select_from_singleton_mv_join_top_n before: - create_singleton_mv @@ -43,17 +38,12 @@ └─BatchExchange { order: [], dist: Single } └─BatchTopN { order: [mv.v ASC], limit: 10, offset: 0 } └─BatchProject { exprs: [mv.v, mv.v] } - └─BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: HashShard(mv.v) } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: Single } - └─BatchExchange { order: [], dist: HashShard(mv.v) } - └─BatchScan { table: mv, columns: [mv.v], distribution: Single } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } batch_local_plan: |- BatchProject { exprs: [mv.v] } └─BatchTopN { order: [mv.v ASC], limit: 10, offset: 0 } └─BatchProject { exprs: [mv.v, mv.v] } - └─BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: Single } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } diff --git a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml index 7a0ac41a4b85b..7ff0a08172cb2 100644 --- a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml @@ -12,29 +12,50 @@ sql: | select max(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [max(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [max(max(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [max(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(s.v)] } - └─StreamSimpleAgg { aggs: [max(s.v), count] } - └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamProject { exprs: [max(max(s.v))] } + └─StreamSimpleAgg { aggs: [max(max(s.v)), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(s.v), count] } + └─StreamProject { exprs: [s.v, s.o, s.t._row_id, Vnode(s.o) as _vnode] } + └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [max(s.v)] } - └── StreamSimpleAgg { aggs: [max(s.v), count] } { tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] } - └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 2 ] - ├── Upstream - └── BatchPlanNode + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [max(max(s.v))] } + └── StreamSimpleAgg { aggs: [max(max(s.v)), count] } + ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] + └── StreamExchange Single from 1 - Table 0 { columns: [ s_v, s_t__row_id, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Fragment 1 + StreamHashAgg { group_key: [_vnode], aggs: [max(s.v), count] } { tables: [ HashAggState: 3, HashAggCall0: 2 ] } + └── StreamProject { exprs: [s.v, s.o, s.t._row_id, Vnode(s.o) as _vnode] } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 4 ] + ├── Upstream + └── BatchPlanNode + + Table 0 { columns: [ max(s_v), _vnode, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ max(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ max(max(s_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 + ├── columns: [ _vnode, s_v, s_t__row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 3 { columns: [ _vnode, max(s_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 4 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4 ] @@ -50,25 +71,34 @@ sql: | select sum(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [sum(sum(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [sum(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(s.v)] } - └─StreamSimpleAgg { aggs: [sum(s.v), count] } - └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamProject { exprs: [sum(sum(s.v))] } + └─StreamSimpleAgg { aggs: [sum(sum(s.v)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(s.v)] } + └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [sum(s.v)] } - └── StreamSimpleAgg { aggs: [sum(s.v), count] } { tables: [ SimpleAggState: 0 ] } - └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 1 ] - ├── Upstream - └── BatchPlanNode + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [sum(sum(s.v))] } + └── StreamSimpleAgg { aggs: [sum(sum(s.v)), count] } + ├── tables: [ SimpleAggState: 0 ] + └── StreamExchange Single from 1 - Table 0 { columns: [ sum(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Fragment 1 + StreamStatelessSimpleAgg { aggs: [sum(s.v)] } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 1 ] + ├── Upstream + └── BatchPlanNode + + Table 0 { columns: [ sum(sum(s_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] @@ -86,25 +116,34 @@ sql: | select count(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [count(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [sum0(count(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [count(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(s.v)] } - └─StreamSimpleAgg { aggs: [count(s.v), count] } - └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamProject { exprs: [sum0(count(s.v))] } + └─StreamSimpleAgg { aggs: [sum0(count(s.v)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [count(s.v)] } + └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [count(s.v)] } - └── StreamSimpleAgg { aggs: [count(s.v), count] } { tables: [ SimpleAggState: 0 ] } - └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 1 ] - ├── Upstream - └── BatchPlanNode + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [sum0(count(s.v))] } + └── StreamSimpleAgg { aggs: [sum0(count(s.v)), count] } + ├── tables: [ SimpleAggState: 0 ] + └── StreamExchange Single from 1 + + Fragment 1 + StreamStatelessSimpleAgg { aggs: [count(s.v)] } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 1 ] + ├── Upstream + └── BatchPlanNode - Table 0 { columns: [ count(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum0(count(s_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] @@ -122,26 +161,32 @@ sql: | select string_agg(s, ',' order by v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [s.s, ',':Varchar, s.v] } - └─BatchScan { table: s, columns: [s.v, s.s], distribution: Single } + └─BatchScan { table: s, columns: [s.v, s.s], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } └─StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } - └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } - └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamExchange { dist: Single } + └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } + └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] └── StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } - └── StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } { tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] } - └── StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } - └── StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 2 ] - ├── Upstream - └── BatchPlanNode + └── StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } + ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] + └── StreamExchange Single from 1 + + Fragment 1 + StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } + └── StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 2 ] + ├── Upstream + └── BatchPlanNode Table 0 ├── columns: [ s_v, s_t__row_id, s_s, ',':Varchar, _rw_timestamp ] @@ -150,12 +195,7 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ string_agg(s_s, ',':Varchar order_by(s_v ASC)), count, _rw_timestamp ] - ├── primary key: [] - ├── value indices: [ 0, 1 ] - ├── distribution key: [] - └── read pk prefix len hint: 0 + Table 1 { columns: [ string_agg(s_s, ',':Varchar order_by(s_v ASC)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] @@ -966,13 +1006,13 @@ └─BatchProject { exprs: [max(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [max(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [max(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [max(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } - └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -983,17 +1023,12 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 2 ] ├── Upstream └── BatchPlanNode - Table 0 - ├── columns: [ s_k, s_v, s_t__row_id, _rw_timestamp ] - ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 + Table 0 { columns: [ s_k, s_v, s_t__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 { columns: [ s_k, max(s_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1135,13 +1170,13 @@ └─BatchProject { exprs: [sum(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [sum(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [sum(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } - └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -1151,7 +1186,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 1 ] ├── Upstream └── BatchPlanNode @@ -1296,13 +1331,13 @@ └─BatchProject { exprs: [count(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [count(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [count(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [count(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } - └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -1312,7 +1347,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 1 ] ├── Upstream └── BatchPlanNode @@ -1484,14 +1519,14 @@ └─BatchHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC))] } └─BatchExchange { order: [], dist: HashShard(s.k) } └─BatchProject { exprs: [s.k, s.s, ',':Varchar, s.o] } - └─BatchScan { table: s, columns: [s.k, s.o, s.s], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.o, s.s], distribution: UpstreamHashShard(s.o) } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } - └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -1503,7 +1538,7 @@ Fragment 1 StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } - └── StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └── StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 2 ] ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index a87b116a658c2..6d92ab8947741 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -620,17 +620,16 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } - └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 2 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); @@ -679,17 +678,16 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, sum, integers._row_id] } - └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, sum, integers._row_id] } + └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 4 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); diff --git a/src/frontend/planner_test/tests/testdata/output/topn.yaml b/src/frontend/planner_test/tests/testdata/output/topn.yaml index 42a7e61cda5a4..21e4d5b48fc5b 100644 --- a/src/frontend/planner_test/tests/testdata/output/topn.yaml +++ b/src/frontend/planner_test/tests/testdata/output/topn.yaml @@ -25,7 +25,7 @@ BatchTopN { order: [t1_mv.a DESC], limit: 50, offset: 50 } └─BatchExchange { order: [], dist: Single } └─BatchLimit { limit: 100, offset: 0 } - └─BatchScan { table: t1_mv, columns: [t1_mv.pk, t1_mv.a, t1_mv.b, t1_mv.c, t1_mv.d], limit: 100, distribution: SomeShard } + └─BatchScan { table: t1_mv, columns: [t1_mv.pk, t1_mv.a, t1_mv.b, t1_mv.c, t1_mv.d], limit: 100, distribution: UpstreamHashShard(t1_mv.a) } - sql: | WITH c1(k, v) AS ( VALUES diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index e90bf2b747ccc..7e6ab186f7dc0 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -146,32 +146,36 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } - └─StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal) as $expr3, (sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)::Decimal) as $expr4, (sum(lineitem.l_discount) / count(lineitem.l_discount)::Decimal) as $expr5, count, Vnode(lineitem.l_returnflag, lineitem.l_linestatus) as $expr6] } - └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } - └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, $expr1, ($expr1 * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamProject { exprs: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └─StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal) as $expr3, (sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)::Decimal) as $expr4, (sum(lineitem.l_discount) / count(lineitem.l_discount)::Decimal) as $expr5, count, Vnode(lineitem.l_returnflag, lineitem.l_linestatus) as $expr6] } + └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, $expr1, ($expr1 * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } - └── StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0, 1]) from 1 Fragment 1 + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └── StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal) as $expr3, (sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)::Decimal) as $expr4, (sum(lineitem.l_discount) / count(lineitem.l_discount)::Decimal) as $expr5, count, Vnode(lineitem.l_returnflag, lineitem.l_linestatus) as $expr6] } └── StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1]) from 2 + └── StreamExchange Hash([0, 1]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, $expr1, ($expr1 * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamProject { exprs: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } @@ -188,7 +192,7 @@ Table 3 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: tpch_q2 before: @@ -312,152 +316,156 @@ └─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, 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: [_vnode] } - └─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 _vnode] } - └─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], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, 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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(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], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + └─StreamExchange { dist: HashShard(supplier.s_acctbal, nation.n_name, supplier.s_name, part.p_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] } + └─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: [_vnode] } + └─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 _vnode] } + └─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], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, 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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(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], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, 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, 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, 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 + └── StreamExchange Hash([0, 2, 1, 3]) from 1 Fragment 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, 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 2 + + Fragment 2 StreamGroupTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } { 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, 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 _vnode] } └── 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 + ├── StreamExchange Hash([2, 3]) from 3 + └── StreamExchange Hash([6, 7]) from 12 - Fragment 2 + Fragment 3 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([0]) from 4 └── 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 + └── StreamExchange Hash([1]) from 5 - Fragment 3 + Fragment 4 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], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 10 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 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 + ├── StreamExchange Hash([2]) from 6 + └── StreamExchange Hash([0]) from 9 - Fragment 5 + Fragment 6 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 + ├── StreamExchange Hash([1]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 6 + Fragment 7 StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 21 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 22 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 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 + ├── StreamExchange Hash([0]) from 10 + └── StreamExchange Hash([1]) from 11 - Fragment 9 + Fragment 10 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 28 ] } ├── Upstream └── BatchPlanNode - Fragment 11 - 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 - 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 + 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 13 + └── StreamExchange Hash([1]) from 18 Fragment 13 - 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 ] } + 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 14 - └── StreamExchange Hash([2]) from 15 + └── StreamExchange Hash([3]) from 17 Fragment 14 + 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 15 + └── StreamExchange Hash([2]) from 16 + + Fragment 15 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 41 ] } ├── Upstream └── BatchPlanNode - Fragment 15 + Fragment 16 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 42 ] } ├── Upstream └── BatchPlanNode - Fragment 16 + Fragment 17 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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 43 ] } ├── Upstream └── BatchPlanNode - Fragment 17 + Fragment 18 StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 44 ] } ├── Upstream @@ -553,7 +561,7 @@ Table 44 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], 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, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey#1, _rw_timestamp ], 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 } + 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, _rw_timestamp ], 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: [ 0, 2, 1, 3 ], read pk prefix len hint: 9 } - id: tpch_q3 before: @@ -627,70 +635,74 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } - └─StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } - └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } - │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ │ └─StreamProject { exprs: [customer.c_custkey] } - │ │ └─StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - │ └─StreamExchange { dist: HashShard(orders.o_custkey) } - │ └─StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(sum($expr1), orders.o_orderdate) } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } + └─StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } + └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } + │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ │ └─StreamProject { exprs: [customer.c_custkey] } + │ │ └─StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + │ └─StreamExchange { dist: HashShard(orders.o_custkey) } + │ └─StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } - └── StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1, 2]) from 1 Fragment 1 + StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } + └── StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } └── StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamExchange Hash([0, 1, 2]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([1]) from 6 - Fragment 4 + Fragment 5 StreamProject { exprs: [customer.c_custkey] } └── StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } └── StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── tables: [ StreamScan: 12 ] ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -726,7 +738,7 @@ Table 13 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ l_orderkey, revenue, o_orderdate, o_shippriority, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ l_orderkey, revenue, o_orderdate, o_shippriority, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2 ], read pk prefix len hint: 4 } - id: tpch_q4 before: @@ -788,43 +800,47 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [orders.o_orderpriority, count] } - └─StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [orders.o_orderpriority, count, Vnode(orders.o_orderpriority) as _vnode] } - └─StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } - └─StreamExchange { dist: HashShard(orders.o_orderpriority) } - └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - │ └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderpriority) } + └─StreamProject { exprs: [orders.o_orderpriority, count] } + └─StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [orders.o_orderpriority, count, Vnode(orders.o_orderpriority) as _vnode] } + └─StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } + └─StreamExchange { dist: HashShard(orders.o_orderpriority) } + └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } + │ └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [orders.o_orderpriority, count] } - └── StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [orders.o_orderpriority, count] } + └── StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [orders.o_orderpriority, count, Vnode(orders.o_orderpriority) as _vnode] } └── StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } └── StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } @@ -832,7 +848,7 @@ ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -858,7 +874,7 @@ Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ o_orderpriority, order_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_orderpriority, order_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q5 before: @@ -949,116 +965,120 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } - └─StreamProject { exprs: [nation.n_name, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [nation.n_name, sum($expr1), Vnode(nation.n_name) as $expr2] } - └─StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(nation.n_name) } - └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey] } - └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey AND nation.n_nationkey = customer.c_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey, customer.c_nationkey] } - ├─StreamExchange { dist: HashShard(nation.n_nationkey, 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 = 'MIDDLE EAST':Varchar) } - │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ └─StreamFilter { predicate: IsNotNull(nation.n_nationkey) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard(customer.c_nationkey, supplier.s_nationkey) } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND customer.c_nationkey = supplier.s_nationkey, output: [customer.c_nationkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey, customer.c_nationkey) } - │ └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } - │ ├─StreamExchange { dist: HashShard(orders.o_custkey) } - │ │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - │ │ └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - │ └─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey, supplier.s_nationkey) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } - ├─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [nation.n_name, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [nation.n_name, sum($expr1), Vnode(nation.n_name) as $expr2] } + └─StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(nation.n_name) } + └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey] } + └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey AND nation.n_nationkey = customer.c_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey, customer.c_nationkey] } + ├─StreamExchange { dist: HashShard(nation.n_nationkey, 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 = 'MIDDLE EAST':Varchar) } + │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(nation.n_nationkey) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(customer.c_nationkey, supplier.s_nationkey) } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND customer.c_nationkey = supplier.s_nationkey, output: [customer.c_nationkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey, customer.c_nationkey) } + │ └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } + │ ├─StreamExchange { dist: HashShard(orders.o_custkey) } + │ │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } + │ │ └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + │ └─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey, supplier.s_nationkey) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } + ├─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [nation.n_name, sum($expr1)] } - └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1]) from 1 Fragment 1 + StreamProject { exprs: [nation.n_name, sum($expr1)] } + └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [nation.n_name, sum($expr1), Vnode(nation.n_name) as $expr2] } └── StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } ├── tables: [ HashAggState: 2 ] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey] } └── StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey AND nation.n_nationkey = customer.c_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey, customer.c_nationkey] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0, 0]) from 3 - └── StreamExchange Hash([0, 3]) from 6 + ├── StreamExchange Hash([0, 0]) from 4 + └── StreamExchange Hash([0, 3]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([2]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([2]) from 6 - Fragment 4 + Fragment 5 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'MIDDLE EAST':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: IsNotNull(nation.n_nationkey) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 12 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND customer.c_nationkey = supplier.s_nationkey, output: [customer.c_nationkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } ├── tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] - ├── StreamExchange Hash([0, 1]) from 7 - └── StreamExchange Hash([0, 3]) from 10 + ├── StreamExchange Hash([0, 1]) from 8 + └── StreamExchange Hash([0, 3]) from 11 - Fragment 7 + Fragment 8 StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } { tables: [ HashJoinLeft: 17, HashJoinDegreeLeft: 18, HashJoinRight: 19, HashJoinDegreeRight: 20 ] } - ├── StreamExchange Hash([1]) from 8 - └── StreamExchange Hash([0]) from 9 + ├── StreamExchange Hash([1]) from 9 + └── StreamExchange Hash([0]) from 10 - Fragment 8 + Fragment 9 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 21 ] } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 22 ] } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } { tables: [ HashJoinLeft: 23, HashJoinDegreeLeft: 24, HashJoinRight: 25, HashJoinDegreeRight: 26 ] } - ├── StreamExchange Hash([1]) from 11 - └── StreamExchange Hash([0]) from 12 + ├── StreamExchange Hash([1]) from 12 + └── StreamExchange Hash([0]) from 13 - Fragment 11 + Fragment 12 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 27 ] ├── Upstream └── BatchPlanNode - Fragment 12 + Fragment 13 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 28 ] } ├── Upstream └── BatchPlanNode @@ -1126,7 +1146,7 @@ Table 28 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: tpch_q6 before: @@ -1290,107 +1310,111 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, nation.n_name, $expr1) as $expr3] } - └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } - └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, customer.c_custkey] } - └─StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: all } - ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_nationkey, supplier.s_suppkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } - │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(orders.o_orderkey) } - └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, nation.n_name, $expr1) as $expr3] } + └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, customer.c_custkey] } + └─StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: all } + ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_nationkey, supplier.s_suppkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } + │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderkey) } + └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } - └── StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 + StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } + └── StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, nation.n_name, $expr1) as $expr3] } └── StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamExchange Hash([0, 1, 2]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, customer.c_custkey] } └── StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } └── StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: all } { tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] } - ├── StreamExchange Hash([1]) from 3 - └── StreamExchange Hash([1]) from 8 + ├── StreamExchange Hash([1]) from 4 + └── StreamExchange Hash([1]) from 9 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_nationkey, supplier.s_suppkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - ├── StreamExchange Hash([1]) from 4 - └── StreamExchange Hash([1]) from 7 + ├── StreamExchange Hash([1]) from 5 + └── StreamExchange Hash([1]) from 8 - Fragment 4 + Fragment 5 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } - ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([1]) from 6 + ├── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([1]) from 7 - Fragment 5 + Fragment 6 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 16 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 17 ] ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } { tables: [ HashJoinLeft: 18, HashJoinDegreeLeft: 19, HashJoinRight: 20, HashJoinDegreeRight: 21 ] } - ├── StreamExchange Hash([1]) from 9 - └── StreamExchange Hash([1]) from 12 + ├── StreamExchange Hash([1]) from 10 + └── StreamExchange Hash([1]) from 13 - Fragment 9 + Fragment 10 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 22, HashJoinDegreeLeft: 23, HashJoinRight: 24, HashJoinDegreeRight: 25 ] } - ├── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([1]) from 11 + ├── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([1]) from 12 - Fragment 10 + Fragment 11 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 26 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 12 + Fragment 13 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 28 ] } ├── Upstream └── BatchPlanNode @@ -1453,7 +1477,7 @@ Table 28 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - id: tpch_q8 before: @@ -1570,146 +1594,150 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr1, $expr4] } - └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } - └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } - └─StreamHashAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [$expr1, Case((nation.n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, 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, region.r_regionkey] } - │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ │ └─StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } - │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } - │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ │ └─StreamProject { exprs: [part.p_partkey] } - │ │ └─StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } - │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_partkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(orders.o_orderkey) } - └─StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, $expr4] } + └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } + └─StreamHashAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, Case((nation.n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, 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, region.r_regionkey] } + │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ │ └─StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } + │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } + │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(part.p_partkey) } + │ │ └─StreamProject { exprs: [part.p_partkey] } + │ │ └─StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } + │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_partkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderkey) } + └─StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [$expr1, $expr4] } - └── StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [$expr1, $expr4] } + └── StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } └── StreamHashAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } ├── tables: [ HashAggState: 2 ] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [$expr1, Case((nation.n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([3]) from 8 - - Fragment 3 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 7 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([3]) from 9 Fragment 4 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([1]) from 6 + └── StreamExchange Hash([1]) from 8 Fragment 5 + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } + ├── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([1]) from 7 + + Fragment 6 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 16 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 17 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } ├── tables: [ HashJoinLeft: 18, HashJoinDegreeLeft: 19, HashJoinRight: 20, HashJoinDegreeRight: 21 ] - ├── StreamExchange Hash([1]) from 9 - └── StreamExchange Hash([0]) from 16 + ├── StreamExchange Hash([1]) from 10 + └── StreamExchange Hash([0]) from 17 - Fragment 9 + Fragment 10 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 22, HashJoinDegreeLeft: 23, HashJoinRight: 24, HashJoinDegreeRight: 25 ] - ├── StreamExchange Hash([1]) from 10 - └── StreamExchange Hash([1]) from 13 + ├── StreamExchange Hash([1]) from 11 + └── StreamExchange Hash([1]) from 14 - Fragment 10 + Fragment 11 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 26, HashJoinDegreeLeft: 27, HashJoinRight: 28, HashJoinDegreeRight: 29 ] } - ├── StreamExchange Hash([0]) from 11 - └── StreamExchange Hash([1]) from 12 + ├── StreamExchange Hash([0]) from 12 + └── StreamExchange Hash([1]) from 13 - Fragment 11 + Fragment 12 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 30 ] } ├── Upstream └── BatchPlanNode - Fragment 12 + Fragment 13 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 31 ] } ├── Upstream └── BatchPlanNode - Fragment 13 + Fragment 14 StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 32, HashJoinDegreeLeft: 33, HashJoinRight: 34, HashJoinDegreeRight: 35 ] } - ├── StreamExchange Hash([0]) from 14 - └── StreamExchange Hash([1]) from 15 + ├── StreamExchange Hash([0]) from 15 + └── StreamExchange Hash([1]) from 16 - Fragment 14 + Fragment 15 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 36 ] } ├── Upstream └── BatchPlanNode - Fragment 15 + Fragment 16 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 37 ] ├── Upstream └── BatchPlanNode - Fragment 16 + Fragment 17 StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 38 ] } ├── Upstream @@ -1798,7 +1826,7 @@ Table 38 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ o_year, mkt_share, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_year, mkt_share, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q9 before: @@ -1894,106 +1922,110 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, $expr1) as $expr3] } - └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } - └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, partsupp.ps_suppkey, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber, partsupp.ps_partkey] } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey AND partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = supplier.s_suppkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, partsupp.ps_suppkey, partsupp.ps_partkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } - │ ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ │ └─StreamProject { exprs: [part.p_partkey] } - │ │ └─StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } - │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } - │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamFilter { predicate: IsNotNull(partsupp.ps_suppkey) } - │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, $expr1) as $expr3] } + └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, partsupp.ps_suppkey, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber, partsupp.ps_partkey] } + └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey AND partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = supplier.s_suppkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, partsupp.ps_suppkey, partsupp.ps_partkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } + │ ├─StreamExchange { dist: HashShard(part.p_partkey) } + │ │ └─StreamProject { exprs: [part.p_partkey] } + │ │ └─StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } + │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamFilter { predicate: IsNotNull(partsupp.ps_suppkey) } + │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } - └── StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0, 1]) from 1 Fragment 1 + StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } + └── StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, $expr1) as $expr3] } └── StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1]) from 2 + └── StreamExchange Hash([0, 1]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, partsupp.ps_suppkey, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber, partsupp.ps_partkey] } └── StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey AND partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = supplier.s_suppkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, partsupp.ps_suppkey, partsupp.ps_partkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([2]) from 3 + ├── StreamExchange Hash([2]) from 4 └── StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] } - ├── StreamExchange Hash([1]) from 6 - └── StreamExchange Hash([2]) from 9 + ├── StreamExchange Hash([1]) from 7 + └── StreamExchange Hash([2]) from 10 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 4 + Fragment 5 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: IsNotNull(partsupp.ps_suppkey) } └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 12 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 17, HashJoinDegreeLeft: 18, HashJoinRight: 19, HashJoinDegreeRight: 20 ] } - ├── StreamExchange Hash([0]) from 7 - └── StreamExchange Hash([1]) from 8 + ├── StreamExchange Hash([0]) from 8 + └── StreamExchange Hash([1]) from 9 - Fragment 7 + Fragment 8 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 21 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 22 ] } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 23, HashJoinDegreeLeft: 24, HashJoinRight: 25, HashJoinDegreeRight: 26 ] } - ├── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + ├── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 10 + Fragment 11 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { tables: [ StreamScan: 28 ] } ├── Upstream @@ -2057,7 +2089,7 @@ Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ nation, o_year, sum_profit, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ nation, o_year, sum_profit, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: tpch_q10 before: @@ -2145,70 +2177,74 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_returnflag], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey) as $expr2] } - └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } - │ └─StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(orders.o_orderkey) } - └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey) as $expr2] } + └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } + │ └─StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderkey) } + └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } + └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └── StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └── StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey) as $expr2] } └── StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } { tables: [ HashAggState: 2 ] } └── StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([1]) from 2 - └── StreamExchange Hash([2]) from 5 + ├── StreamExchange Hash([1]) from 3 + └── StreamExchange Hash([2]) from 6 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, nation.n_nationkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([3]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([3]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── tables: [ StreamScan: 12 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } { tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] } - ├── StreamExchange Hash([0]) from 6 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([0]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 6 + Fragment 7 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -2216,7 +2252,7 @@ ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 18 ] } @@ -2272,7 +2308,7 @@ Table 18 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [], read pk prefix len hint: 8 } + Table 4294967294 { columns: [ c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 2 ], read pk prefix len hint: 8 } - id: tpch_q11 before: @@ -2371,55 +2407,59 @@ └─BatchScan { table: partsupp, columns: [partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } - └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1), Vnode(partsupp.ps_partkey) as _vnode] } - └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } - ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } - │ └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_nationkey] } - │ └─StreamShare { id: 9 } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, 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_availqty, 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_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, 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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } - └─StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } - └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } - └─StreamShare { id: 9 } - └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, 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_availqty, 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_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, 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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(nation.n_nationkey) } - └─StreamProject { exprs: [nation.n_nationkey] } - └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } - └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1), Vnode(partsupp.ps_partkey) as _vnode] } + └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } + ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } + │ └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_nationkey] } + │ └─StreamShare { id: 9 } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, 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_availqty, 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_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, 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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + └─StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } + └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } + └─StreamShare { id: 9 } + └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, 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_availqty, 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_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, 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], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(nation.n_nationkey) } + └─StreamProject { exprs: [nation.n_nationkey] } + └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } + └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } - └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1]) from 1 Fragment 1 + StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } + └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [partsupp.ps_partkey, sum($expr1), Vnode(partsupp.ps_partkey) as _vnode] } @@ -2428,52 +2468,52 @@ ├── StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } │ └── StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } │ ├── tables: [ HashAggState: 4 ] - │ └── StreamExchange Hash([0]) from 2 - └── StreamExchange Broadcast from 8 + │ └── StreamExchange Hash([0]) from 3 + └── StreamExchange Broadcast from 9 - Fragment 2 + Fragment 3 StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_nationkey] } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, nation.n_nationkey] } ├── tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] - ├── StreamExchange Hash([3]) from 4 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([3]) from 5 + └── StreamExchange Hash([0]) from 8 - Fragment 4 + Fragment 5 StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } ├── tables: [ HashJoinLeft: 9, HashJoinDegreeLeft: 10, HashJoinRight: 11, HashJoinDegreeRight: 12 ] - ├── StreamExchange Hash([1]) from 5 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([1]) from 6 + └── StreamExchange Hash([0]) from 7 - Fragment 5 + Fragment 6 StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── tables: [ StreamScan: 13 ] ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 14 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } └── StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } { tables: [ SimpleAggState: 16 ] } - └── StreamExchange Single from 9 + └── StreamExchange Single from 10 - Fragment 9 + Fragment 10 StreamStatelessSimpleAgg { aggs: [sum($expr2)] } └── StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ partsupp_ps_partkey, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } @@ -2509,7 +2549,7 @@ Table 16 { columns: [ sum(sum($expr2)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ ps_partkey, value, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ ps_partkey, value, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: tpch_q12 before: @@ -2576,47 +2616,51 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } - └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } - └─StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2), Vnode(lineitem.l_shipmode) as $expr3] } - └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } - └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } - └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } + └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } + └─StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2), Vnode(lineitem.l_shipmode) as $expr3] } + └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } + └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } + └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } - └── StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } + └── StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2), Vnode(lineitem.l_shipmode) as $expr3] } └── StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 7 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } └── StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -2642,7 +2686,7 @@ Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ l_shipmode, high_line_count, low_line_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ l_shipmode, high_line_count, low_line_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q13 before: @@ -2703,52 +2747,56 @@ └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], distribution: UpstreamHashShard(orders.o_orderkey) } stream_plan: |- StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(orders.o_orderkey), count] } - └─StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [count(orders.o_orderkey), count, Vnode(count(orders.o_orderkey)) as _vnode] } - └─StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } - └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } - └─StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } - └─StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } - └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - └─StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(count, count(orders.o_orderkey)) } + └─StreamProject { exprs: [count(orders.o_orderkey), count] } + └─StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [count(orders.o_orderkey), count, Vnode(count(orders.o_orderkey)) as _vnode] } + └─StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } + └─StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } + └─StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } + └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } + └─StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [count(orders.o_orderkey), count] } - └── StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamProject { exprs: [count(orders.o_orderkey), count] } + └── StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [count(orders.o_orderkey), count, Vnode(count(orders.o_orderkey)) as _vnode] } └── StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([1]) from 2 + └── StreamExchange Hash([1]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } └── StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } { tables: [ HashAggState: 3 ] } └── StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } ├── tables: [ HashJoinLeft: 4, HashJoinDegreeLeft: 5, HashJoinRight: 6, HashJoinDegreeRight: 7 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([1]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([1]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: customer, columns: [customer.c_custkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── tables: [ StreamScan: 8 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } @@ -2776,7 +2824,7 @@ Table 9 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: tpch_q14 before: @@ -2953,70 +3001,74 @@ └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } - └─StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as _vnode] } - └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } - ├─StreamExchange { dist: HashShard(sum($expr1)) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamShare { id: 7 } - │ └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - │ └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } - └─StreamProject { exprs: [max(max(sum($expr1)))] } - └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as _vnode] } - └─StreamShare { id: 7 } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } + └─StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } + ├─StreamExchange { dist: HashShard(sum($expr1)) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamShare { id: 7 } + │ └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } + │ └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } + └─StreamProject { exprs: [max(max(sum($expr1)))] } + └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as _vnode] } + └─StreamShare { id: 7 } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } + └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } - └── StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } + └── StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as _vnode] } └── StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] - ├── StreamExchange Hash([4]) from 2 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([4]) from 3 + └── StreamExchange Hash([0]) from 7 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } ├── tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange NoShuffle from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange NoShuffle from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 10 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } └── StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } { tables: [ HashAggState: 11 ] } - └── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 5 + Fragment 6 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -3024,15 +3076,15 @@ ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [max(max(sum($expr1)))] } └── StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } { tables: [ SimpleAggState: 14, SimpleAggCall0: 13 ] } - └── StreamExchange Single from 7 + └── StreamExchange Single from 8 - Fragment 7 + Fragment 8 StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } { tables: [ HashAggState: 16, HashAggCall0: 15 ] } └── StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as _vnode] } - └── StreamExchange NoShuffle from 4 + └── StreamExchange NoShuffle from 5 Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 0 } @@ -3068,7 +3120,7 @@ Table 16 { columns: [ _vnode, max(sum($expr1)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q16 before: @@ -3141,67 +3193,71 @@ └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_plan: |- StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } - └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } - └─StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey), Vnode(part.p_brand, part.p_type, part.p_size) as $expr1] } - └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } - └─StreamExchange { dist: HashShard(part.p_brand, part.p_type, part.p_size) } - └─StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey] } - ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey, part.p_partkey] } - │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ └─StreamExchange { dist: HashShard(part.p_partkey) } - │ └─StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } - │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamProject { exprs: [supplier.s_suppkey] } - └─StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } - └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(count(distinct partsupp.ps_suppkey), part.p_brand, part.p_type, part.p_size) } + └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } + └─StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey), Vnode(part.p_brand, part.p_type, part.p_size) as $expr1] } + └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } + └─StreamExchange { dist: HashShard(part.p_brand, part.p_type, part.p_size) } + └─StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey] } + ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey, part.p_partkey] } + │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ └─StreamExchange { dist: HashShard(part.p_partkey) } + │ └─StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } + │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamProject { exprs: [supplier.s_suppkey] } + └─StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } + └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } - └── StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } - ├── tables: [ TopN: 0 ] - └── StreamExchange Single from 1 + └── StreamExchange Hash([3, 0, 1, 2]) from 1 Fragment 1 + StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } + └── StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } + ├── tables: [ TopN: 0 ] + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey), Vnode(part.p_brand, part.p_type, part.p_size) as $expr1] } └── StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } ├── tables: [ HashAggState: 2, HashAggDedupForCol0: 3 ] - └── StreamExchange Hash([1, 2, 3]) from 2 + └── StreamExchange Hash([1, 2, 3]) from 3 - Fragment 2 + Fragment 3 StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey] } ├── tables: [ HashJoinLeft: 4, HashJoinDegreeLeft: 5, HashJoinRight: 6, HashJoinDegreeRight: 7 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey, part.p_partkey] } ├── tables: [ HashJoinLeft: 8, HashJoinDegreeLeft: 9, HashJoinRight: 10, HashJoinDegreeRight: 11 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 4 + Fragment 5 StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── tables: [ StreamScan: 12 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 13 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [supplier.s_suppkey] } └── StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } └── StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 14 ] } @@ -3244,7 +3300,7 @@ Table 14 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ p_brand, p_type, p_size, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ p_brand, p_type, p_size, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3, 0, 1, 2 ], read pk prefix len hint: 4 } - id: tpch_q17 before: @@ -3490,37 +3546,41 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } - └─StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } - └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate], aggs: [sum(lineitem.l_quantity), count] } - └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_linenumber] } - ├─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } - │ │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - │ │ └─StreamExchange { dist: HashShard(orders.o_custkey) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamProject { exprs: [lineitem.l_orderkey] } - └─StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Decimal) } - └─StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_totalprice, orders.o_orderdate) } + └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } + └─StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate], aggs: [sum(lineitem.l_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_linenumber] } + ├─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } + │ │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + │ │ └─StreamExchange { dist: HashShard(orders.o_custkey) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamProject { exprs: [lineitem.l_orderkey] } + └─StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Decimal) } + └─StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } - └── StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([4, 3]) from 1 Fragment 1 + StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } + └── StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } └── StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate], aggs: [sum(lineitem.l_quantity), count] } { tables: [ HashAggState: 2 ] } @@ -3528,38 +3588,38 @@ ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] ├── StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } │ ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - │ ├── StreamExchange Hash([2]) from 2 - │ └── StreamExchange Hash([0]) from 5 + │ ├── StreamExchange Hash([2]) from 3 + │ └── StreamExchange Hash([0]) from 6 └── StreamProject { exprs: [lineitem.l_orderkey] } └── StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Decimal) } └── StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } └── StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } { tables: [ HashAggState: 18 ] } - └── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([0]) from 7 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } ├── tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([1]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([1]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── tables: [ StreamScan: 16 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 17 ] ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 19 ] ├── Upstream @@ -3621,7 +3681,7 @@ Table 19 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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: [ c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity, _rw_timestamp ], primary key: [ $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity, _rw_timestamp ], primary key: [ $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4, 3 ], read pk prefix len hint: 5 } - id: tpch_q19 before: @@ -3810,96 +3870,100 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - └─StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [supplier.s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as _vnode] } - └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - └─StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey] } - ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, lineitem.l_suppkey] } - │ └─StreamFilter { predicate: ($expr1 > $expr2) } - │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey, output: all } - │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } - │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } - │ └─StreamHashAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity), count] } - │ └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } - │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(part.p_partkey) } - └─StreamProject { exprs: [part.p_partkey] } - └─StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } - └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + └─StreamExchange { dist: HashShard(supplier.s_name) } + └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + └─StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [supplier.s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as _vnode] } + └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey] } + ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, lineitem.l_suppkey] } + │ └─StreamFilter { predicate: ($expr1 > $expr2) } + │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey, output: all } + │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } + │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } + │ └─StreamHashAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity), count] } + │ └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } + │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } + │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(part.p_partkey) } + └─StreamProject { exprs: [part.p_partkey] } + └─StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } + └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - └── StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + └── StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [supplier.s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as _vnode] } └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 6 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } ├── tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] - ├── StreamExchange Hash([3]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([3]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 10 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey] } { tables: [ HashJoinLeft: 12, HashJoinDegreeLeft: 13, HashJoinRight: 14, HashJoinDegreeRight: 15 ] } - ├── StreamExchange Hash([0]) from 6 - └── StreamExchange Hash([0]) from 9 + ├── StreamExchange Hash([0]) from 7 + └── StreamExchange Hash([0]) from 10 - Fragment 6 + Fragment 7 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, lineitem.l_suppkey] } └── StreamFilter { predicate: ($expr1 > $expr2) } └── StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey, output: all } { tables: [ HashJoinLeft: 16, HashJoinDegreeLeft: 17, HashJoinRight: 18, HashJoinDegreeRight: 19 ] } - ├── StreamExchange Hash([0, 1]) from 7 + ├── StreamExchange Hash([0, 1]) from 8 └── StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } └── StreamHashAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity), count] } { tables: [ HashAggState: 21 ] } - └── StreamExchange Hash([0, 1]) from 8 + └── StreamExchange Hash([0, 1]) from 9 - Fragment 7 + Fragment 8 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── tables: [ StreamScan: 20 ] ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -3907,7 +3971,7 @@ ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 23 ] } @@ -3962,7 +4026,7 @@ Table 23 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q21 before: @@ -4071,102 +4135,106 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_name, count] } - └─StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as _vnode] } - └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(supplier.s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber, lineitem.l_orderkey] } - ├─StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } - │ ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } - │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } - │ │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ │ └─StreamProject { exprs: [nation.n_nationkey] } - │ │ │ │ └─StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } - │ │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } - │ │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ │ │ └─StreamProject { exprs: [orders.o_orderkey] } - │ │ │ └─StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } - │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - │ │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } - │ │ └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - │ │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(count, supplier.s_name) } + └─StreamProject { exprs: [supplier.s_name, count] } + └─StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as _vnode] } + └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(supplier.s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber, lineitem.l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } + │ ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } + │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } + │ │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ │ └─StreamProject { exprs: [nation.n_nationkey] } + │ │ │ │ └─StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } + │ │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } + │ │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ │ │ └─StreamProject { exprs: [orders.o_orderkey] } + │ │ │ └─StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } + │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + │ │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } + │ │ └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } + │ │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_name, count] } - └── StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamProject { exprs: [supplier.s_name, count] } + └── StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as _vnode] } └── StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber, lineitem.l_orderkey] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] ├── StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - │ ├── StreamExchange Hash([1]) from 3 - │ └── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + │ ├── StreamExchange Hash([1]) from 4 + │ └── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 7 - - Fragment 4 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } { tables: [ HashJoinLeft: 15, HashJoinDegreeLeft: 16, HashJoinRight: 17, HashJoinDegreeRight: 18 ] } ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([2]) from 6 + └── StreamExchange Hash([1]) from 8 Fragment 5 + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } { tables: [ HashJoinLeft: 15, HashJoinDegreeLeft: 16, HashJoinRight: 17, HashJoinDegreeRight: 18 ] } + ├── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([2]) from 7 + + Fragment 6 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 19 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 20 ] ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 21, HashJoinDegreeLeft: 22, HashJoinRight: 23, HashJoinDegreeRight: 24 ] - ├── StreamExchange Hash([0]) from 8 - └── StreamExchange Hash([0]) from 9 + ├── StreamExchange Hash([0]) from 9 + └── StreamExchange Hash([0]) from 10 - Fragment 8 + Fragment 9 StreamProject { exprs: [orders.o_orderkey] } └── StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 25 ] } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -4174,12 +4242,12 @@ ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -4245,7 +4313,7 @@ Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], 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_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: tpch_q22 before: @@ -4342,26 +4410,27 @@ └─BatchScan { table: customer, columns: [customer.c_acctbal, customer.c_phone], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [cntrycode, numcust, totacctbal], stream_key: [], pk_columns: [cntrycode], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal)] } - └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal), Vnode($expr2) as _vnode] } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(customer.c_acctbal)] } - └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr2, customer.c_acctbal, customer.c_custkey] } - └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr1), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } - ├─StreamHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } - │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ │ └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - │ └─StreamExchange { dist: HashShard(orders.o_custkey) } - │ └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))::Decimal) as $expr1] } - └─StreamSimpleAgg { aggs: [sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } - └─StreamProject { exprs: [customer.c_acctbal, customer.c_custkey] } - └─StreamFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─StreamTableScan { table: customer, columns: [customer.c_acctbal, customer.c_custkey, customer.c_phone], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal)] } + └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal), Vnode($expr2) as _vnode] } + └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(customer.c_acctbal)] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr2, customer.c_acctbal, customer.c_custkey] } + └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr1), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } + ├─StreamHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } + │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ │ └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + │ └─StreamExchange { dist: HashShard(orders.o_custkey) } + │ └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))::Decimal) as $expr1] } + └─StreamSimpleAgg { aggs: [sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } + └─StreamProject { exprs: [customer.c_acctbal, customer.c_custkey] } + └─StreamFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamTableScan { table: customer, columns: [customer.c_acctbal, customer.c_custkey, customer.c_phone], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml index b1f9c6d5dc88c..fc3a1fc7f5e33 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml @@ -145,18 +145,19 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } - └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } - └─StreamTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } - └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), (sum(l_quantity) / count(l_quantity)::Decimal) as $expr3, (sum(l_extendedprice) / count(l_extendedprice)::Decimal) as $expr4, (sum(l_discount) / count(l_discount)::Decimal) as $expr5, count, Vnode(l_returnflag, l_linestatus) as $expr6] } - └─StreamHashAgg [append_only] { group_key: [l_returnflag, l_linestatus], aggs: [sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count] } - └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } - └─StreamProject { exprs: [l_returnflag, l_linestatus, l_quantity, l_extendedprice, $expr1, ($expr1 * (1:Decimal + l_tax)) as $expr2, l_discount, _row_id] } - └─StreamProject { exprs: [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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, (l_extendedprice * (1:Decimal - l_discount)) as $expr1] } - └─StreamFilter { predicate: (l_shipdate <= '1998-09-21 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └─StreamTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), (sum(l_quantity) / count(l_quantity)::Decimal) as $expr3, (sum(l_extendedprice) / count(l_extendedprice)::Decimal) as $expr4, (sum(l_discount) / count(l_discount)::Decimal) as $expr5, count, Vnode(l_returnflag, l_linestatus) as $expr6] } + └─StreamHashAgg [append_only] { group_key: [l_returnflag, l_linestatus], aggs: [sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count] } + └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } + └─StreamProject { exprs: [l_returnflag, l_linestatus, l_quantity, l_extendedprice, $expr1, ($expr1 * (1:Decimal + l_tax)) as $expr2, l_discount, _row_id] } + └─StreamProject { exprs: [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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, (l_extendedprice * (1:Decimal - l_discount)) as $expr1] } + └─StreamFilter { predicate: (l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q2 before: - create_tables @@ -207,77 +208,78 @@ LIMIT 100; stream_plan: |- 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 } - └─StreamProject { exprs: [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] } - └─StreamTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [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, Vnode(ps_partkey, min(ps_supplycost)) as _vnode] } - └─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) } - │ │ └─StreamFilter { predicate: (p_size = 4:Int32) AND Like(p_type, '%TIN':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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: 6 } - │ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamShare { id: 10 } - │ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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) } - │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ └─StreamShare { id: 17 } - │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } - │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ └─StreamShare { id: 22 } - │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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) } - │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ │ └─StreamShare { id: 17 } - │ │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } - │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 22 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamShare { id: 10 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(ps_suppkey) } - └─StreamFilter { predicate: IsNotNull(ps_partkey) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - └─StreamRowIdGen { row_id_index: 8 } - └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_acctbal, n_name, s_name, p_partkey) } + └─StreamProject { exprs: [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] } + └─StreamTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [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, Vnode(ps_partkey, min(ps_supplycost)) as _vnode] } + └─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) } + │ │ └─StreamFilter { predicate: (p_size = 4:Int32) AND Like(p_type, '%TIN':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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: 6 } + │ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamShare { id: 10 } + │ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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) } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamShare { id: 17 } + │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamShare { id: 22 } + │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _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) } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamShare { id: 17 } + │ │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 22 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamShare { id: 10 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamFilter { predicate: IsNotNull(ps_partkey) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + └─StreamRowIdGen { row_id_index: 8 } + └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q3 before: - create_tables @@ -307,29 +309,30 @@ LIMIT 10; stream_plan: |- StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority] } - └─StreamTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority, Vnode(l_orderkey, o_orderdate, o_shippriority) as $expr2] } - └─StreamHashAgg [append_only] { group_key: [l_orderkey, o_orderdate, o_shippriority], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(l_orderkey, o_orderdate, o_shippriority) } - └─StreamProject { exprs: [l_orderkey, o_orderdate, o_shippriority, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, c_custkey, _row_id, o_orderkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, o_shippriority, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, c_custkey, o_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [o_orderkey, o_orderdate, o_shippriority, _row_id, c_custkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(c_custkey) } - │ │ └─StreamFilter { predicate: (c_mktsegment = 'FURNITURE':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 11 } - │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(o_custkey) } - │ └─StreamFilter { predicate: (o_orderdate < '1995-03-29':Date) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamFilter { predicate: (l_shipdate > '1995-03-29':Date) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1), o_orderdate) } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority] } + └─StreamTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority, Vnode(l_orderkey, o_orderdate, o_shippriority) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey, o_orderdate, o_shippriority], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_orderkey, o_orderdate, o_shippriority) } + └─StreamProject { exprs: [l_orderkey, o_orderdate, o_shippriority, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, c_custkey, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, o_shippriority, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, c_custkey, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [o_orderkey, o_orderdate, o_shippriority, _row_id, c_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: (c_mktsegment = 'FURNITURE':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamFilter { predicate: (o_orderdate < '1995-03-29':Date) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: (l_shipdate > '1995-03-29':Date) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q4 before: - create_tables @@ -358,23 +361,24 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [o_orderpriority, count] } - └─StreamTopN { order: [o_orderpriority ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [o_orderpriority, count, Vnode(o_orderpriority) as _vnode] } - └─StreamHashAgg { group_key: [o_orderpriority], aggs: [count] } - └─StreamExchange { dist: HashShard(o_orderpriority) } - └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, _row_id, o_orderkey] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamFilter { predicate: (o_orderdate >= '1997-07-01':Date) AND (o_orderdate < '1997-10-01 00:00:00':Timestamp) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamProject { exprs: [l_orderkey, _row_id] } - └─StreamFilter { predicate: (l_commitdate < l_receiptdate) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderpriority) } + └─StreamProject { exprs: [o_orderpriority, count] } + └─StreamTopN { order: [o_orderpriority ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [o_orderpriority, count, Vnode(o_orderpriority) as _vnode] } + └─StreamHashAgg { group_key: [o_orderpriority], aggs: [count] } + └─StreamExchange { dist: HashShard(o_orderpriority) } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamFilter { predicate: (o_orderdate >= '1997-07-01':Date) AND (o_orderdate < '1997-10-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, _row_id] } + └─StreamFilter { predicate: (l_commitdate < l_receiptdate) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q5 before: - create_tables @@ -406,44 +410,45 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [n_name, sum($expr1), Vnode(n_name) as $expr2] } - └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(n_name) } - └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } - ├─StreamExchange { dist: HashShard(n_nationkey, 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) } - │ │ └─StreamFilter { predicate: (r_name = 'MIDDLE EAST':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } - ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(o_custkey) } - │ │ └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1995-01-01 00:00:00':Timestamp) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } - ├─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(s_suppkey) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [n_name, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [n_name, sum($expr1), Vnode(n_name) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(n_name) } + └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } + ├─StreamExchange { dist: HashShard(n_nationkey, 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) } + │ │ └─StreamFilter { predicate: (r_name = 'MIDDLE EAST':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } + ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1995-01-01 00:00:00':Timestamp) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q6 before: - create_tables @@ -513,47 +518,48 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2), Vnode(n_name, n_name, $expr1) as $expr3] } - └─StreamHashAgg [append_only] { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } - └─StreamProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, n_nationkey, _row_id, s_suppkey, _row_id, _row_id, n_nationkey, _row_id, c_custkey, l_orderkey] } - └─StreamFilter { predicate: (((n_name = 'ROMANIA':Varchar) AND (n_name = 'IRAN':Varchar)) OR ((n_name = 'IRAN':Varchar) AND (n_name = 'ROMANIA':Varchar))) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: all } - ├─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _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, _row_id, n_nationkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamFilter { predicate: (l_shipdate >= '1983-01-01':Date) AND (l_shipdate <= '2000-12-31':Date) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_orderkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [n_name, o_orderkey, _row_id, _row_id, n_nationkey, c_custkey, _row_id] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, _row_id, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamShare { id: 3 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_nationkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2), Vnode(n_name, n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } + └─StreamProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, n_nationkey, _row_id, s_suppkey, _row_id, _row_id, n_nationkey, _row_id, c_custkey, l_orderkey] } + └─StreamFilter { predicate: (((n_name = 'ROMANIA':Varchar) AND (n_name = 'IRAN':Varchar)) OR ((n_name = 'IRAN':Varchar) AND (n_name = 'ROMANIA':Varchar))) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: all } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _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, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1983-01-01':Date) AND (l_shipdate <= '2000-12-31':Date) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [n_name, o_orderkey, _row_id, _row_id, n_nationkey, c_custkey, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamShare { id: 3 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q8 before: - create_tables @@ -598,59 +604,60 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr1, $expr4] } - └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } - └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } - └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [$expr1, Case((n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } - └─StreamProject { exprs: [Extract('YEAR':Varchar, o_orderdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [l_extendedprice, l_discount, o_orderdate, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, c_custkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [c_custkey, _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, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamFilter { predicate: (r_name = 'ASIA':Varchar) } - │ │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 6 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_nationkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, l_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, p_partkey] } - │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ └─StreamShare { id: 6 } - │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ └─StreamFilter { predicate: (p_type = 'PROMO ANODIZED STEEL':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_partkey) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_orderkey) } - └─StreamFilter { predicate: (o_orderdate >= '1995-01-01':Date) AND (o_orderdate <= '1996-12-31':Date) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, $expr4] } + └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } + └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, Case((n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, o_orderdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [l_extendedprice, l_discount, o_orderdate, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, c_custkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [c_custkey, _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, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamFilter { predicate: (r_name = 'ASIA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 6 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, p_partkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 6 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: (p_type = 'PROMO ANODIZED STEEL':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1995-01-01':Date) AND (o_orderdate <= '1996-12-31':Date) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q9 before: - create_tables @@ -690,43 +697,44 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [n_name, $expr1, sum($expr2), Vnode(n_name, $expr1) as $expr3] } - └─StreamHashAgg [append_only] { group_key: [n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(n_name, $expr1) } - └─StreamProject { exprs: [n_name, Extract('YEAR':Varchar, o_orderdate) as $expr1, ((l_extendedprice * (1:Decimal - l_discount)) - (ps_supplycost * l_quantity)) as $expr2, _row_id, _row_id, p_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, ps_suppkey, ps_partkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey AND ps_suppkey = l_suppkey AND ps_partkey = l_partkey AND ps_suppkey = s_suppkey, output: [l_quantity, l_extendedprice, l_discount, ps_supplycost, o_orderdate, n_name, _row_id, _row_id, p_partkey, ps_suppkey, ps_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } - ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ └─StreamFilter { predicate: Like(p_name, '%yellow%':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ └─StreamFilter { predicate: IsNotNull(ps_suppkey) } - │ └─StreamRowIdGen { row_id_index: 8 } - │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey] } - ├─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_suppkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamFilter { predicate: IsNotNull(l_partkey) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_name, $expr1) } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2), Vnode(n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, $expr1) } + └─StreamProject { exprs: [n_name, Extract('YEAR':Varchar, o_orderdate) as $expr1, ((l_extendedprice * (1:Decimal - l_discount)) - (ps_supplycost * l_quantity)) as $expr2, _row_id, _row_id, p_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, ps_suppkey, ps_partkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey AND ps_suppkey = l_suppkey AND ps_partkey = l_partkey AND ps_suppkey = s_suppkey, output: [l_quantity, l_extendedprice, l_discount, ps_supplycost, o_orderdate, n_name, _row_id, _row_id, p_partkey, ps_suppkey, ps_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: Like(p_name, '%yellow%':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamFilter { predicate: IsNotNull(ps_suppkey) } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: IsNotNull(l_partkey) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q10 before: - create_tables @@ -765,32 +773,33 @@ LIMIT 20; stream_plan: |- StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } - └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment, Vnode(c_custkey) as $expr2] } - └─StreamHashAgg [append_only] { group_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], aggs: [sum($expr1), count] } - └─StreamProject { exprs: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment, (l_extendedprice * (1.00:Decimal - l_discount)) as $expr1, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, l_extendedprice, l_discount, n_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, _row_id, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_nationkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [l_extendedprice, l_discount, o_custkey, _row_id, l_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamFilter { predicate: (l_returnflag = 'R':Varchar) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_orderkey) } - └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1994-04-01 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment, Vnode(c_custkey) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], aggs: [sum($expr1), count] } + └─StreamProject { exprs: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment, (l_extendedprice * (1.00:Decimal - l_discount)) as $expr1, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, l_extendedprice, l_discount, n_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [l_extendedprice, l_discount, o_custkey, _row_id, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamFilter { predicate: (l_returnflag = 'R':Varchar) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1994-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q11 before: - create_tables @@ -826,50 +835,51 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } - └─StreamProject { exprs: [ps_partkey, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [ps_partkey, sum($expr1), Vnode(ps_partkey) as _vnode] } - └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [ps_partkey, sum($expr1)] } - ├─StreamProject { exprs: [ps_partkey, sum($expr1)] } - │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ └─StreamProject { exprs: [ps_partkey, (ps_supplycost * ps_availqty::Decimal) as $expr1, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } - │ └─StreamShare { id: 11 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_nationkey) } - │ └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } - └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr2)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } - └─StreamProject { exprs: [(ps_supplycost * ps_availqty::Decimal) as $expr2, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } - └─StreamShare { id: 11 } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } - ├─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(n_nationkey) } - └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [ps_partkey, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [ps_partkey, sum($expr1), Vnode(ps_partkey) as _vnode] } + └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [ps_partkey, sum($expr1)] } + ├─StreamProject { exprs: [ps_partkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, (ps_supplycost * ps_availqty::Decimal) as $expr1, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + │ └─StreamShare { id: 11 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } + └─StreamProject { exprs: [(ps_supplycost * ps_availqty::Decimal) as $expr2, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + └─StreamShare { id: 11 } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_nationkey) } + └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q12 before: - create_tables @@ -905,22 +915,23 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } - └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2)] } - └─StreamTopN { order: [l_shipmode ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2), Vnode(l_shipmode) as $expr3] } - └─StreamHashAgg [append_only] { group_key: [l_shipmode], aggs: [sum($expr1), sum($expr2), count] } - └─StreamExchange { dist: HashShard(l_shipmode) } - └─StreamProject { exprs: [l_shipmode, Case(((o_orderpriority = '1-URGENT':Varchar) OR (o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((o_orderpriority <> '1-URGENT':Varchar) AND (o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, _row_id, _row_id, o_orderkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, l_shipmode, _row_id, o_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamFilter { predicate: In(l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (l_commitdate < l_receiptdate) AND (l_shipdate < l_commitdate) AND (l_receiptdate >= '1994-01-01':Date) AND (l_receiptdate < '1995-01-01 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_shipmode) } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2)] } + └─StreamTopN { order: [l_shipmode ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2), Vnode(l_shipmode) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [l_shipmode], aggs: [sum($expr1), sum($expr2), count] } + └─StreamExchange { dist: HashShard(l_shipmode) } + └─StreamProject { exprs: [l_shipmode, Case(((o_orderpriority = '1-URGENT':Varchar) OR (o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((o_orderpriority <> '1-URGENT':Varchar) AND (o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, _row_id, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, l_shipmode, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: In(l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (l_commitdate < l_receiptdate) AND (l_shipdate < l_commitdate) AND (l_receiptdate >= '1994-01-01':Date) AND (l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q13 before: - create_tables @@ -948,24 +959,25 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(o_orderkey), count] } - └─StreamTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [count(o_orderkey), count, Vnode(count(o_orderkey)) as _vnode] } - └─StreamHashAgg { group_key: [count(o_orderkey)], aggs: [count] } - └─StreamExchange { dist: HashShard(count(o_orderkey)) } - └─StreamProject { exprs: [c_custkey, count(o_orderkey)] } - └─StreamHashAgg { group_key: [c_custkey], aggs: [count(o_orderkey), count] } - └─StreamHashJoin { type: LeftOuter, predicate: c_custkey = o_custkey, output: [c_custkey, o_orderkey, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamProject { exprs: [o_orderkey, o_custkey, _row_id] } - └─StreamFilter { predicate: Not(Like(o_comment, '%:1%:2%':Varchar)) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count, count(o_orderkey)) } + └─StreamProject { exprs: [count(o_orderkey), count] } + └─StreamTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [count(o_orderkey), count, Vnode(count(o_orderkey)) as _vnode] } + └─StreamHashAgg { group_key: [count(o_orderkey)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(o_orderkey)) } + └─StreamProject { exprs: [c_custkey, count(o_orderkey)] } + └─StreamHashAgg { group_key: [c_custkey], aggs: [count(o_orderkey), count] } + └─StreamHashJoin { type: LeftOuter, predicate: c_custkey = o_custkey, output: [c_custkey, o_orderkey, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamProject { exprs: [o_orderkey, o_custkey, _row_id] } + └─StreamFilter { predicate: Not(Like(o_comment, '%:1%:2%':Varchar)) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q14 before: - create_tables @@ -1036,39 +1048,40 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, _row_id(hidden)], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } - └─StreamTopN { order: [s_suppkey ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, Vnode(sum($expr1)) as _vnode] } - └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } - ├─StreamExchange { dist: HashShard(sum($expr1)) } - │ └─StreamHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, l_suppkey] } - │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamShare { id: 9 } - │ └─StreamProject { exprs: [l_suppkey, sum($expr1)] } - │ └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } - │ └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } - └─StreamProject { exprs: [max(max(sum($expr1)))] } - └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } - └─StreamProject { exprs: [l_suppkey, sum($expr1), Vnode(l_suppkey) as _vnode] } - └─StreamShare { id: 9 } - └─StreamProject { exprs: [l_suppkey, sum($expr1)] } - └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(l_suppkey) } - └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } - └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + └─StreamTopN { order: [s_suppkey ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, Vnode(sum($expr1)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + ├─StreamExchange { dist: HashShard(sum($expr1)) } + │ └─StreamHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, l_suppkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + │ └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } + └─StreamProject { exprs: [max(max(sum($expr1)))] } + └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } + └─StreamProject { exprs: [l_suppkey, sum($expr1), Vnode(l_suppkey) as _vnode] } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q16 before: - create_tables @@ -1106,28 +1119,29 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } - └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey)] } - └─StreamTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey), Vnode(p_brand, p_type, p_size) as $expr1] } - └─StreamHashAgg { group_key: [p_brand, p_type, p_size], aggs: [count(distinct ps_suppkey), count] } - └─StreamExchange { dist: HashShard(p_brand, p_type, p_size) } - └─StreamHashJoin { type: LeftAnti, predicate: ps_suppkey = s_suppkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, _row_id, ps_partkey] } - ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_partkey = p_partkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, ps_partkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamFilter { predicate: (p_brand <> 'Brand#45':Varchar) AND Not(Like(p_type, 'SMALL PLATED%':Varchar)) AND In(p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(s_suppkey) } - └─StreamProject { exprs: [s_suppkey, _row_id] } - └─StreamFilter { predicate: Like(s_comment, '%Customer%Complaints%':Varchar) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count(distinct ps_suppkey), p_brand, p_type, p_size) } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey)] } + └─StreamTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey), Vnode(p_brand, p_type, p_size) as $expr1] } + └─StreamHashAgg { group_key: [p_brand, p_type, p_size], aggs: [count(distinct ps_suppkey), count] } + └─StreamExchange { dist: HashShard(p_brand, p_type, p_size) } + └─StreamHashJoin { type: LeftAnti, predicate: ps_suppkey = s_suppkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_partkey = p_partkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, ps_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamFilter { predicate: (p_brand <> 'Brand#45':Varchar) AND Not(Like(p_type, 'SMALL PLATED%':Varchar)) AND In(p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamProject { exprs: [s_suppkey, _row_id] } + └─StreamFilter { predicate: Like(s_comment, '%Customer%Complaints%':Varchar) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q17 before: - create_tables @@ -1216,36 +1230,37 @@ LIMIT 100; stream_plan: |- StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)] } - └─StreamTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity), Vnode(o_orderkey) as $expr1] } - └─StreamHashAgg { group_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], aggs: [sum(l_quantity), count] } - └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: all } - ├─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, l_quantity, _row_id, _row_id, _row_id] } - │ ├─StreamExchange { dist: HashShard(o_orderkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(c_custkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 11 } - │ │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(o_custkey) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamShare { id: 9 } - │ └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamProject { exprs: [l_orderkey] } - └─StreamFilter { predicate: (sum(l_quantity) > 1:Decimal) } - └─StreamProject { exprs: [l_orderkey, sum(l_quantity)] } - └─StreamHashAgg [append_only] { group_key: [l_orderkey], aggs: [sum(l_quantity), count] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamShare { id: 9 } - └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_totalprice, o_orderdate) } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)] } + └─StreamTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity), Vnode(o_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], aggs: [sum(l_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: all } + ├─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, l_quantity, _row_id, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [l_orderkey] } + └─StreamFilter { predicate: (sum(l_quantity) > 1:Decimal) } + └─StreamProject { exprs: [l_orderkey, sum(l_quantity)] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey], aggs: [sum(l_quantity), count] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q19 before: - create_tables @@ -1308,42 +1323,43 @@ sql: "select\n\ts_name,\n\ts_address\nfrom\n\tsupplier,\n\tnation\nwhere\n\ts_suppkey in (\n\t\tselect\n\t\t\tps_suppkey\n\t\tfrom\n\t\t\tpartsupp,\n\t\t\t(\n\t\t\t\tselect\n\t\t\t\t\tl_partkey agg_partkey,\n\t\t\t\t\tl_suppkey agg_suppkey,\n\t\t\t\t\t0.5 * sum(l_quantity) AS agg_quantity\n\t\t\t\tfrom\n\t\t\t\t\tlineitem\n\t\t\t\twhere\n\t\t\t\t\tl_shipdate >= date '1994-01-01'\n\t\t\t\t\tand l_shipdate < date '1994-01-01' + interval '1' year\n\t\t\t\tgroup by\n\t\t\t\t\tl_partkey,\n\t\t\t\t\tl_suppkey\n\t\t\t) agg_lineitem\n\t\twhere\n\t\t\tagg_partkey = ps_partkey\n\t\t\tand agg_suppkey = ps_suppkey\n\t\t\tand ps_partkey in (\n\t\t\t\tselect\n\t\t\t\t\tp_partkey\n\t\t\t\tfrom\n\t\t\t\t\tpart\n\t\t\t\twhere\n\t\t\t\t\tp_name like 'forest%'\n\t\t\t)\n\t\t\tand ps_availqty > agg_quantity\n\t)\n\tand s_nationkey = n_nationkey\n\tand n_name = 'KENYA'\norder by\n\ts_name\nLIMIT 1;\n" stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } - └─StreamTopN { order: [s_name ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey, Vnode(s_suppkey) as _vnode] } - └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } - ├─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_nationkey) } - │ └─StreamFilter { predicate: (n_name = 'KENYA':Varchar) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(ps_suppkey) } - └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_suppkey, _row_id, ps_partkey] } - ├─StreamExchange { dist: HashShard(ps_partkey) } - │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, _row_id, l_partkey, l_suppkey] } - │ └─StreamFilter { predicate: ($expr1 > $expr2) } - │ └─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] } - │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamProject { exprs: [l_partkey, l_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - │ └─StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } - │ └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } - │ └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(p_partkey) } - └─StreamProject { exprs: [p_partkey, _row_id] } - └─StreamFilter { predicate: Like(p_name, 'forest%':Varchar) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + └─StreamTopN { order: [s_name ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey, Vnode(s_suppkey) as _vnode] } + └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'KENYA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_suppkey, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, _row_id, l_partkey, l_suppkey] } + │ └─StreamFilter { predicate: ($expr1 > $expr2) } + │ └─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] } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamProject { exprs: [l_partkey, l_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } + │ └─StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } + │ └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamProject { exprs: [p_partkey, _row_id] } + └─StreamFilter { predicate: Like(p_name, 'forest%':Varchar) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q21 before: - create_tables @@ -1390,51 +1406,52 @@ LIMIT 100; stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [s_name, count] } - └─StreamTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [s_name, count, Vnode(s_name) as _vnode] } - └─StreamHashAgg { group_key: [s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } - ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } - │ ├─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } - │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } - │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ │ └─StreamFilter { predicate: (n_name = 'GERMANY':Varchar) } - │ │ │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } - │ │ │ └─StreamFilter { predicate: (o_orderstatus = 'F':Varchar) } - │ │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ └─StreamShare { id: 13 } - │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 19 } - │ │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - │ └─StreamShare { id: 13 } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └─StreamShare { id: 13 } - └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count, s_name) } + └─StreamProject { exprs: [s_name, count] } + └─StreamTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, count, Vnode(s_name) as _vnode] } + └─StreamHashAgg { group_key: [s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + │ ├─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } + │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ │ └─StreamFilter { predicate: (n_name = 'GERMANY':Varchar) } + │ │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ │ └─StreamFilter { predicate: (o_orderstatus = 'F':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + │ │ └─StreamShare { id: 13 } + │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 19 } + │ │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + │ └─StreamShare { id: 13 } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + └─StreamShare { id: 13 } + └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { 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, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q22 before: - create_tables @@ -1479,35 +1496,36 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [cntrycode, numcust, totacctbal], stream_key: [], pk_columns: [cntrycode], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, count, sum(c_acctbal)] } - └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [$expr2, count, sum(c_acctbal), Vnode($expr2) as _vnode] } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(c_acctbal)] } - └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [Substr(c_phone, 1:Int32, 2:Int32) as $expr2, c_acctbal, _row_id, c_custkey] } - └─StreamDynamicFilter { predicate: (c_acctbal > $expr1), output: [c_phone, c_acctbal, _row_id, c_custkey] } - ├─StreamHashJoin { type: LeftAnti, predicate: c_custkey = o_custkey, output: [c_phone, c_acctbal, _row_id, c_custkey] } - │ ├─StreamExchange { dist: HashShard(c_custkey) } - │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - │ │ └─StreamShare { id: 4 } - │ │ └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } - │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 11 } - │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(o_custkey) } - │ └─StreamProject { exprs: [o_custkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(c_acctbal)) / sum0(count(c_acctbal))::Decimal) as $expr1] } - └─StreamSimpleAgg [append_only] { aggs: [sum(sum(c_acctbal)), sum0(count(c_acctbal)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(c_acctbal), count(c_acctbal)] } - └─StreamFilter { predicate: (c_acctbal > 0.00:Decimal) AND In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─StreamShare { id: 4 } - └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } - └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─StreamRowIdGen { row_id_index: 11 } - └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal)] } + └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal), Vnode($expr2) as _vnode] } + └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(c_acctbal)] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [Substr(c_phone, 1:Int32, 2:Int32) as $expr2, c_acctbal, _row_id, c_custkey] } + └─StreamDynamicFilter { predicate: (c_acctbal > $expr1), output: [c_phone, c_acctbal, _row_id, c_custkey] } + ├─StreamHashJoin { type: LeftAnti, predicate: c_custkey = o_custkey, output: [c_phone, c_acctbal, _row_id, c_custkey] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamShare { id: 4 } + │ │ └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamProject { exprs: [o_custkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(c_acctbal)) / sum0(count(c_acctbal))::Decimal) as $expr1] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum(c_acctbal)), sum0(count(c_acctbal)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(c_acctbal), count(c_acctbal)] } + └─StreamFilter { predicate: (c_acctbal > 0.00:Decimal) AND In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamShare { id: 4 } + └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamRowIdGen { row_id_index: 11 } + └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } 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 772ff95b58f9e..0c5612ba8dc3a 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -209,7 +209,7 @@ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- 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) } + └─StreamExchange { dist: HashShard(s_acctbal, n_name, s_name, p_partkey) } └─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] } @@ -275,7 +275,7 @@ Fragment 0 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, 17]) from 1 + └── StreamExchange Hash([0, 2, 1, 3]) from 1 Fragment 1 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 ] } @@ -451,7 +451,7 @@ Table 37 { columns: [ n_regionkey, _row_id, _degree, _rw_timestamp ], 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, min(ps_supplycost), ps_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, _rw_timestamp ], 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 } + 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, _rw_timestamp ], 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: [ 0, 2, 1, 3 ], read pk prefix len hint: 14 } - id: tpch_q5 before: @@ -537,96 +537,100 @@ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [n_name], pk_columns: [revenue, n_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, sum($expr1)] } - └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(n_name) } - └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } - ├─StreamExchange { dist: HashShard(n_nationkey, 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) } - │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } - │ └─StreamRowIdGen { row_id_index: 4 } - │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } - ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(o_custkey) } - │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamRowIdGen { row_id_index: 8 } - │ └─StreamSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } - ├─StreamExchange { dist: HashShard(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] } - └─StreamExchange { dist: HashShard(s_suppkey) } - └─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(sum($expr1)) } + └─StreamProject { exprs: [n_name, sum($expr1)] } + └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(n_name) } + └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } + ├─StreamExchange { dist: HashShard(n_nationkey, 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) } + │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 4 } + │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } + ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(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] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─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] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [n_name, revenue], stream_key: [n_name], pk_columns: [revenue, n_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [n_name, sum($expr1)] } - └── StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([1]) from 1 Fragment 1 + StreamProject { exprs: [n_name, sum($expr1)] } + └── StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0]) from 2 + + Fragment 2 StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } └── StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([0, 0]) from 2 - └── StreamExchange Hash([0, 3]) from 5 + ├── StreamExchange Hash([0, 0]) from 3 + └── StreamExchange Hash([0, 3]) from 6 - Fragment 2 + Fragment 3 StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([2]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2]) from 5 - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 3 } └── StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } { tables: [ Source: 9 ] } - Fragment 4 + Fragment 5 StreamFilter { predicate: IsNotNull(n_nationkey) } └── StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { tables: [ Source: 10 ] } - Fragment 5 + Fragment 6 StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } ├── tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] - ├── StreamExchange Hash([0, 1]) from 6 - └── StreamExchange Hash([0, 3]) from 9 + ├── StreamExchange Hash([0, 1]) from 7 + └── StreamExchange Hash([0, 3]) from 10 - Fragment 6 + Fragment 7 StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } { tables: [ HashJoinLeft: 15, HashJoinDegreeLeft: 16, HashJoinRight: 17, HashJoinDegreeRight: 18 ] } - ├── StreamExchange Hash([1]) from 7 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([1]) from 8 + └── StreamExchange Hash([0]) from 9 - Fragment 7 + Fragment 8 StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } { tables: [ Source: 19 ] } - Fragment 8 + Fragment 9 StreamRowIdGen { row_id_index: 8 } └── StreamSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } { tables: [ Source: 20 ] } - Fragment 9 + Fragment 10 StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } { tables: [ HashJoinLeft: 21, HashJoinDegreeLeft: 22, HashJoinRight: 23, HashJoinDegreeRight: 24 ] } - ├── StreamExchange Hash([2]) from 10 - └── StreamExchange Hash([0]) from 11 + ├── StreamExchange Hash([2]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 10 + Fragment 11 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: 25 ] } - Fragment 11 + Fragment 12 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: 26 ] } @@ -704,7 +708,7 @@ Table 26 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - id: tpch_q7 before: @@ -1676,7 +1680,7 @@ └─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] } 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) } + └─StreamExchange { dist: HashShard(s_name) } └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } ├─StreamExchange { dist: HashShard(s_suppkey) } │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } @@ -1709,7 +1713,7 @@ 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 } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([2, 3, 4, 5]) from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } @@ -1802,7 +1806,7 @@ Table 21 { columns: [ partition_id, offset_info, _rw_timestamp ], 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, _rw_timestamp ], 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 } + Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 5 } - id: tpch_q21 before: @@ -1912,108 +1916,112 @@ └─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] } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamHashAgg { group_key: [s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } - ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } - │ ├─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } - │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } - │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ │ └─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(l_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ └─StreamShare { id: 11 } - │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ │ └─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] } - │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - │ └─StreamShare { id: 11 } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ └─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] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └─StreamShare { id: 11 } - └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - └─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] } + └─StreamExchange { dist: HashShard(count, s_name) } + └─StreamHashAgg { group_key: [s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + │ ├─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } + │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ │ └─StreamRowIdGen { row_id_index: 4 } + │ │ │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ │ └─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(l_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + │ │ └─StreamShare { id: 11 } + │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ │ └─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] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + │ └─StreamShare { id: 11 } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ └─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] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + └─StreamShare { id: 11 } + └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + └─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, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamHashAgg { group_key: [s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamHashAgg { group_key: [s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0]) from 2 + + Fragment 2 StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] ├── StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } │ ├── tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] - │ ├── StreamExchange Hash([1]) from 2 - │ └── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + │ ├── StreamExchange Hash([1]) from 3 + │ └── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 2 + Fragment 3 StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } ├── tables: [ HashJoinLeft: 9, HashJoinDegreeLeft: 10, HashJoinRight: 11, HashJoinDegreeRight: 12 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([1]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([1]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } ├── tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([3]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([3]) from 6 - Fragment 4 + Fragment 5 StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { tables: [ Source: 17 ] } - Fragment 5 + Fragment 6 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: 18 ] } - Fragment 6 + Fragment 7 StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } ├── tables: [ HashJoinLeft: 19, HashJoinDegreeLeft: 20, HashJoinRight: 21, HashJoinDegreeRight: 22 ] - ├── StreamExchange Hash([0]) from 7 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([0]) from 8 + └── StreamExchange Hash([0]) from 9 - Fragment 7 + Fragment 8 StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } { tables: [ Source: 23 ] } - Fragment 8 + Fragment 9 StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └── StreamExchange NoShuffle from 9 + └── StreamExchange NoShuffle from 10 - Fragment 9 + Fragment 10 StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } └── 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: 24 ] - Fragment 10 + Fragment 11 StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - └── StreamExchange NoShuffle from 9 + └── StreamExchange NoShuffle from 10 - Fragment 11 + Fragment 12 StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } └── StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └── StreamExchange NoShuffle from 9 + └── StreamExchange NoShuffle from 10 Table 0 { columns: [ s_name, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2090,5 +2098,5 @@ Table 24 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index d5c5cc2054323..0d49457555494 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -126,6 +126,7 @@ pub fn gen_create_mv_plan_bound( } let mut plan_root = Planner::new_for_stream(context).plan_query(query)?; + plan_root.set_req_dist_as_same_as_req_order(); if let Some(col_names) = col_names { for name in &col_names { check_valid_column_name(name)?; diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index df241d76135a8..d60f435648b3a 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -192,6 +192,18 @@ impl PlanRoot { Ok(()) } + pub fn set_req_dist_as_same_as_req_order(&mut self) { + if self.required_order.len() != 0 { + let dist = self + .required_order + .column_orders + .iter() + .map(|o| o.column_index) + .collect_vec(); + self.required_dist = RequiredDist::hash_shard(&dist) + } + } + /// Get the plan root's schema, only including the fields to be output. pub fn schema(&self) -> Schema { // The schema can be derived from the `out_fields` and `out_names`, so we don't maintain it diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index a13a8f0a313e4..6b9aa1ca43bef 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -180,24 +180,33 @@ impl StreamMaterialize { user_distributed_by } TableType::MaterializedView => { - assert_matches!(user_distributed_by, RequiredDist::Any); - // ensure the same pk will not shuffle to different node - let required_dist = - RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key()); - - // If the input is a stream join, enforce the stream key as the materialized - // view distribution key to avoid slow backfilling caused by - // data skew of the dimension table join key. - // See for more information. - let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join)) - || matches!(input.as_stream_temporal_join(), Some(_join)) - || matches!(input.as_stream_delta_join(), Some(_join)); - - if is_stream_join { - return Ok(required_dist.enforce(input, &Order::any())); + match user_distributed_by { + RequiredDist::PhysicalDist(Distribution::HashShard(_)) => { + user_distributed_by + } + RequiredDist::Any => { + // ensure the same pk will not shuffle to different node + let required_dist = RequiredDist::shard_by_key( + input.schema().len(), + input.expect_stream_key(), + ); + + // If the input is a stream join, enforce the stream key as the materialized + // view distribution key to avoid slow backfilling caused by + // data skew of the dimension table join key. + // See for more information. + let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join)) + || matches!(input.as_stream_temporal_join(), Some(_join)) + || matches!(input.as_stream_delta_join(), Some(_join)); + + if is_stream_join { + return Ok(required_dist.enforce(input, &Order::any())); + } + + required_dist + } + _ => unreachable!("{:?}", user_distributed_by), } - - required_dist } TableType::Index => { assert_matches!( From 5b8660ef977bb3f664cea096a58c8b412b4c0bd5 Mon Sep 17 00:00:00 2001 From: st1page <1245835950@qq.com> Date: Thu, 16 Jan 2025 16:28:14 +0800 Subject: [PATCH 2/4] fix test --- src/tests/simulation/src/nexmark.rs | 2 +- src/tests/simulation/src/nexmark/q105.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/tests/simulation/src/nexmark.rs b/src/tests/simulation/src/nexmark.rs index 7959cf392f232..b8b7ecdf39f59 100644 --- a/src/tests/simulation/src/nexmark.rs +++ b/src/tests/simulation/src/nexmark.rs @@ -219,7 +219,7 @@ pub mod queries { pub mod q105 { use super::*; pub const CREATE: &str = include_str!("nexmark/q105.sql"); - pub const SELECT: &str = "SELECT * FROM nexmark_q105;"; + pub const SELECT: &str = "SELECT * FROM nexmark_q105 ORDER BY bid_count;"; pub const DROP: &str = "DROP MATERIALIZED VIEW nexmark_q105;"; pub const INITIAL_INTERVAL: Duration = DEFAULT_INITIAL_INTERVAL; pub const INITIAL_TIMEOUT: Duration = DEFAULT_INITIAL_TIMEOUT; diff --git a/src/tests/simulation/src/nexmark/q105.sql b/src/tests/simulation/src/nexmark/q105.sql index 9c3e566da440a..4b7d21af968a9 100644 --- a/src/tests/simulation/src/nexmark/q105.sql +++ b/src/tests/simulation/src/nexmark/q105.sql @@ -12,4 +12,4 @@ FROM auction a JOIN bid b ON a.id = b.auction GROUP BY a.id, a.item_name ORDER BY bid_count DESC -LIMIT 1000; +LIMIT 1000 WITH TIES; From cba2e46068992359984881fbcf56bd826d2e6be9 Mon Sep 17 00:00:00 2001 From: st1page <1245835950@qq.com> Date: Fri, 17 Jan 2025 13:06:38 +0800 Subject: [PATCH 3/4] fix --- src/tests/simulation/src/nexmark/q105.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tests/simulation/src/nexmark/q105.sql b/src/tests/simulation/src/nexmark/q105.sql index 4b7d21af968a9..9c3e566da440a 100644 --- a/src/tests/simulation/src/nexmark/q105.sql +++ b/src/tests/simulation/src/nexmark/q105.sql @@ -12,4 +12,4 @@ FROM auction a JOIN bid b ON a.id = b.auction GROUP BY a.id, a.item_name ORDER BY bid_count DESC -LIMIT 1000 WITH TIES; +LIMIT 1000; From e1d75deb947399b2020517e85f3040d7737369d5 Mon Sep 17 00:00:00 2001 From: st1page <1245835950@qq.com> Date: Fri, 17 Jan 2025 15:40:13 +0800 Subject: [PATCH 4/4] fix --- src/tests/simulation/src/nexmark.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/tests/simulation/src/nexmark.rs b/src/tests/simulation/src/nexmark.rs index b8b7ecdf39f59..814f751733ada 100644 --- a/src/tests/simulation/src/nexmark.rs +++ b/src/tests/simulation/src/nexmark.rs @@ -219,7 +219,7 @@ pub mod queries { pub mod q105 { use super::*; pub const CREATE: &str = include_str!("nexmark/q105.sql"); - pub const SELECT: &str = "SELECT * FROM nexmark_q105 ORDER BY bid_count;"; + pub const SELECT: &str = "SELECT * FROM nexmark_q105 ORDER BY 1,2,3;"; pub const DROP: &str = "DROP MATERIALIZED VIEW nexmark_q105;"; pub const INITIAL_INTERVAL: Duration = DEFAULT_INITIAL_INTERVAL; pub const INITIAL_TIMEOUT: Duration = DEFAULT_INITIAL_TIMEOUT;