From 0ce9d72263c45eb81e163feea6f9ab9c7fcfc0af Mon Sep 17 00:00:00 2001 From: Clearlove <52417396+Eurekaaw@users.noreply.github.com> Date: Fri, 17 Feb 2023 09:33:10 -0500 Subject: [PATCH] fix(optimizer): Constraining `TopN` Optimization in Batch (#7993) Fix #7992 by apply `TopN` on index optimization into `gen_batch_plan` part. Please explain **IN DETAIL** what the changes are in this PR and why they are needed: - Summarize your change (**mandatory**) Fix #7992 by apply `TopN` on index optimization into `gen_batch_plan` part. One planner test is added. Approved-By: BugenZhao Approved-By: chenzl25 Co-Authored-By: Clearlove Co-Authored-By: Clearlove <52417396+Eurekaaw@users.noreply.github.com> --- .../planner_test/tests/testdata/agg.yaml | 80 +-- .../tests/testdata/dynamic_filter.yaml | 36 +- .../planner_test/tests/testdata/expr.yaml | 14 +- .../tests/testdata/index_selection.yaml | 12 + .../planner_test/tests/testdata/join.yaml | 30 +- .../tests/testdata/mv_column_name.yaml | 4 +- .../planner_test/tests/testdata/nexmark.yaml | 196 +++--- .../tests/testdata/nexmark_source.yaml | 210 +++--- .../planner_test/tests/testdata/order_by.yaml | 16 +- .../tests/testdata/over_window_function.yaml | 16 +- .../tests/testdata/predicate_pushdown.yaml | 10 +- .../planner_test/tests/testdata/share.yaml | 16 +- .../tests/testdata/shared_views.yaml | 26 +- .../testdata/subquery_expr_correlated.yaml | 46 +- .../tests/testdata/temporal_filter.yaml | 22 +- .../tests/testdata/time_window.yaml | 4 +- .../planner_test/tests/testdata/tpch.yaml | 636 +++++++++--------- .../tests/testdata/watermark.yaml | 2 +- src/frontend/src/optimizer/mod.rs | 14 +- 19 files changed, 701 insertions(+), 689 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index 423e20aaf635..b020868b813d 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -31,7 +31,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [v1, agg], pk_columns: [v1] } - └─StreamProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr70] } + └─StreamProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr69] } └─StreamHashAgg { group_key: [t.v1], aggs: [count, min(t.v2), max(t.v3), count(t.v1)] } └─StreamExchange { dist: HashShard(t.v1) } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -51,11 +51,11 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [agg], pk_columns: [] } - └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr71] } + └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr70] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v2)), sum0(count(t.v3))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr69], aggs: [count, min(t.v1), max(t.v2), count(t.v3)] } - └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr69] } + └─StreamHashAgg { group_key: [$expr68], aggs: [count, min(t.v1), max(t.v2), count(t.v3)] } + └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr68] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t(v1 int, v2 int); @@ -83,10 +83,10 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [v3, agg], pk_columns: [v3] } - └─StreamProject { exprs: [t.v3, (min(t.v1) * (sum($expr137)::Decimal / count($expr137))) as $expr139] } - └─StreamHashAgg { group_key: [t.v3], aggs: [count, min(t.v1), sum($expr137), count($expr137)] } + └─StreamProject { exprs: [t.v3, (min(t.v1) * (sum($expr135)::Decimal / count($expr135))) as $expr137] } + └─StreamHashAgg { group_key: [t.v3], aggs: [count, min(t.v1), sum($expr135), count($expr135)] } └─StreamExchange { dist: HashShard(t.v3) } - └─StreamProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr137, t._row_id] } + └─StreamProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr135, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: test logical_agg with complex group expression sql: | @@ -151,11 +151,11 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [cnt, sum], pk_columns: [] } - └─StreamProject { exprs: [sum0(count($expr69)), sum(sum($expr69))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count($expr69)), sum(sum($expr69))] } + └─StreamProject { exprs: [sum0(count($expr68)), sum(sum($expr68))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum0(count($expr68)), sum(sum($expr68))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count($expr69), sum($expr69)] } - └─StreamProject { exprs: [(t.v1 + t.v2) as $expr69, t._row_id] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, count($expr68), sum($expr68)] } + └─StreamProject { exprs: [(t.v1 + t.v2) as $expr68, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t(v1 int, v2 int, v3 int); @@ -169,10 +169,10 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [v1, agg], pk_columns: [v1] } - └─StreamProject { exprs: [t.v1, ((sum($expr93) / count($expr93)) + max(t.v1)) as $expr95] } - └─StreamHashAgg { group_key: [t.v1], aggs: [count, sum($expr93), count($expr93), max(t.v1)] } + └─StreamProject { exprs: [t.v1, ((sum($expr91) / count($expr91)) + max(t.v1)) as $expr93] } + └─StreamHashAgg { group_key: [t.v1], aggs: [count, sum($expr91), count($expr91), max(t.v1)] } └─StreamExchange { dist: HashShard(t.v1) } - └─StreamProject { exprs: [t.v1, (t.v2 + t.v3) as $expr93, t._row_id] } + └─StreamProject { exprs: [t.v1, (t.v2 + t.v3) as $expr91, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 real); @@ -410,22 +410,22 @@ └─LogicalProject { exprs: [t.v1, t.v3, t.v2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } optimized_logical_plan: | - LogicalProject { exprs: [(min(t.v1) + (max(t.v3) * count(t.v2))) as $expr20] } + LogicalProject { exprs: [(min(t.v1) + (max(t.v3) * count(t.v2))) as $expr19] } └─LogicalAgg { aggs: [min(t.v1), max(t.v3), count(t.v2)] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3] } batch_plan: | - BatchProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr42] } + BatchProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr41] } └─BatchSimpleAgg { aggs: [min(min(t.v1)), max(max(t.v3)), sum0(count(t.v2))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1), max(t.v3), count(t.v2)] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [agg], pk_columns: [] } - └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr68] } + └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr66] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v3)), sum0(count(t.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr66], aggs: [count, min(t.v1), max(t.v3), count(t.v2)] } - └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr66] } + └─StreamHashAgg { group_key: [$expr64], aggs: [count, min(t.v1), max(t.v3), count(t.v2)] } + └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr64] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: dup group key sql: | @@ -593,8 +593,8 @@ └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } optimized_logical_plan: | - LogicalAgg { aggs: [sum($expr20) filter(((t.a * t.b) > 0:Int32))] } - └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr20] } + LogicalAgg { aggs: [sum($expr19) filter(((t.a * t.b) > 0:Int32))] } + └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr19] } └─LogicalScan { table: t, columns: [t.a, t.b] } - name: complex filter clause sql: | @@ -606,17 +606,17 @@ └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } optimized_logical_plan: | - LogicalAgg { aggs: [max($expr20) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } - └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr20] } + LogicalAgg { aggs: [max($expr19) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } + └─LogicalProject { exprs: [t.a, t.b, (t.a * t.b) as $expr19] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | StreamMaterialize { columns: [sab], pk_columns: [] } - └─StreamProject { exprs: [max(max($expr44) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max($expr44) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } + └─StreamProject { exprs: [max(max($expr42) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } + └─StreamGlobalSimpleAgg { aggs: [count, max(max($expr42) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32))))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr45], aggs: [count, max($expr44) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } - └─StreamProject { exprs: [t.a, t.b, $expr44, t._row_id, Vnode(t._row_id) as $expr45] } - └─StreamProject { exprs: [t.a, t.b, (t.a * t.b) as $expr44, t._row_id] } + └─StreamHashAgg { group_key: [$expr43], aggs: [count, max($expr42) filter((t.a < t.b) AND ((t.a + t.b) < 100:Int32) AND ((t.a * t.b) <> ((t.a + t.b) - 1:Int32)))] } + └─StreamProject { exprs: [t.a, t.b, $expr42, t._row_id, Vnode(t._row_id) as $expr43] } + └─StreamProject { exprs: [t.a, t.b, (t.a * t.b) as $expr42, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: avg filter clause + group by sql: | @@ -628,12 +628,12 @@ └─LogicalProject { exprs: [t.b, t.a] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } optimized_logical_plan: | - LogicalProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))) as $expr21] } + LogicalProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))) as $expr20] } └─LogicalAgg { group_key: [t.b], aggs: [sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | StreamMaterialize { columns: [avga, t.b(hidden)], pk_columns: [t.b] } - └─StreamProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))) as $expr47, t.b] } + └─StreamProject { exprs: [(sum(t.a) filter((t.a > t.b))::Decimal / count(t.a) filter((t.a > t.b))) as $expr45, t.b] } └─StreamHashAgg { group_key: [t.b], aggs: [count, sum(t.a) filter((t.a > t.b)), count(t.a) filter((t.a > t.b))] } └─StreamExchange { dist: HashShard(t.b) } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -855,16 +855,16 @@ └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr1] } └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } optimized_logical_plan: | - LogicalAgg { aggs: [sum($expr20) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } - └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr20] } + LogicalAgg { aggs: [sum($expr19) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } + └─LogicalProject { exprs: [t.b, (Length(t.a) * t.b) as $expr19] } └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [] } - └─StreamProject { exprs: [sum(sum($expr44) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr44) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } + └─StreamProject { exprs: [sum(sum($expr42) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr42) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32)))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr44) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } - └─StreamProject { exprs: [t.b, (Length(t.a) * t.b) as $expr44, t._row_id] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr42) filter((t.b < 100:Int32) AND ((t.b * 2:Int32) > 10:Int32))] } + └─StreamProject { exprs: [t.b, (Length(t.a) * t.b) as $expr42, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t.b, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t(x int, y varchar); @@ -1097,9 +1097,9 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [stddev_samp, stddev_pop], pk_columns: [] } - └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Decimal::Float64, Pow(((sum(sum($expr205))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / (sum0(count(t.v1)) - 1:Int64))::Float64, 0.5:Float64)) as $expr208, Pow(((sum(sum($expr205))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr209] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr205)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr205)), sum(sum(t.v1)), sum0(count(t.v1))] } + └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Decimal::Float64, Pow(((sum(sum($expr202))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / (sum0(count(t.v1)) - 1:Int64))::Float64, 0.5:Float64)) as $expr205, Pow(((sum(sum($expr202))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr206] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr202)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr202)), sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr205), sum(t.v1), count(t.v1), sum($expr205), sum(t.v1), count(t.v1)] } - └─StreamProject { exprs: [t.v1, (t.v1 * t.v1) as $expr205, t._row_id] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr202), sum(t.v1), count(t.v1), sum($expr202), sum(t.v1), count(t.v1)] } + └─StreamProject { exprs: [t.v1, (t.v1 * t.v1) as $expr202, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml index 62685095a088..b361a6263eec 100644 --- a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml @@ -50,23 +50,23 @@ sql: | with max_v2 as (select max(v2) max from t2) select v1 from t1, max_v2 where v1 + v1 > max; optimized_logical_plan: | - LogicalJoin { type: Inner, on: ($expr10 > max(t2.v2)), output: [t1.v1] } - ├─LogicalProject { exprs: [t1.v1, (t1.v1 + t1.v1) as $expr10] } + LogicalJoin { type: Inner, on: ($expr9 > max(t2.v2)), output: [t1.v1] } + ├─LogicalProject { exprs: [t1.v1, (t1.v1 + t1.v1) as $expr9] } | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.v1, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr23 > max(max(t2.v2))), output: [t1.v1, $expr23, t1._row_id] } - ├─StreamProject { exprs: [t1.v1, (t1.v1 + t1.v1) as $expr23, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr21 > max(max(t2.v2))), output: [t1.v1, $expr21, t1._row_id] } + ├─StreamProject { exprs: [t1.v1, (t1.v1 + t1.v1) as $expr21, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr24], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr24] } + └─StreamHashAgg { group_key: [$expr22], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr22] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Ensure error on multiple rows on inner side before: @@ -126,23 +126,23 @@ create table t2 (v2 bigint); with max_v2 as (select max(v2) max from t2) select v1 from t1, max_v2 where v1 > max; optimized_logical_plan: | - LogicalJoin { type: Inner, on: ($expr10 > max(t2.v2)), output: [t1.v1] } - ├─LogicalProject { exprs: [t1.v1, t1.v1::Int64 as $expr10] } + LogicalJoin { type: Inner, on: ($expr9 > max(t2.v2)), output: [t1.v1] } + ├─LogicalProject { exprs: [t1.v1, t1.v1::Int64 as $expr9] } | └─LogicalScan { table: t1, columns: [t1.v1] } └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.v1, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr23 > max(max(t2.v2))), output: [t1.v1, $expr23, t1._row_id] } - ├─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr23, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr21 > max(max(t2.v2))), output: [t1.v1, $expr21, t1._row_id] } + ├─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr21, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr24], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr24] } + └─StreamHashAgg { group_key: [$expr22], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr22] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Dynamic filter on semi join sql: | @@ -166,19 +166,19 @@ sql: | with max_v2 as (select max(v2) max from t2) select v1 from t1, max_v2 where v1 > 2 * max; optimized_logical_plan: | - LogicalJoin { type: Inner, on: (t1.v1 > $expr10), output: [t1.v1] } + LogicalJoin { type: Inner, on: (t1.v1 > $expr9), output: [t1.v1] } ├─LogicalScan { table: t1, columns: [t1.v1] } - └─LogicalProject { exprs: [(2:Int32 * max(t2.v2)) as $expr10] } + └─LogicalProject { exprs: [(2:Int32 * max(t2.v2)) as $expr9] } └─LogicalAgg { aggs: [max(t2.v2)] } └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } - └─StreamDynamicFilter { predicate: (t1.v1 > $expr25), output: [t1.v1, t1._row_id] } + └─StreamDynamicFilter { predicate: (t1.v1 > $expr23), output: [t1.v1, t1._row_id] } ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(2:Int32 * max(max(t2.v2))) as $expr25] } + └─StreamProject { exprs: [(2:Int32 * max(max(t2.v2))) as $expr23] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr23], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr23] } + └─StreamHashAgg { group_key: [$expr21], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr21] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 619b273edeb4..6803e6020bef 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -178,7 +178,7 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal) as $expr47, t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal) as $expr46, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: case searched form without else sql: | @@ -217,7 +217,7 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [Case((t.v1 = 1:Int32), null:Int32, t.v1) as $expr47, t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), null:Int32, t.v1) as $expr46, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -238,7 +238,7 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [Coalesce(t.v1, 1:Int32) as $expr47, t._row_id] } + └─StreamProject { exprs: [Coalesce(t.v1, 1:Int32) as $expr46, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -264,7 +264,7 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar) as $expr47, t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar) as $expr46, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar); @@ -290,7 +290,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar) as $expr47, t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar) as $expr46, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 float); @@ -457,10 +457,10 @@ └─LogicalScan { table: t, columns: [t.v1, t._row_id] } stream_plan: | StreamMaterialize { columns: [v1, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamDynamicFilter { predicate: (t.v1 >= $expr13), output: [t.v1, t._row_id] } + └─StreamDynamicFilter { predicate: (t.v1 >= $expr12), output: [t.v1, t._row_id] } ├─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '00:00:02':Interval) as $expr13], watermark_columns: [(now - '00:00:02':Interval)] } + └─StreamProject { exprs: [(now - '00:00:02':Interval) as $expr12], watermark_columns: [(now - '00:00:02':Interval)] } └─StreamNow { output: [now] } - name: and of two now expression condition sql: | diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index eeea4d3e5c41..0e28d25116c0 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -608,3 +608,15 @@ └─BatchExchange { order: [idx1.a DESC], dist: Single } └─BatchLimit { limit: 1, offset: 0 } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], distribution: UpstreamHashShard(idx1.a) } +- name: topn on pk streaming case, should NOT optimized + sql: | + create table t1 (a int primary key); + select * from t1 order by a limit 1; + stream_plan: | + StreamMaterialize { columns: [a], pk_columns: [a] } + └─StreamProject { exprs: [t1.a] } + └─StreamTopN { order: "[t1.a ASC]", limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: "[t1.a ASC]", limit: 1, offset: 0, group_key: [1] } + └─StreamProject { exprs: [t1.a, Vnode(t1.a) as $expr1] } + └─StreamTableScan { table: t1, columns: [t1.a], pk: [t1.a], dist: UpstreamHashShard(t1.a) } diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index b8c990b00dfc..28b470b1e124 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -197,7 +197,7 @@ stream_plan: | StreamMaterialize { columns: [x, i.t._row_id(hidden), i.t._row_id#1(hidden), i.x(hidden), i.x#1(hidden), i.t._row_id#2(hidden), i.t._row_id#3(hidden), i.x#2(hidden), i.x#3(hidden)], pk_columns: [i.t._row_id, i.t._row_id#1, i.x, i.x#1, i.t._row_id#2, i.t._row_id#3, i.x#2, i.x#3] } └─StreamExchange { dist: HashShard(i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x) } - └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr47, i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x] } + └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr46, i.t._row_id, i.t._row_id, i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.x] } └─StreamHashJoin { type: FullOuter, predicate: i.x = i.x, output: [i.x, i.x, i.t._row_id, i.t._row_id, i.x, i.t._row_id, i.t._row_id, i.x] } ├─StreamHashJoin { type: Inner, predicate: i.x = i.x, output: [i.x, i.t._row_id, i.t._row_id, i.x] } | ├─StreamExchange { dist: HashShard(i.x) } @@ -414,9 +414,9 @@ | └─LogicalScan { table: b, columns: [b.x, b.z, b._row_id] } └─LogicalScan { table: c, columns: [c.x, c.a, c._row_id] } optimized_logical_plan: | - LogicalProject { exprs: [Coalesce(a.x, b.x, c.x) as $expr30, a.x, b.x, c.x] } - └─LogicalJoin { type: FullOuter, on: ($expr29 = c.x), output: [a.x, b.x, c.x] } - ├─LogicalProject { exprs: [a.x, b.x, Coalesce(a.x, b.x) as $expr29] } + LogicalProject { exprs: [Coalesce(a.x, b.x, c.x) as $expr28, a.x, b.x, c.x] } + └─LogicalJoin { type: FullOuter, on: ($expr27 = c.x), output: [a.x, b.x, c.x] } + ├─LogicalProject { exprs: [a.x, b.x, Coalesce(a.x, b.x) as $expr27] } | └─LogicalJoin { type: FullOuter, on: (a.x = b.x), output: all } | ├─LogicalScan { table: a, columns: [a.x] } | └─LogicalScan { table: b, columns: [b.x] } @@ -499,9 +499,9 @@ └─BatchExchange { order: [], dist: HashShard(b.x) } └─BatchScan { table: b, columns: [b.x], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [y, z, $expr141(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr141, a._row_id, b._row_id, a.x, b.x] } + StreamMaterialize { columns: [y, z, $expr138(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], pk_columns: [a._row_id, b._row_id, a.x, b.x], order_descs: [$expr138, a._row_id, b._row_id, a.x, b.x] } └─StreamExchange { dist: HashShard(a._row_id, b._row_id, a.x, b.x) } - └─StreamProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr139, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr140, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr141, a._row_id, b._row_id, a.x, b.x] } + └─StreamProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr136, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr137, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr138, a._row_id, b._row_id, a.x, b.x] } └─StreamFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) } └─StreamHashJoin { type: FullOuter, predicate: a.x = b.x, output: [a.x, b.x, a._row_id, b._row_id] } ├─StreamExchange { dist: HashShard(a.x) } @@ -558,10 +558,10 @@ create table t2(x int, y int); select * from t1, t2 where t1.x + t1.y = t2.x + t2.y; optimized_logical_plan: | - LogicalJoin { type: Inner, on: ($expr19 = $expr20), output: [t1.x, t1.y, t2.x, t2.y] } - ├─LogicalProject { exprs: [t1.x, t1.y, (t1.x + t1.y) as $expr19] } + LogicalJoin { type: Inner, on: ($expr17 = $expr18), output: [t1.x, t1.y, t2.x, t2.y] } + ├─LogicalProject { exprs: [t1.x, t1.y, (t1.x + t1.y) as $expr17] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t2.x, t2.y, (t2.x + t2.y) as $expr20] } + └─LogicalProject { exprs: [t2.x, t2.y, (t2.x + t2.y) as $expr18] } └─LogicalScan { table: t2, columns: [t2.x, t2.y] } - name: Use project to align return types sql: | @@ -569,8 +569,8 @@ create table t2(x int, y decimal); select * from t1, t2 where t1.x = t2.y; optimized_logical_plan: | - LogicalJoin { type: Inner, on: ($expr10 = t2.y), output: [t1.x, t1.y, t2.x, t2.y] } - ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr10] } + LogicalJoin { type: Inner, on: ($expr9 = t2.y), output: [t1.x, t1.y, t2.x, t2.y] } + ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr9] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalScan { table: t2, columns: [t2.x, t2.y] } - name: Lookup join with no eq keys after pulling up predicate will revert to hash join @@ -600,10 +600,10 @@ └─BatchExchange { order: [], dist: HashShard(t2.v2) } └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr26(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr26, v2] } - └─StreamHashJoin { type: Inner, predicate: $expr26 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr26, t2._row_id] } - ├─StreamExchange { dist: HashShard($expr26) } - | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr26, t1._row_id] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr25(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr25, v2] } + └─StreamHashJoin { type: Inner, predicate: $expr25 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr25, t2._row_id] } + ├─StreamExchange { dist: HashShard($expr25) } + | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr25, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard(t2.v2) } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/mv_column_name.yaml b/src/frontend/planner_test/tests/testdata/mv_column_name.yaml index 87cf00ef488c..30fb7e5a1571 100644 --- a/src/frontend/planner_test/tests/testdata/mv_column_name.yaml +++ b/src/frontend/planner_test/tests/testdata/mv_column_name.yaml @@ -15,7 +15,7 @@ select a is null as is_null from t; stream_plan: | StreamMaterialize { columns: [is_null, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [IsNull(t.a) as $expr25, t._row_id] } + └─StreamProject { exprs: [IsNull(t.a) as $expr24, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: is_true with alias sql: | @@ -23,7 +23,7 @@ select a, a is true as is_true from t; stream_plan: | StreamMaterialize { columns: [a, is_true, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [t.a, IsTrue(t.a) as $expr25, t._row_id] } + └─StreamProject { exprs: [t.a, IsTrue(t.a) as $expr24, t._row_id] } └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: column name specified more than once sql: | diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index 9b3f337f9e50..af7336166d6b 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -70,13 +70,13 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id] } - └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr47, bid.date_time, bid._row_id] } + └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr46, bid.date_time, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, bid._row_id(hidden)], pk_columns: [bid._row_id] } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr47, bid.date_time, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr46, bid.date_time, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -187,7 +187,7 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [category, avg], pk_columns: [category] } - └─StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr48] } + └─StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr47] } └─StreamHashAgg { group_key: [auction.category], aggs: [count, sum(max(bid.price)), count(max(bid.price))] } └─StreamExchange { dist: HashShard(auction.category) } └─StreamProject { exprs: [auction.id, auction.category, max(bid.price)] } @@ -203,7 +203,7 @@ Fragment 0 StreamMaterialize { columns: [category, avg], pk_columns: [category] } materialized table: 4294967294 - StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr48] } + StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr47] } StreamHashAgg { group_key: [auction.category], aggs: [count, sum(max(bid.price)), count(max(bid.price))] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -408,24 +408,24 @@ └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr49, bid.price] } └─BatchScan { table: bid, columns: [bid.price, bid.date_time], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr101(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr101, price, max(bid.price)] } - └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr101, max(bid.price)] } - └─StreamFilter { predicate: (bid.date_time >= $expr103) AND (bid.date_time <= $expr101) } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr99(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr99, price, max(bid.price)] } + └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr99, max(bid.price)] } + └─StreamFilter { predicate: (bid.date_time >= $expr101) AND (bid.date_time <= $expr99) } └─StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } ├─StreamExchange { dist: HashShard(bid.price) } | └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } └─StreamExchange { dist: HashShard(max(bid.price)) } - └─StreamProject { exprs: [max(bid.price), $expr101, ($expr101 - '00:00:10':Interval) as $expr103] } - └─StreamAppendOnlyHashAgg { group_key: [$expr101], aggs: [count, max(bid.price)] } - └─StreamExchange { dist: HashShard($expr101) } - └─StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr101, bid.price, bid._row_id] } + └─StreamProject { exprs: [max(bid.price), $expr99, ($expr99 - '00:00:10':Interval) as $expr101] } + └─StreamAppendOnlyHashAgg { group_key: [$expr99], aggs: [count, max(bid.price)] } + └─StreamExchange { dist: HashShard($expr99) } + └─StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr99, bid.price, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr101(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr101, price, max(bid.price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr99(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr99, price, max(bid.price)] } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr101, max(bid.price)] } - StreamFilter { predicate: (bid.date_time >= $expr103) AND (bid.date_time <= $expr101) } + StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr99, max(bid.price)] } + StreamFilter { predicate: (bid.date_time >= $expr101) AND (bid.date_time <= $expr99) } StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -437,23 +437,23 @@ BatchPlanNode Fragment 2 - StreamProject { exprs: [max(bid.price), $expr101, ($expr101 - '00:00:10':Interval) as $expr103] } - StreamAppendOnlyHashAgg { group_key: [$expr101], aggs: [count, max(bid.price)] } + StreamProject { exprs: [max(bid.price), $expr99, ($expr99 - '00:00:10':Interval) as $expr101] } + StreamAppendOnlyHashAgg { group_key: [$expr99], aggs: [count, max(bid.price)] } result table: 4, state tables: [] StreamExchange Hash([0]) from 3 Fragment 3 - StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr101, bid.price, bid._row_id] } + StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr99, bid.price, bid._row_id] } Chain { table: bid, columns: [bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode Table 0 { columns: [bid_auction, bid_bidder, bid_price, bid_date_time, bid__row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [bid_price, bid__row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(bid_price), $expr101, $expr103], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(bid_price), $expr101, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4 { columns: [$expr101, count, max(bid_price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr101, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 2 { columns: [max(bid_price), $expr99, $expr101], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(bid_price), $expr99, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 4 { columns: [$expr99, count, max(bid_price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, bid._row_id, $expr99, max(bid.price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_tables @@ -502,52 +502,52 @@ └─BatchProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr93, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr94] } └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr196(hidden), auction.seller(hidden), $expr193(hidden), $expr194(hidden)], pk_columns: [id, name, starttime, $expr196, auction.seller, $expr193, $expr194] } - └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr195 = $expr193 AND $expr196 = $expr194, output: all } - ├─StreamExchange { dist: HashShard(person.id, $expr195, $expr196) } - | └─StreamProject { exprs: [person.id, person.name, $expr195, $expr196] } - | └─StreamHashAgg { group_key: [person.id, person.name, $expr195, $expr196], aggs: [count] } - | └─StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr195, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr196] } + StreamMaterialize { columns: [id, name, starttime, $expr192(hidden), auction.seller(hidden), $expr189(hidden), $expr190(hidden)], pk_columns: [id, name, starttime, $expr192, auction.seller, $expr189, $expr190] } + └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr191 = $expr189 AND $expr192 = $expr190, output: all } + ├─StreamExchange { dist: HashShard(person.id, $expr191, $expr192) } + | └─StreamProject { exprs: [person.id, person.name, $expr191, $expr192] } + | └─StreamHashAgg { group_key: [person.id, person.name, $expr191, $expr192], aggs: [count] } + | └─StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr191, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr192] } | └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } - └─StreamProject { exprs: [auction.seller, $expr193, $expr194] } - └─StreamHashAgg { group_key: [auction.seller, $expr193, $expr194], aggs: [count] } - └─StreamExchange { dist: HashShard(auction.seller, $expr193, $expr194) } - └─StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr193, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr194, auction.id] } + └─StreamProject { exprs: [auction.seller, $expr189, $expr190] } + └─StreamHashAgg { group_key: [auction.seller, $expr189, $expr190], aggs: [count] } + └─StreamExchange { dist: HashShard(auction.seller, $expr189, $expr190) } + └─StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr189, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr190, auction.id] } └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr196(hidden), auction.seller(hidden), $expr193(hidden), $expr194(hidden)], pk_columns: [id, name, starttime, $expr196, auction.seller, $expr193, $expr194] } + StreamMaterialize { columns: [id, name, starttime, $expr192(hidden), auction.seller(hidden), $expr189(hidden), $expr190(hidden)], pk_columns: [id, name, starttime, $expr192, auction.seller, $expr189, $expr190] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr195 = $expr193 AND $expr196 = $expr194, output: all } + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr191 = $expr189 AND $expr192 = $expr190, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [auction.seller, $expr193, $expr194] } - StreamHashAgg { group_key: [auction.seller, $expr193, $expr194], aggs: [count] } + StreamProject { exprs: [auction.seller, $expr189, $expr190] } + StreamHashAgg { group_key: [auction.seller, $expr189, $expr190], aggs: [count] } result table: 5, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - StreamProject { exprs: [person.id, person.name, $expr195, $expr196] } - StreamHashAgg { group_key: [person.id, person.name, $expr195, $expr196], aggs: [count] } + StreamProject { exprs: [person.id, person.name, $expr191, $expr192] } + StreamHashAgg { group_key: [person.id, person.name, $expr191, $expr192], aggs: [count] } result table: 4, state tables: [] - StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr195, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr196] } + StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr191, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr192] } Chain { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.id) } Upstream BatchPlanNode Fragment 2 - StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr193, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr194, auction.id] } + StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr189, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr190, auction.id] } Chain { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } Upstream BatchPlanNode - Table 0 { columns: [person_id, person_name, $expr195, $expr196], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [person_id, $expr195, $expr196, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [auction_seller, $expr193, $expr194], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [auction_seller, $expr193, $expr194, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [person_id, person_name, $expr195, $expr196, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 5 { columns: [auction_seller, $expr193, $expr194, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4294967294 { columns: [id, name, starttime, $expr196, auction.seller, $expr193, $expr194], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 0 { columns: [person_id, person_name, $expr191, $expr192], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [person_id, $expr191, $expr192, person_name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [auction_seller, $expr189, $expr190], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [auction_seller, $expr189, $expr190, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [person_id, person_name, $expr191, $expr192, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 5 { columns: [auction_seller, $expr189, $expr190, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4294967294 { columns: [id, name, starttime, $expr192, auction.seller, $expr189, $expr190], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_tables @@ -637,13 +637,13 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id] } - └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr93, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr94, bid._row_id] } + └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr91, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr92, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, bid._row_id(hidden)], pk_columns: [bid._row_id] } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr93, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr94, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr91, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr92, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -725,14 +725,14 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id] } - └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr93, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr94, bid.date_time, bid.extra, bid._row_id] } + └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr91, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr92, bid.date_time, bid.extra, bid._row_id] } └─StreamFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, bid._row_id(hidden)], pk_columns: [bid._row_id] } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr93, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr94, bid.date_time, bid.extra, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr91, Case(((Extract('HOUR':Varchar, bid.date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, bid.date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, bid.date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, bid.date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr92, bid.date_time, bid.extra, bid._row_id] } StreamFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid.extra, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream @@ -771,39 +771,39 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } - └─StreamProject { exprs: [$expr47, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [$expr47], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamExchange { dist: HashShard($expr47) } - └─StreamProject { exprs: [$expr47, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [$expr47, bid.bidder, bid.auction, flag], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard($expr47, bid.bidder, bid.auction, flag) } - └─StreamExpand { column_subsets: [[$expr47], [$expr47, bid.bidder], [$expr47, bid.auction]] } - └─StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr47, bid.price, bid.bidder, bid.auction, bid._row_id] } + └─StreamProject { exprs: [$expr46, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr46], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamExchange { dist: HashShard($expr46) } + └─StreamProject { exprs: [$expr46, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─StreamAppendOnlyHashAgg { group_key: [$expr46, bid.bidder, bid.auction, flag], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr46, bid.bidder, bid.auction, flag) } + └─StreamExpand { column_subsets: [[$expr46], [$expr46, bid.bidder], [$expr46, bid.auction]] } + └─StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr46, bid.price, bid.bidder, bid.auction, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } materialized table: 4294967294 - StreamProject { exprs: [$expr47, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - StreamHashAgg { group_key: [$expr47], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamProject { exprs: [$expr46, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [$expr46], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [$expr47, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [$expr47, bid.bidder, bid.auction, flag], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + StreamProject { exprs: [$expr46, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr46, bid.bidder, bid.auction, flag], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } result table: 1, state tables: [] StreamExchange Hash([0, 2, 3, 10]) from 2 Fragment 2 - StreamExpand { column_subsets: [[$expr47], [$expr47, bid.bidder], [$expr47, bid.auction]] } - StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr47, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamExpand { column_subsets: [[$expr46], [$expr46, bid.bidder], [$expr46, bid.auction]] } + StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr46, bid.price, bid.bidder, bid.auction, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [$expr47, count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } - Table 1 { columns: [$expr47, bid_bidder, bid_auction, flag, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count filter((bid_price < 10000:Int32))_0, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_0, count filter((bid_price >= 1000000:Int32))_0, count filter((bid_price < 10000:Int32))_1, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_1, count filter((bid_price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1, 2, 3] } + Table 0 { columns: [$expr46, count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 1 { columns: [$expr46, bid_bidder, bid_auction, flag, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count filter((bid_price < 10000:Int32))_0, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_0, count filter((bid_price >= 1000000:Int32))_0, count filter((bid_price < 10000:Int32))_1, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_1, count filter((bid_price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1, 2, 3] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 before: @@ -839,40 +839,40 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } - └─StreamProject { exprs: [bid.channel, $expr93, max(max($expr94)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [bid.channel, $expr93], aggs: [count, max(max($expr94)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamExchange { dist: HashShard(bid.channel, $expr93) } - └─StreamProject { exprs: [bid.channel, $expr93, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr94), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr93, bid.bidder, bid.auction, flag], aggs: [count, max($expr94), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard(bid.channel, $expr93, bid.bidder, bid.auction, flag) } - └─StreamExpand { column_subsets: [[bid.channel, $expr93, $expr94], [bid.channel, $expr93, bid.bidder], [bid.channel, $expr93, bid.auction]] } - └─StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr93, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr94, bid.price, bid.bidder, bid.auction, bid._row_id] } + └─StreamProject { exprs: [bid.channel, $expr91, max(max($expr92)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [bid.channel, $expr91], aggs: [count, max(max($expr92)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamExchange { dist: HashShard(bid.channel, $expr91) } + └─StreamProject { exprs: [bid.channel, $expr91, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr92), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr91, bid.bidder, bid.auction, flag], aggs: [count, max($expr92), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(bid.channel, $expr91, bid.bidder, bid.auction, flag) } + └─StreamExpand { column_subsets: [[bid.channel, $expr91, $expr92], [bid.channel, $expr91, bid.bidder], [bid.channel, $expr91, bid.auction]] } + └─StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr91, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr92, bid.price, bid.bidder, bid.auction, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } materialized table: 4294967294 - StreamProject { exprs: [bid.channel, $expr93, max(max($expr94)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - StreamHashAgg { group_key: [bid.channel, $expr93], aggs: [count, max(max($expr94)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamProject { exprs: [bid.channel, $expr91, max(max($expr92)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [bid.channel, $expr91], aggs: [count, max(max($expr92)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid.price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid.price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid.bidder) filter((flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.bidder) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid.auction) filter((flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid.auction) filter((count filter((bid.price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } result table: 1, state tables: [0] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.channel, $expr93, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr94), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr93, bid.bidder, bid.auction, flag], aggs: [count, max($expr94), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + StreamProject { exprs: [bid.channel, $expr91, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr92), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [bid.channel, $expr91, bid.bidder, bid.auction, flag], aggs: [count, max($expr92), count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32))] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 4, 5, 14]) from 2 Fragment 2 - StreamExpand { column_subsets: [[bid.channel, $expr93, $expr94], [bid.channel, $expr93, bid.bidder], [bid.channel, $expr93, bid.auction]] } - StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr93, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr94, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamExpand { column_subsets: [[bid.channel, $expr91, $expr92], [bid.channel, $expr91, bid.bidder], [bid.channel, $expr91, bid.auction]] } + StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr91, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr92, bid.price, bid.bidder, bid.auction, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [bid_channel, $expr93, max($expr94), bid_bidder, bid_auction, flag], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0, 1] } - Table 1 { columns: [bid_channel, $expr93, count, max(max($expr94)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } - Table 2 { columns: [bid_channel, $expr93, bid_bidder, bid_auction, flag, count, max($expr94), count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count filter((bid_price < 10000:Int32))_0, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_0, count filter((bid_price >= 1000000:Int32))_0, count filter((bid_price < 10000:Int32))_1, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_1, count filter((bid_price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0, 1, 2, 3, 4] } + Table 0 { columns: [bid_channel, $expr91, max($expr92), bid_bidder, bid_auction, flag], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0, 1] } + Table 1 { columns: [bid_channel, $expr91, count, max(max($expr92)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((bid_price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((bid_price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bid_bidder) filter((flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_bidder) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bid_auction) filter((flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(bid_auction) filter((count filter((bid_price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 2 { columns: [bid_channel, $expr91, bid_bidder, bid_auction, flag, count, max($expr92), count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), count filter((bid_price < 10000:Int32))_0, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_0, count filter((bid_price >= 1000000:Int32))_0, count filter((bid_price < 10000:Int32))_1, count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32))_1, count filter((bid_price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0, 1, 2, 3, 4] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 before: @@ -900,27 +900,27 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.price, bid.date_time], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } - └─StreamProject { exprs: [bid.auction, $expr93, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr95, sum(bid.price)] } - └─StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr93], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price), sum(bid.price)] } - └─StreamExchange { dist: HashShard(bid.auction, $expr93) } - └─StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr93, bid.price, bid._row_id] } + └─StreamProject { exprs: [bid.auction, $expr91, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr93, sum(bid.price)] } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr91], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price), sum(bid.price)] } + └─StreamExchange { dist: HashShard(bid.auction, $expr91) } + └─StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr91, bid.price, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, $expr93, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr95, sum(bid.price)] } - StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr93], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price), sum(bid.price)] } + StreamProject { exprs: [bid.auction, $expr91, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), (sum(bid.price) / count(bid.price)) as $expr93, sum(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr91], aggs: [count, count, count filter((bid.price < 10000:Int32)), count filter((bid.price >= 10000:Int32) AND (bid.price < 1000000:Int32)), count filter((bid.price >= 1000000:Int32)), min(bid.price), max(bid.price), sum(bid.price), count(bid.price), sum(bid.price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr93, bid.price, bid._row_id] } + StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr91, bid.price, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.price, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode - Table 0 { columns: [bid_auction, $expr93, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price), sum(bid_price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } + Table 0 { columns: [bid_auction, $expr91, count, count_0, count filter((bid_price < 10000:Int32)), count filter((bid_price >= 10000:Int32) AND (bid_price < 1000000:Int32)), count filter((bid_price >= 1000000:Int32)), min(bid_price), max(bid_price), sum(bid_price), count(bid_price), sum(bid_price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 before: @@ -1069,13 +1069,13 @@ └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id] } - └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr139, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr140, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr141, bid._row_id] } + └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr136, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr137, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr138, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, bid._row_id(hidden)], pk_columns: [bid._row_id] } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr139, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr140, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr141, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr136, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr137, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr138, bid._row_id] } Chain { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } Upstream BatchPlanNode @@ -1182,7 +1182,7 @@ stream_plan: | StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } - └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr48), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } + └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr47), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } ├─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } | └─StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction), count(bid.auction)] } | └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } @@ -1191,7 +1191,7 @@ | └─StreamExchange { dist: HashShard(bid.auction) } | └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr48] } + └─StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr47] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(bid.auction)] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [bid.auction, count] } @@ -1203,7 +1203,7 @@ StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } materialized table: 4294967294 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } - StreamDynamicFilter { predicate: (count(bid.auction) >= $expr48), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } + StreamDynamicFilter { predicate: (count(bid.auction) >= $expr47), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } left table: 0, right table 1 StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction), count(bid.auction)] } @@ -1225,7 +1225,7 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr48] } + StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr47] } StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(bid.auction)] } result table: 7, state tables: [] StreamExchange Single from 4 @@ -1242,7 +1242,7 @@ BatchPlanNode Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), count(bid_auction)_0], primary key: [$3 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } - Table 1 { columns: [$expr48], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr47], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [auction_id, auction_item_name, count, count(bid_auction), count(bid_auction)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4], distribution key: [0] } Table 3 { columns: [auction_id, auction_item_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [auction_id, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 4c9f19efa3b0..93417b548a57 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -87,7 +87,7 @@ stream_plan: | StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], pk_columns: [_row_id] } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr49, date_time, _row_id] } + └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr48, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | @@ -97,7 +97,7 @@ StreamExchange Hash([4]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr49, date_time, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr48, date_time, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -230,7 +230,7 @@ └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | StreamMaterialize { columns: [category, avg], pk_columns: [category] } - └─StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr48] } + └─StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr47] } └─StreamHashAgg { group_key: [category], aggs: [count, sum(max(price)), count(max(price))] } └─StreamExchange { dist: HashShard(category) } └─StreamProject { exprs: [id, category, max(price)] } @@ -250,7 +250,7 @@ Fragment 0 StreamMaterialize { columns: [category, avg], pk_columns: [category] } materialized table: 4294967294 - StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr48] } + StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr47] } StreamHashAgg { group_key: [category], aggs: [count, sum(max(price)), count(max(price))] } result table: 0, state tables: [] StreamExchange Hash([1]) from 1 @@ -350,7 +350,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 764 } + | └─StreamShare { id = 754 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -362,7 +362,7 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 764 } + └─StreamShare { id = 754 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -474,31 +474,31 @@ └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr54, price] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr110(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr110, price, max(price)] } - └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr110, max(price)] } - └─StreamFilter { predicate: (date_time >= $expr112) AND (date_time <= $expr110) } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr108(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr108, price, max(price)] } + └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr108, max(price)] } + └─StreamFilter { predicate: (date_time >= $expr110) AND (date_time <= $expr108) } └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 446 } + | └─StreamShare { id = 440 } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(max(price)) } - └─StreamProject { exprs: [max(price), $expr110, ($expr110 - '00:00:10':Interval) as $expr112] } - └─StreamAppendOnlyHashAgg { group_key: [$expr110], aggs: [count, max(price)] } - └─StreamExchange { dist: HashShard($expr110) } - └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr110, price, _row_id] } - └─StreamShare { id = 446 } + └─StreamProject { exprs: [max(price), $expr108, ($expr108 - '00:00:10':Interval) as $expr110] } + └─StreamAppendOnlyHashAgg { group_key: [$expr108], aggs: [count, max(price)] } + └─StreamExchange { dist: HashShard($expr108) } + └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr108, price, _row_id] } + └─StreamShare { id = 440 } └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr110(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr110, price, max(price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr108(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr108, price, max(price)] } materialized table: 4294967294 - StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr110, max(price)] } - StreamFilter { predicate: (date_time >= $expr112) AND (date_time <= $expr110) } + StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr108, max(price)] } + StreamFilter { predicate: (date_time >= $expr110) AND (date_time <= $expr108) } StreamHashJoin { type: Inner, predicate: price = max(price), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([2]) from 1 @@ -515,22 +515,22 @@ source state table: 4 Fragment 3 - StreamProject { exprs: [max(price), $expr110, ($expr110 - '00:00:10':Interval) as $expr112] } - StreamAppendOnlyHashAgg { group_key: [$expr110], aggs: [count, max(price)] } + StreamProject { exprs: [max(price), $expr108, ($expr108 - '00:00:10':Interval) as $expr110] } + StreamAppendOnlyHashAgg { group_key: [$expr108], aggs: [count, max(price)] } result table: 5, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr110, price, _row_id] } + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr108, price, _row_id] } StreamExchange Hash([4]) from 2 Table 0 { columns: [auction, bidder, price, date_time, _row_id], primary key: [$2 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [2] } Table 1 { columns: [price, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 2 { columns: [max(price), $expr110, $expr112], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(price), $expr110, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 2 { columns: [max(price), $expr108, $expr110], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(price), $expr108, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 4 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 5 { columns: [$expr110, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr110, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } + Table 5 { columns: [$expr108, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr108, max(price)], primary key: [$4 ASC, $5 ASC, $1 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [1] } - id: nexmark_q8 before: - create_sources @@ -580,60 +580,60 @@ └─BatchProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr97, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr98] } └─BatchSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"], filter: (None, None) } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr204(hidden), seller(hidden), $expr201(hidden), $expr202(hidden)], pk_columns: [id, name, starttime, $expr204, seller, $expr201, $expr202] } - └─StreamHashJoin { type: Inner, predicate: id = seller AND $expr203 = $expr201 AND $expr204 = $expr202, output: all } - ├─StreamExchange { dist: HashShard(id, $expr203, $expr204) } - | └─StreamProject { exprs: [id, name, $expr203, $expr204] } - | └─StreamAppendOnlyHashAgg { group_key: [id, name, $expr203, $expr204], aggs: [count] } - | └─StreamExchange { dist: HashShard(id, name, $expr203, $expr204) } - | └─StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr203, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr204, _row_id] } + StreamMaterialize { columns: [id, name, starttime, $expr200(hidden), seller(hidden), $expr197(hidden), $expr198(hidden)], pk_columns: [id, name, starttime, $expr200, seller, $expr197, $expr198] } + └─StreamHashJoin { type: Inner, predicate: id = seller AND $expr199 = $expr197 AND $expr200 = $expr198, output: all } + ├─StreamExchange { dist: HashShard(id, $expr199, $expr200) } + | └─StreamProject { exprs: [id, name, $expr199, $expr200] } + | └─StreamAppendOnlyHashAgg { group_key: [id, name, $expr199, $expr200], aggs: [count] } + | └─StreamExchange { dist: HashShard(id, name, $expr199, $expr200) } + | └─StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr199, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr200, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"] } - └─StreamProject { exprs: [seller, $expr201, $expr202] } - └─StreamAppendOnlyHashAgg { group_key: [seller, $expr201, $expr202], aggs: [count] } - └─StreamExchange { dist: HashShard(seller, $expr201, $expr202) } - └─StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr201, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr202, _row_id] } + └─StreamProject { exprs: [seller, $expr197, $expr198] } + └─StreamAppendOnlyHashAgg { group_key: [seller, $expr197, $expr198], aggs: [count] } + └─StreamExchange { dist: HashShard(seller, $expr197, $expr198) } + └─StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr197, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr198, _row_id] } └─StreamRowIdGen { row_id_index: 9 } └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr204(hidden), seller(hidden), $expr201(hidden), $expr202(hidden)], pk_columns: [id, name, starttime, $expr204, seller, $expr201, $expr202] } + StreamMaterialize { columns: [id, name, starttime, $expr200(hidden), seller(hidden), $expr197(hidden), $expr198(hidden)], pk_columns: [id, name, starttime, $expr200, seller, $expr197, $expr198] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: id = seller AND $expr203 = $expr201 AND $expr204 = $expr202, output: all } + StreamHashJoin { type: Inner, predicate: id = seller AND $expr199 = $expr197 AND $expr200 = $expr198, output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([0, 2, 3]) from 1 - StreamProject { exprs: [seller, $expr201, $expr202] } - StreamAppendOnlyHashAgg { group_key: [seller, $expr201, $expr202], aggs: [count] } + StreamProject { exprs: [seller, $expr197, $expr198] } + StreamAppendOnlyHashAgg { group_key: [seller, $expr197, $expr198], aggs: [count] } result table: 6, state tables: [] StreamExchange Hash([0, 1, 2]) from 3 Fragment 1 - StreamProject { exprs: [id, name, $expr203, $expr204] } - StreamAppendOnlyHashAgg { group_key: [id, name, $expr203, $expr204], aggs: [count] } + StreamProject { exprs: [id, name, $expr199, $expr200] } + StreamAppendOnlyHashAgg { group_key: [id, name, $expr199, $expr200], aggs: [count] } result table: 4, state tables: [] StreamExchange Hash([0, 1, 2, 3]) from 2 Fragment 2 - StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr203, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr204, _row_id] } + StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr199, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr200, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"] } source state table: 5 Fragment 3 - StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr201, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr202, _row_id] } + StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr197, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr198, _row_id] } StreamRowIdGen { row_id_index: 9 } StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } source state table: 7 - Table 0 { columns: [id, name, $expr203, $expr204], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } - Table 1 { columns: [id, $expr203, $expr204, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [seller, $expr201, $expr202], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [seller, $expr201, $expr202, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [id, name, $expr203, $expr204, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } + Table 0 { columns: [id, name, $expr199, $expr200], primary key: [$0 ASC, $2 ASC, $3 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0, 2, 3] } + Table 1 { columns: [id, $expr199, $expr200, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [seller, $expr197, $expr198], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [seller, $expr197, $expr198, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [id, name, $expr199, $expr200, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2, 3] } Table 5 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 6 { columns: [seller, $expr201, $expr202, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 6 { columns: [seller, $expr197, $expr198, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } Table 7 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } - Table 4294967294 { columns: [id, name, starttime, $expr204, seller, $expr201, $expr202], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } + Table 4294967294 { columns: [id, name, starttime, $expr200, seller, $expr197, $expr198], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [0, 2, 3] } - id: nexmark_q9 before: - create_sources @@ -728,7 +728,7 @@ stream_plan: | StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], pk_columns: [_row_id] } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr97, ToChar(date_time, 'HH:MI':Varchar) as $expr98, _row_id] } + └─StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr95, ToChar(date_time, 'HH:MI':Varchar) as $expr96, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | @@ -738,7 +738,7 @@ StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr97, ToChar(date_time, 'HH:MI':Varchar) as $expr98, _row_id] } + StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr95, ToChar(date_time, 'HH:MI':Varchar) as $expr96, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -822,7 +822,7 @@ stream_plan: | StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], pk_columns: [_row_id] } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr97, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr98, date_time, extra, _row_id] } + └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr95, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr96, date_time, extra, _row_id] } └─StreamFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -833,7 +833,7 @@ StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr97, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr98, date_time, extra, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr95, Case(((Extract('HOUR':Varchar, date_time) >= 8:Int32) AND (Extract('HOUR':Varchar, date_time) <= 18:Int32)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Int32) OR (Extract('HOUR':Varchar, date_time) >= 20:Int32)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr96, date_time, extra, _row_id] } StreamFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -873,40 +873,40 @@ └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } - └─StreamProject { exprs: [$expr49, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [$expr49], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamExchange { dist: HashShard($expr49) } - └─StreamProject { exprs: [$expr49, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [$expr49, bidder, auction, flag], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard($expr49, bidder, auction, flag) } - └─StreamExpand { column_subsets: [[$expr49], [$expr49, bidder], [$expr49, auction]] } - └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr49, price, bidder, auction, _row_id] } + └─StreamProject { exprs: [$expr48, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr48], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamExchange { dist: HashShard($expr48) } + └─StreamProject { exprs: [$expr48, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamAppendOnlyHashAgg { group_key: [$expr48, bidder, auction, flag], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr48, bidder, auction, flag) } + └─StreamExpand { column_subsets: [[$expr48], [$expr48, bidder], [$expr48, auction]] } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr48, price, bidder, auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [day] } materialized table: 4294967294 - StreamProject { exprs: [$expr49, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - StreamHashAgg { group_key: [$expr49], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamProject { exprs: [$expr48, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [$expr48], aggs: [count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [$expr49, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [$expr49, bidder, auction, flag], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + StreamProject { exprs: [$expr48, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [$expr48, bidder, auction, flag], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } result table: 1, state tables: [] StreamExchange Hash([0, 2, 3, 10]) from 2 Fragment 2 - StreamExpand { column_subsets: [[$expr49], [$expr49, bidder], [$expr49, auction]] } - StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr49, price, bidder, auction, _row_id] } + StreamExpand { column_subsets: [[$expr48], [$expr48, bidder], [$expr48, auction]] } + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr48, price, bidder, auction, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 2 - Table 0 { columns: [$expr49, count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } - Table 1 { columns: [$expr49, bidder, auction, flag, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32))_0, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_0, count filter((price >= 1000000:Int32))_0, count filter((price < 10000:Int32))_1, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_1, count filter((price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1, 2, 3] } + Table 0 { columns: [$expr48, count, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC], value indices: [1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0] } + Table 1 { columns: [$expr48, bidder, auction, flag, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32))_0, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_0, count filter((price >= 1000000:Int32))_0, count filter((price < 10000:Int32))_1, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_1, count filter((price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1, 2, 3] } Table 2 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12], distribution key: [0] } - id: nexmark_q16 @@ -943,41 +943,41 @@ └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } - └─StreamProject { exprs: [channel, $expr97, max(max($expr98)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamHashAgg { group_key: [channel, $expr97], aggs: [count, max(max($expr98)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - └─StreamExchange { dist: HashShard(channel, $expr97) } - └─StreamProject { exprs: [channel, $expr97, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr98), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─StreamAppendOnlyHashAgg { group_key: [channel, $expr97, bidder, auction, flag], aggs: [count, max($expr98), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - └─StreamExchange { dist: HashShard(channel, $expr97, bidder, auction, flag) } - └─StreamExpand { column_subsets: [[channel, $expr97, $expr98], [channel, $expr97, bidder], [channel, $expr97, auction]] } - └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr97, ToChar(date_time, 'HH:mm':Varchar) as $expr98, price, bidder, auction, _row_id] } + └─StreamProject { exprs: [channel, $expr95, max(max($expr96)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [channel, $expr95], aggs: [count, max(max($expr96)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamExchange { dist: HashShard(channel, $expr95) } + └─StreamProject { exprs: [channel, $expr95, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr96), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamAppendOnlyHashAgg { group_key: [channel, $expr95, bidder, auction, flag], aggs: [count, max($expr96), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(channel, $expr95, bidder, auction, flag) } + └─StreamExpand { column_subsets: [[channel, $expr95, $expr96], [channel, $expr95, bidder], [channel, $expr95, auction]] } + └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr95, ToChar(date_time, 'HH:mm':Varchar) as $expr96, price, bidder, auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], pk_columns: [channel, day] } materialized table: 4294967294 - StreamProject { exprs: [channel, $expr97, max(max($expr98)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } - StreamHashAgg { group_key: [channel, $expr97], aggs: [count, max(max($expr98)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamProject { exprs: [channel, $expr95, max(max($expr96)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + StreamHashAgg { group_key: [channel, $expr95], aggs: [count, max(max($expr96)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } result table: 1, state tables: [0] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [channel, $expr97, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr98), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } - StreamAppendOnlyHashAgg { group_key: [channel, $expr97, bidder, auction, flag], aggs: [count, max($expr98), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + StreamProject { exprs: [channel, $expr95, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr96), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + StreamAppendOnlyHashAgg { group_key: [channel, $expr95, bidder, auction, flag], aggs: [count, max($expr96), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 4, 5, 14]) from 2 Fragment 2 - StreamExpand { column_subsets: [[channel, $expr97, $expr98], [channel, $expr97, bidder], [channel, $expr97, auction]] } - StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr97, ToChar(date_time, 'HH:mm':Varchar) as $expr98, price, bidder, auction, _row_id] } + StreamExpand { column_subsets: [[channel, $expr95, $expr96], [channel, $expr95, bidder], [channel, $expr95, auction]] } + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr95, ToChar(date_time, 'HH:mm':Varchar) as $expr96, price, bidder, auction, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 3 - Table 0 { columns: [channel, $expr97, max($expr98), bidder, auction, flag], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0, 1] } - Table 1 { columns: [channel, $expr97, count, max(max($expr98)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } - Table 2 { columns: [channel, $expr97, bidder, auction, flag, count, max($expr98), count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32))_0, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_0, count filter((price >= 1000000:Int32))_0, count filter((price < 10000:Int32))_1, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_1, count filter((price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0, 1, 2, 3, 4] } + Table 0 { columns: [channel, $expr95, max($expr96), bidder, auction, flag], primary key: [$0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0, 1] } + Table 1 { columns: [channel, $expr95, count, max(max($expr96)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder) filter((flag = 1:Int64)), count(bidder) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction) filter((flag = 2:Int64)), count(auction) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15], distribution key: [0, 1] } + Table 2 { columns: [channel, $expr95, bidder, auction, flag, count, max($expr96), count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count filter((price < 10000:Int32))_0, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_0, count filter((price >= 1000000:Int32))_0, count filter((price < 10000:Int32))_1, count filter((price >= 10000:Int32) AND (price < 1000000:Int32))_1, count filter((price >= 1000000:Int32))_1], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [0, 1, 2, 3, 4] } Table 3 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0, 1] } - id: nexmark_q17 @@ -1006,28 +1006,28 @@ └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } stream_plan: | StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } - └─StreamProject { exprs: [auction, $expr95, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr97, sum(price)] } - └─StreamAppendOnlyHashAgg { group_key: [auction, $expr95], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)] } - └─StreamExchange { dist: HashShard(auction, $expr95) } - └─StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr95, price, _row_id] } + └─StreamProject { exprs: [auction, $expr93, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr95, sum(price)] } + └─StreamAppendOnlyHashAgg { group_key: [auction, $expr93], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)] } + └─StreamExchange { dist: HashShard(auction, $expr93) } + └─StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr93, price, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], pk_columns: [auction, day] } materialized table: 4294967294 - StreamProject { exprs: [auction, $expr95, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr97, sum(price)] } - StreamAppendOnlyHashAgg { group_key: [auction, $expr95], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)] } + StreamProject { exprs: [auction, $expr93, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price)::Decimal / count(price)) as $expr95, sum(price)] } + StreamAppendOnlyHashAgg { group_key: [auction, $expr93], aggs: [count, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr95, price, _row_id] } + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr93, price, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 1 - Table 0 { columns: [auction, $expr95, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } + Table 0 { columns: [auction, $expr93, count, count_0, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), sum(price)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11], distribution key: [0, 1] } Table 1 { columns: [partition_id, offset], primary key: [$0 ASC], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: nexmark_q18 @@ -1193,7 +1193,7 @@ stream_plan: | StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], pk_columns: [_row_id] } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr145, SplitPart(url, '/':Varchar, 5:Int32) as $expr146, SplitPart(url, '/':Varchar, 6:Int32) as $expr147, _row_id] } + └─StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr142, SplitPart(url, '/':Varchar, 5:Int32) as $expr143, SplitPart(url, '/':Varchar, 6:Int32) as $expr144, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } stream_dist_plan: | @@ -1203,7 +1203,7 @@ StreamExchange Hash([7]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr145, SplitPart(url, '/':Varchar, 5:Int32) as $expr146, SplitPart(url, '/':Varchar, 6:Int32) as $expr147, _row_id] } + StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr142, SplitPart(url, '/':Varchar, 5:Int32) as $expr143, SplitPart(url, '/':Varchar, 6:Int32) as $expr144, _row_id] } StreamRowIdGen { row_id_index: 7 } StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 0 @@ -1321,7 +1321,7 @@ stream_plan: | StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } └─StreamProject { exprs: [id, item_name, count(auction)] } - └─StreamDynamicFilter { predicate: (count(auction) >= $expr50), output: [id, item_name, count(auction), count(auction)] } + └─StreamDynamicFilter { predicate: (count(auction) >= $expr49), output: [id, item_name, count(auction), count(auction)] } ├─StreamProject { exprs: [id, item_name, count(auction), count(auction)] } | └─StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction), count(auction)] } | └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } @@ -1331,19 +1331,19 @@ | | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "_row_id"] } | └─StreamExchange { dist: HashShard(auction) } | └─StreamProject { exprs: [auction, _row_id] } - | └─StreamShare { id = 571 } + | └─StreamShare { id = 562 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum0(count) / count(auction)) as $expr50] } + └─StreamProject { exprs: [(sum0(count) / count(auction)) as $expr49] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(auction)] } └─StreamExchange { dist: Single } └─StreamProject { exprs: [auction, count] } └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } └─StreamExchange { dist: HashShard(auction) } └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 571 } + └─StreamShare { id = 562 } └─StreamProject { exprs: [auction, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1352,7 +1352,7 @@ StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], pk_columns: [auction_id, auction_item_name] } materialized table: 4294967294 StreamProject { exprs: [id, item_name, count(auction)] } - StreamDynamicFilter { predicate: (count(auction) >= $expr50), output: [id, item_name, count(auction), count(auction)] } + StreamDynamicFilter { predicate: (count(auction) >= $expr49), output: [id, item_name, count(auction), count(auction)] } left table: 0, right table 1 StreamProject { exprs: [id, item_name, count(auction), count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction), count(auction)] } @@ -1380,7 +1380,7 @@ source state table: 8 Fragment 4 - StreamProject { exprs: [(sum0(count) / count(auction)) as $expr50] } + StreamProject { exprs: [(sum0(count) / count(auction)) as $expr49] } StreamGlobalSimpleAgg { aggs: [count, sum0(count), count(auction)] } result table: 9, state tables: [] StreamExchange Single from 5 @@ -1396,7 +1396,7 @@ StreamExchange Hash([1]) from 3 Table 0 { columns: [id, item_name, count(auction), count(auction)_0], primary key: [$3 ASC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } - Table 1 { columns: [$expr50], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr49], primary key: [], value indices: [0], distribution key: [] } Table 2 { columns: [id, item_name, count, count(auction), count(auction)_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4], distribution key: [0] } Table 3 { columns: [id, item_name, _row_id], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 4 { columns: [id, _row_id, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 0db43a65141a..984fbb2cf7b3 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -64,8 +64,8 @@ └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr23] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, $expr47(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr47, t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr47, t._row_id] } + StreamMaterialize { columns: [v1, v2, $expr46(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr46, t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr46, t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); @@ -124,17 +124,17 @@ create table t (x int, y int, z int); select x, y from t order by x + y, z; optimized_logical_plan: | - LogicalProject { exprs: [t.x, t.y, (t.x + t.y) as $expr20, t.z] } + LogicalProject { exprs: [t.x, t.y, (t.x + t.y) as $expr19, t.z] } └─LogicalScan { table: t, columns: [t.x, t.y, t.z] } batch_plan: | BatchProject { exprs: [t.x, t.y] } - └─BatchExchange { order: [$expr42 ASC, t.z ASC], dist: Single } - └─BatchSort { order: [$expr42 ASC, t.z ASC] } - └─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr42, t.z] } + └─BatchExchange { order: [$expr41 ASC, t.z ASC], dist: Single } + └─BatchSort { order: [$expr41 ASC, t.z ASC] } + └─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr41, t.z] } └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [x, y, $expr66(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr66, t.z, t._row_id] } - └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr66, t.z, t._row_id] } + StreamMaterialize { columns: [x, y, $expr64(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr64, t.z, t._row_id] } + └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr64, t.z, t._row_id] } └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by the number of an output column sql: | diff --git a/src/frontend/planner_test/tests/testdata/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/over_window_function.yaml index 4e4cff6b7b27..e3775e9f77c4 100644 --- a/src/frontend/planner_test/tests/testdata/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/over_window_function.yaml @@ -290,11 +290,11 @@ stream_plan: | StreamMaterialize { columns: [window_start, window_end, supplier_id, price, cnt], pk_columns: [window_start, window_end, supplier_id] } └─StreamGroupTopN { order: "[sum(bid.price) DESC]", limit: 3, offset: 0, group_key: [0, 1] } - └─StreamExchange { dist: HashShard($expr49, $expr50) } - └─StreamProject { exprs: [$expr49, $expr50, bid.supplier_id, sum(bid.price), count] } - └─StreamHashAgg { group_key: [$expr49, $expr50, bid.supplier_id], aggs: [count, sum(bid.price), count] } - └─StreamExchange { dist: HashShard($expr49, $expr50, bid.supplier_id) } - └─StreamProject { exprs: [TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr49, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr50, bid.supplier_id, bid.price, bid._row_id] } + └─StreamExchange { dist: HashShard($expr47, $expr48) } + └─StreamProject { exprs: [$expr47, $expr48, bid.supplier_id, sum(bid.price), count] } + └─StreamHashAgg { group_key: [$expr47, $expr48, bid.supplier_id], aggs: [count, sum(bid.price), count] } + └─StreamExchange { dist: HashShard($expr47, $expr48, bid.supplier_id) } + └─StreamProject { exprs: [TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr47, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr48, bid.supplier_id, bid.price, bid._row_id] } └─StreamTableScan { table: bid, columns: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - before: - create_bid @@ -310,10 +310,10 @@ stream_plan: | StreamMaterialize { columns: [window_start, window_end, supplier_id, price, bid._row_id(hidden)], pk_columns: [bid._row_id] } └─StreamExchange { dist: HashShard(bid._row_id) } - └─StreamProject { exprs: [$expr47, $expr48, bid.supplier_id, bid.price, bid._row_id] } + └─StreamProject { exprs: [$expr45, $expr46, bid.supplier_id, bid.price, bid._row_id] } └─StreamGroupTopN { order: "[bid.price DESC]", limit: 3, offset: 0, group_key: [5, 6] } - └─StreamExchange { dist: HashShard($expr47, $expr48) } - └─StreamProject { exprs: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id, TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr47, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr48] } + └─StreamExchange { dist: HashShard($expr45, $expr46) } + └─StreamProject { exprs: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id, TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr45, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr46] } └─StreamTableScan { table: bid, columns: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } - name: Deduplication sql: | diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index 65b089a9421f..59567b31c598 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -108,7 +108,7 @@ └─LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr1, (t.v3 * t.v4) as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4, t.ts, t._row_id] } optimized_logical_plan: | - LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr41, (t.v3 * t.v4) as $expr42] } + LogicalProject { exprs: [t.v1, t.v2, t.v3, t.v4, (t.v1 + t.v2) as $expr39, (t.v3 * t.v4) as $expr40] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t.v4], predicate: ((t.v1 + t.v2) > 10:Int32) AND (t.v1 = 10:Int32) AND (t.v2 = 20:Int32) AND (t.v3 = 30:Int32) AND ((t.v1 + t.v2) > (t.v3 * t.v4)) AND ((t.v1 + t.v2) > t.v1) } - name: filter agg transpose sql: | @@ -264,10 +264,10 @@ StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, v1, v2] } └─StreamHashJoin { type: Inner, predicate: t1.v1 = t2.v2, output: [t1.v1, t2.v2, t1._row_id, t2._row_id] } ├─StreamExchange { dist: HashShard(t1.v1) } - | └─StreamDynamicFilter { predicate: (t1.v1 > $expr13), output: [t1.v1, t1._row_id] } + | └─StreamDynamicFilter { predicate: (t1.v1 > $expr12), output: [t1.v1, t1._row_id] } | ├─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } | └─StreamExchange { dist: Broadcast } - | └─StreamProject { exprs: [(now + '01:00:00':Interval) as $expr13], watermark_columns: [(now + '01:00:00':Interval)] } + | └─StreamProject { exprs: [(now + '01:00:00':Interval) as $expr12], watermark_columns: [(now + '01:00:00':Interval)] } | └─StreamNow { output: [now] } └─StreamExchange { dist: HashShard(t2.v2) } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } @@ -302,11 +302,11 @@ └─LogicalScan { table: t1, columns: [t1.v1, t1.v2], predicate: (t1.v2 > 5:Int32) } stream_plan: | StreamMaterialize { columns: [v1, v2, t1._row_id(hidden)], pk_columns: [t1._row_id] } - └─StreamDynamicFilter { predicate: (t1.v1 > $expr13), output: [t1.v1, t1.v2, t1._row_id] } + └─StreamDynamicFilter { predicate: (t1.v1 > $expr12), output: [t1.v1, t1.v2, t1._row_id] } ├─StreamFilter { predicate: (t1.v2 > 5:Int32) } | └─StreamTableScan { table: t1, columns: [t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now + '00:30:00':Interval) as $expr13], watermark_columns: [(now + '00:30:00':Interval)] } + └─StreamProject { exprs: [(now + '00:30:00':Interval) as $expr12], watermark_columns: [(now + '00:30:00':Interval)] } └─StreamNow { output: [now] } - name: eq-predicate derived condition other side pushdown in inner join sql: | diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 7fa5f6029c68..9444c038a90d 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -48,7 +48,7 @@ | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 539 } + | └─StreamShare { id = 529 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 10 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 539 } + └─StreamShare { id = 529 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 10 } @@ -125,7 +125,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 764 } + | └─StreamShare { id = 754 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -137,7 +137,7 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 764 } + └─StreamShare { id = 754 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 7 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -150,7 +150,7 @@ └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } | └─StreamProject { exprs: [sum0(count), 0:Int32] } - | └─StreamShare { id = 213 } + | └─StreamShare { id = 210 } | └─StreamProject { exprs: [sum0(count)] } | └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } | └─StreamExchange { dist: Single } @@ -158,7 +158,7 @@ | └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } └─StreamExchange { dist: HashShard(1:Int32) } └─StreamProject { exprs: [sum0(count), 1:Int32] } - └─StreamShare { id = 213 } + └─StreamShare { id = 210 } └─StreamProject { exprs: [sum0(count)] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count)] } └─StreamExchange { dist: Single } @@ -198,13 +198,13 @@ └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = id, output: [_row_id, id, _row_id, id] } ├─StreamExchange { dist: HashShard(id) } | └─StreamProject { exprs: [id, _row_id] } - | └─StreamShare { id = 156 } + | └─StreamShare { id = 152 } | └─StreamProject { exprs: [id, _row_id] } | └─StreamRowIdGen { row_id_index: 10 } | └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } └─StreamExchange { dist: HashShard(id) } └─StreamProject { exprs: [id, _row_id] } - └─StreamShare { id = 156 } + └─StreamShare { id = 152 } └─StreamProject { exprs: [id, _row_id] } └─StreamRowIdGen { row_id_index: 10 } └─StreamSource { source: "auction", columns: ["id", "item_name", "description", "initial_bid", "reserve", "date_time", "expires", "seller", "category", "extra", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index 8377a1504c92..206a89e9e22f 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -22,22 +22,22 @@ └─LogicalFilter { predicate: (t1.y > 0:Int32) } └─LogicalScan { table: t1, columns: [t1.x, t1.y, t1._row_id] } stream_plan: | - StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr90(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr90, z, a] } - └─StreamHashJoin { type: Inner, predicate: $expr90 = $expr91, output: [$expr90, $expr91, $expr92, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr90] } - ├─StreamExchange { dist: HashShard($expr90) } - | └─StreamProject { exprs: [$expr90, t1._row_id] } - | └─StreamShare { id = 212 } - | └─StreamProject { exprs: [(t1.x + t1.y) as $expr90, t1._row_id] } + StreamMaterialize { columns: [z, a, b, t1._row_id(hidden), t1._row_id#1(hidden), t1._row_id#2(hidden), t1.x(hidden), $expr86(hidden)], pk_columns: [t1._row_id, t1._row_id#1, t1._row_id#2, t1.x, $expr86, z, a] } + └─StreamHashJoin { type: Inner, predicate: $expr86 = $expr87, output: [$expr86, $expr87, $expr88, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr86] } + ├─StreamExchange { dist: HashShard($expr86) } + | └─StreamProject { exprs: [$expr86, t1._row_id] } + | └─StreamShare { id = 207 } + | └─StreamProject { exprs: [(t1.x + t1.y) as $expr86, t1._row_id] } | └─StreamFilter { predicate: (t1.y > 0:Int32) } | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard($expr91) } - └─StreamProject { exprs: [(t1.x * $expr90) as $expr91, (t1.y * $expr90) as $expr92, t1._row_id, t1._row_id, t1.x, $expr90] } - └─StreamHashJoin { type: Inner, predicate: t1.x = $expr90, output: [t1.x, t1.y, $expr90, t1._row_id, t1._row_id] } + └─StreamExchange { dist: HashShard($expr87) } + └─StreamProject { exprs: [(t1.x * $expr86) as $expr87, (t1.y * $expr86) as $expr88, t1._row_id, t1._row_id, t1.x, $expr86] } + └─StreamHashJoin { type: Inner, predicate: t1.x = $expr86, output: [t1.x, t1.y, $expr86, t1._row_id, t1._row_id] } ├─StreamExchange { dist: HashShard(t1.x) } | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard($expr90) } - └─StreamProject { exprs: [$expr90, t1._row_id] } - └─StreamShare { id = 212 } - └─StreamProject { exprs: [(t1.x + t1.y) as $expr90, t1._row_id] } + └─StreamExchange { dist: HashShard($expr86) } + └─StreamProject { exprs: [$expr86, t1._row_id] } + └─StreamShare { id = 207 } + └─StreamProject { exprs: [(t1.x + t1.y) as $expr86, t1._row_id] } └─StreamFilter { predicate: (t1.y > 0:Int32) } └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml index 9835d86dedf5..2bfef446c612 100644 --- a/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery_expr_correlated.yaml @@ -14,10 +14,10 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) AND (t2.y = 1000:Int32) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan: | - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr31 > $expr32), output: [t1.x, t1.y] } - ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr31] } + LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr29 > $expr30), output: [t1.x, t1.y] } + ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr29] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t1.y, (1.5:Decimal * min(t2.x)) as $expr32] } + └─LogicalProject { exprs: [t1.y, (1.5:Decimal * min(t2.x)) as $expr30] } └─LogicalAgg { group_key: [t1.y], aggs: [min(t2.x)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, t2.x] } ├─LogicalAgg { group_key: [t1.y], aggs: [] } @@ -95,10 +95,10 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan: | - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr31 > $expr32), output: [t1.x, t1.y] } - ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr31] } + LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr29 > $expr30), output: [t1.x, t1.y] } + ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Decimal as $expr29] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t1.y, (1.5:Decimal * min(t2.x)) as $expr32] } + └─LogicalProject { exprs: [t1.y, (1.5:Decimal * min(t2.x)) as $expr30] } └─LogicalAgg { group_key: [t1.y], aggs: [min(t2.x)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, t2.x] } ├─LogicalAgg { group_key: [t1.y], aggs: [] } @@ -120,8 +120,8 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan: | - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr10 > count(1:Int32)), output: [t1.x, t1.y] } - ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr10] } + LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr9 > count(1:Int32)), output: [t1.x, t1.y] } + ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr9] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } └─LogicalAgg { group_key: [t1.y], aggs: [count(1:Int32)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, 1:Int32] } @@ -144,10 +144,10 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 1, correlated_id: 1 } = t2.y) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan: | - LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr31 > $expr32), output: [t1.x, t1.y] } - ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr31] } + LogicalJoin { type: Inner, on: IsNotDistinctFrom(t1.y, t1.y) AND ($expr29 > $expr30), output: [t1.x, t1.y] } + ├─LogicalProject { exprs: [t1.x, t1.y, t1.x::Int64 as $expr29] } | └─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t1.y, (count(1:Int32) + count(1:Int32)) as $expr32] } + └─LogicalProject { exprs: [t1.y, (count(1:Int32) + count(1:Int32)) as $expr30] } └─LogicalAgg { group_key: [t1.y], aggs: [count(1:Int32), count(1:Int32)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(t1.y, t2.y), output: [t1.y, 1:Int32] } ├─LogicalAgg { group_key: [t1.y], aggs: [] } @@ -202,9 +202,9 @@ └─LogicalFilter { predicate: (CorrelatedInputRef { index: 0, correlated_id: 1 } > (t2.x + 1000:Int32)) } └─LogicalScan { table: t2, columns: [t2.x, t2.y, t2._row_id] } optimized_logical_plan: | - LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x > $expr11), output: [t1.x] } + LogicalJoin { type: LeftSemi, on: (t1.y = t2.y) AND (t1.x > $expr10), output: [t1.x] } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalProject { exprs: [t2.y, (t2.x + 1000:Int32) as $expr11] } + └─LogicalProject { exprs: [t2.y, (t2.x + 1000:Int32) as $expr10] } └─LogicalScan { table: t2, columns: [t2.x, t2.y] } - sql: | create table t1(x int, y int); @@ -485,8 +485,8 @@ select count(*) from a where a.x=3 and a.y = (select count(*) from b where b.z = a.z and a.x = 3); optimized_logical_plan: | LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.x, a.x) AND IsNotDistinctFrom(a.z, a.z) AND ($expr10 = count(1:Int32)), output: [] } - ├─LogicalProject { exprs: [a.x, a.z, a.y::Int64 as $expr10] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.x, a.x) AND IsNotDistinctFrom(a.z, a.z) AND ($expr9 = count(1:Int32)), output: [] } + ├─LogicalProject { exprs: [a.x, a.z, a.y::Int64 as $expr9] } | └─LogicalScan { table: a, columns: [a.x, a.y, a.z], predicate: (a.x = 3:Int32) } └─LogicalAgg { group_key: [a.x, a.z], aggs: [count(1:Int32)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.x, a.x) AND IsNotDistinctFrom(a.z, a.z), output: [a.x, a.z, 1:Int32] } @@ -503,8 +503,8 @@ select count(*) from a where a.x=3 and a.y = (select count(*) from b where b.z = a.z); optimized_logical_plan: | LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr10 = count(1:Int32)), output: [] } - ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr10] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr9 = count(1:Int32)), output: [] } + ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr9] } | └─LogicalScan { table: a, output_columns: [a.y, a.z], required_columns: [a.y, a.z, a.x], predicate: (a.x = 3:Int32) } └─LogicalAgg { group_key: [a.z], aggs: [count(1:Int32)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, 1:Int32] } @@ -532,8 +532,8 @@ select count(*) from a where a.y = (select count(distinct x) from b where b.z = a.z); optimized_logical_plan: | LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr10 = count(b.x)), output: [] } - ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr10] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr9 = count(b.x)), output: [] } + ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr9] } | └─LogicalScan { table: a, columns: [a.y, a.z] } └─LogicalAgg { group_key: [a.z], aggs: [count(b.x)] } └─LogicalAgg { group_key: [a.z, b.x], aggs: [] } @@ -548,8 +548,8 @@ select count(*) from a where a.y = (select count(x) filter(where x < 100) from b where b.z = a.z); optimized_logical_plan: | LogicalAgg { aggs: [count] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr10 = count(b.x) filter((b.x < 100:Int32))), output: [] } - ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr10] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(a.z, a.z) AND ($expr9 = count(b.x) filter((b.x < 100:Int32))), output: [] } + ├─LogicalProject { exprs: [a.z, a.y::Int64 as $expr9] } | └─LogicalScan { table: a, columns: [a.y, a.z] } └─LogicalAgg { group_key: [a.z], aggs: [count(b.x) filter((b.x < 100:Int32))] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(a.z, b.z), output: [a.z, b.x] } @@ -658,9 +658,9 @@ optimized_logical_plan: | LogicalJoin { type: LeftSemi, on: IsNotDistinctFrom(t1.x, t2.x), output: all } ├─LogicalScan { table: t1, columns: [t1.x, t1.y] } - └─LogicalJoin { type: LeftSemi, on: (t2.y = $expr25) AND IsNotDistinctFrom(t2.y, t2.y) AND IsNotDistinctFrom(t2.x, t3.x), output: [t2.x] } + └─LogicalJoin { type: LeftSemi, on: (t2.y = $expr24) AND IsNotDistinctFrom(t2.y, t2.y) AND IsNotDistinctFrom(t2.x, t3.x), output: [t2.x] } ├─LogicalScan { table: t2, columns: [t2.x, t2.y], predicate: IsNotNull(t2.x) } - └─LogicalProject { exprs: [t3.x, t2.y, (t3.y + t2.y) as $expr25] } + └─LogicalProject { exprs: [t3.x, t2.y, (t3.y + t2.y) as $expr24] } └─LogicalJoin { type: Inner, on: true, output: [t3.x, t2.y, t3.y] } ├─LogicalAgg { group_key: [t2.y], aggs: [] } | └─LogicalScan { table: t2, columns: [t2.y] } diff --git a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml index 3983e0134979..e0af04db818b 100644 --- a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml @@ -6,8 +6,8 @@ stream_plan: | StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.ts, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr13 > now), output: [t1.ts, $expr13, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, (t1.ts + '01:00:00':Interval) as $expr13, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr12 > now), output: [t1.ts, $expr12, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, (t1.ts + '01:00:00':Interval) as $expr12, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamNow { output: [now] } @@ -18,8 +18,8 @@ stream_plan: | StreamMaterialize { columns: [ts, time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.ts, t1.time_to_live, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr13 > now), output: [t1.ts, t1.time_to_live, $expr13, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (t1.ts + (t1.time_to_live * 1.5:Decimal)) as $expr13, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr12 > now), output: [t1.ts, t1.time_to_live, $expr12, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (t1.ts + (t1.time_to_live * 1.5:Decimal)) as $expr12, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.time_to_live, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } └─StreamNow { output: [now] } @@ -30,11 +30,11 @@ stream_plan: | StreamMaterialize { columns: [ts, additional_time_to_live, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, t1._row_id] } - └─StreamDynamicFilter { predicate: ($expr25 > $expr26), output: [t1.ts, t1.additional_time_to_live, $expr25, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (t1.ts + (t1.additional_time_to_live * 1.5:Decimal)) as $expr25, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr23 > $expr24), output: [t1.ts, t1.additional_time_to_live, $expr23, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (t1.ts + (t1.additional_time_to_live * 1.5:Decimal)) as $expr23, t1._row_id] } | └─StreamTableScan { table: t1, columns: [t1.ts, t1.additional_time_to_live, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '00:15:00':Interval) as $expr26], watermark_columns: [(now - '00:15:00':Interval)] } + └─StreamProject { exprs: [(now - '00:15:00':Interval) as $expr24], watermark_columns: [(now - '00:15:00':Interval)] } └─StreamNow { output: [now] } - name: Temporal filter fails without `now()` in lower bound sql: |- @@ -47,12 +47,12 @@ select * from t1 where ts < now() - interval '1 hour' and ts >= now() - interval '2 hour'; stream_plan: | StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id] } - └─StreamDynamicFilter { predicate: (t1.ts < $expr26), output: [t1.ts, t1._row_id] } - ├─StreamDynamicFilter { predicate: (t1.ts >= $expr25), output: [t1.ts, t1._row_id] } + └─StreamDynamicFilter { predicate: (t1.ts < $expr24), output: [t1.ts, t1._row_id] } + ├─StreamDynamicFilter { predicate: (t1.ts >= $expr23), output: [t1.ts, t1._row_id] } | ├─StreamTableScan { table: t1, columns: [t1.ts, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } | └─StreamExchange { dist: Broadcast } - | └─StreamProject { exprs: [(now - '02:00:00':Interval) as $expr25], watermark_columns: [(now - '02:00:00':Interval)] } + | └─StreamProject { exprs: [(now - '02:00:00':Interval) as $expr23], watermark_columns: [(now - '02:00:00':Interval)] } | └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '01:00:00':Interval) as $expr26], watermark_columns: [(now - '01:00:00':Interval)] } + └─StreamProject { exprs: [(now - '01:00:00':Interval) as $expr24], watermark_columns: [(now - '01:00:00':Interval)] } └─StreamNow { output: [now] } diff --git a/src/frontend/planner_test/tests/testdata/time_window.yaml b/src/frontend/planner_test/tests/testdata/time_window.yaml index 1c9039111a3f..678147aeeadd 100644 --- a/src/frontend/planner_test/tests/testdata/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/time_window.yaml @@ -137,7 +137,7 @@ └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [v1, window_end, avg], pk_columns: [v1, window_end] } - └─StreamProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr48] } + └─StreamProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr47] } └─StreamHashAgg { group_key: [t.v1, window_end], aggs: [count, sum(t.v3), count(t.v3)] } └─StreamExchange { dist: HashShard(t.v1, window_end) } └─StreamHopWindow { time_col: t.v2, slide: 00:01:00, size: 00:10:00, output: [t.v1, t.v3, window_end, t._row_id] } @@ -160,7 +160,7 @@ └─BatchScan { table: t1, columns: [t1.id, t1.v1, t1.created_at], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [id, v1, created_at, window_start, window_end, t1._row_id(hidden)], pk_columns: [t1._row_id] } - └─StreamProject { exprs: [t1.id, t1.v1, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr97, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr98, t1._row_id] } + └─StreamProject { exprs: [t1.id, t1.v1, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr95, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr96, t1._row_id] } └─StreamFilter { predicate: (t1.v1 >= 10:Int32) } └─StreamTableScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 9dd22ffa79cd..7594908c7850 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -125,44 +125,44 @@ └─LogicalFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } optimized_logical_plan: | - LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr96), sum($expr97), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr98, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr99, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr100, count] } - └─LogicalAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr96), sum($expr97), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } - └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr96, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr97, lineitem.l_discount] } + LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr91), sum($expr92), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr93, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr94, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr95, count] } + └─LogicalAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr91), sum($expr92), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + └─LogicalProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr91, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr92, lineitem.l_discount] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus], required_columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } batch_plan: | BatchExchange { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], dist: Single } - └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr208), sum($expr209), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr210, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr211, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr212, count] } + └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr203), sum($expr204), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr205, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr206, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr207, count] } └─BatchSort { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC] } - └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr208), sum($expr209), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr203), sum($expr204), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } - └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr208, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr209, lineitem.l_discount] } + └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr203, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr204, lineitem.l_discount] } └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus] } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr330), sum($expr331), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr335, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr336, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr337, count] } - └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr330), sum($expr331), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr320), sum($expr321), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr325, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr326, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr327, count] } + └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr320), sum($expr321), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr330, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr331, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr320, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr321, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus] } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr330), sum($expr331), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr335, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr336, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr337, count] } - StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr330), sum($expr331), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr320), sum($expr321), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr325, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr326, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr327, count] } + StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr320), sum($expr321), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr330, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr331, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr320, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr321, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, count, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr330), sum($expr331), sum(lineitem_l_quantity)_0, count(lineitem_l_quantity), sum(lineitem_l_extendedprice)_0, count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0, 1] } + Table 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, count, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr320), sum($expr321), sum(lineitem_l_quantity)_0, count(lineitem_l_quantity), sum(lineitem_l_extendedprice)_0, count(lineitem_l_extendedprice), sum(lineitem_l_discount), count(lineitem_l_discount), count_0], primary key: [$0 ASC, $1 ASC], value indices: [2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [0, 1] } Table 4294967294 { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9], distribution key: [0, 1] } - id: tpch_q2 before: @@ -560,23 +560,23 @@ | └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } optimized_logical_plan: | - LogicalTopN { order: "[sum($expr20) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } - └─LogicalProject { exprs: [lineitem.l_orderkey, sum($expr20), orders.o_orderdate, orders.o_shippriority] } - └─LogicalAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr20)] } - └─LogicalProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr20] } + LogicalTopN { order: "[sum($expr19) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + └─LogicalProject { exprs: [lineitem.l_orderkey, sum($expr19), orders.o_orderdate, orders.o_shippriority] } + └─LogicalAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr19)] } + └─LogicalProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr19] } └─LogicalJoin { type: Inner, on: (lineitem.l_orderkey = orders.o_orderkey), output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } ├─LogicalJoin { type: Inner, on: (customer.c_custkey = orders.o_custkey), output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority] } | ├─LogicalScan { table: customer, output_columns: [customer.c_custkey], required_columns: [customer.c_custkey, customer.c_mktsegment], predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } | └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], predicate: (orders.o_orderdate < '1995-03-29':Date) } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate > '1995-03-29':Date) } batch_plan: | - BatchTopN { order: "[sum($expr42) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + BatchTopN { order: "[sum($expr41) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } └─BatchExchange { order: [], dist: Single } - └─BatchTopN { order: "[sum($expr42) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } - └─BatchProject { exprs: [lineitem.l_orderkey, sum($expr42), orders.o_orderdate, orders.o_shippriority] } - └─BatchHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr42)] } + └─BatchTopN { order: "[sum($expr41) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + └─BatchProject { exprs: [lineitem.l_orderkey, sum($expr41), orders.o_orderdate, orders.o_shippriority] } + └─BatchHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr41)] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } - └─BatchProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr42] } + └─BatchProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr41] } └─BatchHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority] } @@ -593,15 +593,15 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority] } - └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr66), orders.o_orderdate, orders.o_shippriority] } - └─StreamTopN { order: "[sum($expr66) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr64), orders.o_orderdate, orders.o_shippriority] } + └─StreamTopN { order: "[sum($expr64) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: "[sum($expr66) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } - └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr66), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr67] } - └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr66), orders.o_orderdate, orders.o_shippriority] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr66)] } + └─StreamGroupTopN { order: "[sum($expr64) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr64), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr65] } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr64), orders.o_orderdate, orders.o_shippriority] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr64)] } └─StreamExchange { dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } - └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr66, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr64, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey, orders.o_custkey] } @@ -620,22 +620,22 @@ Fragment 0 StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], pk_columns: [l_orderkey, o_orderdate, o_shippriority], order_descs: [revenue, o_orderdate, l_orderkey, o_shippriority] } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr66), orders.o_orderdate, orders.o_shippriority] } - StreamTopN { order: "[sum($expr66) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr64), orders.o_orderdate, orders.o_shippriority] } + StreamTopN { order: "[sum($expr64) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr66) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + StreamGroupTopN { order: "[sum($expr64) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } state table: 1 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr66), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr67] } - StreamProject { exprs: [lineitem.l_orderkey, sum($expr66), orders.o_orderdate, orders.o_shippriority] } - StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr66)] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr64), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr65] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr64), orders.o_orderdate, orders.o_shippriority] } + StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr64)] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 2 - StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr66, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr64, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([0]) from 3 @@ -667,9 +667,9 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_orderkey, sum($expr66), orders_o_orderdate, orders_o_shippriority, $expr67], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } - Table 1 { columns: [lineitem_l_orderkey, sum($expr66), orders_o_orderdate, orders_o_shippriority, $expr67], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } - Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, count, sum($expr66)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [lineitem_l_orderkey, sum($expr64), orders_o_orderdate, orders_o_shippriority, $expr65], primary key: [$1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [] } + Table 1 { columns: [lineitem_l_orderkey, sum($expr64), orders_o_orderdate, orders_o_shippriority, $expr65], primary key: [$4 ASC, $1 DESC, $2 ASC, $0 ASC, $3 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0, 2, 3], vnode column idx: 4 } + Table 2 { columns: [lineitem_l_orderkey, orders_o_orderdate, orders_o_shippriority, count, sum($expr64)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate, orders_o_shippriority, customer_c_custkey, orders_o_custkey], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -827,8 +827,8 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } optimized_logical_plan: | - LogicalAgg { group_key: [nation.n_name], aggs: [sum($expr20)] } - └─LogicalProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr20] } + LogicalAgg { group_key: [nation.n_name], aggs: [sum($expr19)] } + └─LogicalProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr19] } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey] } | ├─LogicalJoin { type: Inner, on: (lineitem.l_orderkey = orders.o_orderkey) AND (lineitem.l_suppkey = supplier.s_suppkey), output: [supplier.s_nationkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -841,11 +841,11 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey] } └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'MIDDLE EAST':Varchar) } batch_plan: | - BatchExchange { order: [sum($expr42) DESC], dist: Single } - └─BatchSort { order: [sum($expr42) DESC] } - └─BatchHashAgg { group_key: [nation.n_name], aggs: [sum($expr42)] } + BatchExchange { order: [sum($expr41) DESC], dist: Single } + └─BatchSort { order: [sum($expr41) DESC] } + └─BatchHashAgg { group_key: [nation.n_name], aggs: [sum($expr41)] } └─BatchExchange { order: [], dist: HashShard(nation.n_name) } - └─BatchProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr42] } + └─BatchProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr41] } └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'MIDDLE EAST':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey] } @@ -867,10 +867,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name] } - └─StreamProject { exprs: [nation.n_name, sum($expr66)] } - └─StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr66)] } + └─StreamProject { exprs: [nation.n_name, sum($expr64)] } + └─StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr64)] } └─StreamExchange { dist: HashShard(nation.n_name) } - └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr66, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr64, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_regionkey, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey] } @@ -900,13 +900,13 @@ Fragment 0 StreamMaterialize { columns: [n_name, revenue], pk_columns: [n_name], order_descs: [revenue, n_name] } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, sum($expr66)] } - StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr66)] } + StreamProject { exprs: [nation.n_name, sum($expr64)] } + StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr64)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr66, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr64, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, customer.c_custkey, orders.o_orderkey, orders.o_custkey, supplier.s_suppkey, customer.c_nationkey, supplier.s_nationkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([3]) from 2 @@ -970,7 +970,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, count, sum($expr66)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [nation_n_name, count, sum($expr64)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey], primary key: [$3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [3] } Table 2 { columns: [nation_n_regionkey, customer_c_custkey, orders_o_orderkey, orders_o_custkey, supplier_s_suppkey, customer_c_nationkey, supplier_s_nationkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC], value indices: [11], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1012,43 +1012,43 @@ └─LogicalFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } optimized_logical_plan: | - LogicalAgg { aggs: [sum($expr20)] } - └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr20] } + LogicalAgg { aggs: [sum($expr19)] } + └─LogicalProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr19] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } batch_plan: | - BatchSimpleAgg { aggs: [sum(sum($expr43))] } + BatchSimpleAgg { aggs: [sum(sum($expr42))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum($expr43)] } - └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr43] } + └─BatchSimpleAgg { aggs: [sum($expr42)] } + └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr42] } └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─BatchScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [revenue], pk_columns: [] } - └─StreamProject { exprs: [sum(sum($expr68))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr68))] } + └─StreamProject { exprs: [sum(sum($expr66))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr66))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr68)] } - └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr68, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr66)] } + └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr66, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 StreamMaterialize { columns: [revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr68))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr68))] } + StreamProject { exprs: [sum(sum($expr66))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr66))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr68)] } - StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr68, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr66)] } + StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr66, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr68))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr66))], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [revenue], primary key: [], value indices: [0], distribution key: [] } - id: tpch_q7 before: @@ -1111,8 +1111,8 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } optimized_logical_plan: | - LogicalAgg { group_key: [nation.n_name, nation.n_name, $expr41], aggs: [sum($expr42)] } - └─LogicalProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr41, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr42] } + LogicalAgg { group_key: [nation.n_name, nation.n_name, $expr39], aggs: [sum($expr40)] } + └─LogicalProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr39, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr40] } └─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey) AND (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))), output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, nation.n_name] } ├─LogicalJoin { type: Inner, on: (customer.c_custkey = orders.o_custkey), output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey] } | ├─LogicalJoin { type: Inner, on: (orders.o_orderkey = lineitem.l_orderkey), output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, orders.o_custkey] } @@ -1125,11 +1125,11 @@ | └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name] } batch_plan: | - BatchExchange { order: [nation.n_name ASC, nation.n_name ASC, $expr87 ASC], dist: Single } - └─BatchSort { order: [nation.n_name ASC, nation.n_name ASC, $expr87 ASC] } - └─BatchHashAgg { group_key: [nation.n_name, nation.n_name, $expr87], aggs: [sum($expr88)] } - └─BatchExchange { order: [], dist: HashShard(nation.n_name, nation.n_name, $expr87) } - └─BatchProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr87, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr88] } + BatchExchange { order: [nation.n_name ASC, nation.n_name ASC, $expr85 ASC], dist: Single } + └─BatchSort { order: [nation.n_name ASC, nation.n_name ASC, $expr85 ASC] } + └─BatchHashAgg { group_key: [nation.n_name, nation.n_name, $expr85], aggs: [sum($expr86)] } + └─BatchExchange { order: [], dist: HashShard(nation.n_name, nation.n_name, $expr85) } + └─BatchProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr85, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr86] } └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey AND (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))), output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, nation.n_name] } └─BatchExchange { order: [], dist: UpstreamHashShard(customer.c_nationkey) } └─BatchLookupJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_name, customer.c_nationkey] } @@ -1146,10 +1146,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year] } - └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr139, sum($expr140)] } - └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr139], aggs: [count, sum($expr140)] } - └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr139) } - └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr139, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr140, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } + └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr135, sum($expr136)] } + └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr135], aggs: [count, sum($expr136)] } + └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr135) } + └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr135, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr136, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } └─StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: all } ├─StreamExchange { dist: HashShard(customer.c_nationkey) } @@ -1177,13 +1177,13 @@ Fragment 0 StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], pk_columns: [supp_nation, cust_nation, l_year] } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, nation.n_name, $expr139, sum($expr140)] } - StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr139], aggs: [count, sum($expr140)] } + StreamProject { exprs: [nation.n_name, nation.n_name, $expr135, sum($expr136)] } + StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr135], aggs: [count, sum($expr136)] } result table: 0, state tables: [] StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr139, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr140, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } + StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr135, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr136, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey] } StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -1245,7 +1245,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, nation_n_name_0, $expr139, count, sum($expr140)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [nation_n_name, nation_n_name_0, $expr135, count, sum($expr136)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_shipdate, nation_n_name, customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, orders_o_custkey, customer_c_custkey], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $12 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13], distribution key: [4] } Table 2 { columns: [customer_c_nationkey, supplier_s_suppkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [nation_n_nationkey, nation_n_name], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1330,9 +1330,9 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } └─LogicalScan { table: region, columns: [region.r_regionkey, region.r_name, region.r_comment] } optimized_logical_plan: | - LogicalProject { exprs: [$expr80, RoundDigit((sum($expr81) / sum($expr82)), 6:Int32) as $expr83] } - └─LogicalAgg { group_key: [$expr80], aggs: [sum($expr81), sum($expr82)] } - └─LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr80, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr81, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr82] } + LogicalProject { exprs: [$expr76, RoundDigit((sum($expr77) / sum($expr78)), 6:Int32) as $expr79] } + └─LogicalAgg { group_key: [$expr76], aggs: [sum($expr77), sum($expr78)] } + └─LogicalProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr76, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr77, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr78] } └─LogicalJoin { type: Inner, on: (nation.n_regionkey = region.r_regionkey), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } | ├─LogicalJoin { type: Inner, on: (orders.o_custkey = customer.c_custkey), output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, customer.c_nationkey] } @@ -1349,12 +1349,12 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey] } └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'ASIA':Varchar) } batch_plan: | - BatchExchange { order: [$expr171 ASC], dist: Single } - └─BatchProject { exprs: [$expr171, RoundDigit((sum($expr172) / sum($expr173)), 6:Int32) as $expr174] } - └─BatchSort { order: [$expr171 ASC] } - └─BatchHashAgg { group_key: [$expr171], aggs: [sum($expr172), sum($expr173)] } - └─BatchExchange { order: [], dist: HashShard($expr171) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr171, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr172, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr173] } + BatchExchange { order: [$expr167 ASC], dist: Single } + └─BatchProject { exprs: [$expr167, RoundDigit((sum($expr168) / sum($expr169)), 6:Int32) as $expr170] } + └─BatchSort { order: [$expr167 ASC] } + └─BatchHashAgg { group_key: [$expr167], aggs: [sum($expr168), sum($expr169)] } + └─BatchExchange { order: [], dist: HashShard($expr167) } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr167, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr168, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr169] } └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } @@ -1376,10 +1376,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } - └─StreamProject { exprs: [$expr270, RoundDigit((sum($expr271) / sum($expr272)), 6:Int32) as $expr274] } - └─StreamHashAgg { group_key: [$expr270], aggs: [count, sum($expr271), sum($expr272)] } - └─StreamExchange { dist: HashShard($expr270) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr270, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr271, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr272, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamProject { exprs: [$expr262, RoundDigit((sum($expr263) / sum($expr264)), 6:Int32) as $expr266] } + └─StreamHashAgg { group_key: [$expr262], aggs: [count, sum($expr263), sum($expr264)] } + └─StreamExchange { dist: HashShard($expr262) } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr262, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr263, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr264, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } @@ -1418,13 +1418,13 @@ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } materialized table: 4294967294 - StreamProject { exprs: [$expr270, RoundDigit((sum($expr271) / sum($expr272)), 6:Int32) as $expr274] } - StreamHashAgg { group_key: [$expr270], aggs: [count, sum($expr271), sum($expr272)] } + StreamProject { exprs: [$expr262, RoundDigit((sum($expr263) / sum($expr264)), 6:Int32) as $expr266] } + StreamHashAgg { group_key: [$expr262], aggs: [count, sum($expr263), sum($expr264)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr270, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr271, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr272, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr262, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr263, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr264, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 @@ -1511,7 +1511,7 @@ Upstream BatchPlanNode - Table 0 { columns: [$expr270, count, sum($expr271), sum($expr272)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [$expr262, count, sum($expr263), sum($expr264)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1595,9 +1595,9 @@ | └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } optimized_logical_plan: | - LogicalProject { exprs: [nation.n_name, $expr60, RoundDigit(sum($expr61), 2:Int32) as $expr62] } - └─LogicalAgg { group_key: [nation.n_name, $expr60], aggs: [sum($expr61)] } - └─LogicalProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr60, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr61] } + LogicalProject { exprs: [nation.n_name, $expr57, RoundDigit(sum($expr58), 2:Int32) as $expr59] } + └─LogicalAgg { group_key: [nation.n_name, $expr57], aggs: [sum($expr58)] } + └─LogicalProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr57, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr58] } └─LogicalJoin { type: Inner, on: (orders.o_orderkey = lineitem.l_orderkey), output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name] } ├─LogicalJoin { type: Inner, on: (partsupp.ps_suppkey = lineitem.l_suppkey) AND (partsupp.ps_partkey = lineitem.l_partkey), output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost] } | ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } @@ -1610,12 +1610,12 @@ | └─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost] } └─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate] } batch_plan: | - BatchExchange { order: [nation.n_name ASC, $expr128 DESC], dist: Single } - └─BatchProject { exprs: [nation.n_name, $expr128, RoundDigit(sum($expr129), 2:Int32) as $expr130] } - └─BatchSort { order: [nation.n_name ASC, $expr128 DESC] } - └─BatchHashAgg { group_key: [nation.n_name, $expr128], aggs: [sum($expr129)] } - └─BatchExchange { order: [], dist: HashShard(nation.n_name, $expr128) } - └─BatchProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr128, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr129] } + BatchExchange { order: [nation.n_name ASC, $expr125 DESC], dist: Single } + └─BatchProject { exprs: [nation.n_name, $expr125, RoundDigit(sum($expr126), 2:Int32) as $expr127] } + └─BatchSort { order: [nation.n_name ASC, $expr125 DESC] } + └─BatchHashAgg { group_key: [nation.n_name, $expr125], aggs: [sum($expr126)] } + └─BatchExchange { order: [], dist: HashShard(nation.n_name, $expr125) } + └─BatchProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr125, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr126] } └─BatchLookupJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name] } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_orderkey) } └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = partsupp.ps_partkey AND lineitem.l_suppkey = partsupp.ps_suppkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost] } @@ -1633,10 +1633,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year] } - └─StreamProject { exprs: [nation.n_name, $expr202, RoundDigit(sum($expr203), 2:Int32) as $expr205] } - └─StreamHashAgg { group_key: [nation.n_name, $expr202], aggs: [count, sum($expr203)] } - └─StreamExchange { dist: HashShard(nation.n_name, $expr202) } - └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr202, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr203, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } + └─StreamProject { exprs: [nation.n_name, $expr196, RoundDigit(sum($expr197), 2:Int32) as $expr199] } + └─StreamHashAgg { group_key: [nation.n_name, $expr196], aggs: [count, sum($expr197)] } + └─StreamExchange { dist: HashShard(nation.n_name, $expr196) } + └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr196, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr197, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } | └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = partsupp.ps_suppkey AND lineitem.l_partkey = partsupp.ps_partkey, output: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, partsupp.ps_supplycost, part.p_partkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey] } @@ -1664,13 +1664,13 @@ Fragment 0 StreamMaterialize { columns: [nation, o_year, sum_profit], pk_columns: [nation, o_year] } materialized table: 4294967294 - StreamProject { exprs: [nation.n_name, $expr202, RoundDigit(sum($expr203), 2:Int32) as $expr205] } - StreamHashAgg { group_key: [nation.n_name, $expr202], aggs: [count, sum($expr203)] } + StreamProject { exprs: [nation.n_name, $expr196, RoundDigit(sum($expr197), 2:Int32) as $expr199] } + StreamHashAgg { group_key: [nation.n_name, $expr196], aggs: [count, sum($expr197)] } result table: 0, state tables: [] StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr202, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr203, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } + StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr196, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr197, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, orders.o_orderkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -1732,7 +1732,7 @@ Upstream BatchPlanNode - Table 0 { columns: [nation_n_name, $expr202, count, sum($expr203)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [nation_n_name, $expr196, count, sum($expr197)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } Table 1 { columns: [lineitem_l_orderkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, partsupp_ps_supplycost, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey], primary key: [$0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14], distribution key: [0] } Table 2 { columns: [lineitem_l_orderkey, part_p_partkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC], value indices: [10], distribution key: [0] } Table 3 { columns: [orders_o_orderkey, orders_o_orderdate], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -1804,10 +1804,10 @@ | └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } optimized_logical_plan: | - LogicalTopN { order: "[sum($expr20) DESC]", limit: 20, offset: 0 } - └─LogicalProject { exprs: [customer.c_custkey, customer.c_name, sum($expr20), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └─LogicalAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr20)] } - └─LogicalProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr20] } + LogicalTopN { order: "[sum($expr19) DESC]", limit: 20, offset: 0 } + └─LogicalProject { exprs: [customer.c_custkey, customer.c_name, sum($expr19), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └─LogicalAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr19)] } + └─LogicalProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr19] } └─LogicalJoin { type: Inner, on: (lineitem.l_orderkey = orders.o_orderkey), output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } ├─LogicalJoin { type: Inner, on: (customer.c_nationkey = nation.n_nationkey), output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name] } | ├─LogicalJoin { type: Inner, on: (customer.c_custkey = orders.o_custkey), output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey] } @@ -1816,13 +1816,13 @@ | └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_returnflag], predicate: (lineitem.l_returnflag = 'R':Varchar) } batch_plan: | - BatchTopN { order: "[sum($expr42) DESC]", limit: 20, offset: 0 } + BatchTopN { order: "[sum($expr41) DESC]", limit: 20, offset: 0 } └─BatchExchange { order: [], dist: Single } - └─BatchTopN { order: "[sum($expr42) DESC]", limit: 20, offset: 0 } - └─BatchProject { exprs: [customer.c_custkey, customer.c_name, sum($expr42), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └─BatchHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr42)] } + └─BatchTopN { order: "[sum($expr41) DESC]", limit: 20, offset: 0 } + └─BatchProject { exprs: [customer.c_custkey, customer.c_name, sum($expr41), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └─BatchHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr41)] } └─BatchExchange { order: [], dist: HashShard(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) } - └─BatchProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr42] } + └─BatchProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr41] } └─BatchHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name] } @@ -1840,15 +1840,15 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_returnflag], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment] } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr66), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └─StreamTopN { order: "[sum($expr66) DESC]", limit: 20, offset: 0 } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr64), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └─StreamTopN { order: "[sum($expr64) DESC]", limit: 20, offset: 0 } └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: "[sum($expr66) DESC]", limit: 20, offset: 0, group_key: [8] } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr66), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr67] } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr66), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr66)] } + └─StreamGroupTopN { order: "[sum($expr64) DESC]", limit: 20, offset: 0, group_key: [8] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr64), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr65] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr64), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr64)] } └─StreamExchange { dist: HashShard(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr66, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr64, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, orders.o_orderkey, nation.n_name, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } @@ -1870,22 +1870,22 @@ Fragment 0 StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], pk_columns: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], order_descs: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment] } materialized table: 4294967294 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr66), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamTopN { order: "[sum($expr66) DESC]", limit: 20, offset: 0 } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr64), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamTopN { order: "[sum($expr64) DESC]", limit: 20, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr66) DESC]", limit: 20, offset: 0, group_key: [8] } + StreamGroupTopN { order: "[sum($expr64) DESC]", limit: 20, offset: 0, group_key: [8] } state table: 1 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr66), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr67] } - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr66), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr66)] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr64), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment) as $expr65] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr64), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [count, sum($expr64)] } result table: 2, state tables: [] StreamExchange Hash([0, 1, 2, 3, 4, 5, 6]) from 2 Fragment 2 - StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr66, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr64, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([6]) from 3 @@ -1927,9 +1927,9 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr66), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr67], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } - Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr66), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr67], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } - Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, count, sum($expr66)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } + Table 0 { columns: [customer_c_custkey, customer_c_name, sum($expr64), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr65], primary key: [$2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [] } + Table 1 { columns: [customer_c_custkey, customer_c_name, sum($expr64), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr65], primary key: [$8 ASC, $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [0, 1, 3, 6, 4, 5, 7], vnode column idx: 8 } + Table 2 { columns: [customer_c_custkey, customer_c_name, customer_c_acctbal, customer_c_phone, nation_n_name, customer_c_address, customer_c_comment, count, sum($expr64)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7, 8], distribution key: [0, 1, 2, 3, 4, 5, 6] } Table 3 { columns: [customer_c_custkey, customer_c_name, customer_c_address, customer_c_phone, customer_c_acctbal, customer_c_comment, orders_o_orderkey, nation_n_name, orders_o_custkey, customer_c_nationkey, nation_n_nationkey], primary key: [$6 ASC, $0 ASC, $8 ASC, $10 ASC, $9 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10], distribution key: [6] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } Table 5 { columns: [lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -1997,39 +1997,39 @@ | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment] } └─LogicalScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey, nation.n_comment] } optimized_logical_plan: | - LogicalJoin { type: Inner, on: (sum($expr58) > $expr60), output: [partsupp.ps_partkey, sum($expr58)] } - ├─LogicalAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr58), sum($expr58)] } - | └─LogicalProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr58] } + LogicalJoin { type: Inner, on: (sum($expr55) > $expr57), output: [partsupp.ps_partkey, sum($expr55)] } + ├─LogicalAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr55), sum($expr55)] } + | └─LogicalProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr55] } | └─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost] } | ├─LogicalJoin { type: Inner, on: (partsupp.ps_suppkey = supplier.s_suppkey), output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey] } | | ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost] } | | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey] } | └─LogicalScan { table: nation, output_columns: [nation.n_nationkey], required_columns: [nation.n_nationkey, nation.n_name], predicate: (nation.n_name = 'ARGENTINA':Varchar) } - └─LogicalProject { exprs: [(sum($expr59) * 0.0001000000:Decimal) as $expr60] } - └─LogicalAgg { aggs: [sum($expr59)] } - └─LogicalProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr59] } + └─LogicalProject { exprs: [(sum($expr56) * 0.0001000000:Decimal) as $expr57] } + └─LogicalAgg { aggs: [sum($expr56)] } + └─LogicalProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr56] } └─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [partsupp.ps_availqty, partsupp.ps_supplycost] } ├─LogicalJoin { type: Inner, on: (partsupp.ps_suppkey = supplier.s_suppkey), output: [partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey] } | ├─LogicalScan { table: partsupp, columns: [partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost] } | └─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey] } └─LogicalScan { table: nation, output_columns: [nation.n_nationkey], required_columns: [nation.n_nationkey, nation.n_name], predicate: (nation.n_name = 'ARGENTINA':Varchar) } batch_plan: | - BatchSort { order: [sum($expr124) DESC] } - └─BatchNestedLoopJoin { type: Inner, predicate: (sum($expr124) > $expr126), output: [partsupp.ps_partkey, sum($expr124)] } + BatchSort { order: [sum($expr121) DESC] } + └─BatchNestedLoopJoin { type: Inner, predicate: (sum($expr121) > $expr123), output: [partsupp.ps_partkey, sum($expr121)] } ├─BatchExchange { order: [], dist: Single } - | └─BatchHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr124), sum($expr124)] } + | └─BatchHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr121), sum($expr121)] } | └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } - | └─BatchProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr124] } + | └─BatchProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr121] } | └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'ARGENTINA':Varchar), output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost] } | └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } | └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey] } | └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_suppkey) } | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - └─BatchProject { exprs: [(sum(sum($expr125)) * 0.0001000000:Decimal) as $expr126] } - └─BatchSimpleAgg { aggs: [sum(sum($expr125))] } + └─BatchProject { exprs: [(sum(sum($expr122)) * 0.0001000000:Decimal) as $expr123] } + └─BatchSimpleAgg { aggs: [sum(sum($expr122))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum($expr125)] } - └─BatchProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr125] } + └─BatchSimpleAgg { aggs: [sum($expr122)] } + └─BatchProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr122] } └─BatchLookupJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey AND (nation.n_name = 'ARGENTINA':Varchar), output: [partsupp.ps_availqty, partsupp.ps_supplycost] } └─BatchExchange { order: [], dist: UpstreamHashShard(supplier.s_nationkey) } └─BatchLookupJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey] } @@ -2037,12 +2037,12 @@ └─BatchScan { table: partsupp, columns: [partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey] } - └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr196)] } - └─StreamDynamicFilter { predicate: (sum($expr196) > $expr199), output: [partsupp.ps_partkey, sum($expr196), sum($expr196)] } - ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr196), sum($expr196)] } - | └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr196), sum($expr196)] } + └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr190)] } + └─StreamDynamicFilter { predicate: (sum($expr190) > $expr193), output: [partsupp.ps_partkey, sum($expr190), sum($expr190)] } + ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr190), sum($expr190)] } + | └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr190), sum($expr190)] } | └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - | └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr196, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + | └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr190, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } | └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } | ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } @@ -2055,11 +2055,11 @@ | └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum($expr197)) * 0.0001000000:Decimal) as $expr199] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr197))] } + └─StreamProject { exprs: [(sum(sum($expr191)) * 0.0001000000:Decimal) as $expr193] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr191))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr197)] } - └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr197, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr191)] } + └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr191, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } | └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey] } @@ -2075,17 +2075,17 @@ Fragment 0 StreamMaterialize { columns: [ps_partkey, value], pk_columns: [ps_partkey], order_descs: [value, ps_partkey] } materialized table: 4294967294 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr196)] } - StreamDynamicFilter { predicate: (sum($expr196) > $expr199), output: [partsupp.ps_partkey, sum($expr196), sum($expr196)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr190)] } + StreamDynamicFilter { predicate: (sum($expr190) > $expr193), output: [partsupp.ps_partkey, sum($expr190), sum($expr190)] } left table: 0, right table 1 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr196), sum($expr196)] } - StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr196), sum($expr196)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr190), sum($expr190)] } + StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr190), sum($expr190)] } result table: 2, state tables: [] StreamExchange Hash([0]) from 1 StreamExchange Broadcast from 6 Fragment 1 - StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr196, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr190, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } left table: 3, right table 5, left degree table: 4, right degree table: 6, StreamExchange Hash([3]) from 2 @@ -2115,14 +2115,14 @@ BatchPlanNode Fragment 6 - StreamProject { exprs: [(sum(sum($expr197)) * 0.0001000000:Decimal) as $expr199] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr197))] } + StreamProject { exprs: [(sum(sum($expr191)) * 0.0001000000:Decimal) as $expr193] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr191))] } result table: 11, state tables: [] StreamExchange Single from 7 Fragment 7 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr197)] } - StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr197, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr191)] } + StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr191, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } left table: 12, right table 14, left degree table: 13, right degree table: 15, StreamExchange Hash([2]) from 8 @@ -2151,9 +2151,9 @@ Upstream BatchPlanNode - Table 0 { columns: [partsupp_ps_partkey, sum($expr196), sum($expr196)_0], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [$expr199], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [partsupp_ps_partkey, count, sum($expr196), sum($expr196)_0], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [partsupp_ps_partkey, sum($expr190), sum($expr190)_0], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 1 { columns: [$expr193], primary key: [], value indices: [0], distribution key: [] } + Table 2 { columns: [partsupp_ps_partkey, count, sum($expr190), sum($expr190)_0], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 3 { columns: [partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey], primary key: [$3 ASC, $0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [3] } Table 4 { columns: [supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 5 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2162,7 +2162,7 @@ Table 8 { columns: [partsupp_ps_suppkey, partsupp_ps_partkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 9 { columns: [supplier_s_suppkey, supplier_s_nationkey], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 10 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 11 { columns: [count, sum(sum($expr197))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [count, sum(sum($expr191))], primary key: [], value indices: [0, 1], distribution key: [] } Table 12 { columns: [partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey], primary key: [$2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } Table 13 { columns: [supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } Table 14 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -2213,17 +2213,17 @@ ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderstatus, orders.o_totalprice, orders.o_orderdate, orders.o_orderpriority, orders.o_clerk, orders.o_shippriority, orders.o_comment] } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } optimized_logical_plan: | - LogicalAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr39), sum($expr40)] } - └─LogicalProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr39, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr40] } + LogicalAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr37), sum($expr38)] } + └─LogicalProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr37, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr38] } └─LogicalJoin { type: Inner, on: (orders.o_orderkey = lineitem.l_orderkey), output: [orders.o_orderpriority, lineitem.l_shipmode] } ├─LogicalScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_orderkey, lineitem.l_shipmode], required_columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } batch_plan: | BatchExchange { order: [lineitem.l_shipmode ASC], dist: Single } └─BatchSort { order: [lineitem.l_shipmode ASC] } - └─BatchHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr83), sum($expr84)] } + └─BatchHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr81), sum($expr82)] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_shipmode) } - └─BatchProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr83, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr84] } + └─BatchProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr81, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr82] } └─BatchHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } @@ -2233,10 +2233,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode] } - └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr131), sum($expr132)] } - └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr131), sum($expr132)] } + └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr127), sum($expr128)] } + └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr127), sum($expr128)] } └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } - └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr131, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr132, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr127, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr128, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } @@ -2248,13 +2248,13 @@ Fragment 0 StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode] } materialized table: 4294967294 - StreamProject { exprs: [lineitem.l_shipmode, sum($expr131), sum($expr132)] } - StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr131), sum($expr132)] } + StreamProject { exprs: [lineitem.l_shipmode, sum($expr127), sum($expr128)] } + StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr127), sum($expr128)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr131, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr132, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr127, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr128, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2272,7 +2272,7 @@ Upstream BatchPlanNode - Table 0 { columns: [lineitem_l_shipmode, count, sum($expr131), sum($expr132)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [lineitem_l_shipmode, count, sum($expr127), sum($expr128)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [orders_o_orderkey, orders_o_orderpriority], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 2 { columns: [orders_o_orderkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 3 { columns: [lineitem_l_orderkey, lineitem_l_shipmode, lineitem_l_linenumber], primary key: [$0 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0] } @@ -2408,18 +2408,18 @@ ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } optimized_logical_plan: | - LogicalProject { exprs: [((100.00:Decimal * sum($expr58)) / sum($expr59)) as $expr60] } - └─LogicalAgg { aggs: [sum($expr58), sum($expr59)] } - └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr58, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr59] } + LogicalProject { exprs: [((100.00:Decimal * sum($expr55)) / sum($expr56)) as $expr57] } + └─LogicalAgg { aggs: [sum($expr55), sum($expr56)] } + └─LogicalProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr55, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr56] } └─LogicalJoin { type: Inner, on: (lineitem.l_partkey = part.p_partkey), output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_type] } batch_plan: | - BatchProject { exprs: [((100.00:Decimal * sum(sum($expr124))) / sum(sum($expr125))) as $expr126] } - └─BatchSimpleAgg { aggs: [sum(sum($expr124)), sum(sum($expr125))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum($expr121))) / sum(sum($expr122))) as $expr123] } + └─BatchSimpleAgg { aggs: [sum(sum($expr121)), sum(sum($expr122))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum($expr124), sum($expr125)] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr124, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr125] } + └─BatchSimpleAgg { aggs: [sum($expr121), sum($expr122)] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr121, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr122] } └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -2427,11 +2427,11 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [promo_revenue], pk_columns: [] } - └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr196))) / sum(sum($expr197))) as $expr199] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr196)), sum(sum($expr197))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr190))) / sum(sum($expr191))) as $expr193] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr190)), sum(sum($expr191))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr196), sum($expr197)] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr196, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr197, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr190), sum($expr191)] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr190, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr191, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } | └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } @@ -2443,14 +2443,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum($expr196))) / sum(sum($expr197))) as $expr199] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr196)), sum(sum($expr197))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum($expr190))) / sum(sum($expr191))) as $expr193] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr190)), sum(sum($expr191))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr196), sum($expr197)] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr196, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr197, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr190), sum($expr191)] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr190, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr191, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr196)), sum(sum($expr197))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr190)), sum(sum($expr191))], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2535,79 +2535,79 @@ ├─LogicalJoin { type: Inner, on: (supplier.s_suppkey = lineitem.l_suppkey), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr20)] } | ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone] } | └─LogicalShare { id = 165 } - | └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr34)] } - | └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr34] } + | └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr32)] } + | └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr32] } | └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─LogicalAgg { aggs: [max(sum($expr20))] } └─LogicalShare { id = 165 } - └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr34)] } - └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr34] } + └─LogicalAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr32)] } + └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr32] } └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } batch_plan: | BatchExchange { order: [supplier.s_suppkey ASC], dist: Single } └─BatchSort { order: [supplier.s_suppkey ASC] } - └─BatchHashJoin { type: Inner, predicate: sum($expr75) = max(max(sum($expr74))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr75)] } - ├─BatchExchange { order: [], dist: HashShard(sum($expr75)) } - | └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr75)] } + └─BatchHashJoin { type: Inner, predicate: sum($expr73) = max(max(sum($expr72))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr73)] } + ├─BatchExchange { order: [], dist: HashShard(sum($expr73)) } + | └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr73)] } | ├─BatchExchange { order: [], dist: HashShard(supplier.s_suppkey) } | | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], distribution: UpstreamHashShard(supplier.s_suppkey) } - | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr75)] } + | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr73)] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } - | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr75] } + | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr73] } | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } | └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } - └─BatchExchange { order: [], dist: HashShard(max(max(sum($expr74)))) } - └─BatchSimpleAgg { aggs: [max(max(sum($expr74)))] } + └─BatchExchange { order: [], dist: HashShard(max(max(sum($expr72)))) } + └─BatchSimpleAgg { aggs: [max(max(sum($expr72)))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [max(sum($expr74))] } - └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr74)] } + └─BatchSimpleAgg { aggs: [max(sum($expr72))] } + └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr72)] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } - └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr74] } + └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr72] } └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr115)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr115)))] } - └─StreamHashJoin { type: Inner, predicate: sum($expr115) = max(max(sum($expr115))), output: all } - ├─StreamExchange { dist: HashShard(sum($expr115)) } - | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr115), lineitem.l_suppkey] } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr111)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr111)))] } + └─StreamHashJoin { type: Inner, predicate: sum($expr111) = max(max(sum($expr111))), output: all } + ├─StreamExchange { dist: HashShard(sum($expr111)) } + | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr111), lineitem.l_suppkey] } | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } - | └─StreamShare { id = 921 } - | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } - | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr115)] } + | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } + | └─StreamShare { id = 907 } + | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } + | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr111)] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr115, lineitem.l_orderkey, lineitem.l_linenumber] } + | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr111, lineitem.l_orderkey, lineitem.l_linenumber] } | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(max(max(sum($expr115)))) } - └─StreamProject { exprs: [max(max(sum($expr115)))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr115)))] } + └─StreamExchange { dist: HashShard(max(max(sum($expr111)))) } + └─StreamProject { exprs: [max(max(sum($expr111)))] } + └─StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr111)))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr116], aggs: [count, max(sum($expr115))] } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115), Vnode(lineitem.l_suppkey) as $expr116] } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } - └─StreamShare { id = 921 } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } - └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr115)] } + └─StreamHashAgg { group_key: [$expr112], aggs: [count, max(sum($expr111))] } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr111), Vnode(lineitem.l_suppkey) as $expr112] } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } + └─StreamShare { id = 907 } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } + └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr111)] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr115, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr111, lineitem.l_orderkey, lineitem.l_linenumber] } └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr115)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr115)))] } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr111)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr111)))] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: sum($expr115) = max(max(sum($expr115))), output: all } + StreamHashJoin { type: Inner, predicate: sum($expr111) = max(max(sum($expr111))), output: all } left table: 0, right table 2, left degree table: 1, right degree table: 3, StreamExchange Hash([4]) from 1 StreamExchange Hash([0]) from 5 Fragment 1 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr115), lineitem.l_suppkey] } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr111), lineitem.l_suppkey] } left table: 4, right table 6, left degree table: 5, right degree table: 7, StreamExchange Hash([0]) from 2 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } StreamExchange Hash([0]) from 3 Fragment 2 @@ -2616,45 +2616,45 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } - StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr115)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } + StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr111)] } result table: 8, state tables: [] StreamExchange Hash([0]) from 4 Fragment 4 - StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr115, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr111, lineitem.l_orderkey, lineitem.l_linenumber] } StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode Fragment 5 - StreamProject { exprs: [max(max(sum($expr115)))] } - StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr115)))] } + StreamProject { exprs: [max(max(sum($expr111)))] } + StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr111)))] } result table: 10, state tables: [9] StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr116], aggs: [count, max(sum($expr115))] } + StreamHashAgg { group_key: [$expr112], aggs: [count, max(sum($expr111))] } result table: 12, state tables: [11] - StreamProject { exprs: [lineitem.l_suppkey, sum($expr115), Vnode(lineitem.l_suppkey) as $expr116] } - StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr111), Vnode(lineitem.l_suppkey) as $expr112] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } StreamExchange Hash([0]) from 3 - Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr115), lineitem_l_suppkey], primary key: [$4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } - Table 1 { columns: [sum($expr115), supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } - Table 2 { columns: [max(max(sum($expr115)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } - Table 3 { columns: [max(max(sum($expr115))), _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } + Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr111), lineitem_l_suppkey], primary key: [$4 ASC, $0 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [4] } + Table 1 { columns: [sum($expr111), supplier_s_suppkey, lineitem_l_suppkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } + Table 2 { columns: [max(max(sum($expr111)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } + Table 3 { columns: [max(max(sum($expr111))), _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 4 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } Table 5 { columns: [supplier_s_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 6 { columns: [lineitem_l_suppkey, sum($expr115)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 6 { columns: [lineitem_l_suppkey, sum($expr111)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 7 { columns: [lineitem_l_suppkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [lineitem_l_suppkey, count, sum($expr115)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 9 { columns: [max(sum($expr115)), $expr116], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 10 { columns: [count, max(max(sum($expr115)))], primary key: [], value indices: [0, 1], distribution key: [] } - Table 11 { columns: [$expr116, sum($expr115), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 12 { columns: [$expr116, count, max(sum($expr115))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } - Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr115)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } + Table 8 { columns: [lineitem_l_suppkey, count, sum($expr111)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 9 { columns: [max(sum($expr111)), $expr112], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 10 { columns: [count, max(max(sum($expr111)))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [$expr112, sum($expr111), lineitem_l_suppkey], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 12 { columns: [$expr112, count, max(sum($expr111))], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 4294967294 { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey, max(max(sum($expr111)))], primary key: [$0 ASC, $5 ASC, $4 ASC, $6 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [4] } - id: tpch_q16 before: - create_tables @@ -2828,31 +2828,31 @@ └─LogicalFilter { predicate: (lineitem.l_partkey = CorrelatedInputRef { index: 16, correlated_id: 1 }) } └─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } optimized_logical_plan: | - LogicalProject { exprs: [RoundDigit((sum(lineitem.l_extendedprice) / 7.0:Decimal), 16:Int32) as $expr43] } + LogicalProject { exprs: [RoundDigit((sum(lineitem.l_extendedprice) / 7.0:Decimal), 16:Int32) as $expr41] } └─LogicalAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(part.p_partkey, part.p_partkey) AND (lineitem.l_quantity < $expr42), output: [lineitem.l_extendedprice] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(part.p_partkey, part.p_partkey) AND (lineitem.l_quantity < $expr40), output: [lineitem.l_extendedprice] } ├─LogicalJoin { type: Inner, on: (part.p_partkey = lineitem.l_partkey), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey] } | ├─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice] } | └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_brand, part.p_container], predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } - └─LogicalProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr42] } + └─LogicalProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr40] } └─LogicalAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(part.p_partkey, lineitem.l_partkey), output: [part.p_partkey, lineitem.l_quantity] } ├─LogicalAgg { group_key: [part.p_partkey], aggs: [] } | └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_brand, part.p_container], predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } └─LogicalScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], predicate: IsNotNull(lineitem.l_partkey) } batch_plan: | - BatchProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr90] } + BatchProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr88] } └─BatchSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } └─BatchProject { exprs: [lineitem.l_extendedprice] } - └─BatchFilter { predicate: (lineitem.l_quantity < $expr89) } + └─BatchFilter { predicate: (lineitem.l_quantity < $expr87) } └─BatchHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } | └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar), output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey] } | └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } | └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice], distribution: SomeShard } - └─BatchProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr89] } + └─BatchProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr87] } └─BatchHashAgg { group_key: [part.p_partkey], aggs: [sum(lineitem.l_quantity), count(lineitem.l_quantity)] } └─BatchHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity] } ├─BatchExchange { order: [], dist: HashShard(part.p_partkey) } @@ -2865,12 +2865,12 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [avg_yearly], pk_columns: [] } - └─StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr143] } + └─StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr139] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(lineitem.l_extendedprice))] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(lineitem.l_extendedprice)] } └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } - └─StreamFilter { predicate: (lineitem.l_quantity < $expr141) } + └─StreamFilter { predicate: (lineitem.l_quantity < $expr137) } └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } ├─StreamExchange { dist: HashShard(part.p_partkey) } | └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } @@ -2880,7 +2880,7 @@ | └─StreamProject { exprs: [part.p_partkey] } | └─StreamFilter { predicate: (part.p_brand = 'Brand#13':Varchar) AND (part.p_container = 'JUMBO PKG':Varchar) } | └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_container], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr141] } + └─StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr137] } └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count, sum(lineitem.l_quantity), count(lineitem.l_quantity)] } └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } ├─StreamExchange { dist: HashShard(part.p_partkey) } @@ -2896,7 +2896,7 @@ Fragment 0 StreamMaterialize { columns: [avg_yearly], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr143] } + StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr139] } StreamGlobalSimpleAgg { aggs: [count, sum(sum(lineitem.l_extendedprice))] } result table: 0, state tables: [] StreamExchange Single from 1 @@ -2904,11 +2904,11 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum(lineitem.l_extendedprice)] } StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } - StreamFilter { predicate: (lineitem.l_quantity < $expr141) } + StreamFilter { predicate: (lineitem.l_quantity < $expr137) } StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([2]) from 2 - StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr141] } + StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr137] } StreamHashAgg { group_key: [part.p_partkey], aggs: [count, sum(lineitem.l_quantity), count(lineitem.l_quantity)] } result table: 9, state tables: [] StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM lineitem.l_partkey, output: [part.p_partkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } @@ -2953,7 +2953,7 @@ Table 0 { columns: [count, sum(sum(lineitem_l_extendedprice))], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_quantity, lineitem_l_extendedprice, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey], primary key: [$2 ASC, $3 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [2] } Table 2 { columns: [part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 3 { columns: [part_p_partkey, $expr141], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 3 { columns: [part_p_partkey, $expr137], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } Table 4 { columns: [part_p_partkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } Table 5 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 6 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } @@ -3199,16 +3199,16 @@ ├─LogicalScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate, lineitem.l_shipinstruct, lineitem.l_shipmode, lineitem.l_comment] } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_name, part.p_mfgr, part.p_brand, part.p_type, part.p_size, part.p_container, part.p_retailprice, part.p_comment] } optimized_logical_plan: | - LogicalAgg { aggs: [sum($expr20)] } - └─LogicalProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr20] } + LogicalAgg { aggs: [sum($expr19)] } + └─LogicalProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr19] } └─LogicalJoin { type: Inner, on: (part.p_partkey = lineitem.l_partkey) AND (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))), output: [lineitem.l_extendedprice, lineitem.l_discount] } ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipinstruct, lineitem.l_shipmode], predicate: In(lineitem.l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (lineitem.l_shipinstruct = 'DELIVER IN PERSON':Varchar) } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_size, part.p_container], predicate: (part.p_size >= 1:Int32) } batch_plan: | - BatchSimpleAgg { aggs: [sum(sum($expr42))] } + BatchSimpleAgg { aggs: [sum(sum($expr41))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum($expr42)] } - └─BatchProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr42] } + └─BatchSimpleAgg { aggs: [sum($expr41)] } + └─BatchProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr41] } └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey AND (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) AND (part.p_size >= 1:Int32), output: [lineitem.l_extendedprice, lineitem.l_discount] } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount] } @@ -3216,11 +3216,11 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipinstruct, lineitem.l_shipmode], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [revenue], pk_columns: [] } - └─StreamProject { exprs: [sum(sum($expr67))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr67))] } + └─StreamProject { exprs: [sum(sum($expr65))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr65))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr67)] } - └─StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr67, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr65)] } + └─StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr65, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } @@ -3234,14 +3234,14 @@ Fragment 0 StreamMaterialize { columns: [revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [sum(sum($expr67))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr67))] } + StreamProject { exprs: [sum(sum($expr65))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr65))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr67)] } - StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr67, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr65)] } + StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr65, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Int32)) AND (lineitem.l_quantity <= 11:Int32)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Int32)) AND (lineitem.l_quantity <= 40:Int32)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Int32)) AND (lineitem.l_quantity <= 20:Int32)) AND (part.p_size <= 15:Int32))) } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } left table: 1, right table 3, left degree table: 2, right degree table: 4, @@ -3261,7 +3261,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum($expr67))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr65))], primary key: [], value indices: [0, 1], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_quantity, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $4 ASC, $5 ASC], value indices: [0, 1, 2, 3, 4, 5], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_brand, part_p_size, part_p_container], primary key: [$0 ASC], value indices: [0, 1, 2, 3], distribution key: [0] } @@ -3333,12 +3333,12 @@ ├─LogicalJoin { type: Inner, on: (supplier.s_nationkey = nation.n_nationkey), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address] } | ├─LogicalScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey] } | └─LogicalScan { table: nation, output_columns: [nation.n_nationkey], required_columns: [nation.n_nationkey, nation.n_name], predicate: (nation.n_name = 'KENYA':Varchar) } - └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(partsupp.ps_partkey, partsupp.ps_partkey) AND IsNotDistinctFrom(partsupp.ps_suppkey, partsupp.ps_suppkey) AND ($expr31 > $expr32), output: [partsupp.ps_suppkey] } - ├─LogicalProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr31] } + └─LogicalJoin { type: Inner, on: IsNotDistinctFrom(partsupp.ps_partkey, partsupp.ps_partkey) AND IsNotDistinctFrom(partsupp.ps_suppkey, partsupp.ps_suppkey) AND ($expr29 > $expr30), output: [partsupp.ps_suppkey] } + ├─LogicalProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr29] } | └─LogicalJoin { type: LeftSemi, on: (partsupp.ps_partkey = part.p_partkey), output: all } | ├─LogicalScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty] } | └─LogicalScan { table: part, output_columns: [part.p_partkey], required_columns: [part.p_partkey, part.p_name], predicate: Like(part.p_name, 'forest%':Varchar) } - └─LogicalProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr32] } + └─LogicalProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr30] } └─LogicalAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity)] } └─LogicalJoin { type: LeftOuter, on: IsNotDistinctFrom(partsupp.ps_partkey, lineitem.l_partkey) AND IsNotDistinctFrom(partsupp.ps_suppkey, lineitem.l_suppkey), output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity] } ├─LogicalAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [] } @@ -3354,14 +3354,14 @@ | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard(partsupp.ps_suppkey) } └─BatchProject { exprs: [partsupp.ps_suppkey] } - └─BatchFilter { predicate: ($expr69 > $expr68) } + └─BatchFilter { predicate: ($expr67 > $expr66) } └─BatchHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } ├─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - | └─BatchProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr69] } + | └─BatchProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr67] } | └─BatchLookupJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey AND Like(part.p_name, 'forest%':Varchar), output: all } | └─BatchExchange { order: [], dist: UpstreamHashShard(partsupp.ps_partkey) } | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - └─BatchProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr68] } + └─BatchProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr66] } └─BatchHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [sum(lineitem.l_quantity)] } └─BatchHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity] } ├─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } @@ -3384,10 +3384,10 @@ | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } └─StreamProject { exprs: [partsupp.ps_suppkey, partsupp.ps_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } - └─StreamFilter { predicate: ($expr109 > $expr108) } + └─StreamFilter { predicate: ($expr105 > $expr104) } └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - | └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr109] } + | └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr105] } | └─StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: all } | ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } | | └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } @@ -3395,7 +3395,7 @@ | └─StreamProject { exprs: [part.p_partkey] } | └─StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } | └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr108] } + └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr104] } └─StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count, sum(lineitem.l_quantity)] } └─StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } @@ -3435,11 +3435,11 @@ Fragment 4 StreamProject { exprs: [partsupp.ps_suppkey, partsupp.ps_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } - StreamFilter { predicate: ($expr109 > $expr108) } + StreamFilter { predicate: ($expr105 > $expr104) } StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM partsupp.ps_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM partsupp.ps_suppkey, output: all } left table: 8, right table 10, left degree table: 9, right degree table: 11, StreamExchange Hash([0, 1]) from 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr108] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr104] } StreamHashAgg { group_key: [partsupp.ps_partkey, partsupp.ps_suppkey], aggs: [count, sum(lineitem.l_quantity)] } result table: 16, state tables: [] StreamHashJoin { type: LeftOuter, predicate: partsupp.ps_partkey IS NOT DISTINCT FROM lineitem.l_partkey AND partsupp.ps_suppkey IS NOT DISTINCT FROM lineitem.l_suppkey, output: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, lineitem.l_suppkey] } @@ -3448,7 +3448,7 @@ StreamExchange Hash([0, 1]) from 9 Fragment 5 - StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr109] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr105] } StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: all } left table: 12, right table 14, left degree table: 13, right degree table: 15, StreamExchange Hash([0]) from 6 @@ -3489,9 +3489,9 @@ Table 5 { columns: [supplier_s_nationkey, supplier_s_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } Table 6 { columns: [nation_n_nationkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } Table 7 { columns: [nation_n_nationkey, _degree], primary key: [$0 ASC], value indices: [1], distribution key: [0] } - Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr109], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr105], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 9 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } - Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr108], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr104], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } Table 11 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0, 1] } Table 12 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, partsupp_ps_availqty], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } Table 13 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } @@ -3809,22 +3809,22 @@ └─LogicalFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } └─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_mktsegment, customer.c_comment] } optimized_logical_plan: | - LogicalAgg { group_key: [$expr41], aggs: [count, sum(customer.c_acctbal)] } - └─LogicalProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr41, customer.c_acctbal] } - └─LogicalJoin { type: Inner, on: (customer.c_acctbal > $expr40), output: [customer.c_phone, customer.c_acctbal] } + LogicalAgg { group_key: [$expr39], aggs: [count, sum(customer.c_acctbal)] } + └─LogicalProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr39, customer.c_acctbal] } + └─LogicalJoin { type: Inner, on: (customer.c_acctbal > $expr38), output: [customer.c_phone, customer.c_acctbal] } ├─LogicalJoin { type: LeftAnti, on: (orders.o_custkey = customer.c_custkey), output: [customer.c_phone, customer.c_acctbal] } | ├─LogicalScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } | └─LogicalScan { table: orders, columns: [orders.o_custkey] } - └─LogicalProject { exprs: [(sum(customer.c_acctbal) / count(customer.c_acctbal)) as $expr40] } + └─LogicalProject { exprs: [(sum(customer.c_acctbal) / count(customer.c_acctbal)) as $expr38] } └─LogicalAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } └─LogicalScan { table: customer, output_columns: [customer.c_acctbal], required_columns: [customer.c_acctbal, customer.c_phone], predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } batch_plan: | - BatchExchange { order: [$expr86 ASC], dist: Single } - └─BatchSort { order: [$expr86 ASC] } - └─BatchHashAgg { group_key: [$expr86], aggs: [count, sum(customer.c_acctbal)] } - └─BatchExchange { order: [], dist: HashShard($expr86) } - └─BatchProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr86, customer.c_acctbal] } - └─BatchNestedLoopJoin { type: Inner, predicate: (customer.c_acctbal > $expr85), output: [customer.c_phone, customer.c_acctbal] } + BatchExchange { order: [$expr84 ASC], dist: Single } + └─BatchSort { order: [$expr84 ASC] } + └─BatchHashAgg { group_key: [$expr84], aggs: [count, sum(customer.c_acctbal)] } + └─BatchExchange { order: [], dist: HashShard($expr84) } + └─BatchProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr84, customer.c_acctbal] } + └─BatchNestedLoopJoin { type: Inner, predicate: (customer.c_acctbal > $expr83), output: [customer.c_phone, customer.c_acctbal] } ├─BatchExchange { order: [], dist: Single } | └─BatchHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal] } | ├─BatchExchange { order: [], dist: HashShard(customer.c_custkey) } @@ -3832,7 +3832,7 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchScan { table: orders, columns: [orders.o_custkey], distribution: SomeShard } - └─BatchProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr85] } + └─BatchProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr83] } └─BatchSimpleAgg { aggs: [sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } @@ -3841,11 +3841,11 @@ └─BatchScan { table: customer, columns: [customer.c_acctbal, customer.c_phone], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [cntrycode, numcust, totacctbal], pk_columns: [cntrycode] } - └─StreamProject { exprs: [$expr136, count, sum(customer.c_acctbal)] } - └─StreamHashAgg { group_key: [$expr136], aggs: [count, count, sum(customer.c_acctbal)] } - └─StreamExchange { dist: HashShard($expr136) } - └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr136, customer.c_acctbal, customer.c_custkey] } - └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr135), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } + └─StreamProject { exprs: [$expr132, count, sum(customer.c_acctbal)] } + └─StreamHashAgg { group_key: [$expr132], aggs: [count, count, sum(customer.c_acctbal)] } + └─StreamExchange { dist: HashShard($expr132) } + └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr132, customer.c_acctbal, customer.c_custkey] } + └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr131), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } ├─StreamHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } | ├─StreamExchange { dist: HashShard(customer.c_custkey) } | | └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } @@ -3853,7 +3853,7 @@ | └─StreamExchange { dist: HashShard(orders.o_custkey) } | └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr135] } + └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr131] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal))] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(customer.c_acctbal), count(customer.c_acctbal)] } diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index f536f4c5cd48..a0237485407c 100644 --- a/src/frontend/planner_test/tests/testdata/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/watermark.yaml @@ -9,7 +9,7 @@ stream_plan: | StreamMaterialize { columns: [v1, _row_id(hidden)], pk_columns: [_row_id] } └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [(v1 - '00:00:02':Interval) as $expr26, _row_id], watermark_columns: [(v1 - '00:00:02':Interval)] } + └─StreamProject { exprs: [(v1 - '00:00:02':Interval) as $expr25, _row_id], watermark_columns: [(v1 - '00:00:02':Interval)] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [idx: 0, expr: (v1 - '00:00:01':Interval)] } └─StreamSource { source: "t", columns: ["v1", "_row_id"] } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 66712899764d..181d82a7decb 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -443,13 +443,6 @@ impl PlanRoot { ApplyOrder::TopDown, ); - plan = self.optimize_by_rules( - plan, - "Agg on Index".to_string(), - vec![TopNOnIndexRule::create()], - ApplyOrder::TopDown, - ); - #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -471,6 +464,13 @@ impl PlanRoot { ApplyOrder::TopDown, ); + plan = self.optimize_by_rules( + plan, + "Agg on Index".to_string(), + vec![TopNOnIndexRule::create()], + ApplyOrder::TopDown, + ); + // Convert to physical plan node plan = plan.to_batch_with_order_required(&self.required_order)?;