From 2b6ebcd3b09ccec4285c690efa9c283794222eff Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Tue, 28 May 2024 17:32:42 +0800 Subject: [PATCH 1/4] improve scalar subquery optimization time --- .../input/subquery_expr_correlated.yaml | 46 + .../output/subquery_expr_correlated.yaml | 976 ++++++++++++++++++ .../testdata/output/with_ordinality.yaml | 43 +- .../src/optimizer/logical_optimization.rs | 4 +- .../src/optimizer/plan_node/logical_apply.rs | 18 +- .../rule/apply_agg_transpose_rule.rs | 4 +- .../rule/apply_dedup_transpose_rule.rs | 5 +- .../rule/apply_expand_transpose_rule.rs | 5 +- .../rule/apply_hop_window_transpose_rule.rs | 5 +- .../rule/apply_limit_transpose_rule.rs | 5 +- .../rule/apply_over_window_transpose_rule.rs | 5 +- .../rule/apply_topn_transpose_rule.rs | 5 +- .../optimizer/rule/translate_apply_rule.rs | 146 ++- 13 files changed, 1195 insertions(+), 72 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml index 5ba9930be99b..679ffe51e31a 100644 --- a/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml @@ -508,3 +508,49 @@ expected_outputs: - batch_plan - stream_plan +- name: improve multi scalar subqueries optimization time. issue 16952. case 1. + sql: | + create table t1(a int, b int); + create table t2(c int primary key, d int); + select + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col1, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col2, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col3, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col4, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col5, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col6, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col7, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col8, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col9, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col10, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col11, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col12, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col13, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col14, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col15, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col16, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col17, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col18 + from t1; + expected_outputs: + - batch_plan + - stream_plan +- name: improve multi scalar subqueries optimization time. issue 16952. case 2. + sql: | + create table t1(a int, b int); + create table t2(c int primary key, d int); + create table t3(e int, f int); + create table t4(g int, h int); + create table t5(i int, j int); + create table t6(k int, l int); + select + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col1, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col2, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col3, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col4, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col5, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col6 + from t1; + expected_outputs: + - batch_plan + - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 762f4bea8fe5..9798b11cafc3 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -1132,3 +1132,979 @@ └─StreamHashAgg { group_key: [t2.d], aggs: [sum(t2.c), count(t2.c), count] } └─StreamExchange { dist: HashShard(t2.d) } └─StreamTableScan { table: t2, columns: [t2.c, t2.d, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } +- name: improve multi scalar subqueries optimization time. issue 16952. case 1. + sql: | + create table t1(a int, b int); + create table t2(c int primary key, d int); + select + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col1, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col2, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col3, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col4, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col5, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col6, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col7, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col8, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col9, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col10, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col11, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col12, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col13, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col14, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col15, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col16, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col17, + COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col18 + from t1; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [Coalesce(t1.b, 0:Int32) as $expr1, Coalesce(t1.b, 0:Int32) as $expr2, Coalesce(t1.b, 0:Int32) as $expr3, Coalesce(t1.b, 0:Int32) as $expr4, Coalesce(t1.b, 0:Int32) as $expr5, Coalesce(t1.b, 0:Int32) as $expr6, Coalesce(t1.b, 0:Int32) as $expr7, Coalesce(t1.b, 0:Int32) as $expr8, Coalesce(t1.b, 0:Int32) as $expr9, Coalesce(t1.b, 0:Int32) as $expr10, Coalesce(t1.b, 0:Int32) as $expr11, Coalesce(t1.b, 0:Int32) as $expr12, Coalesce(t1.b, 0:Int32) as $expr13, Coalesce(t1.b, 0:Int32) as $expr14, Coalesce(t1.b, 0:Int32) as $expr15, Coalesce(t1.b, 0:Int32) as $expr16, Coalesce(t1.b, 0:Int32) as $expr17, Coalesce(t1.b, 0:Int32) as $expr18] } + └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + └─BatchProject { exprs: [t1.a, t1.b, t1.b] } + └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: [t1.a, t1.b], lookup table: t2 } + └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [col1, col2, col3, col4, col5, col6, col7, col8, col9, col10, col11, col12, col13, col14, col15, col16, col17, col18, t1._row_id(hidden), t1.b(hidden), t1.a(hidden), t1.b#1(hidden), t1.b#2(hidden), t1.b#3(hidden), t1.b#4(hidden), t1.b#5(hidden), t1.b#6(hidden), t1.b#7(hidden), t1.b#8(hidden), t1.b#9(hidden), t1.b#10(hidden), t1.b#11(hidden), t1.b#12(hidden), t1.b#13(hidden), t1.b#14(hidden), t1.b#15(hidden), t1.b#16(hidden), t1.b#17(hidden), t1.b#18(hidden)], stream_key: [t1._row_id, t1.b, t1.a, t1.b#1, t1.b#2, t1.b#3, t1.b#4, t1.b#5, t1.b#6, t1.b#7, t1.b#8, t1.b#9, t1.b#10, t1.b#11, t1.b#12, t1.b#13, t1.b#14, t1.b#15, t1.b#16, t1.b#17, t1.b#18], pk_columns: [t1._row_id, t1.b, t1.a, t1.b#1, t1.b#2, t1.b#3, t1.b#4, t1.b#5, t1.b#6, t1.b#7, t1.b#8, t1.b#9, t1.b#10, t1.b#11, t1.b#12, t1.b#13, t1.b#14, t1.b#15, t1.b#16, t1.b#17, t1.b#18], pk_conflict: NoCheck } + └─StreamProject { exprs: [Coalesce(t1.b, 0:Int32) as $expr1, Coalesce(t1.b, 0:Int32) as $expr2, Coalesce(t1.b, 0:Int32) as $expr3, Coalesce(t1.b, 0:Int32) as $expr4, Coalesce(t1.b, 0:Int32) as $expr5, Coalesce(t1.b, 0:Int32) as $expr6, Coalesce(t1.b, 0:Int32) as $expr7, Coalesce(t1.b, 0:Int32) as $expr8, Coalesce(t1.b, 0:Int32) as $expr9, Coalesce(t1.b, 0:Int32) as $expr10, Coalesce(t1.b, 0:Int32) as $expr11, Coalesce(t1.b, 0:Int32) as $expr12, Coalesce(t1.b, 0:Int32) as $expr13, Coalesce(t1.b, 0:Int32) as $expr14, Coalesce(t1.b, 0:Int32) as $expr15, Coalesce(t1.b, 0:Int32) as $expr16, Coalesce(t1.b, 0:Int32) as $expr17, Coalesce(t1.b, 0:Int32) as $expr18, t1._row_id, t1.b, t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b] } + └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a, t1.b, t1.a] } + ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t1.b, t1._row_id, t1.a] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t2.c) } + │ └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + └─StreamProject { exprs: [t1.a, t1.b, t1.b] } + └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + ├─StreamExchange { dist: HashShard(t1.a) } + │ └─StreamProject { exprs: [t1.a, t1.b] } + │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t2.c) } + └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } +- name: improve multi scalar subqueries optimization time. issue 16952. case 2. + sql: "create table t1(a int, b int);\ncreate table t2(c int primary key, d int);\ncreate table t3(e int, f int);\ncreate table t4(g int, h int);\ncreate table t5(i int, j int);\ncreate table t6(k int, l int);\nselect \nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col1,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col2,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col3,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col4,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col5,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col6\nfrom t1;\n" + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [Coalesce(sum(t2.d), 0:Int64) as $expr1, Coalesce(sum(t2.d), 0:Int64) as $expr2, Coalesce(sum(t2.d), 0:Int64) as $expr3, Coalesce(sum(t2.d), 0:Int64) as $expr4, Coalesce(sum(t2.d), 0:Int64) as $expr5, Coalesce(sum(t2.d), 0:Int64) as $expr6] } + └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d)] } + ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d)] } + │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d)] } + │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d), sum(t2.d)] } + │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d)] } + │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d)] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d)] } + │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j] } + │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: all, lookup table: t2 } + │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f] } + │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ │ │ │ └─BatchScan { table: t3, columns: [t3.e, t3.f], distribution: SomeShard } + │ │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h] } + │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t4.g, t4.h) } + │ │ │ │ │ │ │ └─BatchScan { table: t4, columns: [t4.g, t4.h], distribution: SomeShard } + │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j] } + │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t5.i, t5.j) } + │ │ │ │ │ │ └─BatchScan { table: t5, columns: [t5.i, t5.j], distribution: SomeShard } + │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t6.l) } + │ │ │ │ │ └─BatchScan { table: t6, columns: [t6.k, t6.l], distribution: SomeShard } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d)] } + │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ └─BatchHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j] } + │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: all, lookup table: t2 } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f] } + │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ │ │ └─BatchScan { table: t3, columns: [t3.e, t3.f], distribution: SomeShard } + │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h] } + │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t4.g, t4.h) } + │ │ │ │ │ │ └─BatchScan { table: t4, columns: [t4.g, t4.h], distribution: SomeShard } + │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t5.i, t5.j) } + │ │ │ │ │ └─BatchScan { table: t5, columns: [t5.i, t5.j], distribution: SomeShard } + │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t6.l) } + │ │ │ │ └─BatchScan { table: t6, columns: [t6.k, t6.l], distribution: SomeShard } + │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d)] } + │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ └─BatchHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j] } + │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: all, lookup table: t2 } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f] } + │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ │ └─BatchScan { table: t3, columns: [t3.e, t3.f], distribution: SomeShard } + │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t4.g, t4.h) } + │ │ │ │ │ └─BatchScan { table: t4, columns: [t4.g, t4.h], distribution: SomeShard } + │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t5.i, t5.j) } + │ │ │ │ └─BatchScan { table: t5, columns: [t5.i, t5.j], distribution: SomeShard } + │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t6.l) } + │ │ │ └─BatchScan { table: t6, columns: [t6.k, t6.l], distribution: SomeShard } + │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d)] } + │ │ └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ └─BatchHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j] } + │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: all, lookup table: t2 } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ └─BatchScan { table: t3, columns: [t3.e, t3.f], distribution: SomeShard } + │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t4.g, t4.h) } + │ │ │ │ └─BatchScan { table: t4, columns: [t4.g, t4.h], distribution: SomeShard } + │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t5.i, t5.j) } + │ │ │ └─BatchScan { table: t5, columns: [t5.i, t5.j], distribution: SomeShard } + │ │ └─BatchHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k] } + │ │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ └─BatchExchange { order: [], dist: HashShard(t6.l) } + │ │ └─BatchScan { table: t6, columns: [t6.k, t6.l], distribution: SomeShard } + │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d)] } + │ └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ └─BatchHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ └─BatchHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j] } + │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: all, lookup table: t2 } + │ │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t3.e, t3.f) } + │ │ │ │ └─BatchScan { table: t3, columns: [t3.e, t3.f], distribution: SomeShard } + │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t4.g, t4.h) } + │ │ │ └─BatchScan { table: t4, columns: [t4.g, t4.h], distribution: SomeShard } + │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j] } + │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ └─BatchExchange { order: [], dist: HashShard(t5.i, t5.j) } + │ │ └─BatchScan { table: t5, columns: [t5.i, t5.j], distribution: SomeShard } + │ └─BatchHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k] } + │ ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ └─BatchExchange { order: [], dist: HashShard(t6.l) } + │ └─BatchScan { table: t6, columns: [t6.k, t6.l], distribution: SomeShard } + └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d)] } + └─BatchHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + └─BatchHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d] } + ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ └─BatchHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j] } + │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ ├─BatchHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─BatchLookupJoin { type: Inner, predicate: t1.a = t2.c, output: all, lookup table: t2 } + │ │ │ │ └─BatchExchange { order: [], dist: UpstreamHashShard(t1.a) } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f] } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t3.e, t3.f) } + │ │ │ └─BatchScan { table: t3, columns: [t3.e, t3.f], distribution: SomeShard } + │ │ └─BatchHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h] } + │ │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ │ └─BatchExchange { order: [], dist: HashShard(t4.g, t4.h) } + │ │ └─BatchScan { table: t4, columns: [t4.g, t4.h], distribution: SomeShard } + │ └─BatchHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j] } + │ ├─BatchExchange { order: [], dist: HashShard(t1.a, t1.a) } + │ │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + │ └─BatchExchange { order: [], dist: HashShard(t5.i, t5.j) } + │ └─BatchScan { table: t5, columns: [t5.i, t5.j], distribution: SomeShard } + └─BatchHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k] } + ├─BatchExchange { order: [], dist: HashShard(t1.b) } + │ └─BatchHashAgg { group_key: [t1.a, t1.b], aggs: [] } + │ └─BatchExchange { order: [], dist: HashShard(t1.a, t1.b) } + │ └─BatchScan { table: t1, columns: [t1.a, t1.b], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(t6.l) } + └─BatchScan { table: t6, columns: [t6.k, t6.l], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [col1, col2, col3, col4, col5, col6, t1._row_id(hidden), t1.a(hidden), t1.b(hidden)], stream_key: [t1._row_id, t1.a, t1.b], pk_columns: [t1._row_id, t1.a, t1.b], pk_conflict: NoCheck } + └─StreamProject { exprs: [Coalesce(sum(t2.d), 0:Int64) as $expr1, Coalesce(sum(t2.d), 0:Int64) as $expr2, Coalesce(sum(t2.d), 0:Int64) as $expr3, Coalesce(sum(t2.d), 0:Int64) as $expr4, Coalesce(sum(t2.d), 0:Int64) as $expr5, Coalesce(sum(t2.d), 0:Int64) as $expr6, t1._row_id, t1.a, t1.b] } + └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), t1._row_id, t1.a, t1.b, t1.a, t1.b] } + ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), t1._row_id, t1.a, t1.b] } + │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d), sum(t2.d), sum(t2.d), t1._row_id, t1.a, t1.b] } + │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d), sum(t2.d), t1._row_id, t1.a, t1.b] } + │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), sum(t2.d), t1._row_id, t1.a, t1.b] } + │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, sum(t2.d), t1._row_id, t1.a, t1.b] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, sum(t2.d)] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } + │ │ │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j, t3._row_id, t2.c, t4._row_id, t1.a, t1.b, t5._row_id] } + │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t3._row_id, t1.a, t1.b, t4._row_id] } + │ │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t1.a, t1.b, t3._row_id] } + │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } + │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } + │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } + │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t4.g, t4.h) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } + │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } + │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t5.i, t5.j) } + │ │ │ │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } + │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t6.l) } + │ │ │ │ │ └─StreamTableScan { table: t6, columns: [t6.k, t6.l, t6._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t6._row_id], pk: [_row_id], dist: UpstreamHashShard(t6._row_id) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b, sum(t2.d)] } + │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } + │ │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ └─StreamHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j, t3._row_id, t2.c, t4._row_id, t1.a, t1.b, t5._row_id] } + │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t3._row_id, t1.a, t1.b, t4._row_id] } + │ │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t1.a, t1.b, t3._row_id] } + │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } + │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } + │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } + │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t4.g, t4.h) } + │ │ │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } + │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t5.i, t5.j) } + │ │ │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } + │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ └─StreamExchange { dist: HashShard(t6.l) } + │ │ │ │ └─StreamTableScan { table: t6, columns: [t6.k, t6.l, t6._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t6._row_id], pk: [_row_id], dist: UpstreamHashShard(t6._row_id) } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b, sum(t2.d)] } + │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } + │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ └─StreamHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j, t3._row_id, t2.c, t4._row_id, t1.a, t1.b, t5._row_id] } + │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t3._row_id, t1.a, t1.b, t4._row_id] } + │ │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t1.a, t1.b, t3._row_id] } + │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } + │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } + │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t4.g, t4.h) } + │ │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } + │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ └─StreamExchange { dist: HashShard(t5.i, t5.j) } + │ │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ └─StreamExchange { dist: HashShard(t6.l) } + │ │ │ └─StreamTableScan { table: t6, columns: [t6.k, t6.l, t6._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t6._row_id], pk: [_row_id], dist: UpstreamHashShard(t6._row_id) } + │ │ └─StreamProject { exprs: [t1.a, t1.b, sum(t2.d)] } + │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } + │ │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ └─StreamHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j, t3._row_id, t2.c, t4._row_id, t1.a, t1.b, t5._row_id] } + │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t3._row_id, t1.a, t1.b, t4._row_id] } + │ │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t1.a, t1.b, t3._row_id] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t3.e, t3.f) } + │ │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } + │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ └─StreamExchange { dist: HashShard(t4.g, t4.h) } + │ │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ └─StreamExchange { dist: HashShard(t5.i, t5.j) } + │ │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } + │ │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } + │ │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: HashShard(t6.l) } + │ │ └─StreamTableScan { table: t6, columns: [t6.k, t6.l, t6._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t6._row_id], pk: [_row_id], dist: UpstreamHashShard(t6._row_id) } + │ └─StreamProject { exprs: [t1.a, t1.b, sum(t2.d)] } + │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } + │ └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ └─StreamHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j, t3._row_id, t2.c, t4._row_id, t1.a, t1.b, t5._row_id] } + │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t3._row_id, t1.a, t1.b, t4._row_id] } + │ │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t1.a, t1.b, t3._row_id] } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ └─StreamExchange { dist: HashShard(t3.e, t3.f) } + │ │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ └─StreamExchange { dist: HashShard(t4.g, t4.h) } + │ │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } + │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } + │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: HashShard(t5.i, t5.j) } + │ │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } + │ └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.b) } + │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t6.l) } + │ └─StreamTableScan { table: t6, columns: [t6.k, t6.l, t6._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t6._row_id], pk: [_row_id], dist: UpstreamHashShard(t6._row_id) } + └─StreamProject { exprs: [t1.a, t1.b, sum(t2.d)] } + └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [sum(t2.d), count] } + └─StreamHashJoin { type: LeftOuter, predicate: t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t1.a, t1.b, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + ├─StreamExchange { dist: HashShard(t1.b) } + │ └─StreamProject { exprs: [t1.a, t1.b] } + │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: t5.j = t6.k AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t3._row_id, t2.c, t4._row_id, t5._row_id, t5.j] } + ├─StreamExchange { dist: HashShard(t1.b) } + │ └─StreamHashJoin { type: LeftOuter, predicate: t2.c = t5.j AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.d, t5.j, t3._row_id, t2.c, t4._row_id, t1.a, t1.b, t5._row_id] } + │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t4.h AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t3._row_id, t1.a, t1.b, t4._row_id] } + │ │ ├─StreamHashJoin { type: LeftOuter, predicate: t2.c = t3.f AND t1.a IS NOT DISTINCT FROM t1.a AND t1.b IS NOT DISTINCT FROM t1.b, output: [t1.a, t1.b, t2.c, t2.d, t1.a, t1.b, t3._row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t2.c, output: all } + │ │ │ │ ├─StreamExchange { dist: HashShard(t1.a) } + │ │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ │ └─StreamExchange { dist: HashShard(t2.c) } + │ │ │ │ └─StreamTableScan { table: t2, columns: [t2.c, t2.d], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t3.e AND t1.a = t3.f, output: [t1.a, t1.b, t3.f, t3._row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ │ └─StreamExchange { dist: HashShard(t3.e, t3.f) } + │ │ │ └─StreamTableScan { table: t3, columns: [t3.e, t3.f, t3._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t3._row_id], pk: [_row_id], dist: UpstreamHashShard(t3._row_id) } + │ │ └─StreamHashJoin { type: Inner, predicate: t1.a = t4.g AND t1.a = t4.h, output: [t1.a, t1.b, t4.h, t4._row_id] } + │ │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ │ └─StreamExchange { dist: HashShard(t4.g, t4.h) } + │ │ └─StreamTableScan { table: t4, columns: [t4.g, t4.h, t4._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t4._row_id], pk: [_row_id], dist: UpstreamHashShard(t4._row_id) } + │ └─StreamHashJoin { type: Inner, predicate: t1.a = t5.i AND t1.a = t5.j, output: [t1.a, t1.b, t5.j, t5._row_id] } + │ ├─StreamExchange { dist: HashShard(t1.a, t1.a) } + │ │ └─StreamProject { exprs: [t1.a, t1.b] } + │ │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + │ └─StreamExchange { dist: HashShard(t5.i, t5.j) } + │ └─StreamTableScan { table: t5, columns: [t5.i, t5.j, t5._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t5._row_id], pk: [_row_id], dist: UpstreamHashShard(t5._row_id) } + └─StreamHashJoin { type: Inner, predicate: t1.b = t6.l, output: [t1.a, t1.b, t6.k, t6._row_id] } + ├─StreamExchange { dist: HashShard(t1.b) } + │ └─StreamProject { exprs: [t1.a, t1.b] } + │ └─StreamHashAgg { group_key: [t1.a, t1.b], aggs: [count] } + │ └─StreamExchange { dist: HashShard(t1.a, t1.b) } + │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t6.l) } + └─StreamTableScan { table: t6, columns: [t6.k, t6.l, t6._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t6._row_id], pk: [_row_id], dist: UpstreamHashShard(t6._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml index 37a68dba7f36..be7c3a19a566 100644 --- a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml +++ b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml @@ -151,40 +151,25 @@ │ └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } └─BatchProjectSet { select_list: [$0, Unnest($0)] } └─BatchHashAgg { group_key: [t.arr], aggs: [] } - └─BatchHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.arr] } - ├─BatchExchange { order: [], dist: HashShard(t.arr) } - │ └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } - └─BatchProjectSet { select_list: [$0, Unnest($0)] } - └─BatchHashAgg { group_key: [t.arr], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(t.arr) } - └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(t.arr) } + └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, arr, unnest, ordinality, arr_2, ordinality_2, t._row_id(hidden), projected_row_id(hidden), projected_row_id#1(hidden)], stream_key: [t._row_id, projected_row_id, arr, projected_row_id#1], pk_columns: [t._row_id, projected_row_id, arr, projected_row_id#1], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), $expr1, Unnest($0), (projected_row_id + 1:Int64) as $expr2, t._row_id, projected_row_id, projected_row_id] } - └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), $expr1, projected_row_id, t.arr, Unnest($0), t._row_id, projected_row_id] } - ├─StreamShare { id: 8 } - │ └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } - │ └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } - │ ├─StreamExchange { dist: HashShard(t.arr) } - │ │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - │ └─StreamProjectSet { select_list: [$0, Unnest($0)] } - │ └─StreamProject { exprs: [t.arr] } - │ └─StreamHashAgg { group_key: [t.arr], aggs: [count] } - │ └─StreamExchange { dist: HashShard(t.arr) } - │ └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, Unnest($0), (projected_row_id + 1:Int64) as $expr2, t._row_id, projected_row_id, projected_row_id] } + └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), projected_row_id, t.arr, Unnest($0), t._row_id] } + ├─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } + │ ├─StreamExchange { dist: HashShard(t.arr) } + │ │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + │ └─StreamProjectSet { select_list: [$0, Unnest($0)] } + │ └─StreamProject { exprs: [t.arr] } + │ └─StreamHashAgg { group_key: [t.arr], aggs: [count] } + │ └─StreamExchange { dist: HashShard(t.arr) } + │ └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } └─StreamProjectSet { select_list: [$0, Unnest($0)] } └─StreamProject { exprs: [t.arr] } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } - └─StreamShare { id: 8 } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } - └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } - ├─StreamExchange { dist: HashShard(t.arr) } - │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - └─StreamProjectSet { select_list: [$0, Unnest($0)] } - └─StreamProject { exprs: [t.arr] } - └─StreamHashAgg { group_key: [t.arr], aggs: [count] } - └─StreamExchange { dist: HashShard(t.arr) } - └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.arr) } + └─StreamTableScan { table: t, columns: [t.arr, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - sql: | select * from abs(1) WITH ORDINALITY; batch_plan: |- diff --git a/src/frontend/src/optimizer/logical_optimization.rs b/src/frontend/src/optimizer/logical_optimization.rs index 1c73245a0afc..d452626bb941 100644 --- a/src/frontend/src/optimizer/logical_optimization.rs +++ b/src/frontend/src/optimizer/logical_optimization.rs @@ -172,7 +172,7 @@ static GENERAL_UNNESTING_TRANS_APPLY_WITH_SHARE: LazyLock = // can't handle a join with `output_indices`. ProjectJoinSeparateRule::create(), ], - ApplyOrder::BottomUp, + ApplyOrder::TopDown, ) }); @@ -186,7 +186,7 @@ static GENERAL_UNNESTING_TRANS_APPLY_WITHOUT_SHARE: LazyLock // can't handle a join with `output_indices`. ProjectJoinSeparateRule::create(), ], - ApplyOrder::BottomUp, + ApplyOrder::TopDown, ) }); diff --git a/src/frontend/src/optimizer/plan_node/logical_apply.rs b/src/frontend/src/optimizer/plan_node/logical_apply.rs index b7a564ee3298..9b11927a1c4e 100644 --- a/src/frontend/src/optimizer/plan_node/logical_apply.rs +++ b/src/frontend/src/optimizer/plan_node/logical_apply.rs @@ -53,6 +53,10 @@ pub struct LogicalApply { /// Whether we require the subquery to produce at most one row. If `true`, we have to report an /// error if the subquery produces more than one row. max_one_row: bool, + + /// An apply has been translated by `translate_apply()`, so we should not translate it in `translate_apply_rule` again. + /// This flag is used to avoid infinite loop in General Unnesting(Translate Apply), since we use a top-down apply order instead of bottom-up to improve the multi-scalar subqueries optimization time. + translated: bool, } impl Distill for LogicalApply { @@ -85,6 +89,7 @@ impl LogicalApply { correlated_id: CorrelatedId, correlated_indices: Vec, max_one_row: bool, + translated: bool, ) -> Self { let ctx = left.ctx(); let join_core = generic::Join::with_full_output(left, right, join_type, on); @@ -105,6 +110,7 @@ impl LogicalApply { correlated_id, correlated_indices, max_one_row, + translated, } } @@ -125,6 +131,7 @@ impl LogicalApply { correlated_id, correlated_indices, max_one_row, + false, ) .into() } @@ -164,6 +171,10 @@ impl LogicalApply { self.correlated_indices.to_owned() } + pub fn translated(&self) -> bool { + self.translated + } + pub fn max_one_row(&self) -> bool { self.max_one_row } @@ -202,7 +213,7 @@ impl LogicalApply { let apply_left_len = apply_left.schema().len(); let correlated_indices_len = correlated_indices.len(); - let new_apply = LogicalApply::create( + let new_apply = LogicalApply::new( domain, apply_right, JoinType::Inner, @@ -210,7 +221,9 @@ impl LogicalApply { correlated_id, correlated_indices, max_one_row, - ); + true, + ) + .into(); let on = Self::rewrite_on(on, correlated_indices_len, apply_left_len).and(Condition { conjunctions: eq_predicates, @@ -285,6 +298,7 @@ impl PlanTreeNodeBinary for LogicalApply { self.correlated_id, self.correlated_indices.clone(), self.max_one_row, + self.translated, ) } } diff --git a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs index 3ac1412d1360..48c3366d1d82 100644 --- a/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_agg_transpose_rule.rs @@ -106,10 +106,11 @@ impl Rule for ApplyAggTransposeRule { correlated_id, correlated_indices.clone(), false, + false, ) .translate_apply(left, eq_predicates) } else { - LogicalApply::new( + LogicalApply::create( left, input, JoinType::Inner, @@ -118,7 +119,6 @@ impl Rule for ApplyAggTransposeRule { correlated_indices.clone(), false, ) - .into() }; let group_agg = { diff --git a/src/frontend/src/optimizer/rule/apply_dedup_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_dedup_transpose_rule.rs index 798744168c42..1b793cddd281 100644 --- a/src/frontend/src/optimizer/rule/apply_dedup_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_dedup_transpose_rule.rs @@ -57,7 +57,7 @@ impl Rule for ApplyDedupTransposeRule { return None; } - let new_apply = LogicalApply::new( + let new_apply = LogicalApply::create( left, dedup_input, JoinType::Inner, @@ -65,8 +65,7 @@ impl Rule for ApplyDedupTransposeRule { correlated_id, correlated_indices, false, - ) - .into(); + ); let new_dedup = { let mut new_dedup_cols: Vec = (0..apply_left_len).collect(); diff --git a/src/frontend/src/optimizer/rule/apply_expand_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_expand_transpose_rule.rs index 694fc803f87f..9f03d9e42098 100644 --- a/src/frontend/src/optimizer/rule/apply_expand_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_expand_transpose_rule.rs @@ -61,7 +61,7 @@ impl Rule for ApplyExpandTransposeRule { return None; } - let new_apply: PlanRef = LogicalApply::new( + let new_apply: PlanRef = LogicalApply::create( left, expand_input, JoinType::Inner, @@ -69,8 +69,7 @@ impl Rule for ApplyExpandTransposeRule { correlated_id, correlated_indices, false, - ) - .into(); + ); let new_apply_schema_len = new_apply.schema().len(); diff --git a/src/frontend/src/optimizer/rule/apply_hop_window_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_hop_window_transpose_rule.rs index b3d5d0841bdc..634a8c6feb45 100644 --- a/src/frontend/src/optimizer/rule/apply_hop_window_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_hop_window_transpose_rule.rs @@ -62,7 +62,7 @@ impl Rule for ApplyHopWindowTransposeRule { return None; } - let new_apply = LogicalApply::new( + let new_apply = LogicalApply::create( left, hop_window_input, JoinType::Inner, @@ -70,8 +70,7 @@ impl Rule for ApplyHopWindowTransposeRule { correlated_id, correlated_indices, false, - ) - .into(); + ); let new_hop_window = LogicalHopWindow::create( new_apply, diff --git a/src/frontend/src/optimizer/rule/apply_limit_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_limit_transpose_rule.rs index 8dc01c195cee..56395ed61a6b 100644 --- a/src/frontend/src/optimizer/rule/apply_limit_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_limit_transpose_rule.rs @@ -63,7 +63,7 @@ impl Rule for ApplyLimitTransposeRule { return None; } - let new_apply = LogicalApply::new( + let new_apply = LogicalApply::create( left, limit_input, JoinType::Inner, @@ -71,8 +71,7 @@ impl Rule for ApplyLimitTransposeRule { correlated_id, correlated_indices, false, - ) - .into(); + ); let new_topn = { // use the first column as an order to provide determinism for streaming queries. diff --git a/src/frontend/src/optimizer/rule/apply_over_window_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_over_window_transpose_rule.rs index 34ae1902b4d2..bb848561ce0b 100644 --- a/src/frontend/src/optimizer/rule/apply_over_window_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_over_window_transpose_rule.rs @@ -58,7 +58,7 @@ impl Rule for ApplyOverWindowTransposeRule { let apply_left_len = left.schema().len(); let apply_left_schema = left.schema().clone(); - let new_apply = LogicalApply::new( + let new_apply = LogicalApply::create( left, window_input, JoinType::Inner, @@ -66,8 +66,7 @@ impl Rule for ApplyOverWindowTransposeRule { correlated_id, correlated_indices, false, - ) - .into(); + ); let new_over_window = { // Shift index of window functions' `InputRef` with `apply_left_len`. diff --git a/src/frontend/src/optimizer/rule/apply_topn_transpose_rule.rs b/src/frontend/src/optimizer/rule/apply_topn_transpose_rule.rs index 61b887af7ea5..9a6884b33e9a 100644 --- a/src/frontend/src/optimizer/rule/apply_topn_transpose_rule.rs +++ b/src/frontend/src/optimizer/rule/apply_topn_transpose_rule.rs @@ -57,7 +57,7 @@ impl Rule for ApplyTopNTransposeRule { return None; } - let new_apply = LogicalApply::new( + let new_apply = LogicalApply::create( left, topn_input, JoinType::Inner, @@ -65,8 +65,7 @@ impl Rule for ApplyTopNTransposeRule { correlated_id, correlated_indices, false, - ) - .into(); + ); let new_topn = { // shift index of topn's `InputRef` with `apply_left_len`. diff --git a/src/frontend/src/optimizer/rule/translate_apply_rule.rs b/src/frontend/src/optimizer/rule/translate_apply_rule.rs index 53a656fa296a..f46aa9656512 100644 --- a/src/frontend/src/optimizer/rule/translate_apply_rule.rs +++ b/src/frontend/src/optimizer/rule/translate_apply_rule.rs @@ -54,6 +54,9 @@ pub struct TranslateApplyRule { impl Rule for TranslateApplyRule { fn apply(&self, plan: PlanRef) -> Option { let apply: &LogicalApply = plan.as_logical_apply()?; + if apply.translated() { + return None; + } let mut left: PlanRef = apply.left(); let right: PlanRef = apply.right(); let apply_left_len = left.schema().len(); @@ -167,6 +170,15 @@ impl TranslateApplyRule { data_types, index, ) + } else if let Some(apply) = plan.as_logical_apply() { + Self::rewrite_apply( + apply, + correlated_indices, + offset, + index_mapping, + data_types, + index, + ) } else if let Some(scan) = plan.as_logical_scan() { Self::rewrite_scan( scan, @@ -199,18 +211,6 @@ impl TranslateApplyRule { data_types: &mut HashMap, index: &mut usize, ) -> Option { - // Only accept join which doesn't generate null columns. - if !matches!( - join.join_type(), - JoinType::Inner - | JoinType::LeftSemi - | JoinType::RightSemi - | JoinType::LeftAnti - | JoinType::RightAnti - ) { - return None; - } - // TODO: Do we need to take the `on` into account? let left_len = join.left().schema().len(); let (left_idxs, right_idxs): (Vec<_>, Vec<_>) = required_col_idx @@ -224,6 +224,8 @@ impl TranslateApplyRule { } if let Some(join) = plan.as_logical_join() { Self::rewrite_join(join, indices, offset, index_mapping, data_types, index) + } else if let Some(apply) = plan.as_logical_apply() { + Self::rewrite_apply(apply, indices, offset, index_mapping, data_types, index) } else if let Some(scan) = plan.as_logical_scan() { Self::rewrite_scan(scan, indices, offset, index_mapping, data_types, index) } else { @@ -231,19 +233,125 @@ impl TranslateApplyRule { } }; match (left_idxs.is_empty(), right_idxs.is_empty()) { - (true, false) => rewrite(join.right(), right_idxs, true), - (false, true) => rewrite(join.left(), left_idxs, false), + (true, false) => { + // Only accept join which doesn't generate null columns. + match join.join_type() { + JoinType::Inner + | JoinType::LeftSemi + | JoinType::RightSemi + | JoinType::LeftAnti + | JoinType::RightAnti + | JoinType::RightOuter => rewrite(join.right(), right_idxs, true), + JoinType::LeftOuter | JoinType::FullOuter => None, + JoinType::Unspecified => unreachable!(), + } + } + (false, true) => { + // Only accept join which doesn't generate null columns. + match join.join_type() { + JoinType::Inner + | JoinType::LeftSemi + | JoinType::RightSemi + | JoinType::LeftAnti + | JoinType::RightAnti + | JoinType::LeftOuter => rewrite(join.left(), left_idxs, false), + JoinType::RightOuter | JoinType::FullOuter => None, + JoinType::Unspecified => unreachable!(), + } + } (false, false) => { - let left = rewrite(join.left(), left_idxs, false)?; - let right = rewrite(join.right(), right_idxs, true)?; - let new_join = - LogicalJoin::new(left, right, join.join_type(), Condition::true_cond()); - Some(new_join.into()) + // Only accept join which doesn't generate null columns. + match join.join_type() { + JoinType::Inner + | JoinType::LeftSemi + | JoinType::RightSemi + | JoinType::LeftAnti + | JoinType::RightAnti => { + let left = rewrite(join.left(), left_idxs, false)?; + let right = rewrite(join.right(), right_idxs, true)?; + let new_join = + LogicalJoin::new(left, right, join.join_type(), Condition::true_cond()); + Some(new_join.into()) + } + JoinType::LeftOuter | JoinType::RightOuter | JoinType::FullOuter => None, + JoinType::Unspecified => unreachable!(), + } } _ => None, } } + /// ```text + /// LogicalApply + /// / \ + /// LogicalApply RHS1 + /// / \ + /// LHS RHS2 + /// ``` + /// + /// A common structure of multi scalar subqueries is a chain of `LogicalApply`. To avoid exponential growth of the domain operator, we need to rewrite the apply and try to simplify the domain as much as possible. + /// We use a top-down apply order to rewrite the apply, so that we don't need to handle operator like project and aggregation generated by the domain calculation. + /// As a cost, we need to add a flag `translated` to the apply operator to remind `translate_apply_rule` that the apply has been transalted. + fn rewrite_apply( + apply: &LogicalApply, + required_col_idx: Vec, + offset: usize, + index_mapping: &mut ColIndexMapping, + data_types: &mut HashMap, + index: &mut usize, + ) -> Option { + // TODO: Do we need to take the `on` into account? + let left_len = apply.left().schema().len(); + let (left_idxs, right_idxs): (Vec<_>, Vec<_>) = required_col_idx + .into_iter() + .partition(|idx| *idx < left_len); + if !left_idxs.is_empty() && right_idxs.is_empty() { + // Deal with multi scalar subqueries + match apply.join_type() { + JoinType::Inner | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftOuter => { + let plan = apply.left(); + if let Some(join) = plan.as_logical_join() { + Self::rewrite_join( + join, + left_idxs, + offset, + index_mapping, + data_types, + index, + ) + } else if let Some(apply) = plan.as_logical_apply() { + Self::rewrite_apply( + apply, + left_idxs, + offset, + index_mapping, + data_types, + index, + ) + } else if let Some(scan) = plan.as_logical_scan() { + Self::rewrite_scan( + scan, + left_idxs, + offset, + index_mapping, + data_types, + index, + ) + } else { + None + } + } + JoinType::RightOuter + | JoinType::RightAnti + | JoinType::RightSemi + | JoinType::FullOuter => None, + JoinType::Unspecified => unreachable!(), + } + } else { + None + } + } + fn rewrite_scan( scan: &LogicalScan, required_col_idx: Vec, From 7e7ed831f616608f10519c65265eb4e65ad9bc36 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Tue, 28 May 2024 19:50:40 +0800 Subject: [PATCH 2/4] typo --- src/frontend/src/optimizer/rule/translate_apply_rule.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/rule/translate_apply_rule.rs b/src/frontend/src/optimizer/rule/translate_apply_rule.rs index f46aa9656512..c29eaaf4ee22 100644 --- a/src/frontend/src/optimizer/rule/translate_apply_rule.rs +++ b/src/frontend/src/optimizer/rule/translate_apply_rule.rs @@ -291,7 +291,7 @@ impl TranslateApplyRule { /// /// A common structure of multi scalar subqueries is a chain of `LogicalApply`. To avoid exponential growth of the domain operator, we need to rewrite the apply and try to simplify the domain as much as possible. /// We use a top-down apply order to rewrite the apply, so that we don't need to handle operator like project and aggregation generated by the domain calculation. - /// As a cost, we need to add a flag `translated` to the apply operator to remind `translate_apply_rule` that the apply has been transalted. + /// As a cost, we need to add a flag `translated` to the apply operator to remind `translate_apply_rule` that the apply has been translated. fn rewrite_apply( apply: &LogicalApply, required_col_idx: Vec, From 5006a6677efab6529682f678ed44aad24257dabe Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Wed, 29 May 2024 13:15:17 +0800 Subject: [PATCH 3/4] fmt --- .../testdata/input/subquery_expr_correlated.yaml | 2 +- .../output/subquery_expr_correlated.yaml | 16 +++++++++++++++- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml index 679ffe51e31a..f35703128ae7 100644 --- a/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/input/subquery_expr_correlated.yaml @@ -543,7 +543,7 @@ create table t4(g int, h int); create table t5(i int, j int); create table t6(k int, l int); - select + select COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col1, COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col2, COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col3, diff --git a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml index 9798b11cafc3..3bb2639f1293 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery_expr_correlated.yaml @@ -1491,7 +1491,21 @@ └─StreamExchange { dist: HashShard(t2.c) } └─StreamTableScan { table: t2, columns: [t2.c], stream_scan_type: ArrangementBackfill, stream_key: [t2.c], pk: [c], dist: UpstreamHashShard(t2.c) } - name: improve multi scalar subqueries optimization time. issue 16952. case 2. - sql: "create table t1(a int, b int);\ncreate table t2(c int primary key, d int);\ncreate table t3(e int, f int);\ncreate table t4(g int, h int);\ncreate table t5(i int, j int);\ncreate table t6(k int, l int);\nselect \nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col1,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col2,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col3,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col4,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col5,\nCOALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col6\nfrom t1;\n" + sql: | + create table t1(a int, b int); + create table t2(c int primary key, d int); + create table t3(e int, f int); + create table t4(g int, h int); + create table t5(i int, j int); + create table t6(k int, l int); + select + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col1, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col2, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col3, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col4, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col5, + COALESCE((SELECT sum(d) FROM t2 left join t3 on e = a and f = c left join t4 on g = a and h = c left join t5 on i = a and j = c WHERE t1.a = t2.c and t1.a = t2.c and j in (select k from t6 where b = l) ), 0) col6 + from t1; batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [Coalesce(sum(t2.d), 0:Int64) as $expr1, Coalesce(sum(t2.d), 0:Int64) as $expr2, Coalesce(sum(t2.d), 0:Int64) as $expr3, Coalesce(sum(t2.d), 0:Int64) as $expr4, Coalesce(sum(t2.d), 0:Int64) as $expr5, Coalesce(sum(t2.d), 0:Int64) as $expr6] } From 0c794e9e502b00c8adc12c864e549077d0ffe148 Mon Sep 17 00:00:00 2001 From: Dylan Chen Date: Wed, 29 May 2024 13:20:04 +0800 Subject: [PATCH 4/4] refactor --- .../optimizer/rule/translate_apply_rule.rs | 41 +------------------ 1 file changed, 2 insertions(+), 39 deletions(-) diff --git a/src/frontend/src/optimizer/rule/translate_apply_rule.rs b/src/frontend/src/optimizer/rule/translate_apply_rule.rs index c29eaaf4ee22..876ca7d6285b 100644 --- a/src/frontend/src/optimizer/rule/translate_apply_rule.rs +++ b/src/frontend/src/optimizer/rule/translate_apply_rule.rs @@ -222,15 +222,7 @@ impl TranslateApplyRule { indices.iter_mut().for_each(|index| *index -= left_len); offset += left_len; } - if let Some(join) = plan.as_logical_join() { - Self::rewrite_join(join, indices, offset, index_mapping, data_types, index) - } else if let Some(apply) = plan.as_logical_apply() { - Self::rewrite_apply(apply, indices, offset, index_mapping, data_types, index) - } else if let Some(scan) = plan.as_logical_scan() { - Self::rewrite_scan(scan, indices, offset, index_mapping, data_types, index) - } else { - None - } + Self::rewrite(&plan, indices, offset, index_mapping, data_types, index) }; match (left_idxs.is_empty(), right_idxs.is_empty()) { (true, false) => { @@ -310,36 +302,7 @@ impl TranslateApplyRule { match apply.join_type() { JoinType::Inner | JoinType::LeftSemi | JoinType::LeftAnti | JoinType::LeftOuter => { let plan = apply.left(); - if let Some(join) = plan.as_logical_join() { - Self::rewrite_join( - join, - left_idxs, - offset, - index_mapping, - data_types, - index, - ) - } else if let Some(apply) = plan.as_logical_apply() { - Self::rewrite_apply( - apply, - left_idxs, - offset, - index_mapping, - data_types, - index, - ) - } else if let Some(scan) = plan.as_logical_scan() { - Self::rewrite_scan( - scan, - left_idxs, - offset, - index_mapping, - data_types, - index, - ) - } else { - None - } + Self::rewrite(&plan, left_idxs, offset, index_mapping, data_types, index) } JoinType::RightOuter | JoinType::RightAnti