From 4b24e1b3a1d3b7cdfc8a3985bbc9eb26339eecf9 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 11:09:20 +0800 Subject: [PATCH 01/30] apply for stream --- .../planner_test/tests/testdata/array.yaml | 14 +-- .../tests/testdata/basic_query.yaml | 10 +- .../planner_test/tests/testdata/cast.yaml | 4 +- .../planner_test/tests/testdata/explain.yaml | 9 +- .../planner_test/tests/testdata/expr.yaml | 74 ++++++------ .../tests/testdata/index_selection.yaml | 6 +- .../planner_test/tests/testdata/insert.yaml | 8 +- .../planner_test/tests/testdata/order_by.yaml | 10 +- .../tests/testdata/project_set.yaml | 12 +- .../tests/testdata/range_scan.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 8 +- .../tests/testdata/sysinfo_funcs.yaml | 4 +- .../planner_test/tests/testdata/tpch.yaml | 108 +++++++++--------- src/frontend/src/optimizer/mod.rs | 21 +++- .../optimizer/plan_node/expr_rewritable.rs | 12 -- .../optimizer/plan_node/stream_delta_join.rs | 22 +++- .../src/optimizer/plan_node/stream_dml.rs | 4 +- .../plan_node/stream_dynamic_filter.rs | 4 +- .../optimizer/plan_node/stream_exchange.rs | 4 +- .../src/optimizer/plan_node/stream_expand.rs | 20 +++- .../src/optimizer/plan_node/stream_filter.rs | 21 +++- .../plan_node/stream_global_simple_agg.rs | 20 +++- .../optimizer/plan_node/stream_group_topn.rs | 4 +- .../optimizer/plan_node/stream_hash_agg.rs | 21 +++- .../optimizer/plan_node/stream_hash_join.rs | 24 +++- .../optimizer/plan_node/stream_hop_window.rs | 4 +- .../optimizer/plan_node/stream_index_scan.rs | 21 +++- .../plan_node/stream_local_simple_agg.rs | 20 +++- .../optimizer/plan_node/stream_materialize.rs | 4 +- .../src/optimizer/plan_node/stream_now.rs | 4 +- .../src/optimizer/plan_node/stream_project.rs | 21 +++- .../optimizer/plan_node/stream_project_set.rs | 21 +++- .../optimizer/plan_node/stream_row_id_gen.rs | 4 +- .../src/optimizer/plan_node/stream_share.rs | 4 +- .../src/optimizer/plan_node/stream_sink.rs | 4 +- .../src/optimizer/plan_node/stream_source.rs | 4 +- .../optimizer/plan_node/stream_table_scan.rs | 22 +++- .../src/optimizer/plan_node/stream_topn.rs | 3 +- .../src/optimizer/plan_node/stream_union.rs | 4 +- 39 files changed, 408 insertions(+), 180 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index b436cbdff2d4e..fd26d4a94ac4a 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | - BatchValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } + BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | - BatchValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } + BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } - sql: | create table t (v1 int); select (ARRAY[1, v1]) from t; @@ -52,7 +52,7 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } + BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[array[66]], array[233]); @@ -60,7 +60,7 @@ LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } + BatchProject { exprs: [ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[66]]); @@ -68,7 +68,7 @@ LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } + BatchProject { exprs: [ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[array[66]]]); @@ -85,7 +85,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } + BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_append(123, 234); @@ -102,7 +102,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } + BatchProject { exprs: [ARRAY[123, 66]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_prepend(123, 234); diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 5b141abcc0d1f..51ac43d99d7fd 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (11, 22), (33+(1+2), 44); batch_plan: | - BatchValues { rows: [[11:Int32, 22:Int32], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32]] } + BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | @@ -22,11 +22,11 @@ select * from t where 1>2 and 1=1 and 3<1 and 4<>1 or 1=1 and 2>=1 and 1<=2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + └─BatchFilter { predicate: true:Boolean AND true:Boolean } └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + └─StreamFilter { predicate: true:Boolean AND true:Boolean } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -130,11 +130,11 @@ - sql: | select * from unnest(Array[1,2,3]); batch_plan: | - BatchTableFunction { Unnest(Array(1:Int32, 2:Int32, 3:Int32)) } + BatchTableFunction { Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 }) } - sql: | select * from unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchTableFunction { Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32))) } + BatchTableFunction { Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } }) } - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index e122dd0ace778..b9385ce48355e 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchProject { exprs: [Case(null:Boolean, 1:Int32)] } + BatchProject { exprs: [null:Int32] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON NULL) sql: | @@ -65,7 +65,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchProject { exprs: [Case(true:Boolean, 1:Int32)] } + BatchProject { exprs: [1:Int32] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index c89502dce3ade..541b3be817f90 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,6 +36,11 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + Const eval exprs: + + BatchProject { exprs: [1:Int32] } + └─BatchValues { rows: [[]] } + To Batch Physical Plan: BatchProject { exprs: [1:Int32] } @@ -51,7 +56,7 @@ "stages": { "0": { "root": { - "plan_node_id": 28, + "plan_node_id": 34, "plan_node_type": "BatchProject", "schema": [ { @@ -64,7 +69,7 @@ ], "children": [ { - "plan_node_id": 26, + "plan_node_id": 32, "plan_node_type": "BatchValues", "schema": [], "children": [], diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 6ff7530e9f2c3..3f8d2435e0dfd 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -19,17 +19,17 @@ - sql: | values(cast(1 as bigint)); batch_plan: | - BatchValues { rows: [[1:Int32::Int64]] } + BatchValues { rows: [[1:Int64]] } - sql: | values(not true); batch_plan: | - BatchValues { rows: [[Not(true:Boolean)]] } + BatchValues { rows: [[false:Boolean]] } - sql: | create table t (); select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean))))))] } + └─BatchProject { exprs: [false:Boolean] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,13 +65,13 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)] } + BatchProject { exprs: [true:Boolean] } └─BatchValues { rows: [[]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))] } + BatchProject { exprs: [false:Boolean] } └─BatchValues { rows: [[]] } - name: in-list with misaligned types sql: | @@ -82,7 +82,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1))))] } + BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1))))] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -94,7 +94,7 @@ SELECT b2 from b where 1 in (3, 1.0, (select min(v1) from t)); batch_plan: | BatchProject { exprs: [b.b2] } - └─BatchFilter { predicate: (In(1:Int32::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (1:Int32 = min(min(t.v1)))) } + └─BatchFilter { predicate: (true:Boolean OR (1:Int32 = min(min(t.v1)))) } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b2], distribution: SomeShard } @@ -108,7 +108,7 @@ create table b (b1 int, b2 int); SELECT b2 from b where exists (select 2 from t where v1 in (3, 1.0, b1)); batch_plan: | - BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } + BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b1, b.b2], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } @@ -122,19 +122,19 @@ - sql: | values(round(42.4382, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42.4382:Decimal, 2:Int32)]] } + BatchValues { rows: [[42.44:Decimal]] } - sql: | values(round(42.4382)); batch_plan: | - BatchValues { rows: [[Round(42.4382:Decimal)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(42)); batch_plan: | - BatchValues { rows: [[Round(42:Int32::Float64)]] } + BatchValues { rows: [[42:Float64]] } - sql: | values(round(42, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42:Int32::Decimal, 2:Int32)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(true)); binder_error: |- @@ -144,18 +144,18 @@ -- Single quoted literal can be treated as number without error. values(round('123')); batch_plan: | - BatchValues { rows: [[Round(123:Float64)]] } + BatchValues { rows: [[123:Float64]] } - sql: | -- When it is invalid, PostgreSQL reports error during explain, but we have to wait until execution as of now. #4235 values(round('abc')); - sql: | values(extract(hour from timestamp '2001-02-16 20:38:40')); batch_plan: | - BatchValues { rows: [[Extract('HOUR':Varchar, '2001-02-16 20:38:40':Timestamp)]] } + BatchValues { rows: [[20:Decimal]] } - sql: | values('Postgres' not like 'Post%'); batch_plan: | - BatchValues { rows: [[Not(Like('Postgres':Varchar, 'Post%':Varchar))]] } + BatchValues { rows: [[false:Boolean]] } - sql: | values(1 not like 1.23); binder_error: |- @@ -164,13 +164,13 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchProject { exprs: [((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))] } + BatchProject { exprs: [4:Int32] } └─BatchValues { rows: [[]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar)] } - └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } + BatchProject { exprs: [0:Int32] } + └─BatchFilter { predicate: false:Boolean } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -178,11 +178,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal)] } └─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), t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal), 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: | @@ -190,7 +190,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -198,7 +198,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -264,18 +264,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar)] } └─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), t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar), 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); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -290,11 +290,11 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar)] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar)] } └─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), t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar), 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); @@ -310,22 +310,22 @@ - sql: | select concat(':', true); batch_plan: | - BatchProject { exprs: [ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))] } + BatchProject { exprs: [':t':Varchar] } └─BatchValues { rows: [[]] } - sql: | select ':' || true; batch_plan: | - BatchProject { exprs: [ConcatOp(':':Varchar, true:Boolean::Varchar)] } + BatchProject { exprs: [':true':Varchar] } └─BatchValues { rows: [[]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchProject { exprs: [Substr('hello':Varchar, null:Int32)] } + BatchProject { exprs: [null:Varchar] } └─BatchValues { rows: [[]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchProject { exprs: [Substr(null:Varchar, 1:Int32)] } + BatchProject { exprs: [null:Varchar] } └─BatchValues { rows: [[]] } - sql: | select pg_typeof('123'); @@ -419,10 +419,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [Array(1:Int32)] } + └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select 1 < ALL(array[null]::integer[]); @@ -443,10 +443,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } + BatchProject { exprs: [All((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [Array(1:Int32)] } + └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - name: now expression sql: | @@ -540,7 +540,7 @@ sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) batch_plan: | - BatchProject { exprs: [(false:Boolean >= Not(In('LN1O0QP1yi':Varchar, Md5('4SeUPZhUbH':Varchar))))] } + BatchProject { exprs: [false:Boolean] } └─BatchValues { rows: [[]] } - name: const_eval of const expr sql: | @@ -548,7 +548,7 @@ select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1)] } + └─BatchProject { exprs: [(3:Int32 + t.v1)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index af18731a327f8..c98da2bf8e0bc 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -77,7 +77,7 @@ select a,b from t1 where a in (1,2) and b in (2,3) batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: In(idx1.b, 2:Int32::Decimal, 3:Int32::Decimal) } + └─BatchFilter { predicate: In(idx1.b, 2:Decimal, 3:Decimal) } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], scan_ranges: [idx1.a = Int32(1) , idx1.a = Int32(2)], distribution: UpstreamHashShard(idx1.a, idx1.b) } - sql: | create table t1 (a int, b numeric, c bigint); @@ -213,13 +213,13 @@ update t1 set c = 3 where a = 1 and b = 2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } + └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchExchange { order: [], dist: Single } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: | - BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } + BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index aa67c803d90a2..985277c9273f7 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } + └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } - name: insert values on non-assign-castable types sql: | create table t (v1 real, v2 int); @@ -73,7 +73,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [['2020-01-01 01:02:03':Timestamp::Time], ['03:04:05':Time]] } + └─BatchValues { rows: [['01:02:03':Time], ['03:04:05':Time]] } - name: a `VALUES` without insert context may be invalid on its own (compare with above) sql: | create table t (v1 time); @@ -93,7 +93,7 @@ sql: | values (1), (null), (2.3); batch_plan: | - BatchValues { rows: [[1:Int32::Decimal], [null:Decimal], [2.3:Decimal]] } + BatchValues { rows: [[1:Decimal], [null:Decimal], [2.3:Decimal]] } - name: rows of different number of columns sql: | values (1), (2, 3); @@ -115,7 +115,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time, 11:Int32, 4.5:Decimal::Float32] } + └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 4f460213c6791..5a74f834a74f5 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [(1:Int32 + 1:Int32) ASC], dist: Single } - └─BatchSort { order: [(1:Int32 + 1:Int32) ASC] } - └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32)] } + └─BatchExchange { order: [2:Int32 ASC], dist: Single } + └─BatchSort { order: [2:Int32 ASC] } + └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, (1:Int32 + 1:Int32)(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(1:Int32 + 1:Int32), t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32), t._row_id] } + StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, 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); diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 4adbeadc2bc68..fde593e397a9c 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -8,14 +8,14 @@ - sql: | select unnest(Array[1,2,3]); batch_plan: | - BatchProject { exprs: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } - └─BatchProjectSet { select_list: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } + BatchProject { exprs: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } └─BatchValues { rows: [[]] } - sql: | select unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchProject { exprs: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } - └─BatchProjectSet { select_list: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } + BatchProject { exprs: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } └─BatchValues { rows: [[]] } - sql: | create table t(x int[]); @@ -43,9 +43,9 @@ create table t(x int[]); select unnest(x), unnest(Array[1,2]) from t; batch_plan: | - BatchProject { exprs: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + BatchProject { exprs: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchExchange { order: [], dist: Single } - └─BatchProjectSet { select_list: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + └─BatchProjectSet { select_list: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions sql: | diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index df8dbca94c355..4c6678d46ccbd 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -87,7 +87,7 @@ SELECT * FROM orders_count_by_user WHERE user_id > 42 AND date = 1111 AND 2>1 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (2:Int32 > 1:Int32) AND (orders_count_by_user.date = 1111:Int32) } + └─BatchFilter { predicate: true:Boolean AND (orders_count_by_user.date = 1111:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id > Int64(42)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv @@ -95,7 +95,7 @@ SELECT * FROM orders_count_by_user WHERE date > 1111 AND user_id = 42 AND 5<6 AND date <= 6666 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (5:Int32 < 6:Int32) } + └─BatchFilter { predicate: true:Boolean } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id = Int64(42) AND orders_count_by_user.date > Int32(1111) AND orders_count_by_user.date <= Int32(6666)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index d150ff23dcd82..9273d92b13642 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 = 519 } + | └─StreamShare { id = 559 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 519 } + └─StreamShare { id = 559 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } @@ -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 = 744 } + | └─StreamShare { id = 788 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_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 = 744 } + └─StreamShare { id = 788 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index 3d9034c7ddfc5..db6be75eb1831 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -36,6 +36,6 @@ No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml - sql: | select current_timestamp; - batch_plan: |- - BatchProject { exprs: [Now] } + batch_plan: | + BatchProject { exprs: ['2021-04-01 00:00:00+00:00':Timestamptz] } └─BatchValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 953cdbb3bcfd3..34d7fa4e6246d 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -136,7 +136,7 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), 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)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─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] } @@ -144,7 +144,7 @@ └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), 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)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -157,7 +157,7 @@ Fragment 1 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } 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 @@ -727,7 +727,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey] } @@ -741,7 +741,7 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } @@ -764,7 +764,7 @@ Fragment 2 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -859,7 +859,7 @@ | | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], distribution: UpstreamHashShard(customer.c_custkey) } | | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } | └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey) } | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } @@ -884,7 +884,7 @@ | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | | | └─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -943,7 +943,7 @@ Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -1020,7 +1020,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─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) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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: [] } @@ -1029,7 +1029,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), 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) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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 @@ -1043,7 +1043,7 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), 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) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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 @@ -1350,11 +1350,11 @@ └─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: [Extract('YEAR':Varchar, orders.o_orderdate) ASC], dist: Single } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } └─BatchSort { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC] } - └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─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: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], 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] } @@ -1832,7 +1832,7 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -1858,7 +1858,7 @@ | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -1910,7 +1910,7 @@ Fragment 6 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -2229,7 +2229,7 @@ | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode] } - └─BatchFilter { 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)) } + └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └─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] } @@ -2242,7 +2242,7 @@ | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2267,7 +2267,7 @@ Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2415,27 +2415,27 @@ ├─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(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─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] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } └─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(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, 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(part.p_partkey) } └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } @@ -2443,14 +2443,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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 @@ -2458,7 +2458,7 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_partkey, 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 @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], 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] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─BatchSimpleAgg { aggs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2563,7 +2563,7 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2573,12 +2573,12 @@ | ├─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 900 } + | └─StreamShare { id = 943 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─StreamProject { exprs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2587,12 +2587,12 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 900 } + └─StreamShare { id = 943 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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)) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2623,7 +2623,7 @@ Fragment 4 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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)) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } 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 @@ -3369,7 +3369,7 @@ | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } @@ -3404,7 +3404,7 @@ | └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -3476,7 +3476,7 @@ Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, 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 diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index f53161351b2c2..25dd1c80679ce 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -441,6 +441,7 @@ impl PlanRoot { fn gen_batch_plan(&mut self) -> Result { // Logical optimization let mut plan = self.gen_optimized_logical_plan()?; + let ctx = plan.ctx(); // Convert the dag back to the tree, because we don't support physical dag plan for now. plan = self.optimize_by_rules( @@ -453,14 +454,14 @@ impl PlanRoot { // Convert to physical plan node plan = plan.to_batch_with_order_required(&self.required_order)?; - // SessionTimezone substitution + // TODO: SessionTimezone substitution // Const eval of exprs at the last minute - // plan = const_eval_exprs(plan)?; + plan = const_eval_exprs(plan)?; - // if explain_trace { - // ctx.trace("Const eval exprs:"); - // ctx.trace(plan.explain_to_string().unwrap()); - // } + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -614,6 +615,14 @@ impl PlanRoot { ); } + // Const eval of exprs at the last minute + plan = const_eval_exprs(plan)?; + + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); diff --git a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs index baeb4caa1d61a..2957ac89fa1ac 100644 --- a/src/frontend/src/optimizer/plan_node/expr_rewritable.rs +++ b/src/frontend/src/optimizer/plan_node/expr_rewritable.rs @@ -14,11 +14,8 @@ use std::ops::Deref; -use paste::paste; - use super::*; use crate::expr::ExprRewriter; -use crate::for_stream_plan_nodes; /// Rewrites expressions in a `PlanRef`. Due to `Share` operator, /// the `ExprRewriter` needs to be idempotent i.e. applying it more than once @@ -48,12 +45,3 @@ impl ExprRewritable for PlanRef { } } } - -macro_rules! ban_expr_rewritable { - ($( { $convention:ident, $name:ident }),*) => { - paste!{ - $(impl ExprRewritable for [<$convention $name>] {} )* - } - } -} -for_stream_plan_nodes! {ban_expr_rewritable} diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index 7eff29860cedd..272e6115ddfa1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -21,8 +21,8 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ArrangementInfo, DeltaIndexJoinNode}; use super::generic::GenericPlanRef; -use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; -use crate::expr::Expr; +use super::{ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamNode}; +use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{EqJoinPredicate, EqJoinPredicateDisplay}; @@ -231,3 +231,21 @@ impl StreamNode for StreamDeltaJoin { }) } } + +impl ExprRewritable for StreamDeltaJoin { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_join() + .unwrap() + .clone(), + self.eq_join_predicate.rewrite_exprs(r), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_dml.rs b/src/frontend/src/optimizer/plan_node/stream_dml.rs index 2efe3d9981662..a96124eb9b170 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dml.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dml.rs @@ -18,7 +18,7 @@ use fixedbitset::FixedBitSet; use risingwave_common::catalog::ColumnDesc; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use super::{PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Clone, Debug)] @@ -88,3 +88,5 @@ impl StreamNode for StreamDml { }) } } + +impl ExprRewritable for StreamDml {} diff --git a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs index 5e6f20de78a0d..186e699087b7e 100644 --- a/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_dynamic_filter.rs @@ -21,8 +21,8 @@ pub use risingwave_pb::expr::expr_node::Type as ExprType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::DynamicFilterNode; -use super::generic; use super::utils::IndicesDisplay; +use super::{generic, ExprRewritable}; use crate::expr::Expr; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::{PlanBase, PlanTreeNodeBinary, StreamNode}; @@ -150,3 +150,5 @@ impl StreamNode for StreamDynamicFilter { }) } } + +impl ExprRewritable for StreamDynamicFilter {} diff --git a/src/frontend/src/optimizer/plan_node/stream_exchange.rs b/src/frontend/src/optimizer/plan_node/stream_exchange.rs index 825a77fa39155..e8f5d36cf53ed 100644 --- a/src/frontend/src/optimizer/plan_node/stream_exchange.rs +++ b/src/frontend/src/optimizer/plan_node/stream_exchange.rs @@ -17,7 +17,7 @@ use std::fmt; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{DispatchStrategy, DispatcherType, ExchangeNode}; -use super::{PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -91,3 +91,5 @@ impl StreamNode for StreamExchange { }) } } + +impl ExprRewritable for StreamExchange {} diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index 5ff700297171d..b78d11a5c5411 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -19,7 +19,8 @@ use risingwave_pb::stream_plan::expand_node::Subset; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::ExpandNode; -use super::{LogicalExpand, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalExpand, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::ExprRewriter; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -101,3 +102,20 @@ fn subset_to_protobuf(subset: &[usize]) -> Subset { let column_indices = subset.iter().map(|key| *key as u32).collect(); Subset { column_indices } } + +impl ExprRewritable for StreamExpand { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_expand() + .unwrap() + .clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 7bc9f3f7a56b7..6599c69344576 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -18,8 +18,8 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::FilterNode; use super::generic::GenericPlanRef; -use super::{LogicalFilter, PlanRef, PlanTreeNodeUnary, StreamNode}; -use crate::expr::{Expr, ExprImpl}; +use super::{ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::PlanBase; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::utils::Condition; @@ -85,3 +85,20 @@ impl StreamNode for StreamFilter { }) } } + +impl ExprRewritable for StreamFilter { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_filter() + .unwrap() + .clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index dbabbf23a8008..0928a9da4f7a4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -18,7 +18,8 @@ use fixedbitset::FixedBitSet; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -119,3 +120,20 @@ impl StreamNode for StreamGlobalSimpleAgg { }) } } + +impl ExprRewritable for StreamGlobalSimpleAgg { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_agg() + .unwrap() + .clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs index 56cee4e84d140..b37c9c3a37875 100644 --- a/src/frontend/src/optimizer/plan_node/stream_group_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_group_topn.rs @@ -17,7 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use super::{LogicalTopN, PlanBase, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalTopN, PlanBase, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Order, OrderDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::PlanRef; @@ -151,3 +151,5 @@ impl PlanTreeNodeUnary for StreamGroupTopN { Self::new(self.logical.clone_with_input(input), self.vnode_col_idx) } } + +impl ExprRewritable for StreamGroupTopN {} diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 57e59c965199d..49a5901ca3bef 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -18,7 +18,8 @@ use fixedbitset::FixedBitSet; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -128,3 +129,21 @@ impl StreamNode for StreamHashAgg { }) } } + +impl ExprRewritable for StreamHashAgg { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_agg() + .unwrap() + .clone(), + self.vnode_col_idx.clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 166283092d08c..3c6e9b72d80d8 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -21,8 +21,10 @@ use risingwave_pb::plan_common::JoinType; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::HashJoinNode; -use super::{LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamDeltaJoin, StreamNode}; -use crate::expr::Expr; +use super::{ + ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamDeltaJoin, StreamNode, +}; +use crate::expr::{Expr, ExprRewriter}; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{EqJoinPredicate, EqJoinPredicateDisplay}; @@ -283,3 +285,21 @@ impl StreamNode for StreamHashJoin { }) } } + +impl ExprRewritable for StreamHashJoin { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_join() + .unwrap() + .clone(), + self.eq_join_predicate.rewrite_exprs(r), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs index 72dbbfbbfdd40..afe318cb66a77 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hop_window.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hop_window.rs @@ -17,7 +17,7 @@ use std::fmt; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::HopWindowNode; -use super::{LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalHopWindow, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; /// [`StreamHopWindow`] represents a hop window table function. @@ -91,3 +91,5 @@ impl StreamNode for StreamHopWindow { }) } } + +impl ExprRewritable for StreamHopWindow {} diff --git a/src/frontend/src/optimizer/plan_node/stream_index_scan.rs b/src/frontend/src/optimizer/plan_node/stream_index_scan.rs index 3a613a95f1379..62486104469f5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_index_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_index_scan.rs @@ -19,8 +19,9 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::{ChainType, StreamNode as ProstStreamPlan}; -use super::{LogicalScan, PlanBase, PlanNodeId, StreamNode}; +use super::{ExprRewritable, LogicalScan, PlanBase, PlanNodeId, PlanRef, StreamNode}; use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -206,3 +207,21 @@ impl StreamIndexScan { } } } + +impl ExprRewritable for StreamIndexScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_scan() + .unwrap() + .clone(), + self.chain_type, + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index 6f5884e0117be..2c945a92782ad 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -19,7 +19,8 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; -use super::{LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::RequiredDist; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -109,3 +110,20 @@ impl StreamNode for StreamLocalSimpleAgg { }) } } + +impl ExprRewritable for StreamLocalSimpleAgg { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_agg() + .unwrap() + .clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 38ebd07e9956e..026629e9151a9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -22,7 +22,7 @@ use risingwave_common::catalog::{ColumnDesc, TableId}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use super::{PlanRef, PlanTreeNodeUnary, StreamNode, StreamSink}; +use super::{ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode, StreamSink}; use crate::catalog::column_catalog::ColumnCatalog; use crate::catalog::table_catalog::{TableCatalog, TableType, TableVersion}; use crate::catalog::{FragmentId, USER_COLUMN_ID_OFFSET}; @@ -358,3 +358,5 @@ impl StreamNode for StreamMaterialize { }) } } + +impl ExprRewritable for StreamMaterialize {} diff --git a/src/frontend/src/optimizer/plan_node/stream_now.rs b/src/frontend/src/optimizer/plan_node/stream_now.rs index fb2ae37a17625..9d7ec3999d876 100644 --- a/src/frontend/src/optimizer/plan_node/stream_now.rs +++ b/src/frontend/src/optimizer/plan_node/stream_now.rs @@ -24,7 +24,7 @@ use risingwave_pb::stream_plan::NowNode; use super::generic::GenericPlanRef; use super::stream::StreamPlanRef; use super::utils::{IndicesDisplay, TableCatalogBuilder}; -use super::{LogicalNow, PlanBase, StreamNode}; +use super::{ExprRewritable, LogicalNow, PlanBase, StreamNode}; use crate::optimizer::property::{Distribution, FunctionalDependencySet}; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::OptimizerContextRef; @@ -97,3 +97,5 @@ impl StreamNode for StreamNow { }) } } + +impl ExprRewritable for StreamNow {} diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index e67d16777f5ee..613e9abd5b275 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -20,8 +20,8 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::ProjectNode; use super::generic::GenericPlanRef; -use super::{LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; -use crate::expr::{try_derive_watermark, Expr, ExprDisplay, ExprImpl}; +use super::{ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::{try_derive_watermark, Expr, ExprDisplay, ExprImpl, ExprRewriter}; use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamProject` implements [`super::LogicalProject`] to evaluate specified expressions on input @@ -150,3 +150,20 @@ impl StreamNode for StreamProject { }) } } + +impl ExprRewritable for StreamProject { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_project() + .unwrap() + .clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_project_set.rs b/src/frontend/src/optimizer/plan_node/stream_project_set.rs index 7c832a23dcafd..e23eb10128664 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project_set.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project_set.rs @@ -19,8 +19,8 @@ use itertools::Itertools; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::ProjectSetNode; -use super::{LogicalProjectSet, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; -use crate::expr::try_derive_watermark; +use super::{ExprRewritable, LogicalProjectSet, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use crate::expr::{try_derive_watermark, ExprRewriter}; use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone)] @@ -94,3 +94,20 @@ impl StreamNode for StreamProjectSet { }) } } + +impl ExprRewritable for StreamProjectSet { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_project_set() + .unwrap() + .clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index 10dca6d19d5b6..689007ae92e1d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -16,7 +16,7 @@ use std::fmt; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use super::{PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Clone, Debug)] @@ -76,3 +76,5 @@ impl StreamNode for StreamRowIdGen { }) } } + +impl ExprRewritable for StreamRowIdGen {} diff --git a/src/frontend/src/optimizer/plan_node/stream_share.rs b/src/frontend/src/optimizer/plan_node/stream_share.rs index a215037da6672..66361b3b89174 100644 --- a/src/frontend/src/optimizer/plan_node/stream_share.rs +++ b/src/frontend/src/optimizer/plan_node/stream_share.rs @@ -20,7 +20,7 @@ use risingwave_pb::stream_plan::{ DispatchStrategy, DispatcherType, ExchangeNode, StreamNode as ProstStreamPlan, }; -use super::{PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::plan_node::{LogicalShare, PlanBase, PlanTreeNode}; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -143,3 +143,5 @@ impl StreamShare { } } } + +impl ExprRewritable for StreamShare {} diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 9ef1a843d07ab..02b77dea42ef1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -17,7 +17,7 @@ use std::fmt; use risingwave_common::catalog::Field; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use super::{PlanBase, PlanRef, StreamNode}; +use super::{ExprRewritable, PlanBase, PlanRef, StreamNode}; use crate::optimizer::plan_node::PlanTreeNodeUnary; use crate::stream_fragmenter::BuildFragmentGraphState; use crate::TableCatalog; @@ -93,3 +93,5 @@ impl StreamNode for StreamSink { }) } } + +impl ExprRewritable for StreamSink {} diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index 27747c590005b..236698ebf445c 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -20,7 +20,7 @@ use risingwave_pb::catalog::ColumnIndex; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::{SourceNode, StreamSource as ProstStreamSource}; -use super::{LogicalSource, PlanBase, StreamNode}; +use super::{ExprRewritable, LogicalSource, PlanBase, StreamNode}; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -103,3 +103,5 @@ impl StreamNode for StreamSource { ProstStreamNode::Source(SourceNode { source_inner }) } } + +impl ExprRewritable for StreamSource {} diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 3f8a68bd4c355..d1f13a9f838be 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -22,8 +22,11 @@ use risingwave_common::catalog::{Field, TableDesc}; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::{ChainType, StreamNode as ProstStreamPlan}; -use super::{LogicalScan, PlanBase, PlanNodeId, StreamIndexScan, StreamNode}; +use super::{ + ExprRewritable, LogicalScan, PlanBase, PlanNodeId, PlanRef, StreamIndexScan, StreamNode, +}; use crate::catalog::ColumnId; +use crate::expr::ExprRewriter; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::property::{Distribution, DistributionDisplay}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -233,3 +236,20 @@ impl StreamTableScan { } } } + +impl ExprRewritable for StreamTableScan { + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new( + self.logical + .rewrite_exprs(r) + .as_logical_scan() + .unwrap() + .clone(), + ) + .into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_topn.rs b/src/frontend/src/optimizer/plan_node/stream_topn.rs index c73d1dd6ab5b1..a69be61f66a20 100644 --- a/src/frontend/src/optimizer/plan_node/stream_topn.rs +++ b/src/frontend/src/optimizer/plan_node/stream_topn.rs @@ -17,7 +17,7 @@ use std::fmt; use fixedbitset::FixedBitSet; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; -use super::{LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; +use super::{ExprRewritable, LogicalTopN, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::optimizer::property::{Distribution, Order}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -114,3 +114,4 @@ impl StreamNode for StreamTopN { } } } +impl ExprRewritable for StreamTopN {} diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index f000cf23877e5..4eccf732f1904 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -19,7 +19,7 @@ use fixedbitset::FixedBitSet; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::UnionNode; -use super::PlanRef; +use super::{ExprRewritable, PlanRef}; use crate::optimizer::plan_node::stream::StreamPlanRef; use crate::optimizer::plan_node::{LogicalUnion, PlanBase, PlanTreeNode, StreamNode}; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -88,3 +88,5 @@ impl StreamNode for StreamUnion { ProstStreamNode::Union(UnionNode {}) } } + +impl ExprRewritable for StreamUnion {} From 53cd4f4509e03e47c74bcc755dc331920d5444f8 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 11:14:04 +0800 Subject: [PATCH 02/30] fix --- .../src/optimizer/plan_node/stream_expand.rs | 18 +----------------- .../src/optimizer/plan_node/stream_hash_agg.rs | 2 +- 2 files changed, 2 insertions(+), 18 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_expand.rs b/src/frontend/src/optimizer/plan_node/stream_expand.rs index b78d11a5c5411..c4193ea15b623 100644 --- a/src/frontend/src/optimizer/plan_node/stream_expand.rs +++ b/src/frontend/src/optimizer/plan_node/stream_expand.rs @@ -20,7 +20,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::ExpandNode; use super::{ExprRewritable, LogicalExpand, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; -use crate::expr::ExprRewriter; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; @@ -103,19 +102,4 @@ fn subset_to_protobuf(subset: &[usize]) -> Subset { Subset { column_indices } } -impl ExprRewritable for StreamExpand { - fn has_rewritable_expr(&self) -> bool { - true - } - - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self::new( - self.logical - .rewrite_exprs(r) - .as_logical_expand() - .unwrap() - .clone(), - ) - .into() - } -} +impl ExprRewritable for StreamExpand {} diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 49a5901ca3bef..60cb7ac010f93 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -142,7 +142,7 @@ impl ExprRewritable for StreamHashAgg { .as_logical_agg() .unwrap() .clone(), - self.vnode_col_idx.clone(), + self.vnode_col_idx, ) .into() } From 7e2e10a80fd07b7d1a6bcf164fb30d67b9687194 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 11:18:43 +0800 Subject: [PATCH 03/30] remove the const eval, still need session timezone --- .../planner_test/tests/testdata/array.yaml | 14 +-- .../tests/testdata/basic_query.yaml | 10 +- .../planner_test/tests/testdata/cast.yaml | 4 +- .../planner_test/tests/testdata/explain.yaml | 9 +- .../planner_test/tests/testdata/expr.yaml | 74 ++++++------ .../tests/testdata/index_selection.yaml | 6 +- .../planner_test/tests/testdata/insert.yaml | 8 +- .../planner_test/tests/testdata/order_by.yaml | 10 +- .../tests/testdata/project_set.yaml | 12 +- .../tests/testdata/range_scan.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 8 +- .../tests/testdata/sysinfo_funcs.yaml | 2 +- .../planner_test/tests/testdata/tpch.yaml | 108 +++++++++--------- src/frontend/src/optimizer/mod.rs | 22 ++-- 14 files changed, 143 insertions(+), 148 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index fd26d4a94ac4a..b436cbdff2d4e 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | - BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } + BatchValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | - BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } + BatchValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } - sql: | create table t (v1 int); select (ARRAY[1, v1]) from t; @@ -52,7 +52,7 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[array[66]], array[233]); @@ -60,7 +60,7 @@ LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }] } + BatchProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[66]]); @@ -68,7 +68,7 @@ LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }] } + BatchProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[array[66]]]); @@ -85,7 +85,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─BatchValues { rows: [[]] } - sql: | select array_append(123, 234); @@ -102,7 +102,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[123, 66]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_prepend(123, 234); diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 51ac43d99d7fd..5b141abcc0d1f 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (11, 22), (33+(1+2), 44); batch_plan: | - BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } + BatchValues { rows: [[11:Int32, 22:Int32], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | @@ -22,11 +22,11 @@ select * from t where 1>2 and 1=1 and 3<1 and 4<>1 or 1=1 and 2>=1 and 1<=2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean AND true:Boolean } + └─BatchFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamFilter { predicate: true:Boolean AND true:Boolean } + └─StreamFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -130,11 +130,11 @@ - sql: | select * from unnest(Array[1,2,3]); batch_plan: | - BatchTableFunction { Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 }) } + BatchTableFunction { Unnest(Array(1:Int32, 2:Int32, 3:Int32)) } - sql: | select * from unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchTableFunction { Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } }) } + BatchTableFunction { Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32))) } - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index b9385ce48355e..e122dd0ace778 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchProject { exprs: [null:Int32] } + BatchProject { exprs: [Case(null:Boolean, 1:Int32)] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON NULL) sql: | @@ -65,7 +65,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchProject { exprs: [1:Int32] } + BatchProject { exprs: [Case(true:Boolean, 1:Int32)] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 541b3be817f90..c89502dce3ade 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,11 +36,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Const eval exprs: - - BatchProject { exprs: [1:Int32] } - └─BatchValues { rows: [[]] } - To Batch Physical Plan: BatchProject { exprs: [1:Int32] } @@ -56,7 +51,7 @@ "stages": { "0": { "root": { - "plan_node_id": 34, + "plan_node_id": 28, "plan_node_type": "BatchProject", "schema": [ { @@ -69,7 +64,7 @@ ], "children": [ { - "plan_node_id": 32, + "plan_node_id": 26, "plan_node_type": "BatchValues", "schema": [], "children": [], diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 3f8d2435e0dfd..6ff7530e9f2c3 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -19,17 +19,17 @@ - sql: | values(cast(1 as bigint)); batch_plan: | - BatchValues { rows: [[1:Int64]] } + BatchValues { rows: [[1:Int32::Int64]] } - sql: | values(not true); batch_plan: | - BatchValues { rows: [[false:Boolean]] } + BatchValues { rows: [[Not(true:Boolean)]] } - sql: | create table t (); select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [false:Boolean] } + └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean))))))] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,13 +65,13 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [true:Boolean] } + BatchProject { exprs: [In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)] } └─BatchValues { rows: [[]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [false:Boolean] } + BatchProject { exprs: [Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))] } └─BatchValues { rows: [[]] } - name: in-list with misaligned types sql: | @@ -82,7 +82,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1))))] } + BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1))))] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -94,7 +94,7 @@ SELECT b2 from b where 1 in (3, 1.0, (select min(v1) from t)); batch_plan: | BatchProject { exprs: [b.b2] } - └─BatchFilter { predicate: (true:Boolean OR (1:Int32 = min(min(t.v1)))) } + └─BatchFilter { predicate: (In(1:Int32::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (1:Int32 = min(min(t.v1)))) } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b2], distribution: SomeShard } @@ -108,7 +108,7 @@ create table b (b1 int, b2 int); SELECT b2 from b where exists (select 2 from t where v1 in (3, 1.0, b1)); batch_plan: | - BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } + BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b1, b.b2], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } @@ -122,19 +122,19 @@ - sql: | values(round(42.4382, 2)); batch_plan: | - BatchValues { rows: [[42.44:Decimal]] } + BatchValues { rows: [[RoundDigit(42.4382:Decimal, 2:Int32)]] } - sql: | values(round(42.4382)); batch_plan: | - BatchValues { rows: [[42:Decimal]] } + BatchValues { rows: [[Round(42.4382:Decimal)]] } - sql: | values(round(42)); batch_plan: | - BatchValues { rows: [[42:Float64]] } + BatchValues { rows: [[Round(42:Int32::Float64)]] } - sql: | values(round(42, 2)); batch_plan: | - BatchValues { rows: [[42:Decimal]] } + BatchValues { rows: [[RoundDigit(42:Int32::Decimal, 2:Int32)]] } - sql: | values(round(true)); binder_error: |- @@ -144,18 +144,18 @@ -- Single quoted literal can be treated as number without error. values(round('123')); batch_plan: | - BatchValues { rows: [[123:Float64]] } + BatchValues { rows: [[Round(123:Float64)]] } - sql: | -- When it is invalid, PostgreSQL reports error during explain, but we have to wait until execution as of now. #4235 values(round('abc')); - sql: | values(extract(hour from timestamp '2001-02-16 20:38:40')); batch_plan: | - BatchValues { rows: [[20:Decimal]] } + BatchValues { rows: [[Extract('HOUR':Varchar, '2001-02-16 20:38:40':Timestamp)]] } - sql: | values('Postgres' not like 'Post%'); batch_plan: | - BatchValues { rows: [[false:Boolean]] } + BatchValues { rows: [[Not(Like('Postgres':Varchar, 'Post%':Varchar))]] } - sql: | values(1 not like 1.23); binder_error: |- @@ -164,13 +164,13 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchProject { exprs: [4:Int32] } + BatchProject { exprs: [((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))] } └─BatchValues { rows: [[]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [0:Int32] } - └─BatchFilter { predicate: false:Boolean } + BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar)] } + └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -178,11 +178,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal)] } └─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:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal), t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal), 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: | @@ -190,7 +190,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -198,7 +198,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -264,18 +264,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar)] } └─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':Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar), 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); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -290,11 +290,11 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar)] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar)] } └─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':Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar), 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); @@ -310,22 +310,22 @@ - sql: | select concat(':', true); batch_plan: | - BatchProject { exprs: [':t':Varchar] } + BatchProject { exprs: [ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))] } └─BatchValues { rows: [[]] } - sql: | select ':' || true; batch_plan: | - BatchProject { exprs: [':true':Varchar] } + BatchProject { exprs: [ConcatOp(':':Varchar, true:Boolean::Varchar)] } └─BatchValues { rows: [[]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchProject { exprs: [null:Varchar] } + BatchProject { exprs: [Substr('hello':Varchar, null:Int32)] } └─BatchValues { rows: [[]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchProject { exprs: [null:Varchar] } + BatchProject { exprs: [Substr(null:Varchar, 1:Int32)] } └─BatchValues { rows: [[]] } - sql: | select pg_typeof('123'); @@ -419,10 +419,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } + └─BatchProject { exprs: [Array(1:Int32)] } └─BatchValues { rows: [[]] } - sql: | select 1 < ALL(array[null]::integer[]); @@ -443,10 +443,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } + └─BatchProject { exprs: [Array(1:Int32)] } └─BatchValues { rows: [[]] } - name: now expression sql: | @@ -540,7 +540,7 @@ sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) batch_plan: | - BatchProject { exprs: [false:Boolean] } + BatchProject { exprs: [(false:Boolean >= Not(In('LN1O0QP1yi':Varchar, Md5('4SeUPZhUbH':Varchar))))] } └─BatchValues { rows: [[]] } - name: const_eval of const expr sql: | @@ -548,7 +548,7 @@ select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [(3:Int32 + t.v1)] } + └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index c98da2bf8e0bc..af18731a327f8 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -77,7 +77,7 @@ select a,b from t1 where a in (1,2) and b in (2,3) batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: In(idx1.b, 2:Decimal, 3:Decimal) } + └─BatchFilter { predicate: In(idx1.b, 2:Int32::Decimal, 3:Int32::Decimal) } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], scan_ranges: [idx1.a = Int32(1) , idx1.a = Int32(2)], distribution: UpstreamHashShard(idx1.a, idx1.b) } - sql: | create table t1 (a int, b numeric, c bigint); @@ -213,13 +213,13 @@ update t1 set c = 3 where a = 1 and b = 2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } + └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } └─BatchExchange { order: [], dist: Single } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: | - BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } + BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 985277c9273f7..aa67c803d90a2 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } + └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } - name: insert values on non-assign-castable types sql: | create table t (v1 real, v2 int); @@ -73,7 +73,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [['01:02:03':Time], ['03:04:05':Time]] } + └─BatchValues { rows: [['2020-01-01 01:02:03':Timestamp::Time], ['03:04:05':Time]] } - name: a `VALUES` without insert context may be invalid on its own (compare with above) sql: | create table t (v1 time); @@ -93,7 +93,7 @@ sql: | values (1), (null), (2.3); batch_plan: | - BatchValues { rows: [[1:Decimal], [null:Decimal], [2.3:Decimal]] } + BatchValues { rows: [[1:Int32::Decimal], [null:Decimal], [2.3:Decimal]] } - name: rows of different number of columns sql: | values (1), (2, 3); @@ -115,7 +115,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } + └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time, 11:Int32, 4.5:Decimal::Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 5a74f834a74f5..4f460213c6791 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [2:Int32 ASC], dist: Single } - └─BatchSort { order: [2:Int32 ASC] } - └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } + └─BatchExchange { order: [(1:Int32 + 1:Int32) ASC], dist: Single } + └─BatchSort { order: [(1:Int32 + 1:Int32) ASC] } + └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32)] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } + StreamMaterialize { columns: [v1, v2, (1:Int32 + 1:Int32)(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(1:Int32 + 1:Int32), t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32), 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); diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index fde593e397a9c..4adbeadc2bc68 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -8,14 +8,14 @@ - sql: | select unnest(Array[1,2,3]); batch_plan: | - BatchProject { exprs: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } - └─BatchProjectSet { select_list: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } + BatchProject { exprs: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } + └─BatchProjectSet { select_list: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } └─BatchValues { rows: [[]] } - sql: | select unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchProject { exprs: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } - └─BatchProjectSet { select_list: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } + BatchProject { exprs: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } + └─BatchProjectSet { select_list: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } └─BatchValues { rows: [[]] } - sql: | create table t(x int[]); @@ -43,9 +43,9 @@ create table t(x int[]); select unnest(x), unnest(Array[1,2]) from t; batch_plan: | - BatchProject { exprs: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } + BatchProject { exprs: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } └─BatchExchange { order: [], dist: Single } - └─BatchProjectSet { select_list: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } + └─BatchProjectSet { select_list: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions sql: | diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index 4c6678d46ccbd..df8dbca94c355 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -87,7 +87,7 @@ SELECT * FROM orders_count_by_user WHERE user_id > 42 AND date = 1111 AND 2>1 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean AND (orders_count_by_user.date = 1111:Int32) } + └─BatchFilter { predicate: (2:Int32 > 1:Int32) AND (orders_count_by_user.date = 1111:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id > Int64(42)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv @@ -95,7 +95,7 @@ SELECT * FROM orders_count_by_user WHERE date > 1111 AND user_id = 42 AND 5<6 AND date <= 6666 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean } + └─BatchFilter { predicate: (5:Int32 < 6:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id = Int64(42) AND orders_count_by_user.date > Int32(1111) AND orders_count_by_user.date <= Int32(6666)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 9273d92b13642..d150ff23dcd82 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 = 559 } + | └─StreamShare { id = 519 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 559 } + └─StreamShare { id = 519 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } @@ -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 = 788 } + | └─StreamShare { id = 744 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_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 = 788 } + └─StreamShare { id = 744 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index db6be75eb1831..c40ca8c25279d 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -37,5 +37,5 @@ - sql: | select current_timestamp; batch_plan: | - BatchProject { exprs: ['2021-04-01 00:00:00+00:00':Timestamptz] } + BatchProject { exprs: [Now] } └─BatchValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 34d7fa4e6246d..953cdbb3bcfd3 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -136,7 +136,7 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), 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)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─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] } @@ -144,7 +144,7 @@ └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), 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)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─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 @@ -157,7 +157,7 @@ Fragment 1 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + 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 @@ -727,7 +727,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey] } @@ -741,7 +741,7 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } @@ -764,7 +764,7 @@ Fragment 2 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -859,7 +859,7 @@ | | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], distribution: UpstreamHashShard(customer.c_custkey) } | | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } | | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } | └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey) } | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } @@ -884,7 +884,7 @@ | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } | | | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | | | └─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -943,7 +943,7 @@ Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -1020,7 +1020,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + └─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: [] } @@ -1029,7 +1029,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + └─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 @@ -1043,7 +1043,7 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + 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 @@ -1350,11 +1350,11 @@ └─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: [Extract('YEAR':Varchar, orders.o_orderdate) ASC], dist: Single } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } └─BatchSort { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC] } - └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─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: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], 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] } @@ -1832,7 +1832,7 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -1858,7 +1858,7 @@ | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -1910,7 +1910,7 @@ Fragment 6 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -2229,7 +2229,7 @@ | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode] } - └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─BatchFilter { 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)) } └─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] } @@ -2242,7 +2242,7 @@ | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2267,7 +2267,7 @@ Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2415,27 +2415,27 @@ ├─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(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─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] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─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(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, 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(part.p_partkey) } └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } @@ -2443,14 +2443,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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 @@ -2458,7 +2458,7 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } Chain { table: lineitem, columns: [lineitem.l_partkey, 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 @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], 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] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─BatchSimpleAgg { aggs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2563,7 +2563,7 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2573,12 +2573,12 @@ | ├─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 943 } + | └─StreamShare { id = 900 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─StreamProject { exprs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2587,12 +2587,12 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 943 } + └─StreamShare { id = 900 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─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 @@ -2623,7 +2623,7 @@ Fragment 4 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + 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 @@ -3369,7 +3369,7 @@ | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } @@ -3404,7 +3404,7 @@ | └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, 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 @@ -3476,7 +3476,7 @@ Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, 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 diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 25dd1c80679ce..c511b80f358f0 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -441,7 +441,6 @@ impl PlanRoot { fn gen_batch_plan(&mut self) -> Result { // Logical optimization let mut plan = self.gen_optimized_logical_plan()?; - let ctx = plan.ctx(); // Convert the dag back to the tree, because we don't support physical dag plan for now. plan = self.optimize_by_rules( @@ -456,12 +455,13 @@ impl PlanRoot { // TODO: SessionTimezone substitution // Const eval of exprs at the last minute - plan = const_eval_exprs(plan)?; + // plan = const_eval_exprs(plan)?; - if ctx.is_explain_trace() { - ctx.trace("Const eval exprs:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + // let ctx = plan.ctx(); + // if ctx.is_explain_trace() { + // ctx.trace("Const eval exprs:"); + // ctx.trace(plan.explain_to_string().unwrap()); + // } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -616,12 +616,12 @@ impl PlanRoot { } // Const eval of exprs at the last minute - plan = const_eval_exprs(plan)?; + // plan = const_eval_exprs(plan)?; - if ctx.is_explain_trace() { - ctx.trace("Const eval exprs:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + // if ctx.is_explain_trace() { + // ctx.trace("Const eval exprs:"); + // ctx.trace(plan.explain_to_string().unwrap()); + // } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); From 41632e49d2ceaab6b8f3e9b44e0dc8ff74125f29 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 12:18:14 +0800 Subject: [PATCH 04/30] commit --- .../planner_test/tests/testdata/array.yaml | 14 +-- .../tests/testdata/basic_query.yaml | 10 +- .../planner_test/tests/testdata/cast.yaml | 4 +- .../planner_test/tests/testdata/explain.yaml | 14 ++- .../planner_test/tests/testdata/expr.yaml | 74 ++++++------ .../tests/testdata/index_selection.yaml | 6 +- .../planner_test/tests/testdata/insert.yaml | 8 +- .../planner_test/tests/testdata/order_by.yaml | 10 +- .../tests/testdata/project_set.yaml | 12 +- .../tests/testdata/range_scan.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 8 +- .../tests/testdata/sysinfo_funcs.yaml | 2 +- .../planner_test/tests/testdata/tpch.yaml | 108 +++++++++--------- src/frontend/src/optimizer/mod.rs | 47 +++++--- .../src/optimizer/optimizer_context.rs | 7 +- .../src/optimizer/plan_node/batch_filter.rs | 5 +- .../src/optimizer/plan_node/batch_hash_agg.rs | 2 +- .../optimizer/plan_node/batch_hash_join.rs | 7 +- .../optimizer/plan_node/batch_lookup_join.rs | 14 +-- .../plan_node/batch_nested_loop_join.rs | 7 +- .../src/optimizer/plan_node/batch_project.rs | 7 +- .../optimizer/plan_node/batch_simple_agg.rs | 2 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 9 +- .../src/optimizer/plan_node/batch_update.rs | 7 +- .../src/optimizer/plan_node/batch_values.rs | 10 +- .../src/optimizer/plan_node/generic/agg.rs | 7 +- .../src/optimizer/plan_node/logical_scan.rs | 12 +- .../src/optimizer/plan_node/logical_source.rs | 1 - .../src/optimizer/plan_node/stream.rs | 29 ++--- .../optimizer/plan_node/stream_delta_join.rs | 7 +- .../src/optimizer/plan_node/stream_filter.rs | 7 +- .../plan_node/stream_global_simple_agg.rs | 2 +- .../optimizer/plan_node/stream_hash_agg.rs | 2 +- .../optimizer/plan_node/stream_hash_join.rs | 7 +- .../plan_node/stream_local_simple_agg.rs | 2 +- .../src/optimizer/plan_node/stream_project.rs | 7 +- 36 files changed, 209 insertions(+), 272 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index b436cbdff2d4e..fd26d4a94ac4a 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | - BatchValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } + BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | - BatchValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } + BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } - sql: | create table t (v1 int); select (ARRAY[1, v1]) from t; @@ -52,7 +52,7 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } + BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[array[66]], array[233]); @@ -60,7 +60,7 @@ LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } + BatchProject { exprs: [ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[66]]); @@ -68,7 +68,7 @@ LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } + BatchProject { exprs: [ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[array[66]]]); @@ -85,7 +85,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } + BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_append(123, 234); @@ -102,7 +102,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } + BatchProject { exprs: [ARRAY[123, 66]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select array_prepend(123, 234); diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 5b141abcc0d1f..51ac43d99d7fd 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (11, 22), (33+(1+2), 44); batch_plan: | - BatchValues { rows: [[11:Int32, 22:Int32], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32]] } + BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | @@ -22,11 +22,11 @@ select * from t where 1>2 and 1=1 and 3<1 and 4<>1 or 1=1 and 2>=1 and 1<=2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + └─BatchFilter { predicate: true:Boolean AND true:Boolean } └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } + └─StreamFilter { predicate: true:Boolean AND true:Boolean } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -130,11 +130,11 @@ - sql: | select * from unnest(Array[1,2,3]); batch_plan: | - BatchTableFunction { Unnest(Array(1:Int32, 2:Int32, 3:Int32)) } + BatchTableFunction { Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 }) } - sql: | select * from unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchTableFunction { Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32))) } + BatchTableFunction { Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } }) } - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index e122dd0ace778..b9385ce48355e 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchProject { exprs: [Case(null:Boolean, 1:Int32)] } + BatchProject { exprs: [null:Int32] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON NULL) sql: | @@ -65,7 +65,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchProject { exprs: [Case(true:Boolean, 1:Int32)] } + BatchProject { exprs: [1:Int32] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index c89502dce3ade..e1fe8632fe021 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,6 +36,16 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } + Const eval exprs: + + BatchProject { exprs: [1:Int32] } + └─BatchValues { rows: [[]] } + + Const eval exprs: + + BatchProject { exprs: [1:Int32] } + └─BatchValues { rows: [[]] } + To Batch Physical Plan: BatchProject { exprs: [1:Int32] } @@ -51,7 +61,7 @@ "stages": { "0": { "root": { - "plan_node_id": 28, + "plan_node_id": 40, "plan_node_type": "BatchProject", "schema": [ { @@ -64,7 +74,7 @@ ], "children": [ { - "plan_node_id": 26, + "plan_node_id": 38, "plan_node_type": "BatchValues", "schema": [], "children": [], diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 6ff7530e9f2c3..3f8d2435e0dfd 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -19,17 +19,17 @@ - sql: | values(cast(1 as bigint)); batch_plan: | - BatchValues { rows: [[1:Int32::Int64]] } + BatchValues { rows: [[1:Int64]] } - sql: | values(not true); batch_plan: | - BatchValues { rows: [[Not(true:Boolean)]] } + BatchValues { rows: [[false:Boolean]] } - sql: | create table t (); select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean))))))] } + └─BatchProject { exprs: [false:Boolean] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,13 +65,13 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)] } + BatchProject { exprs: [true:Boolean] } └─BatchValues { rows: [[]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))] } + BatchProject { exprs: [false:Boolean] } └─BatchValues { rows: [[]] } - name: in-list with misaligned types sql: | @@ -82,7 +82,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1))))] } + BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1))))] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -94,7 +94,7 @@ SELECT b2 from b where 1 in (3, 1.0, (select min(v1) from t)); batch_plan: | BatchProject { exprs: [b.b2] } - └─BatchFilter { predicate: (In(1:Int32::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (1:Int32 = min(min(t.v1)))) } + └─BatchFilter { predicate: (true:Boolean OR (1:Int32 = min(min(t.v1)))) } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b2], distribution: SomeShard } @@ -108,7 +108,7 @@ create table b (b1 int, b2 int); SELECT b2 from b where exists (select 2 from t where v1 in (3, 1.0, b1)); batch_plan: | - BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } + BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b1, b.b2], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } @@ -122,19 +122,19 @@ - sql: | values(round(42.4382, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42.4382:Decimal, 2:Int32)]] } + BatchValues { rows: [[42.44:Decimal]] } - sql: | values(round(42.4382)); batch_plan: | - BatchValues { rows: [[Round(42.4382:Decimal)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(42)); batch_plan: | - BatchValues { rows: [[Round(42:Int32::Float64)]] } + BatchValues { rows: [[42:Float64]] } - sql: | values(round(42, 2)); batch_plan: | - BatchValues { rows: [[RoundDigit(42:Int32::Decimal, 2:Int32)]] } + BatchValues { rows: [[42:Decimal]] } - sql: | values(round(true)); binder_error: |- @@ -144,18 +144,18 @@ -- Single quoted literal can be treated as number without error. values(round('123')); batch_plan: | - BatchValues { rows: [[Round(123:Float64)]] } + BatchValues { rows: [[123:Float64]] } - sql: | -- When it is invalid, PostgreSQL reports error during explain, but we have to wait until execution as of now. #4235 values(round('abc')); - sql: | values(extract(hour from timestamp '2001-02-16 20:38:40')); batch_plan: | - BatchValues { rows: [[Extract('HOUR':Varchar, '2001-02-16 20:38:40':Timestamp)]] } + BatchValues { rows: [[20:Decimal]] } - sql: | values('Postgres' not like 'Post%'); batch_plan: | - BatchValues { rows: [[Not(Like('Postgres':Varchar, 'Post%':Varchar))]] } + BatchValues { rows: [[false:Boolean]] } - sql: | values(1 not like 1.23); binder_error: |- @@ -164,13 +164,13 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchProject { exprs: [((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))] } + BatchProject { exprs: [4:Int32] } └─BatchValues { rows: [[]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar)] } - └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } + BatchProject { exprs: [0:Int32] } + └─BatchFilter { predicate: false:Boolean } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -178,11 +178,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal)] } └─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), t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal), 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: | @@ -190,7 +190,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -198,7 +198,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -264,18 +264,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar)] } └─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), t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar), 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); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -290,11 +290,11 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar)] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar)] } └─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), t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar), 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); @@ -310,22 +310,22 @@ - sql: | select concat(':', true); batch_plan: | - BatchProject { exprs: [ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))] } + BatchProject { exprs: [':t':Varchar] } └─BatchValues { rows: [[]] } - sql: | select ':' || true; batch_plan: | - BatchProject { exprs: [ConcatOp(':':Varchar, true:Boolean::Varchar)] } + BatchProject { exprs: [':true':Varchar] } └─BatchValues { rows: [[]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchProject { exprs: [Substr('hello':Varchar, null:Int32)] } + BatchProject { exprs: [null:Varchar] } └─BatchValues { rows: [[]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchProject { exprs: [Substr(null:Varchar, 1:Int32)] } + BatchProject { exprs: [null:Varchar] } └─BatchValues { rows: [[]] } - sql: | select pg_typeof('123'); @@ -419,10 +419,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [Array(1:Int32)] } + └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - sql: | select 1 < ALL(array[null]::integer[]); @@ -443,10 +443,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } + BatchProject { exprs: [All((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [Array(1:Int32)] } + └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } └─BatchValues { rows: [[]] } - name: now expression sql: | @@ -540,7 +540,7 @@ sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) batch_plan: | - BatchProject { exprs: [(false:Boolean >= Not(In('LN1O0QP1yi':Varchar, Md5('4SeUPZhUbH':Varchar))))] } + BatchProject { exprs: [false:Boolean] } └─BatchValues { rows: [[]] } - name: const_eval of const expr sql: | @@ -548,7 +548,7 @@ select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1)] } + └─BatchProject { exprs: [(3:Int32 + t.v1)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index af18731a327f8..c98da2bf8e0bc 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -77,7 +77,7 @@ select a,b from t1 where a in (1,2) and b in (2,3) batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: In(idx1.b, 2:Int32::Decimal, 3:Int32::Decimal) } + └─BatchFilter { predicate: In(idx1.b, 2:Decimal, 3:Decimal) } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], scan_ranges: [idx1.a = Int32(1) , idx1.a = Int32(2)], distribution: UpstreamHashShard(idx1.a, idx1.b) } - sql: | create table t1 (a int, b numeric, c bigint); @@ -213,13 +213,13 @@ update t1 set c = 3 where a = 1 and b = 2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } + └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchExchange { order: [], dist: Single } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: | - BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } + BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index aa67c803d90a2..985277c9273f7 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } + └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } - name: insert values on non-assign-castable types sql: | create table t (v1 real, v2 int); @@ -73,7 +73,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [['2020-01-01 01:02:03':Timestamp::Time], ['03:04:05':Time]] } + └─BatchValues { rows: [['01:02:03':Time], ['03:04:05':Time]] } - name: a `VALUES` without insert context may be invalid on its own (compare with above) sql: | create table t (v1 time); @@ -93,7 +93,7 @@ sql: | values (1), (null), (2.3); batch_plan: | - BatchValues { rows: [[1:Int32::Decimal], [null:Decimal], [2.3:Decimal]] } + BatchValues { rows: [[1:Decimal], [null:Decimal], [2.3:Decimal]] } - name: rows of different number of columns sql: | values (1), (2, 3); @@ -115,7 +115,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time, 11:Int32, 4.5:Decimal::Float32] } + └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 4f460213c6791..5a74f834a74f5 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [(1:Int32 + 1:Int32) ASC], dist: Single } - └─BatchSort { order: [(1:Int32 + 1:Int32) ASC] } - └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32)] } + └─BatchExchange { order: [2:Int32 ASC], dist: Single } + └─BatchSort { order: [2:Int32 ASC] } + └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, (1:Int32 + 1:Int32)(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(1:Int32 + 1:Int32), t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32), t._row_id] } + StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, 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); diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 4adbeadc2bc68..fde593e397a9c 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -8,14 +8,14 @@ - sql: | select unnest(Array[1,2,3]); batch_plan: | - BatchProject { exprs: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } - └─BatchProjectSet { select_list: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } + BatchProject { exprs: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } └─BatchValues { rows: [[]] } - sql: | select unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchProject { exprs: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } - └─BatchProjectSet { select_list: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } + BatchProject { exprs: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } + └─BatchProjectSet { select_list: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } └─BatchValues { rows: [[]] } - sql: | create table t(x int[]); @@ -43,9 +43,9 @@ create table t(x int[]); select unnest(x), unnest(Array[1,2]) from t; batch_plan: | - BatchProject { exprs: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + BatchProject { exprs: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchExchange { order: [], dist: Single } - └─BatchProjectSet { select_list: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } + └─BatchProjectSet { select_list: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions sql: | diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index df8dbca94c355..4c6678d46ccbd 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -87,7 +87,7 @@ SELECT * FROM orders_count_by_user WHERE user_id > 42 AND date = 1111 AND 2>1 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (2:Int32 > 1:Int32) AND (orders_count_by_user.date = 1111:Int32) } + └─BatchFilter { predicate: true:Boolean AND (orders_count_by_user.date = 1111:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id > Int64(42)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv @@ -95,7 +95,7 @@ SELECT * FROM orders_count_by_user WHERE date > 1111 AND user_id = 42 AND 5<6 AND date <= 6666 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: (5:Int32 < 6:Int32) } + └─BatchFilter { predicate: true:Boolean } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id = Int64(42) AND orders_count_by_user.date > Int32(1111) AND orders_count_by_user.date <= Int32(6666)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index d150ff23dcd82..6475ea7f149e9 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 = 519 } + | └─StreamShare { id = 599 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 519 } + └─StreamShare { id = 599 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } @@ -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 = 744 } + | └─StreamShare { id = 832 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_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 = 744 } + └─StreamShare { id = 832 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index c40ca8c25279d..db6be75eb1831 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -37,5 +37,5 @@ - sql: | select current_timestamp; batch_plan: | - BatchProject { exprs: [Now] } + BatchProject { exprs: ['2021-04-01 00:00:00+00:00':Timestamptz] } └─BatchValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 953cdbb3bcfd3..320c48d8f538f 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -136,7 +136,7 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), 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)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─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] } @@ -144,7 +144,7 @@ └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), 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)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -157,7 +157,7 @@ Fragment 1 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } 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 @@ -727,7 +727,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey] } @@ -741,7 +741,7 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } @@ -764,7 +764,7 @@ Fragment 2 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -859,7 +859,7 @@ | | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], distribution: UpstreamHashShard(customer.c_custkey) } | | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } | └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey) } | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } @@ -884,7 +884,7 @@ | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } | | | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | | | └─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -943,7 +943,7 @@ Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -1020,7 +1020,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─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) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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: [] } @@ -1029,7 +1029,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), 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) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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 @@ -1043,7 +1043,7 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), 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) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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 @@ -1350,11 +1350,11 @@ └─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: [Extract('YEAR':Varchar, orders.o_orderdate) ASC], dist: Single } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } └─BatchSort { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC] } - └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─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: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], 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] } @@ -1832,7 +1832,7 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -1858,7 +1858,7 @@ | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -1910,7 +1910,7 @@ Fragment 6 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -2229,7 +2229,7 @@ | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode] } - └─BatchFilter { 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)) } + └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └─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] } @@ -2242,7 +2242,7 @@ | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2267,7 +2267,7 @@ Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2415,27 +2415,27 @@ ├─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(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─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] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } └─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(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, 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(part.p_partkey) } └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } @@ -2443,14 +2443,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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 @@ -2458,7 +2458,7 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_partkey, 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 @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], 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] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─BatchSimpleAgg { aggs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2563,7 +2563,7 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2573,12 +2573,12 @@ | ├─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((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 900 } + | └─StreamShare { id = 986 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─StreamProject { exprs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2587,12 +2587,12 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 900 } + └─StreamShare { id = 986 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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)) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2623,7 +2623,7 @@ Fragment 4 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 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)) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } 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 @@ -3369,7 +3369,7 @@ | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } @@ -3404,7 +3404,7 @@ | └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -3476,7 +3476,7 @@ Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, 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 diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index c511b80f358f0..0f61954f772aa 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -13,6 +13,8 @@ // limitations under the License. pub mod plan_node; +use std::ops::DerefMut; + pub use plan_node::PlanRef; pub mod property; @@ -453,15 +455,22 @@ impl PlanRoot { // Convert to physical plan node plan = plan.to_batch_with_order_required(&self.required_order)?; - // TODO: SessionTimezone substitution + let ctx = plan.ctx(); + // Inline session timezone + plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?; + + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + // Const eval of exprs at the last minute - // plan = const_eval_exprs(plan)?; + plan = const_eval_exprs(plan)?; - // let ctx = plan.ctx(); - // if ctx.is_explain_trace() { - // ctx.trace("Const eval exprs:"); - // ctx.trace(plan.explain_to_string().unwrap()); - // } + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -615,13 +624,21 @@ impl PlanRoot { ); } + // Inline session timezone + plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?; + + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } + // Const eval of exprs at the last minute - // plan = const_eval_exprs(plan)?; + plan = const_eval_exprs(plan)?; - // if ctx.is_explain_trace() { - // ctx.trace("Const eval exprs:"); - // ctx.trace(plan.explain_to_string().unwrap()); - // } + if ctx.is_explain_trace() { + ctx.trace("Const eval exprs:"); + ctx.trace(plan.explain_to_string().unwrap()); + } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -732,7 +749,6 @@ impl PlanRoot { } } -#[allow(dead_code)] fn const_eval_exprs(plan: PlanRef) -> Result { let mut const_eval_rewriter = ConstEvalRewriter { error: None }; @@ -743,6 +759,11 @@ fn const_eval_exprs(plan: PlanRef) -> Result { Ok(plan) } +fn inline_session_timezone_in_exprs(ctx: OptimizerContextRef, plan: PlanRef) -> Result { + let plan = plan.rewrite_exprs_recursive(ctx.session_timezone().deref_mut()); + Ok(plan) +} + #[cfg(test)] mod tests { use risingwave_common::catalog::Field; diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 81c4e52eb3a2c..2c3b18007831f 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -14,7 +14,7 @@ use core::convert::Into; use core::fmt::Formatter; -use std::cell::RefCell; +use std::cell::{RefCell, RefMut}; use std::rc::Rc; use std::sync::Arc; @@ -155,9 +155,8 @@ impl OptimizerContext { &self.normalized_sql } - pub fn expr_with_session_timezone(&self, expr: ExprImpl) -> ExprImpl { - let mut session_timezone = self.session_timezone.borrow_mut(); - session_timezone.rewrite_expr(expr) + pub fn session_timezone(&self) -> RefMut<'_, SessionTimezone> { + self.session_timezone.borrow_mut() } /// Appends any information that the optimizer needs to alert the user about to the PG NOTICE diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index c92634c2713e1..b574ef48cfd15 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -80,10 +80,7 @@ impl ToBatchProst for BatchFilter { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::Filter(FilterNode { search_condition: Some( - self.base - .ctx() - .expr_with_session_timezone(ExprImpl::from(self.logical.predicate().clone())) - .to_expr_proto(), + ExprImpl::from(self.logical.predicate().clone()).to_expr_proto(), ), }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index db1847f18cb7e..7698b64e18b73 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -89,7 +89,7 @@ impl ToBatchProst for BatchHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), group_key: self .group_key() diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index 77fdef9ffefeb..6bb3a1df42648 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -236,12 +236,7 @@ impl ToBatchProst for BatchHashJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), output_indices: self .logical .output_indices() diff --git a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs index 3a2b7a124cf99..640afea315d02 100644 --- a/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_lookup_join.rs @@ -199,12 +199,7 @@ impl ToBatchProst for BatchLookupJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), outer_side_key: self .eq_join_predicate .left_eq_indexes() @@ -239,12 +234,7 @@ impl ToBatchProst for BatchLookupJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), outer_side_key: self .eq_join_predicate .left_eq_indexes() diff --git a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs index 490cc65d733c8..5f4d92021570a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_nested_loop_join.rs @@ -131,12 +131,7 @@ impl ToBatchProst for BatchNestedLoopJoin { fn to_batch_prost_body(&self) -> NodeBody { NodeBody::NestedLoopJoin(NestedLoopJoinNode { join_type: self.logical.join_type() as i32, - join_cond: Some( - self.base - .ctx() - .expr_with_session_timezone(ExprImpl::from(self.logical.on().clone())) - .to_expr_proto(), - ), + join_cond: Some(ExprImpl::from(self.logical.on().clone()).to_expr_proto()), output_indices: self .logical .output_indices() diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 538575bd2f903..9dd72632abcb8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -89,12 +89,7 @@ impl ToBatchProst for BatchProject { .logical .exprs() .iter() - .map(|expr| { - self.base - .ctx() - .expr_with_session_timezone(expr.clone()) - .to_expr_proto() - }) + .map(|expr| expr.to_expr_proto()) .collect::>(); NodeBody::Project(ProjectNode { select_list }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index afcb5b62a72ab..41b6adedd461e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -114,7 +114,7 @@ impl ToBatchProst for BatchSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), // We treat simple agg as a special sort agg without group key. group_key: vec![], diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 1e3056dff4427..96699616aaacf 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -119,18 +119,13 @@ impl ToBatchProst for BatchSortAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), group_key: self .group_key() .iter() .map(|idx| ExprImpl::InputRef(Box::new(InputRef::new(*idx, DataType::Int32)))) - .map(|expr| { - self.base - .ctx() - .expr_with_session_timezone(expr) - .to_expr_proto() - }) + .map(|expr| expr.to_expr_proto()) .collect::>(), }) } diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index 381072d592051..ec9cb871b08c8 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -79,12 +79,7 @@ impl ToBatchProst for BatchUpdate { .logical .exprs() .iter() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) + .map(|x| x.to_expr_proto()) .collect(); NodeBody::Update(UpdateNode { diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 21b719d3821eb..3ab64e8fb839e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -55,15 +55,7 @@ impl BatchValues { } fn row_to_protobuf(&self, row: &[ExprImpl]) -> ExprTuple { - let cells = row - .iter() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) - .collect(); + let cells = row.iter().map(|x| x.to_expr_proto()).collect(); ExprTuple { cells } } } diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index b90e1b9cd1511..5ff2158e45738 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -541,7 +541,7 @@ impl PlanAggCall { }); } - pub fn to_protobuf(&self, ctx: OptimizerContextRef) -> ProstAggCall { + pub fn to_protobuf(&self) -> ProstAggCall { ProstAggCall { r#type: self.agg_kind.to_prost().into(), return_type: Some(self.return_type.to_protobuf()), @@ -552,10 +552,7 @@ impl PlanAggCall { .iter() .map(PlanAggOrderByField::to_protobuf) .collect(), - filter: self - .filter - .as_expr_unless_true() - .map(|x| ctx.expr_with_session_timezone(x).to_expr_proto()), + filter: self.filter.as_expr_unless_true().map(|x| x.to_expr_proto()), } } diff --git a/src/frontend/src/optimizer/plan_node/logical_scan.rs b/src/frontend/src/optimizer/plan_node/logical_scan.rs index 9528e02a58685..846b4c061f084 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -586,17 +586,7 @@ impl ToBatch for LogicalScan { } fn to_batch_with_order_required(&self, required_order: &Order) -> Result { - // rewrite the condition before converting to batch as we will handle the expressions in a - // special way - let new_predicate = Condition { - conjunctions: self - .predicate() - .conjunctions - .iter() - .map(|expr| self.base.ctx().expr_with_session_timezone(expr.clone())) - .collect(), - }; - let new = self.clone_with_predicate(new_predicate); + let new = self.clone_with_predicate(self.predicate().clone()); if !new.indexes().is_empty() { let index_selection_rule = IndexSelectionRule::create(); diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 789edaf9ced68..3e9dff4b41297 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -325,7 +325,6 @@ impl PredicatePushdown for LogicalSource { let mut new_conjunctions = Vec::with_capacity(predicate.conjunctions.len()); for expr in predicate.conjunctions { - let expr = self.base.ctx().expr_with_session_timezone(expr); if let Some(e) = expr_to_kafka_timestamp_range(expr, &mut range, &self.base.schema) { // Not recognized, so push back new_conjunctions.push(e); diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 9ad6af663c5e0..903e114b63aa4 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -436,7 +436,7 @@ pub fn to_stream_prost_body( let condition = me .predicate() .as_expr_unless_true() - .map(|x| base.ctx().expr_with_session_timezone(x).to_expr_proto()); + .map(|x| x.to_expr_proto()); let left_table = infer_left_internal_table_catalog(base, me.left_index) .with_id(state.gen_table_id_wrapped()); let right_table = infer_right_internal_table_catalog(&me.right.0) @@ -482,7 +482,7 @@ pub fn to_stream_prost_body( .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| base.ctx().expr_with_session_timezone(x).to_expr_proto()), + .map(|x| x.to_expr_proto()), left_table_id: left_table_desc.table_id.table_id(), right_table_id: right_table_desc.table_id.table_id(), left_info: Some(ArrangementInfo { @@ -526,11 +526,7 @@ pub fn to_stream_prost_body( Node::Filter(me) => { let me = &me.core; ProstNode::Filter(FilterNode { - search_condition: Some( - base.ctx() - .expr_with_session_timezone(ExprImpl::from(me.predicate.clone())) - .to_expr_proto(), - ), + search_condition: Some(ExprImpl::from(me.predicate.clone()).to_expr_proto()), }) } Node::GlobalSimpleAgg(me) => { @@ -542,7 +538,7 @@ pub fn to_stream_prost_body( agg_calls: me .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), distribution_key: base .dist @@ -588,7 +584,7 @@ pub fn to_stream_prost_body( .core .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), is_append_only: me.core.input.0.append_only, @@ -650,7 +646,7 @@ pub fn to_stream_prost_body( .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| base.ctx().expr_with_session_timezone(x).to_expr_proto()), + .map(|x| x.to_expr_proto()), left_table: Some(left_table.to_internal_table_prost()), right_table: Some(right_table.to_internal_table_prost()), left_degree_table: Some(left_degree_table.to_internal_table_prost()), @@ -676,7 +672,7 @@ pub fn to_stream_prost_body( agg_calls: me .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), distribution_key: base .dist @@ -709,16 +705,7 @@ pub fn to_stream_prost_body( ProstNode::ProjectSet(ProjectSetNode { select_list }) } Node::Project(me) => ProstNode::Project(ProjectNode { - select_list: me - .core - .exprs - .iter() - .map(|x| { - base.ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) - .collect(), + select_list: me.core.exprs.iter().map(|x| x.to_expr_proto()).collect(), watermark_input_key: me .watermark_derivations .iter() diff --git a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs index 272e6115ddfa1..d0a6a63a25242 100644 --- a/src/frontend/src/optimizer/plan_node/stream_delta_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_delta_join.rs @@ -192,12 +192,7 @@ impl StreamNode for StreamDeltaJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), left_table_id: left_table_desc.table_id.table_id(), right_table_id: right_table_desc.table_id.table_id(), left_info: Some(ArrangementInfo { diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index 6599c69344576..ad71756a78061 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -76,12 +76,7 @@ impl_plan_tree_node_for_unary! { StreamFilter } impl StreamNode for StreamFilter { fn to_stream_prost_body(&self, _state: &mut BuildFragmentGraphState) -> ProstStreamNode { ProstStreamNode::Filter(FilterNode { - search_condition: Some( - self.base - .ctx() - .expr_with_session_timezone(ExprImpl::from(self.predicate().clone())) - .to_expr_proto(), - ), + search_condition: Some(ExprImpl::from(self.predicate().clone()).to_expr_proto()), }) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index 0928a9da4f7a4..87ab9e55de7e7 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -98,7 +98,7 @@ impl StreamNode for StreamGlobalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), distribution_key: self .base diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 60cb7ac010f93..288e2ad46dd07 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -113,7 +113,7 @@ impl StreamNode for StreamHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), is_append_only: self.input().append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs index 3c6e9b72d80d8..f4a97229c93d6 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_join.rs @@ -263,12 +263,7 @@ impl StreamNode for StreamHashJoin { .eq_join_predicate .other_cond() .as_expr_unless_true() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x) - .to_expr_proto() - }), + .map(|x| x.to_expr_proto()), left_table: Some(left_table.to_internal_table_prost()), right_table: Some(right_table.to_internal_table_prost()), left_degree_table: Some(left_degree_table.to_internal_table_prost()), diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index 2c945a92782ad..da6aaf02e43e1 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -96,7 +96,7 @@ impl StreamNode for StreamLocalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(|x| PlanAggCall::to_protobuf(x)) .collect(), distribution_key: self .distribution() diff --git a/src/frontend/src/optimizer/plan_node/stream_project.rs b/src/frontend/src/optimizer/plan_node/stream_project.rs index 613e9abd5b275..5fdfeed19d329 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -130,12 +130,7 @@ impl StreamNode for StreamProject { .logical .exprs() .iter() - .map(|x| { - self.base - .ctx() - .expr_with_session_timezone(x.clone()) - .to_expr_proto() - }) + .map(|x| x.to_expr_proto()) .collect(), watermark_input_key: self .watermark_derivations From 7fd06c6e1114968f7f210d32bab06b0f7dc20a45 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 12:23:02 +0800 Subject: [PATCH 05/30] fix clippy --- src/frontend/src/optimizer/optimizer_context.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_hash_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_sort_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/stream.rs | 6 +++--- .../src/optimizer/plan_node/stream_global_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_hash_agg.rs | 2 +- .../src/optimizer/plan_node/stream_local_simple_agg.rs | 2 +- 8 files changed, 10 insertions(+), 10 deletions(-) diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 2c3b18007831f..13b4fa095b700 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -20,7 +20,7 @@ use std::sync::Arc; use risingwave_sqlparser::ast::{ExplainOptions, ExplainType}; -use crate::expr::{CorrelatedId, ExprImpl, ExprRewriter, SessionTimezone}; +use crate::expr::{CorrelatedId, SessionTimezone}; use crate::handler::HandlerArgs; use crate::optimizer::plan_node::PlanNodeId; use crate::session::SessionImpl; diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs index 7698b64e18b73..abf73cc4549d0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -89,7 +89,7 @@ impl ToBatchProst for BatchHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), group_key: self .group_key() diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index 41b6adedd461e..0d36c48103270 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -114,7 +114,7 @@ impl ToBatchProst for BatchSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), // We treat simple agg as a special sort agg without group key. group_key: vec![], diff --git a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs index 96699616aaacf..a6b9bd5b6b07a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_sort_agg.rs @@ -119,7 +119,7 @@ impl ToBatchProst for BatchSortAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), group_key: self .group_key() diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 903e114b63aa4..9396f1be0d383 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -538,7 +538,7 @@ pub fn to_stream_prost_body( agg_calls: me .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), distribution_key: base .dist @@ -584,7 +584,7 @@ pub fn to_stream_prost_body( .core .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), is_append_only: me.core.input.0.append_only, @@ -672,7 +672,7 @@ pub fn to_stream_prost_body( agg_calls: me .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), distribution_key: base .dist diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index 87ab9e55de7e7..d75dea4979263 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -98,7 +98,7 @@ impl StreamNode for StreamGlobalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), distribution_key: self .base diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 288e2ad46dd07..890e772999b53 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -113,7 +113,7 @@ impl StreamNode for StreamHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), is_append_only: self.input().append_only(), diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index da6aaf02e43e1..10769eb15d36d 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -96,7 +96,7 @@ impl StreamNode for StreamLocalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x)) + .map(PlanAggCall::to_protobuf) .collect(), distribution_key: self .distribution() From e3fac9d60304f0583c467f017f5a505c4363e38a Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 12:29:01 +0800 Subject: [PATCH 06/30] stash --- src/frontend/src/optimizer/plan_node/stream.rs | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream.rs b/src/frontend/src/optimizer/plan_node/stream.rs index 9396f1be0d383..c2a81fb152043 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -535,11 +535,7 @@ pub fn to_stream_prost_body( let agg_states = me.infer_stream_agg_state(base, None); ProstNode::GlobalSimpleAgg(SimpleAggNode { - agg_calls: me - .agg_calls - .iter() - .map(PlanAggCall::to_protobuf) - .collect(), + agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), distribution_key: base .dist .dist_column_indices() @@ -669,11 +665,7 @@ pub fn to_stream_prost_body( Node::LocalSimpleAgg(me) => { let me = &me.core; ProstNode::LocalSimpleAgg(SimpleAggNode { - agg_calls: me - .agg_calls - .iter() - .map(PlanAggCall::to_protobuf) - .collect(), + agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), distribution_key: base .dist .dist_column_indices() From ec5ea0def6c92ee1e061a76aab5f8ba68b1c3de8 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 12:41:05 +0800 Subject: [PATCH 07/30] fix --- src/frontend/src/optimizer/plan_node/batch_filter.rs | 1 - src/frontend/src/optimizer/plan_node/batch_project.rs | 1 - src/frontend/src/optimizer/plan_node/batch_simple_agg.rs | 2 +- src/frontend/src/optimizer/plan_node/batch_update.rs | 1 - src/frontend/src/optimizer/plan_node/batch_values.rs | 1 - src/frontend/src/optimizer/plan_node/logical_source.rs | 2 +- src/frontend/src/optimizer/plan_node/stream_filter.rs | 1 - .../src/optimizer/plan_node/stream_global_simple_agg.rs | 1 - src/frontend/src/optimizer/plan_node/stream_hash_agg.rs | 1 - src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs | 1 - 10 files changed, 2 insertions(+), 10 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/batch_filter.rs b/src/frontend/src/optimizer/plan_node/batch_filter.rs index b574ef48cfd15..9099b7712e1b9 100644 --- a/src/frontend/src/optimizer/plan_node/batch_filter.rs +++ b/src/frontend/src/optimizer/plan_node/batch_filter.rs @@ -18,7 +18,6 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::FilterNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, }; diff --git a/src/frontend/src/optimizer/plan_node/batch_project.rs b/src/frontend/src/optimizer/plan_node/batch_project.rs index 9dd72632abcb8..9dd674374fdf0 100644 --- a/src/frontend/src/optimizer/plan_node/batch_project.rs +++ b/src/frontend/src/optimizer/plan_node/batch_project.rs @@ -19,7 +19,6 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::ProjectNode; use risingwave_pb::expr::ExprNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalProject, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index 0d36c48103270..938154b897aed 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -18,7 +18,7 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; -use super::generic::{GenericPlanRef, PlanAggCall}; +use super::generic::PlanAggCall; use super::{ ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, diff --git a/src/frontend/src/optimizer/plan_node/batch_update.rs b/src/frontend/src/optimizer/plan_node/batch_update.rs index ec9cb871b08c8..c0c1fe7119e1a 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -18,7 +18,6 @@ use risingwave_common::error::Result; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::UpdateNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalUpdate, PlanBase, PlanRef, PlanTreeNodeUnary, ToBatchProst, ToDistributedBatch, diff --git a/src/frontend/src/optimizer/plan_node/batch_values.rs b/src/frontend/src/optimizer/plan_node/batch_values.rs index 3ab64e8fb839e..0cffc31e12c96 100644 --- a/src/frontend/src/optimizer/plan_node/batch_values.rs +++ b/src/frontend/src/optimizer/plan_node/batch_values.rs @@ -19,7 +19,6 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::values_node::ExprTuple; use risingwave_pb::batch_plan::ValuesNode; -use super::generic::GenericPlanRef; use super::{ ExprRewritable, LogicalValues, PlanBase, PlanRef, PlanTreeNodeLeaf, ToBatchProst, ToDistributedBatch, diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 3e9dff4b41297..e313ce255ec57 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -22,7 +22,7 @@ use risingwave_common::catalog::{ColumnDesc, Schema}; use risingwave_common::error::Result; use risingwave_connector::source::DataType; -use super::generic::{GenericPlanNode, GenericPlanRef}; +use super::generic::GenericPlanNode; use super::{ generic, BatchSource, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, PlanRef, PredicatePushdown, StreamRowIdGen, StreamSource, ToBatch, ToStream, diff --git a/src/frontend/src/optimizer/plan_node/stream_filter.rs b/src/frontend/src/optimizer/plan_node/stream_filter.rs index ad71756a78061..c3c7aa77e3946 100644 --- a/src/frontend/src/optimizer/plan_node/stream_filter.rs +++ b/src/frontend/src/optimizer/plan_node/stream_filter.rs @@ -17,7 +17,6 @@ use std::fmt; use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use risingwave_pb::stream_plan::FilterNode; -use super::generic::GenericPlanRef; use super::{ExprRewritable, LogicalFilter, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::{Expr, ExprImpl, ExprRewriter}; use crate::optimizer::plan_node::PlanBase; diff --git a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs index d75dea4979263..c09d5a7ed1736 100644 --- a/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_global_simple_agg.rs @@ -20,7 +20,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index 890e772999b53..c8cf0b687d943 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -20,7 +20,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; diff --git a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs index 10769eb15d36d..2483bc092bf50 100644 --- a/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_local_simple_agg.rs @@ -21,7 +21,6 @@ use risingwave_pb::stream_plan::stream_node::NodeBody as ProstStreamNode; use super::generic::PlanAggCall; use super::{ExprRewritable, LogicalAgg, PlanBase, PlanRef, PlanTreeNodeUnary, StreamNode}; use crate::expr::ExprRewriter; -use crate::optimizer::plan_node::generic::GenericPlanRef; use crate::optimizer::property::RequiredDist; use crate::stream_fragmenter::BuildFragmentGraphState; From df8a23183f3a8917859b1bbd1bc9dc6d6c769afe Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 13:40:56 +0800 Subject: [PATCH 08/30] try disable --- .../optimizer/plan_node/stream_table_scan.rs | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index d1f13a9f838be..43ccd2aa4e53b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -238,18 +238,18 @@ impl StreamTableScan { } impl ExprRewritable for StreamTableScan { - fn has_rewritable_expr(&self) -> bool { - true - } - - fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self::new( - self.logical - .rewrite_exprs(r) - .as_logical_scan() - .unwrap() - .clone(), - ) - .into() - } + // fn has_rewritable_expr(&self) -> bool { + // true + // } + + // fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + // Self::new( + // self.logical + // .rewrite_exprs(r) + // .as_logical_scan() + // .unwrap() + // .clone(), + // ) + // .into() + // } } From 36970f084c1b2e300d11755fc53977fbdfc9877f Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 13:46:28 +0800 Subject: [PATCH 09/30] fix --- .../optimizer/plan_node/stream_table_scan.rs | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 43ccd2aa4e53b..894fcbe1d7fb9 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -238,18 +238,19 @@ impl StreamTableScan { } impl ExprRewritable for StreamTableScan { - // fn has_rewritable_expr(&self) -> bool { - // true - // } - - // fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - // Self::new( - // self.logical - // .rewrite_exprs(r) - // .as_logical_scan() - // .unwrap() - // .clone(), - // ) - // .into() - // } + fn has_rewritable_expr(&self) -> bool { + true + } + + fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { + Self::new_with_chain_type( + self.logical + .rewrite_exprs(r) + .as_logical_scan() + .unwrap() + .clone(), + self.chain_type + ) + .into() + } } From f94290382ec6e5a76b244d2a5a800cdbfca23796 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 13:48:11 +0800 Subject: [PATCH 10/30] minor --- src/frontend/src/optimizer/plan_node/stream_table_scan.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs index 894fcbe1d7fb9..70b72368bde69 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -249,7 +249,7 @@ impl ExprRewritable for StreamTableScan { .as_logical_scan() .unwrap() .clone(), - self.chain_type + self.chain_type, ) .into() } From 2c49c0adc951c3d8541b8716bdfcc5c035864a8e Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 14:45:59 +0800 Subject: [PATCH 11/30] add test changes --- e2e_test/batch/functions/now.slt.part | 12 ++++++------ src/frontend/planner_test/tests/testdata/expr.yaml | 1 + .../planner_test/tests/testdata/sysinfo_funcs.yaml | 2 +- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index 8d3ce7867b3d9..0d3373546267f 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -10,12 +10,12 @@ create table t (a timestamp); statement ok insert into t values(now()); -# constant eval of now in batch plan -# query T -# explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; -# ---- -# BatchProject { exprs: [true:Boolean] } -# └─BatchValues { rows: [[]] } +constant eval of now in batch plan +query T +explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; +---- +BatchProject { exprs: [true:Boolean] } + └─BatchValues { rows: [[]] } statement ok drop table tz diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 3f8d2435e0dfd..59e9aa06cb7a7 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -552,3 +552,4 @@ └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; + batch_error: 'Expr error: Division by zero' diff --git a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index 3466daebe9b9b..6ca6d0a1a33ad 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -37,5 +37,5 @@ - sql: | select current_timestamp > now() - interval '10 minutes'; batch_plan: | - BatchProject { exprs: [t::bolean] } + BatchProject { exprs: [true:Boolean] } └─BatchValues { rows: [[]] } From 7fa7c228942045d0babf86d22447f589dbf513ab Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 14:46:54 +0800 Subject: [PATCH 12/30] add fix --- src/expr/src/expr/expr_field.rs | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/src/expr/src/expr/expr_field.rs b/src/expr/src/expr/expr_field.rs index 555d87baf29af..2b2df2910a866 100644 --- a/src/expr/src/expr/expr_field.rs +++ b/src/expr/src/expr/expr_field.rs @@ -17,7 +17,7 @@ use std::convert::TryFrom; use anyhow::anyhow; use risingwave_common::array::{ArrayImpl, ArrayRef, DataChunk}; use risingwave_common::row::OwnedRow; -use risingwave_common::types::{DataType, Datum}; +use risingwave_common::types::{DataType, Datum, ScalarImpl}; use risingwave_common::util::value_encoding::deserialize_datum; use risingwave_pb::expr::expr_node::{RexNode, Type}; use risingwave_pb::expr::ExprNode; @@ -47,8 +47,12 @@ impl Expression for FieldExpression { } } - fn eval_row(&self, _input: &OwnedRow) -> Result { - Err(anyhow!("expects a struct array ref").into()) + fn eval_row(&self, input: &OwnedRow) -> Result { + let struct_datum = self.input.eval_row(input)?; + struct_datum.map(|s| match s { + ScalarImpl::Struct(v) => Ok(v.fields()[self.index].clone()), + _ => Err(anyhow!("expects a struct array ref").into()) + }).transpose().map(|x| x.flatten()) } } From e286e1bfd43339d0e4d2d06ce02116a21880ef12 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 14:57:29 +0800 Subject: [PATCH 13/30] fix --- e2e_test/batch/functions/now.slt.part | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index 0d3373546267f..5adb66c83db2a 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -10,7 +10,7 @@ create table t (a timestamp); statement ok insert into t values(now()); -constant eval of now in batch plan +# constant eval of now in batch plan query T explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; ---- From 1c639976c5970fd4165360cf03cc6e6c3b9a093f Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 15:00:26 +0800 Subject: [PATCH 14/30] minor --- src/expr/src/expr/expr_field.rs | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/src/expr/src/expr/expr_field.rs b/src/expr/src/expr/expr_field.rs index 2b2df2910a866..06e91130d1c2f 100644 --- a/src/expr/src/expr/expr_field.rs +++ b/src/expr/src/expr/expr_field.rs @@ -49,10 +49,13 @@ impl Expression for FieldExpression { fn eval_row(&self, input: &OwnedRow) -> Result { let struct_datum = self.input.eval_row(input)?; - struct_datum.map(|s| match s { - ScalarImpl::Struct(v) => Ok(v.fields()[self.index].clone()), - _ => Err(anyhow!("expects a struct array ref").into()) - }).transpose().map(|x| x.flatten()) + struct_datum + .map(|s| match s { + ScalarImpl::Struct(v) => Ok(v.fields()[self.index].clone()), + _ => Err(anyhow!("expects a struct array ref").into()), + }) + .transpose() + .map(|x| x.flatten()) } } From 3e483d22ca848d9dcb005ce773b09f34bce51de3 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 17:03:03 +0800 Subject: [PATCH 15/30] fix --- src/common/src/types/interval.rs | 21 ++++++++++++++++----- 1 file changed, 16 insertions(+), 5 deletions(-) diff --git a/src/common/src/types/interval.rs b/src/common/src/types/interval.rs index aacd623cff7a6..d172cfe9239da 100644 --- a/src/common/src/types/interval.rs +++ b/src/common/src/types/interval.rs @@ -686,11 +686,17 @@ impl Display for IntervalUnit { write(format_args!("{days} days"))?; } if self.ms != 0 || self.months == 0 && self.days == 0 { - let hours = self.ms / 1000 / 3600; - let minutes = (self.ms / 1000 / 60) % 60; - let seconds = self.ms % 60000 / 1000; - let secs_fract = self.ms % 1000; - write(format_args!("{hours:0>2}:{minutes:0>2}:{seconds:0>2}"))?; + let ms = self.ms.abs(); + let hours = ms / 1000 / 3600; + let minutes = (ms / 1000 / 60) % 60; + let seconds = ms % 60000 / 1000; + let secs_fract = ms % 1000; + + if self.ms < 0 { + write(format_args!("-{hours:0>2}:{minutes:0>2}:{seconds:0>2}"))?; + } else { + write(format_args!("{hours:0>2}:{minutes:0>2}:{seconds:0>2}"))?; + } if secs_fract != 0 { let mut buf = [0u8; 4]; write!(buf.as_mut_slice(), ".{:03}", secs_fract).unwrap(); @@ -1122,6 +1128,11 @@ mod tests { "-1 years -2 mons 3 days" ); assert_eq!(IntervalUnit::default().to_string(), "00:00:00"); + assert_eq!( + IntervalUnit::new(-14, 3, -(11 * 3600 * 1000 + 45 * 60 * 1000 + 14 * 1000 + 233)) + .to_string(), + "-1 years -2 mons 3 days -11:45:14.233" + ); } #[test] From 4ffa9e5311be9876fe634a68f85909aa576ecd80 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 8 Feb 2023 17:18:20 +0800 Subject: [PATCH 16/30] minor --- src/common/src/types/interval.rs | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/src/common/src/types/interval.rs b/src/common/src/types/interval.rs index d172cfe9239da..2f4c73f9a5417 100644 --- a/src/common/src/types/interval.rs +++ b/src/common/src/types/interval.rs @@ -1129,8 +1129,12 @@ mod tests { ); assert_eq!(IntervalUnit::default().to_string(), "00:00:00"); assert_eq!( - IntervalUnit::new(-14, 3, -(11 * 3600 * 1000 + 45 * 60 * 1000 + 14 * 1000 + 233)) - .to_string(), + IntervalUnit::new( + -14, + 3, + -(11 * 3600 * 1000 + 45 * 60 * 1000 + 14 * 1000 + 233) + ) + .to_string(), "-1 years -2 mons 3 days -11:45:14.233" ); } From d4cad1edbeedac149f85040208eef3efad946f5a Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Mon, 20 Feb 2023 12:04:06 +0800 Subject: [PATCH 17/30] fix --- e2e_test/source/basic/kafka_batch.slt | 4 +- .../planner_test/tests/testdata/agg.yaml | 126 ++-- .../planner_test/tests/testdata/array.yaml | 12 +- .../planner_test/tests/testdata/cast.yaml | 4 +- .../planner_test/tests/testdata/delete.yaml | 2 +- .../tests/testdata/dynamic_filter.yaml | 36 +- .../planner_test/tests/testdata/explain.yaml | 10 +- .../planner_test/tests/testdata/expr.yaml | 68 +- .../tests/testdata/index_selection.yaml | 2 +- .../planner_test/tests/testdata/insert.yaml | 12 +- .../planner_test/tests/testdata/join.yaml | 30 +- .../tests/testdata/mv_column_name.yaml | 8 +- .../planner_test/tests/testdata/nexmark.yaml | 278 ++++----- .../tests/testdata/nexmark_source.yaml | 294 ++++----- .../planner_test/tests/testdata/order_by.yaml | 36 +- .../tests/testdata/over_window_function.yaml | 16 +- .../tests/testdata/predicate_pushdown.yaml | 8 +- .../tests/testdata/project_set.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 8 +- .../tests/testdata/shared_views.yaml | 10 +- .../tests/testdata/stream_dist_agg.yaml | 28 +- .../planner_test/tests/testdata/subquery.yaml | 2 +- .../tests/testdata/sysinfo_funcs.yaml | 2 +- .../tests/testdata/temporal_filter.yaml | 22 +- .../tests/testdata/time_window.yaml | 12 +- .../planner_test/tests/testdata/tpch.yaml | 582 +++++++++--------- .../planner_test/tests/testdata/update.yaml | 2 +- .../tests/testdata/watermark.yaml | 2 +- 28 files changed, 815 insertions(+), 805 deletions(-) diff --git a/e2e_test/source/basic/kafka_batch.slt b/e2e_test/source/basic/kafka_batch.slt index 20aa4ff74d8f4..85e18bfad33e4 100644 --- a/e2e_test/source/basic/kafka_batch.slt +++ b/e2e_test/source/basic/kafka_batch.slt @@ -159,8 +159,10 @@ select count(*) from s5 ---- 100 -statement ok +statement stream err drop table s5 +--- + query I select count(*) from s6 diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index b020868b813da..6418275aebe3e 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -20,18 +20,18 @@ select v1, min(v2) + max(v3) * count(v1) as agg from t group by v1; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr23] } + └─BatchProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr25] } └─BatchHashAgg { group_key: [t.v1], aggs: [min(t.v2), max(t.v3), count(t.v1)] } └─BatchExchange { order: [], dist: HashShard(t.v1) } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } batch_local_plan: | - BatchProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr45] } + BatchProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr49] } └─BatchHashAgg { group_key: [t.v1], aggs: [min(t.v2), max(t.v3), count(t.v1)] } └─BatchExchange { order: [], dist: Single } └─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 $expr69] } + └─StreamProject { exprs: [t.v1, (min(t.v2) + (max(t.v3) * count(t.v1))) as $expr75] } └─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) } @@ -39,23 +39,23 @@ create table t(v1 int, v2 int, v3 int); select min(v1) + max(v2) * count(v3) as agg from t; batch_plan: | - BatchProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr23] } + BatchProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr25] } └─BatchSimpleAgg { aggs: [min(min(t.v1)), max(max(t.v2)), sum0(count(t.v3))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1), max(t.v2), count(t.v3)] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } batch_local_plan: | - BatchProject { exprs: [(min(t.v1) + (max(t.v2) * count(t.v3))) as $expr45] } + BatchProject { exprs: [(min(t.v1) + (max(t.v2) * count(t.v3))) as $expr49] } └─BatchSimpleAgg { aggs: [min(t.v1), max(t.v2), count(t.v3)] } └─BatchExchange { order: [], dist: Single } └─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 $expr70] } + └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v2)) * sum0(count(t.v3)))) as $expr78] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v2)), sum0(count(t.v3))] } └─StreamExchange { dist: Single } - └─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] } + └─StreamHashAgg { group_key: [$expr77], 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 $expr77] } └─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); @@ -70,23 +70,23 @@ select v3, min(v1) * avg(v1+v2) as agg from t group by v3; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.v3, (min(t.v1) * (sum($expr45)::Decimal / count($expr45))) as $expr46] } - └─BatchHashAgg { group_key: [t.v3], aggs: [min(t.v1), sum($expr45), count($expr45)] } + └─BatchProject { exprs: [t.v3, (min(t.v1) * (sum($expr49)::Decimal / count($expr49))) as $expr50] } + └─BatchHashAgg { group_key: [t.v3], aggs: [min(t.v1), sum($expr49), count($expr49)] } └─BatchExchange { order: [], dist: HashShard(t.v3) } - └─BatchProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr45] } + └─BatchProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr49] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } batch_local_plan: | - BatchProject { exprs: [t.v3, (min(t.v1) * (sum($expr89)::Decimal / count($expr89))) as $expr90] } - └─BatchHashAgg { group_key: [t.v3], aggs: [min(t.v1), sum($expr89), count($expr89)] } + BatchProject { exprs: [t.v3, (min(t.v1) * (sum($expr97)::Decimal / count($expr97))) as $expr98] } + └─BatchHashAgg { group_key: [t.v3], aggs: [min(t.v1), sum($expr97), count($expr97)] } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr89] } + └─BatchProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr97] } └─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($expr135)::Decimal / count($expr135))) as $expr137] } - └─StreamHashAgg { group_key: [t.v3], aggs: [count, min(t.v1), sum($expr135), count($expr135)] } + └─StreamProject { exprs: [t.v3, (min(t.v1) * (sum($expr148)::Decimal / count($expr148))) as $expr149] } + └─StreamHashAgg { group_key: [t.v3], aggs: [count, min(t.v1), sum($expr148), count($expr148)] } └─StreamExchange { dist: HashShard(t.v3) } - └─StreamProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr135, t._row_id] } + └─StreamProject { exprs: [t.v3, t.v1, (t.v1 + t.v2) as $expr148, 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: | @@ -139,40 +139,40 @@ create table t(v1 int, v2 int); select count(v1 + v2) as cnt, sum(v1 + v2) as sum from t; batch_plan: | - BatchSimpleAgg { aggs: [sum0(count($expr23)), sum(sum($expr23))] } + BatchSimpleAgg { aggs: [sum0(count($expr25)), sum(sum($expr25))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [count($expr23), sum($expr23)] } - └─BatchProject { exprs: [(t.v1 + t.v2) as $expr23] } + └─BatchSimpleAgg { aggs: [count($expr25), sum($expr25)] } + └─BatchProject { exprs: [(t.v1 + t.v2) as $expr25] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } batch_local_plan: | - BatchSimpleAgg { aggs: [count($expr45), sum($expr45)] } + BatchSimpleAgg { aggs: [count($expr49), sum($expr49)] } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [(t.v1 + t.v2) as $expr45] } + └─BatchProject { exprs: [(t.v1 + t.v2) as $expr49] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [cnt, sum], pk_columns: [] } - └─StreamProject { exprs: [sum0(count($expr68)), sum(sum($expr68))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum0(count($expr68)), sum(sum($expr68))] } + └─StreamProject { exprs: [sum0(count($expr74)), sum(sum($expr74))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum0(count($expr74)), sum(sum($expr74))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, count($expr68), sum($expr68)] } - └─StreamProject { exprs: [(t.v1 + t.v2) as $expr68, t._row_id] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, count($expr74), sum($expr74)] } + └─StreamProject { exprs: [(t.v1 + t.v2) as $expr74, 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); select v1, sum(v2 + v3) / count(v2 + v3) + max(v1) as agg from t group by v1; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.v1, ((sum($expr45) / count($expr45)) + max(t.v1)) as $expr46] } - └─BatchHashAgg { group_key: [t.v1], aggs: [sum($expr45), count($expr45), max(t.v1)] } + └─BatchProject { exprs: [t.v1, ((sum($expr49) / count($expr49)) + max(t.v1)) as $expr50] } + └─BatchHashAgg { group_key: [t.v1], aggs: [sum($expr49), count($expr49), max(t.v1)] } └─BatchExchange { order: [], dist: HashShard(t.v1) } - └─BatchProject { exprs: [t.v1, (t.v2 + t.v3) as $expr45] } + └─BatchProject { exprs: [t.v1, (t.v2 + t.v3) as $expr49] } └─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($expr91) / count($expr91)) + max(t.v1)) as $expr93] } - └─StreamHashAgg { group_key: [t.v1], aggs: [count, sum($expr91), count($expr91), max(t.v1)] } + └─StreamProject { exprs: [t.v1, ((sum($expr100) / count($expr100)) + max(t.v1)) as $expr101] } + └─StreamHashAgg { group_key: [t.v1], aggs: [count, sum($expr100), count($expr100), max(t.v1)] } └─StreamExchange { dist: HashShard(t.v1) } - └─StreamProject { exprs: [t.v1, (t.v2 + t.v3) as $expr91, t._row_id] } + └─StreamProject { exprs: [t.v1, (t.v2 + t.v3) as $expr100, 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); @@ -377,11 +377,11 @@ └─LogicalProject { exprs: [(t.v2 + t.v3) as $expr1, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } batch_plan: | - BatchProject { exprs: [$expr23] } + BatchProject { exprs: [$expr25] } └─BatchExchange { order: [t.v1 ASC], dist: Single } └─BatchGroupTopN { order: "[t.v1 ASC]", limit: 1, offset: 0, group_key: [1] } └─BatchExchange { order: [], dist: HashShard(t.v1) } - └─BatchProject { exprs: [(t.v2 + t.v3) as $expr23, t.v1] } + └─BatchProject { exprs: [(t.v2 + t.v3) as $expr25, t.v1] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } - name: arguments out-of-order sql: | @@ -414,18 +414,18 @@ └─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 $expr41] } + BatchProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr43] } └─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 $expr66] } + └─StreamProject { exprs: [(min(min(t.v1)) + (max(max(t.v3)) * sum0(count(t.v2)))) as $expr72] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), max(max(t.v3)), sum0(count(t.v2))] } └─StreamExchange { dist: Single } - └─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] } + └─StreamHashAgg { group_key: [$expr71], 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 $expr71] } └─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: | @@ -611,12 +611,12 @@ └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | StreamMaterialize { columns: [sab], pk_columns: [] } - └─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))))] } + └─StreamProject { exprs: [max(max($expr46) 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($expr46) 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: [$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] } + └─StreamHashAgg { group_key: [$expr47], aggs: [count, max($expr46) 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, $expr46, t._row_id, Vnode(t._row_id) as $expr47] } + └─StreamProject { exprs: [t.a, t.b, (t.a * t.b) as $expr46, 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: | @@ -633,7 +633,7 @@ └─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 $expr45, 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] } └─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) } @@ -715,8 +715,8 @@ └─StreamProject { exprs: [min(min(t.v3)), sum(sum(t.v1)), t.v1, t.v3, t.v2] } └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2], aggs: [count, min(min(t.v3)), sum(sum(t.v1))] } └─StreamExchange { dist: HashShard(t.v1, t.v3, t.v2) } - └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2, $expr1], aggs: [count, min(t.v3), sum(t.v1)] } - └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2, $expr3], aggs: [count, min(t.v3), sum(t.v1)] } + └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr3] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: enable two phase aggregation sql: | @@ -734,8 +734,8 @@ └─StreamProject { exprs: [min(min(t.v1)), sum(sum(t.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), sum(sum(t.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, min(t.v1), sum(t.v2)] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamHashAgg { group_key: [$expr3], aggs: [count, min(t.v1), sum(t.v2)] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr3] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: disable two phase aggregation sql: | @@ -860,11 +860,11 @@ └─LogicalScan { table: t, columns: [t.a, t.b] } stream_plan: | StreamMaterialize { columns: [s1], pk_columns: [] } - └─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)))] } + └─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)))] } └─StreamExchange { dist: Single } - └─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] } + └─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] } └─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); @@ -1083,23 +1083,23 @@ └─LogicalProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } batch_plan: | - BatchProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Decimal::Float64, Pow(((sum(sum($expr67))::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 $expr68, Pow(((sum(sum($expr67))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr69] } - └─BatchSimpleAgg { aggs: [sum(sum($expr67)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr67)), sum(sum(t.v1)), sum0(count(t.v1))] } + BatchProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Float64, Pow(((sum(sum($expr73))::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 $expr74, Pow(((sum(sum($expr73))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr75] } + └─BatchSimpleAgg { aggs: [sum(sum($expr73)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr73)), sum(sum(t.v1)), sum0(count(t.v1))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum($expr67), sum(t.v1), count(t.v1), sum($expr67), sum(t.v1), count(t.v1)] } - └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr67] } + └─BatchSimpleAgg { aggs: [sum($expr73), sum(t.v1), count(t.v1), sum($expr73), sum(t.v1), count(t.v1)] } + └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr73] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } batch_local_plan: | - BatchProject { exprs: [Case((count(t.v1) <= 1:Int64), null:Decimal::Float64, Pow(((sum($expr133)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / (count(t.v1) - 1:Int64))::Float64, 0.5:Float64)) as $expr134, Pow(((sum($expr133)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / count(t.v1))::Float64, 0.5:Float64) as $expr135] } - └─BatchSimpleAgg { aggs: [sum($expr133), sum(t.v1), count(t.v1), sum($expr133), sum(t.v1), count(t.v1)] } + BatchProject { exprs: [Case((count(t.v1) <= 1:Int64), null:Float64, Pow(((sum($expr145)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / (count(t.v1) - 1:Int64))::Float64, 0.5:Float64)) as $expr146, Pow(((sum($expr145)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / count(t.v1))::Float64, 0.5:Float64) as $expr147] } + └─BatchSimpleAgg { aggs: [sum($expr145), sum(t.v1), count(t.v1), sum($expr145), sum(t.v1), count(t.v1)] } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr133] } + └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr145] } └─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($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))] } + └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Float64, Pow(((sum(sum($expr222))::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 $expr223, Pow(((sum(sum($expr222))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr224] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr222)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr222)), sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamExchange { dist: Single } - └─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] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr222), sum(t.v1), count(t.v1), sum($expr222), sum(t.v1), count(t.v1)] } + └─StreamProject { exprs: [t.v1, (t.v1 * t.v1) as $expr222, 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/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index b285506cdb479..b1c8ea5080bb1 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -19,7 +19,7 @@ └─LogicalScan { table: t, columns: [t.v1, t._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Array(1:Int32, t.v1) as $expr23] } + └─BatchProject { exprs: [Array(1:Int32, t.v1) as $expr25] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | select ARRAY[null]; @@ -52,21 +52,21 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayCat(Array(66:Int32), Array(123:Int32))]] } + BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | select array_cat(array[array[66]], array[233]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayCat(Array(Array(66:Int32)), Array(233:Int32))]] } + BatchValues { rows: [[ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }]] } - sql: | select array_cat(array[233], array[array[66]]); logical_plan: | LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32))) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayCat(Array(233:Int32), Array(Array(66:Int32)))]] } + BatchValues { rows: [[ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }]] } - sql: | select array_cat(array[233], array[array[array[66]]]); binder_error: 'Bind error: unable to find least restrictive type between integer[] and integer[][][]' @@ -82,7 +82,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayAppend(Array(66:Int32), 123:Int32)]] } + BatchValues { rows: [[ARRAY[66, 123]:List { datatype: Int32 }]] } - sql: | select array_append(123, 234); binder_error: 'Bind error: Cannot append integer to integer' @@ -98,7 +98,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32)) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchValues { rows: [[ArrayPrepend(123:Int32, Array(66:Int32))]] } + BatchValues { rows: [[ARRAY[123, 66]:List { datatype: Int32 }]] } - sql: | select array_prepend(123, 234); binder_error: 'Bind error: Cannot prepend integer to integer' diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index 83c69c2134794..6dbb89aecc8db 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchValues { rows: [[Case(null:Boolean, 1:Int32)]] } + BatchValues { rows: [[null:Int32]] } - name: implicit cast boolean (JOIN ON NULL) sql: | create table t1(v1 int); @@ -60,7 +60,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchValues { rows: [[Case(true:Boolean, 1:Int32)]] } + BatchValues { rows: [[1:Int32]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | create table t1(v1 int); diff --git a/src/frontend/planner_test/tests/testdata/delete.yaml b/src/frontend/planner_test/tests/testdata/delete.yaml index d86b07028f370..cfd4c58f43265 100644 --- a/src/frontend/planner_test/tests/testdata/delete.yaml +++ b/src/frontend/planner_test/tests/testdata/delete.yaml @@ -9,7 +9,7 @@ └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.a, t.b, t.a, (t.a + t.b) as $expr23] } + └─BatchProject { exprs: [t.a, t.b, t.a, (t.a + t.b) as $expr25] } └─BatchDelete { table: t, returning: true } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t, columns: [t.a, t.b, t._row_id], distribution: 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 b361a6263eec4..affee3134df45 100644 --- a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml @@ -21,8 +21,8 @@ └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } + └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr3] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: | With Top-1 on inner side @@ -58,15 +58,15 @@ stream_plan: | StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.v1, 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] } + └─StreamDynamicFilter { predicate: ($expr25 > max(max(t2.v2))), output: [t1.v1, $expr25, t1._row_id] } + ├─StreamProject { exprs: [t1.v1, (t1.v1 + t1.v1) 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: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr22], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr22] } + └─StreamHashAgg { group_key: [$expr26], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr26] } └─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: @@ -117,8 +117,8 @@ └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } + └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr3] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Dynamic filter join on unequal types sql: | @@ -134,15 +134,15 @@ stream_plan: | StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.v1, 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] } + └─StreamDynamicFilter { predicate: ($expr25 > max(max(t2.v2))), output: [t1.v1, $expr25, t1._row_id] } + ├─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: Broadcast } └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr22], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr22] } + └─StreamHashAgg { group_key: [$expr26], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr26] } └─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: | @@ -157,8 +157,8 @@ └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } + └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr3] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Complex expression on RHS of condition will still result in dynamic filter before: @@ -173,12 +173,12 @@ └─LogicalScan { table: t2, columns: [t2.v2] } stream_plan: | StreamMaterialize { columns: [v1, t1._row_id(hidden)], pk_columns: [t1._row_id] } - └─StreamDynamicFilter { predicate: (t1.v1 > $expr23), output: [t1.v1, t1._row_id] } + └─StreamDynamicFilter { predicate: (t1.v1 > $expr27), 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 $expr23] } + └─StreamProject { exprs: [(2:Int32 * max(max(t2.v2))) as $expr27] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr21], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr21] } + └─StreamHashAgg { group_key: [$expr26], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr26] } └─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/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index f71b9299f9bb9..12665fed66d9e 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -42,6 +42,14 @@ LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } + Const eval exprs: + + BatchValues { rows: [[1:Int32]] } + + Const eval exprs: + + BatchValues { rows: [[1:Int32]] } + To Batch Physical Plan: BatchValues { rows: [[1:Int32]] } @@ -55,7 +63,7 @@ "stages": { "0": { "root": { - "plan_node_id": 22, + "plan_node_id": 26, "plan_node_type": "BatchValues", "schema": [ { diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index d4274af677bdb..0e6c2ac933ee0 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -29,7 +29,7 @@ select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean)))))) as $expr23] } + └─BatchProject { exprs: [false:Boolean] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,12 +65,12 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchValues { rows: [[In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)]] } + BatchValues { rows: [[true:Boolean]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchValues { rows: [[Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))]] } + BatchValues { rows: [[false:Boolean]] } - name: in-list with misaligned types sql: | SELECT true in (3, 1.0, 2); @@ -80,7 +80,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1)))) as $expr23] } + BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1)))) as $expr25] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -161,12 +161,12 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchValues { rows: [[((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))]] } + BatchValues { rows: [[4:Int32]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar) as $expr23] } - └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } + BatchProject { exprs: [0:Int32] } + └─BatchFilter { predicate: false:Boolean } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -174,11 +174,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal) as $expr23] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal) as $expr25] } └─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 $expr46, t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal) as $expr50, 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: | @@ -186,7 +186,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal) as $expr23] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal) as $expr25] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -194,7 +194,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal) as $expr23] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal) as $expr25] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -213,11 +213,11 @@ select nullif(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), null:Int32, t.v1) as $expr23] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), null:Int32, t.v1) as $expr25] } └─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 $expr46, t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), null:Int32, t.v1) as $expr50, 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); @@ -234,18 +234,18 @@ select coalesce(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Coalesce(t.v1, 1:Int32) as $expr23] } + └─BatchProject { exprs: [Coalesce(t.v1, 1:Int32) as $expr25] } └─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 $expr46, t._row_id] } + └─StreamProject { exprs: [Coalesce(t.v1, 1:Int32) as $expr50, 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); select coalesce(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Coalesce(t.v1::Decimal, 1.2:Decimal) as $expr23] } + └─BatchProject { exprs: [Coalesce(t.v1::Decimal, 1.2:Decimal) as $expr25] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -260,18 +260,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar) as $expr23] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar) as $expr25] } └─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 $expr46, t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar) as $expr50, 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); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar) as $expr23] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar) as $expr25] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -286,18 +286,18 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar) as $expr23] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar) as $expr25] } └─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 $expr46, t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar) as $expr50, 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); select concat(v1) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1::Varchar) as $expr23] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1::Varchar) as $expr25] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -306,19 +306,19 @@ - sql: | select concat(':', true); batch_plan: | - BatchValues { rows: [[ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))]] } + BatchValues { rows: [[':t':Varchar]] } - sql: | select ':' || true; batch_plan: | - BatchValues { rows: [[ConcatOp(':':Varchar, true:Boolean::Varchar)]] } + BatchValues { rows: [[':true':Varchar]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchValues { rows: [[Substr('hello':Varchar, null:Int32)]] } + BatchValues { rows: [[null:Varchar]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchValues { rows: [[Substr(null:Varchar, 1:Int32)]] } + BatchValues { rows: [[null:Varchar]] } - sql: | select pg_typeof('123'); batch_plan: | @@ -406,10 +406,10 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr33, Array(2:Int32)))) as $expr40] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr33, ARRAY[2]:List { datatype: Int32 }))) as $expr42] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchValues { rows: [[Array(1:Int32)]] } + └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } - sql: | select 1 < ALL(array[null]::integer[]); logical_plan: | @@ -429,10 +429,10 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr33, Array(2:Int32)))) as $expr40] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr33, ARRAY[2]:List { datatype: Int32 }))) as $expr42] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchValues { rows: [[Array(1:Int32)]] } + └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } - name: now expression sql: | create table t (v1 timestamp with time zone); @@ -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 >= $expr12), output: [t.v1, t._row_id] } + └─StreamDynamicFilter { predicate: (t.v1 >= $expr14), 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 $expr12], watermark_columns: [(now - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr14], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval)] } └─StreamNow { output: [now] } - name: and of two now expression condition sql: | @@ -525,14 +525,14 @@ sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) batch_plan: | - BatchValues { rows: [[(false:Boolean >= Not(In('LN1O0QP1yi':Varchar, Md5('4SeUPZhUbH':Varchar))))]] } + BatchValues { rows: [[false:Boolean]] } - name: const_eval of const expr sql: | create table t(v1 int); select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1) as $expr23] } + └─BatchProject { exprs: [(3:Int32 + t.v1) as $expr25] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index c20174089d5d4..1761a4bb32ea3 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -618,5 +618,5 @@ └─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] } + └─StreamProject { exprs: [t1.a, Vnode(t1.a) as $expr3] } └─StreamTableScan { table: t1, columns: [t1.a], pk: [t1.a], dist: UpstreamHashShard(t1.a) } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index f043a9de04d6b..db20c94fbffa2 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } + └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } - name: prohibit inserting different number of nulls per row sql: | create table t (v1 real, v2 int, v3 varchar); @@ -27,7 +27,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[1:Int32::Float32, 2:Int32, null:Varchar]] } + └─BatchValues { rows: [[1:Float32, 2:Int32, null:Varchar]] } - name: insert values with implicit null (multiple rows) sql: | create table t (v1 real, v2 int, v3 varchar); @@ -35,7 +35,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[1:Int32::Float32, 2:Int32, null:Varchar], [3:Int32::Float32, 4:Int32, null:Varchar]] } + └─BatchValues { rows: [[1:Float32, 2:Int32, null:Varchar], [3:Float32, 4:Int32, null:Varchar]] } - name: implicit null user defined columns 1 sql: | create table t (v1 int, v2 int); @@ -171,7 +171,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time as $expr47, 11:Int32, 4.5:Decimal::Float32 as $expr48] } + └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | @@ -208,7 +208,7 @@ └─LogicalValues { rows: [[0:Int32, 1:Int32], [1:Int32, 2:Int32]], schema: Schema { fields: [*VALUES*_0.column_0:Int32, *VALUES*_0.column_1:Int32] } } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_0, (*VALUES*_0.column_0 + *VALUES*_0.column_1) as $expr23] } + └─BatchProject { exprs: [*VALUES*_0.column_0, *VALUES*_0.column_1, *VALUES*_0.column_0, (*VALUES*_0.column_0 + *VALUES*_0.column_1) as $expr25] } └─BatchInsert { table: t, returning: true } └─BatchValues { rows: [[0:Int32, 1:Int32], [1:Int32, 2:Int32]] } - name: insert with select and returning @@ -223,7 +223,7 @@ └─LogicalScan { table: t, columns: [t.a, t.b, t._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.a, t.b, t.a, (t.a + t.b) as $expr23] } + └─BatchProject { exprs: [t.a, t.b, t.a, (t.a + t.b) as $expr25] } └─BatchInsert { table: t, returning: true } └─BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [t.a, t.b], aggs: [] } diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index 28b470b1e124e..4de014616461e 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -186,7 +186,7 @@ using (x); batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Coalesce(i.x, i.x) as $expr23] } + └─BatchProject { exprs: [Coalesce(i.x, i.x) as $expr25] } └─BatchHashJoin { type: FullOuter, predicate: i.x = i.x, output: all } ├─BatchExchange { order: [], dist: HashShard(i.x) } | └─BatchLookupJoin { type: Inner, predicate: i.x = i.x, output: [i.x] } @@ -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 $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] } + └─StreamProject { exprs: [Coalesce(i.x, i.x) as $expr50, 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) } @@ -488,10 +488,10 @@ ├─LogicalScan { table: a, columns: [a.x, a._row_id] } └─LogicalScan { table: b, columns: [b.x, b._row_id] } batch_plan: | - BatchProject { exprs: [$expr67, $expr68] } - └─BatchExchange { order: [$expr69 ASC], dist: Single } - └─BatchSort { order: [$expr69 ASC] } - └─BatchProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr67, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr68, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr69] } + BatchProject { exprs: [$expr73, $expr74] } + └─BatchExchange { order: [$expr75 ASC], dist: Single } + └─BatchSort { order: [$expr75 ASC] } + └─BatchProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr73, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr74, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr75] } └─BatchFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) } └─BatchHashJoin { type: FullOuter, predicate: a.x = b.x, output: all } ├─BatchExchange { order: [], dist: HashShard(a.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, $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] } + StreamMaterialize { columns: [y, z, $expr150(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: [$expr150, 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 $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] } + └─StreamProject { exprs: [(2:Int32 * Coalesce(a.x, b.x)) as $expr148, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr149, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr150, 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) } @@ -593,17 +593,17 @@ select * from t1 join t2 on v1 IS NOT DISTINCT FROM v2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: $expr13 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2] } - ├─BatchExchange { order: [], dist: HashShard($expr13) } - | └─BatchProject { exprs: [t1.v1, t1.v1::Int64 as $expr13] } + └─BatchHashJoin { type: Inner, predicate: $expr15 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2] } + ├─BatchExchange { order: [], dist: HashShard($expr15) } + | └─BatchProject { exprs: [t1.v1, t1.v1::Int64 as $expr15] } | └─BatchScan { table: t1, columns: [t1.v1], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(t2.v2) } └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: | - 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] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr29(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr29, v2] } + └─StreamHashJoin { type: Inner, predicate: $expr29 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr29, t2._row_id] } + ├─StreamExchange { dist: HashShard($expr29) } + | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr29, 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 30fb7e5a15717..b6efceca8034c 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 $expr24, t._row_id] } + └─StreamProject { exprs: [IsNull(t.a) as $expr26, 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 $expr24, t._row_id] } + └─StreamProject { exprs: [t.a, IsTrue(t.a) as $expr26, 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: | @@ -57,6 +57,6 @@ └─StreamProject { exprs: [sum0(count), max(max(t.a))] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), max(max(t.a))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, count, max(t.a)] } - └─StreamProject { exprs: [t.a, t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamHashAgg { group_key: [$expr3], aggs: [count, count, max(t.a)] } + └─StreamProject { exprs: [t.a, t._row_id, Vnode(t._row_id) as $expr3] } └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index af7336166d6ba..0411203bc3128 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -66,17 +66,17 @@ FROM bid; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr23, bid.date_time] } + └─BatchProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr25, bid.date_time] } └─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 $expr46, bid.date_time, bid._row_id] } + └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr50, 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 $expr46, bid.date_time, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr50, 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 @@ -174,7 +174,7 @@ GROUP BY Q.category; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr23] } + └─BatchProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr25] } └─BatchHashAgg { group_key: [auction.category], aggs: [sum(max(bid.price)), count(max(bid.price))] } └─BatchExchange { order: [], dist: HashShard(auction.category) } └─BatchHashAgg { group_key: [auction.id, auction.category], aggs: [max(bid.price)] } @@ -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 $expr47] } + └─StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr51] } └─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 $expr47] } + StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr51] } 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 @@ -397,35 +397,35 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time] } - └─BatchFilter { predicate: (bid.date_time >= $expr50) AND (bid.date_time <= $expr49) } + └─BatchFilter { predicate: (bid.date_time >= $expr54) AND (bid.date_time <= $expr53) } └─BatchHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } ├─BatchExchange { order: [], dist: HashShard(bid.price) } | └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.date_time], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(max(bid.price)) } - └─BatchProject { exprs: [max(bid.price), $expr49, ($expr49 - '00:00:10':Interval) as $expr50] } - └─BatchHashAgg { group_key: [$expr49], aggs: [max(bid.price)] } - └─BatchExchange { order: [], dist: HashShard($expr49) } - └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr49, bid.price] } + └─BatchProject { exprs: [max(bid.price), $expr53, ($expr53 - '00:00:10':Interval) as $expr54] } + └─BatchHashAgg { group_key: [$expr53], aggs: [max(bid.price)] } + └─BatchExchange { order: [], dist: HashShard($expr53) } + └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr53, 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), $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) } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr108(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr108, price, max(bid.price)] } + └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr108, max(bid.price)] } + └─StreamFilter { predicate: (bid.date_time >= $expr109) AND (bid.date_time <= $expr108) } └─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), $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] } + └─StreamProject { exprs: [max(bid.price), $expr108, ($expr108 - '00:00:10':Interval) as $expr109] } + └─StreamAppendOnlyHashAgg { group_key: [$expr108], aggs: [count, max(bid.price)] } + └─StreamExchange { dist: HashShard($expr108) } + └─StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr108, 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), $expr99(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr99, price, max(bid.price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr108(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr108, price, max(bid.price)] } materialized table: 4294967294 - 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) } + StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr108, max(bid.price)] } + StreamFilter { predicate: (bid.date_time >= $expr109) AND (bid.date_time <= $expr108) } 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), $expr99, ($expr99 - '00:00:10':Interval) as $expr101] } - StreamAppendOnlyHashAgg { group_key: [$expr99], aggs: [count, max(bid.price)] } + StreamProject { exprs: [max(bid.price), $expr108, ($expr108 - '00:00:10':Interval) as $expr109] } + StreamAppendOnlyHashAgg { group_key: [$expr108], 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 $expr99, bid.price, bid._row_id] } + StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr108, 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), $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] } + Table 2 { columns: [max(bid_price), $expr108, $expr109], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(bid_price), $expr108, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 4 { columns: [$expr108, 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, $expr108, 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 @@ -492,62 +492,62 @@ AND P.endtime = A.endtime; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr95 = $expr93 AND $expr96 = $expr94, output: [person.id, person.name, $expr95] } - ├─BatchExchange { order: [], dist: HashShard(person.id, $expr95, $expr96) } - | └─BatchHashAgg { group_key: [person.id, person.name, $expr95, $expr96], aggs: [] } - | └─BatchProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr95, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr96] } + └─BatchHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr103 = $expr101 AND $expr104 = $expr102, output: [person.id, person.name, $expr103] } + ├─BatchExchange { order: [], dist: HashShard(person.id, $expr103, $expr104) } + | └─BatchHashAgg { group_key: [person.id, person.name, $expr103, $expr104], aggs: [] } + | └─BatchProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr103, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr104] } | └─BatchScan { table: person, columns: [person.id, person.name, person.date_time], distribution: UpstreamHashShard(person.id) } - └─BatchHashAgg { group_key: [auction.seller, $expr93, $expr94], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(auction.seller, $expr93, $expr94) } - └─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] } + └─BatchHashAgg { group_key: [auction.seller, $expr101, $expr102], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(auction.seller, $expr101, $expr102) } + └─BatchProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr101, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr102] } └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: | - 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] } + StreamMaterialize { columns: [id, name, starttime, $expr206(hidden), auction.seller(hidden), $expr207(hidden), $expr208(hidden)], pk_columns: [id, name, starttime, $expr206, auction.seller, $expr207, $expr208] } + └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr205 = $expr207 AND $expr206 = $expr208, output: all } + ├─StreamExchange { dist: HashShard(person.id, $expr205, $expr206) } + | └─StreamProject { exprs: [person.id, person.name, $expr205, $expr206] } + | └─StreamHashAgg { group_key: [person.id, person.name, $expr205, $expr206], aggs: [count] } + | └─StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr205, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr206] } | └─StreamTableScan { table: person, columns: [person.id, person.name, person.date_time], pk: [person.id], dist: UpstreamHashShard(person.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] } + └─StreamProject { exprs: [auction.seller, $expr207, $expr208] } + └─StreamHashAgg { group_key: [auction.seller, $expr207, $expr208], aggs: [count] } + └─StreamExchange { dist: HashShard(auction.seller, $expr207, $expr208) } + └─StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr207, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr208, 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, $expr192(hidden), auction.seller(hidden), $expr189(hidden), $expr190(hidden)], pk_columns: [id, name, starttime, $expr192, auction.seller, $expr189, $expr190] } + StreamMaterialize { columns: [id, name, starttime, $expr206(hidden), auction.seller(hidden), $expr207(hidden), $expr208(hidden)], pk_columns: [id, name, starttime, $expr206, auction.seller, $expr207, $expr208] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr191 = $expr189 AND $expr192 = $expr190, output: all } + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr205 = $expr207 AND $expr206 = $expr208, 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, $expr189, $expr190] } - StreamHashAgg { group_key: [auction.seller, $expr189, $expr190], aggs: [count] } + StreamProject { exprs: [auction.seller, $expr207, $expr208] } + StreamHashAgg { group_key: [auction.seller, $expr207, $expr208], aggs: [count] } result table: 5, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - 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, $expr205, $expr206] } + StreamHashAgg { group_key: [person.id, person.name, $expr205, $expr206], aggs: [count] } result table: 4, state tables: [] - 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] } + StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr205, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr206] } 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 $expr189, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr190, auction.id] } + StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr207, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr208, 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, $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] } + Table 0 { columns: [person_id, person_name, $expr205, $expr206], 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, $expr205, $expr206, 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, $expr207, $expr208], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [auction_seller, $expr207, $expr208, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [person_id, person_name, $expr205, $expr206, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 5 { columns: [auction_seller, $expr207, $expr208, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4294967294 { columns: [id, name, starttime, $expr206, auction.seller, $expr207, $expr208], 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 @@ -633,17 +633,17 @@ SELECT auction, bidder, price, date_time, TO_CHAR(date_time, 'YYYY-MM-DD') as date, TO_CHAR(date_time, 'HH:MI') as time FROM bid; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr45, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr46] } + └─BatchProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr49, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr50] } └─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 $expr91, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr92, bid._row_id] } + └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr99, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr100, 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 $expr91, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr92, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr99, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr100, 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 @@ -720,19 +720,19 @@ WHERE 0.908 * price > 1000000 AND 0.908 * price < 50000000; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr45, 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 $expr46, bid.date_time, bid.extra] } + └─BatchProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr49, 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 $expr50, bid.date_time, bid.extra] } └─BatchFilter { predicate: ((0.908:Decimal * bid.price) > 1000000:Int32) AND ((0.908:Decimal * bid.price) < 50000000:Int32) } └─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 $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] } + └─StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr99, 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 $expr100, 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 $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] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr99, 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 $expr100, 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 @@ -761,49 +761,49 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [$expr23], aggs: [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))] } - └─BatchExchange { order: [], dist: HashShard($expr23) } - └─BatchProject { exprs: [$expr23, 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))] } - └─BatchHashAgg { group_key: [$expr23, bid.bidder, bid.auction, flag], aggs: [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))] } - └─BatchExchange { order: [], dist: HashShard($expr23, bid.bidder, bid.auction, flag) } - └─BatchExpand { column_subsets: [[$expr23], [$expr23, bid.bidder], [$expr23, bid.auction]] } - └─BatchProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr23, bid.price, bid.bidder, bid.auction] } + └─BatchHashAgg { group_key: [$expr25], aggs: [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))] } + └─BatchExchange { order: [], dist: HashShard($expr25) } + └─BatchProject { exprs: [$expr25, 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))] } + └─BatchHashAgg { group_key: [$expr25, bid.bidder, bid.auction, flag], aggs: [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))] } + └─BatchExchange { order: [], dist: HashShard($expr25, bid.bidder, bid.auction, flag) } + └─BatchExpand { column_subsets: [[$expr25], [$expr25, bid.bidder], [$expr25, bid.auction]] } + └─BatchProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr25, bid.price, bid.bidder, bid.auction] } └─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: [$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] } + └─StreamProject { exprs: [$expr50, 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: [$expr50], 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($expr50) } + └─StreamProject { exprs: [$expr50, 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: [$expr50, 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($expr50, bid.bidder, bid.auction, flag) } + └─StreamExpand { column_subsets: [[$expr50], [$expr50, bid.bidder], [$expr50, bid.auction]] } + └─StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr50, 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: [$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))] } + StreamProject { exprs: [$expr50, 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: [$expr50], 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: [$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))] } + StreamProject { exprs: [$expr50, 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: [$expr50, 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: [[$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] } + StreamExpand { column_subsets: [[$expr50], [$expr50, bid.bidder], [$expr50, bid.auction]] } + StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr50, 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: [$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 0 { columns: [$expr50, 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: [$expr50, 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: @@ -829,50 +829,50 @@ GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [bid.channel, $expr45], aggs: [max(max($expr46)) 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))] } - └─BatchExchange { order: [], dist: HashShard(bid.channel, $expr45) } - └─BatchProject { exprs: [bid.channel, $expr45, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr46), 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))] } - └─BatchHashAgg { group_key: [bid.channel, $expr45, bid.bidder, bid.auction, flag], aggs: [max($expr46), 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))] } - └─BatchExchange { order: [], dist: HashShard(bid.channel, $expr45, bid.bidder, bid.auction, flag) } - └─BatchExpand { column_subsets: [[bid.channel, $expr45, $expr46], [bid.channel, $expr45, bid.bidder], [bid.channel, $expr45, bid.auction]] } - └─BatchProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr45, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr46, bid.price, bid.bidder, bid.auction] } + └─BatchHashAgg { group_key: [bid.channel, $expr49], aggs: [max(max($expr50)) 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))] } + └─BatchExchange { order: [], dist: HashShard(bid.channel, $expr49) } + └─BatchProject { exprs: [bid.channel, $expr49, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr50), 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))] } + └─BatchHashAgg { group_key: [bid.channel, $expr49, bid.bidder, bid.auction, flag], aggs: [max($expr50), 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))] } + └─BatchExchange { order: [], dist: HashShard(bid.channel, $expr49, bid.bidder, bid.auction, flag) } + └─BatchExpand { column_subsets: [[bid.channel, $expr49, $expr50], [bid.channel, $expr49, bid.bidder], [bid.channel, $expr49, bid.auction]] } + └─BatchProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr49, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr50, bid.price, bid.bidder, bid.auction] } └─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, $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] } + └─StreamProject { exprs: [bid.channel, $expr99, max(max($expr100)) 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, $expr99], aggs: [count, max(max($expr100)) 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, $expr99) } + └─StreamProject { exprs: [bid.channel, $expr99, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr100), 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, $expr99, bid.bidder, bid.auction, flag], aggs: [count, max($expr100), 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, $expr99, bid.bidder, bid.auction, flag) } + └─StreamExpand { column_subsets: [[bid.channel, $expr99, $expr100], [bid.channel, $expr99, bid.bidder], [bid.channel, $expr99, bid.auction]] } + └─StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr99, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr100, 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, $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))] } + StreamProject { exprs: [bid.channel, $expr99, max(max($expr100)) 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, $expr99], aggs: [count, max(max($expr100)) 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, $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))] } + StreamProject { exprs: [bid.channel, $expr99, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr100), 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, $expr99, bid.bidder, bid.auction, flag], aggs: [count, max($expr100), 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, $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] } + StreamExpand { column_subsets: [[bid.channel, $expr99, $expr100], [bid.channel, $expr99, bid.bidder], [bid.channel, $expr99, bid.auction]] } + StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr99, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr100, 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, $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 0 { columns: [bid_channel, $expr99, max($expr100), 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, $expr99, count, max(max($expr100)) 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, $expr99, bid_bidder, bid_auction, flag, count, max($expr100), 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: @@ -893,34 +893,34 @@ GROUP BY auction, to_char(date_time, 'YYYY-MM-DD'); batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [bid.auction, $expr45, 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 $expr46, sum(bid.price)] } - └─BatchHashAgg { group_key: [bid.auction, $expr45], aggs: [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)] } - └─BatchExchange { order: [], dist: HashShard(bid.auction, $expr45) } - └─BatchProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr45, bid.price] } + └─BatchProject { exprs: [bid.auction, $expr49, 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 $expr50, sum(bid.price)] } + └─BatchHashAgg { group_key: [bid.auction, $expr49], aggs: [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)] } + └─BatchExchange { order: [], dist: HashShard(bid.auction, $expr49) } + └─BatchProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr49, bid.price] } └─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, $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] } + └─StreamProject { exprs: [bid.auction, $expr100, 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 $expr101, sum(bid.price)] } + └─StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr100], 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, $expr100) } + └─StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr100, 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, $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)] } + StreamProject { exprs: [bid.auction, $expr100, 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 $expr101, sum(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr100], 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 $expr91, bid.price, bid._row_id] } + StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr100, 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, $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 0 { columns: [bid_auction, $expr100, 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: @@ -1065,17 +1065,17 @@ SPLIT_PART(url, '/', 6) as dir3 FROM bid; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr67, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr68, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr69] } + └─BatchProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr73, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr74, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr75] } └─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 $expr136, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr137, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr138, bid._row_id] } + └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr148, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr149, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr150, 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 $expr136, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr137, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr138, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr148, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr149, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr150, 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 @@ -1165,7 +1165,7 @@ SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid ) batch_plan: | - BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr23), output: [auction.id, auction.item_name, count(bid.auction)] } + BatchNestedLoopJoin { type: Inner, predicate: (count(bid.auction) >= $expr25), output: [auction.id, auction.item_name, count(bid.auction)] } ├─BatchExchange { order: [], dist: Single } | └─BatchHashAgg { group_key: [auction.id, auction.item_name], aggs: [count(bid.auction), count(bid.auction)] } | └─BatchHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } @@ -1173,7 +1173,7 @@ | | └─BatchScan { table: auction, columns: [auction.id, auction.item_name], distribution: UpstreamHashShard(auction.id) } | └─BatchExchange { order: [], dist: HashShard(bid.auction) } | └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } - └─BatchProject { exprs: [(sum0(count) / count(bid.auction)) as $expr23] } + └─BatchProject { exprs: [(sum0(count) / count(bid.auction)) as $expr25] } └─BatchSimpleAgg { aggs: [sum0(count), count(bid.auction)] } └─BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [bid.auction], aggs: [count] } @@ -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) >= $expr47), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } + └─StreamDynamicFilter { predicate: (count(bid.auction) >= $expr51), 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 $expr47] } + └─StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr51] } └─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) >= $expr47), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } + StreamDynamicFilter { predicate: (count(bid.auction) >= $expr51), 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 $expr47] } + StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr51] } 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: [$expr47], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr51], 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] } @@ -1417,7 +1417,7 @@ └─StreamTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0, group_key: [3] } - └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr1] } + └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr3] } └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } └─StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } @@ -1437,7 +1437,7 @@ Fragment 1 StreamGroupTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr1] } + StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr3] } StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } result table: 2, state tables: [] @@ -1456,8 +1456,8 @@ Upstream BatchPlanNode - Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr1], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr1], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [auction_id, auction_item_name, count(bid_auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [auction_id, auction_item_name, count, count(bid_auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], 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 93417b548a57e..6ea7c936941c2 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -82,12 +82,12 @@ FROM bid; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr24, date_time] } + └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr26, date_time] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } 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 $expr48, date_time, _row_id] } + └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr52, 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 $expr48, date_time, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr52, 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 @@ -215,7 +215,7 @@ GROUP BY Q.category; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr23] } + └─BatchProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr25] } └─BatchHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price))] } └─BatchExchange { order: [], dist: HashShard(category) } └─BatchHashAgg { group_key: [id, category], aggs: [max(price)] } @@ -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 $expr47] } + └─StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr51] } └─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 $expr47] } + StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr51] } 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 = 754 } + | └─StreamShare { id = 842 } | └─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 = 754 } + └─StreamShare { id = 842 } └─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"] } @@ -462,43 +462,43 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [auction, price, bidder, date_time] } - └─BatchFilter { predicate: (date_time >= $expr55) AND (date_time <= $expr54) } + └─BatchFilter { predicate: (date_time >= $expr59) AND (date_time <= $expr58) } └─BatchHashJoin { type: Inner, predicate: price = max(price), output: all } ├─BatchExchange { order: [], dist: HashShard(price) } | └─BatchProject { exprs: [auction, bidder, price, date_time] } | └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } └─BatchExchange { order: [], dist: HashShard(max(price)) } - └─BatchProject { exprs: [max(price), $expr54, ($expr54 - '00:00:10':Interval) as $expr55] } - └─BatchHashAgg { group_key: [$expr54], aggs: [max(price)] } - └─BatchExchange { order: [], dist: HashShard($expr54) } - └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr54, price] } + └─BatchProject { exprs: [max(price), $expr58, ($expr58 - '00:00:10':Interval) as $expr59] } + └─BatchHashAgg { group_key: [$expr58], aggs: [max(price)] } + └─BatchExchange { order: [], dist: HashShard($expr58) } + └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr58, 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), $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) } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr117(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr117, price, max(price)] } + └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr117, max(price)] } + └─StreamFilter { predicate: (date_time >= $expr118) AND (date_time <= $expr117) } └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 440 } + | └─StreamShare { id = 496 } | └─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), $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: [max(price), $expr117, ($expr117 - '00:00:10':Interval) as $expr118] } + └─StreamAppendOnlyHashAgg { group_key: [$expr117], aggs: [count, max(price)] } + └─StreamExchange { dist: HashShard($expr117) } + └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr117, price, _row_id] } + └─StreamShare { id = 496 } └─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), $expr108(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr108, price, max(price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr117(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr117, price, max(price)] } materialized table: 4294967294 - StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr108, max(price)] } - StreamFilter { predicate: (date_time >= $expr110) AND (date_time <= $expr108) } + StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr117, max(price)] } + StreamFilter { predicate: (date_time >= $expr118) AND (date_time <= $expr117) } 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), $expr108, ($expr108 - '00:00:10':Interval) as $expr110] } - StreamAppendOnlyHashAgg { group_key: [$expr108], aggs: [count, max(price)] } + StreamProject { exprs: [max(price), $expr117, ($expr117 - '00:00:10':Interval) as $expr118] } + StreamAppendOnlyHashAgg { group_key: [$expr117], 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 $expr108, price, _row_id] } + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr117, 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), $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 2 { columns: [max(price), $expr117, $expr118], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(price), $expr117, _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: [$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] } + Table 5 { columns: [$expr117, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr117, 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 @@ -569,71 +569,71 @@ AND P.endtime = A.endtime; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: id = seller AND $expr99 = $expr97 AND $expr100 = $expr98, output: [id, name, $expr99] } - ├─BatchExchange { order: [], dist: HashShard(id, $expr99, $expr100) } - | └─BatchHashAgg { group_key: [id, name, $expr99, $expr100], aggs: [] } - | └─BatchExchange { order: [], dist: HashShard(id, name, $expr99, $expr100) } - | └─BatchProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr99, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr100] } + └─BatchHashJoin { type: Inner, predicate: id = seller AND $expr107 = $expr105 AND $expr108 = $expr106, output: [id, name, $expr107] } + ├─BatchExchange { order: [], dist: HashShard(id, $expr107, $expr108) } + | └─BatchHashAgg { group_key: [id, name, $expr107, $expr108], aggs: [] } + | └─BatchExchange { order: [], dist: HashShard(id, name, $expr107, $expr108) } + | └─BatchProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr107, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr108] } | └─BatchSource { source: "person", columns: ["id", "name", "email_address", "credit_card", "city", "state", "date_time", "_row_id"], filter: (None, None) } - └─BatchHashAgg { group_key: [seller, $expr97, $expr98], aggs: [] } - └─BatchExchange { order: [], dist: HashShard(seller, $expr97, $expr98) } - └─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] } + └─BatchHashAgg { group_key: [seller, $expr105, $expr106], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(seller, $expr105, $expr106) } + └─BatchProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr105, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr106] } └─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, $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] } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, seller, $expr215, $expr216] } + └─StreamHashJoin { type: Inner, predicate: id = seller AND $expr213 = $expr215 AND $expr214 = $expr216, output: all } + ├─StreamExchange { dist: HashShard(id, $expr213, $expr214) } + | └─StreamProject { exprs: [id, name, $expr213, $expr214] } + | └─StreamAppendOnlyHashAgg { group_key: [id, name, $expr213, $expr214], aggs: [count] } + | └─StreamExchange { dist: HashShard(id, name, $expr213, $expr214) } + | └─StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr213, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr214, _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, $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] } + └─StreamProject { exprs: [seller, $expr215, $expr216] } + └─StreamAppendOnlyHashAgg { group_key: [seller, $expr215, $expr216], aggs: [count] } + └─StreamExchange { dist: HashShard(seller, $expr215, $expr216) } + └─StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr215, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr216, _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, $expr200(hidden), seller(hidden), $expr197(hidden), $expr198(hidden)], pk_columns: [id, name, starttime, $expr200, seller, $expr197, $expr198] } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, seller, $expr215, $expr216] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: id = seller AND $expr199 = $expr197 AND $expr200 = $expr198, output: all } + StreamHashJoin { type: Inner, predicate: id = seller AND $expr213 = $expr215 AND $expr214 = $expr216, 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, $expr197, $expr198] } - StreamAppendOnlyHashAgg { group_key: [seller, $expr197, $expr198], aggs: [count] } + StreamProject { exprs: [seller, $expr215, $expr216] } + StreamAppendOnlyHashAgg { group_key: [seller, $expr215, $expr216], aggs: [count] } result table: 6, state tables: [] StreamExchange Hash([0, 1, 2]) from 3 Fragment 1 - StreamProject { exprs: [id, name, $expr199, $expr200] } - StreamAppendOnlyHashAgg { group_key: [id, name, $expr199, $expr200], aggs: [count] } + StreamProject { exprs: [id, name, $expr213, $expr214] } + StreamAppendOnlyHashAgg { group_key: [id, name, $expr213, $expr214], 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 $expr199, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr200, _row_id] } + StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr213, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr214, _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 $expr197, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr198, _row_id] } + StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr215, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr216, _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, $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 0 { columns: [id, name, $expr213, $expr214], 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, $expr213, $expr214, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [seller, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [seller, $expr215, $expr216, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [id, name, $expr213, $expr214, 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, $expr197, $expr198, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 6 { columns: [seller, $expr215, $expr216, 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, $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] } + Table 4294967294 { columns: [id, name, starttime, $expr214, seller, $expr215, $expr216], 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 @@ -723,12 +723,12 @@ SELECT auction, bidder, price, date_time, TO_CHAR(date_time, 'YYYY-MM-DD') as date, TO_CHAR(date_time, 'HH:MI') as time FROM bid; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr47, ToChar(date_time, 'HH:MI':Varchar) as $expr48] } + └─BatchProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr51, ToChar(date_time, 'HH:MI':Varchar) as $expr52] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } 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 $expr95, ToChar(date_time, 'HH:MI':Varchar) as $expr96, _row_id] } + └─StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr103, ToChar(date_time, 'HH:MI':Varchar) as $expr104, _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 $expr95, ToChar(date_time, 'HH:MI':Varchar) as $expr96, _row_id] } + StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr103, ToChar(date_time, 'HH:MI':Varchar) as $expr104, _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 @@ -816,13 +816,13 @@ WHERE 0.908 * price > 1000000 AND 0.908 * price < 50000000; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr47, 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 $expr48, date_time, extra] } + └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr51, 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 $expr52, date_time, extra] } └─BatchFilter { predicate: ((0.908:Decimal * price) > 1000000:Int32) AND ((0.908:Decimal * price) < 50000000:Int32) } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } 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 $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] } + └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr103, 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 $expr104, 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 $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] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr103, 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 $expr104, 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"] } @@ -863,50 +863,50 @@ GROUP BY to_char(date_time, 'yyyy-MM-dd'); batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [$expr24], aggs: [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))] } - └─BatchExchange { order: [], dist: HashShard($expr24) } - └─BatchProject { exprs: [$expr24, 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))] } - └─BatchHashAgg { group_key: [$expr24, bidder, auction, flag], aggs: [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))] } - └─BatchExchange { order: [], dist: HashShard($expr24, bidder, auction, flag) } - └─BatchExpand { column_subsets: [[$expr24], [$expr24, bidder], [$expr24, auction]] } - └─BatchProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr24, price, bidder, auction] } + └─BatchHashAgg { group_key: [$expr26], aggs: [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))] } + └─BatchExchange { order: [], dist: HashShard($expr26) } + └─BatchProject { exprs: [$expr26, 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))] } + └─BatchHashAgg { group_key: [$expr26, bidder, auction, flag], aggs: [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))] } + └─BatchExchange { order: [], dist: HashShard($expr26, bidder, auction, flag) } + └─BatchExpand { column_subsets: [[$expr26], [$expr26, bidder], [$expr26, auction]] } + └─BatchProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr26, price, bidder, auction] } └─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: [$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] } + └─StreamProject { exprs: [$expr52, 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: [$expr52], 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($expr52) } + └─StreamProject { exprs: [$expr52, 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: [$expr52, 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($expr52, bidder, auction, flag) } + └─StreamExpand { column_subsets: [[$expr52], [$expr52, bidder], [$expr52, auction]] } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr52, 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: [$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))] } + StreamProject { exprs: [$expr52, 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: [$expr52], 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: [$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))] } + StreamProject { exprs: [$expr52, 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: [$expr52, 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: [[$expr48], [$expr48, bidder], [$expr48, auction]] } - StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr48, price, bidder, auction, _row_id] } + StreamExpand { column_subsets: [[$expr52], [$expr52, bidder], [$expr52, auction]] } + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr52, 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: [$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 0 { columns: [$expr52, 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: [$expr52, 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 @@ -933,51 +933,51 @@ GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchHashAgg { group_key: [channel, $expr47], aggs: [max(max($expr48)) 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))] } - └─BatchExchange { order: [], dist: HashShard(channel, $expr47) } - └─BatchProject { exprs: [channel, $expr47, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr48), 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))] } - └─BatchHashAgg { group_key: [channel, $expr47, bidder, auction, flag], aggs: [max($expr48), 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))] } - └─BatchExchange { order: [], dist: HashShard(channel, $expr47, bidder, auction, flag) } - └─BatchExpand { column_subsets: [[channel, $expr47, $expr48], [channel, $expr47, bidder], [channel, $expr47, auction]] } - └─BatchProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr47, ToChar(date_time, 'HH:mm':Varchar) as $expr48, price, bidder, auction] } + └─BatchHashAgg { group_key: [channel, $expr51], aggs: [max(max($expr52)) 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))] } + └─BatchExchange { order: [], dist: HashShard(channel, $expr51) } + └─BatchProject { exprs: [channel, $expr51, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr52), 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))] } + └─BatchHashAgg { group_key: [channel, $expr51, bidder, auction, flag], aggs: [max($expr52), 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))] } + └─BatchExchange { order: [], dist: HashShard(channel, $expr51, bidder, auction, flag) } + └─BatchExpand { column_subsets: [[channel, $expr51, $expr52], [channel, $expr51, bidder], [channel, $expr51, auction]] } + └─BatchProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr51, ToChar(date_time, 'HH:mm':Varchar) as $expr52, price, bidder, auction] } └─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, $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] } + └─StreamProject { exprs: [channel, $expr103, max(max($expr104)) 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, $expr103], aggs: [count, max(max($expr104)) 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, $expr103) } + └─StreamProject { exprs: [channel, $expr103, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr104), 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, $expr103, bidder, auction, flag], aggs: [count, max($expr104), 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, $expr103, bidder, auction, flag) } + └─StreamExpand { column_subsets: [[channel, $expr103, $expr104], [channel, $expr103, bidder], [channel, $expr103, auction]] } + └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr103, ToChar(date_time, 'HH:mm':Varchar) as $expr104, 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, $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))] } + StreamProject { exprs: [channel, $expr103, max(max($expr104)) 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, $expr103], aggs: [count, max(max($expr104)) 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, $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))] } + StreamProject { exprs: [channel, $expr103, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr104), 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, $expr103, bidder, auction, flag], aggs: [count, max($expr104), 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, $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] } + StreamExpand { column_subsets: [[channel, $expr103, $expr104], [channel, $expr103, bidder], [channel, $expr103, auction]] } + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr103, ToChar(date_time, 'HH:mm':Varchar) as $expr104, 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, $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 0 { columns: [channel, $expr103, max($expr104), 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, $expr103, count, max(max($expr104)) 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, $expr103, bidder, auction, flag, count, max($expr104), 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 @@ -999,35 +999,35 @@ GROUP BY auction, to_char(date_time, 'YYYY-MM-DD'); batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction, $expr46, 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 $expr47, sum(price)] } - └─BatchHashAgg { group_key: [auction, $expr46], aggs: [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)] } - └─BatchExchange { order: [], dist: HashShard(auction, $expr46) } - └─BatchProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr46, price] } + └─BatchProject { exprs: [auction, $expr50, 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 $expr51, sum(price)] } + └─BatchHashAgg { group_key: [auction, $expr50], aggs: [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)] } + └─BatchExchange { order: [], dist: HashShard(auction, $expr50) } + └─BatchProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr50, price] } └─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, $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] } + └─StreamProject { exprs: [auction, $expr102, 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 $expr103, sum(price)] } + └─StreamAppendOnlyHashAgg { group_key: [auction, $expr102], 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, $expr102) } + └─StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr102, 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, $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)] } + StreamProject { exprs: [auction, $expr102, 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 $expr103, sum(price)] } + StreamAppendOnlyHashAgg { group_key: [auction, $expr102], 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 $expr93, price, _row_id] } + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr102, 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, $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 0 { columns: [auction, $expr102, 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 @@ -1188,12 +1188,12 @@ SPLIT_PART(url, '/', 6) as dir3 FROM bid; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr70, SplitPart(url, '/':Varchar, 5:Int32) as $expr71, SplitPart(url, '/':Varchar, 6:Int32) as $expr72] } + └─BatchProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr76, SplitPart(url, '/':Varchar, 5:Int32) as $expr77, SplitPart(url, '/':Varchar, 6:Int32) as $expr78] } └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } 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 $expr142, SplitPart(url, '/':Varchar, 5:Int32) as $expr143, SplitPart(url, '/':Varchar, 6:Int32) as $expr144, _row_id] } + └─StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr154, SplitPart(url, '/':Varchar, 5:Int32) as $expr155, SplitPart(url, '/':Varchar, 6:Int32) as $expr156, _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 $expr142, SplitPart(url, '/':Varchar, 5:Int32) as $expr143, SplitPart(url, '/':Varchar, 6:Int32) as $expr144, _row_id] } + StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr154, SplitPart(url, '/':Varchar, 5:Int32) as $expr155, SplitPart(url, '/':Varchar, 6:Int32) as $expr156, _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 @@ -1301,7 +1301,7 @@ SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid ) batch_plan: | - BatchNestedLoopJoin { type: Inner, predicate: (count(auction) >= $expr24), output: [id, item_name, count(auction)] } + BatchNestedLoopJoin { type: Inner, predicate: (count(auction) >= $expr26), output: [id, item_name, count(auction)] } ├─BatchExchange { order: [], dist: Single } | └─BatchHashAgg { group_key: [id, item_name], aggs: [count(auction), count(auction)] } | └─BatchHashJoin { type: Inner, predicate: id = auction, output: all } @@ -1311,7 +1311,7 @@ | └─BatchExchange { order: [], dist: HashShard(auction) } | └─BatchProject { exprs: [auction] } | └─BatchSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"], filter: (None, None) } - └─BatchProject { exprs: [(sum0(count) / count(auction)) as $expr24] } + └─BatchProject { exprs: [(sum0(count) / count(auction)) as $expr26] } └─BatchSimpleAgg { aggs: [sum0(count), count(auction)] } └─BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [auction], aggs: [count] } @@ -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) >= $expr49), output: [id, item_name, count(auction), count(auction)] } + └─StreamDynamicFilter { predicate: (count(auction) >= $expr53), 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 = 562 } + | └─StreamShare { id = 634 } | └─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 $expr49] } + └─StreamProject { exprs: [(sum0(count) / count(auction)) as $expr53] } └─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 = 562 } + └─StreamShare { id = 634 } └─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) >= $expr49), output: [id, item_name, count(auction), count(auction)] } + StreamDynamicFilter { predicate: (count(auction) >= $expr53), 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 $expr49] } + StreamProject { exprs: [(sum0(count) / count(auction)) as $expr53] } 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: [$expr49], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr53], 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] } @@ -1589,7 +1589,7 @@ └─StreamTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0, group_key: [3] } - └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr3] } └─StreamProject { exprs: [id, item_name, count(auction)] } └─StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } @@ -1613,7 +1613,7 @@ Fragment 1 StreamGroupTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0, group_key: [3] } state table: 1 - StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr3] } StreamProject { exprs: [id, item_name, count(auction)] } StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } result table: 2, state tables: [] @@ -1634,8 +1634,8 @@ StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } source state table: 8 - Table 0 { columns: [id, item_name, count(auction), $expr1], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } - Table 1 { columns: [id, item_name, count(auction), $expr1], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } + Table 0 { columns: [id, item_name, count(auction), $expr3], primary key: [$2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [] } + Table 1 { columns: [id, item_name, count(auction), $expr3], primary key: [$3 ASC, $2 DESC, $0 ASC, $1 ASC], value indices: [0, 1, 2, 3], distribution key: [0], vnode column idx: 3 } Table 2 { columns: [id, item_name, count, count(auction)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], 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 984fbb2cf7b37..da18db53acfb2 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -23,7 +23,7 @@ select v1, v1+1 from t order by v1; batch_plan: | BatchExchange { order: [t.v1 ASC], dist: Single } - └─BatchProject { exprs: [t.v1, (t.v1 + 1:Int32) as $expr23] } + └─BatchProject { exprs: [t.v1, (t.v1 + 1:Int32) as $expr25] } └─BatchSort { order: [t.v1 ASC] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [$expr23 ASC], dist: Single } - └─BatchSort { order: [$expr23 ASC] } - └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr23] } + └─BatchExchange { order: [2:Int32 ASC], dist: Single } + └─BatchSort { order: [2:Int32 ASC] } + └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - 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] } + StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, 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); @@ -85,7 +85,7 @@ └─StreamTopN { order: "[t.v1 DESC]", limit: 5, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[t.v1 DESC]", limit: 5, offset: 0, group_key: [3] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr3] } └─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); @@ -117,7 +117,7 @@ └─StreamTopN { order: "[t.v1 DESC]", limit: 5, offset: 7 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[t.v1 DESC]", limit: 12, offset: 0, group_key: [3] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr3] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by expression that would be valid in select list sql: | @@ -128,13 +128,13 @@ └─LogicalScan { table: t, columns: [t.x, t.y, t.z] } batch_plan: | BatchProject { exprs: [t.x, t.y] } - └─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] } + └─BatchExchange { order: [$expr43 ASC, t.z ASC], dist: Single } + └─BatchSort { order: [$expr43 ASC, t.z ASC] } + └─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr43, t.z] } └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: | - 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] } + StreamMaterialize { columns: [x, y, $expr68(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr68, t.z, t._row_id] } + └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr68, 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: | @@ -208,9 +208,9 @@ CREATE TABLE test (a INTEGER, b INTEGER); SELECT b % 2 AS f, SUM(a) FROM test GROUP BY b % 2 ORDER BY f; batch_plan: | - BatchExchange { order: [$expr23 ASC], dist: Single } - └─BatchSortAgg { group_key: [$expr23], aggs: [sum(test.a)] } - └─BatchExchange { order: [$expr23 ASC], dist: HashShard($expr23) } - └─BatchSort { order: [$expr23 ASC] } - └─BatchProject { exprs: [(test.b % 2:Int32) as $expr23, test.a] } + BatchExchange { order: [$expr25 ASC], dist: Single } + └─BatchSortAgg { group_key: [$expr25], aggs: [sum(test.a)] } + └─BatchExchange { order: [$expr25 ASC], dist: HashShard($expr25) } + └─BatchSort { order: [$expr25 ASC] } + └─BatchProject { exprs: [(test.b % 2:Int32) as $expr25, test.a] } └─BatchScan { table: test, columns: [test.a, test.b], distribution: SomeShard } 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 e3775e9f77c4f..fc304eede4248 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($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] } + └─StreamExchange { dist: HashShard($expr51, $expr52) } + └─StreamProject { exprs: [$expr51, $expr52, bid.supplier_id, sum(bid.price), count] } + └─StreamHashAgg { group_key: [$expr51, $expr52, bid.supplier_id], aggs: [count, sum(bid.price), count] } + └─StreamExchange { dist: HashShard($expr51, $expr52, bid.supplier_id) } + └─StreamProject { exprs: [TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr51, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr52, 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: [$expr45, $expr46, bid.supplier_id, bid.price, bid._row_id] } + └─StreamProject { exprs: [$expr49, $expr50, bid.supplier_id, bid.price, bid._row_id] } └─StreamGroupTopN { order: "[bid.price DESC]", limit: 3, offset: 0, group_key: [5, 6] } - └─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] } + └─StreamExchange { dist: HashShard($expr49, $expr50) } + └─StreamProject { exprs: [bid.bidtime, bid.price, bid.item, bid.supplier_id, bid._row_id, TumbleStart(bid.bidtime, '00:10:00':Interval) as $expr49, (TumbleStart(bid.bidtime, '00:10:00':Interval) + '00:10:00':Interval) as $expr50] } └─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 59567b31c5987..eb2e6de2cd0b4 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -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 > $expr12), output: [t1.v1, t1._row_id] } + | └─StreamDynamicFilter { predicate: (t1.v1 > $expr14), 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 $expr12], watermark_columns: [(now + '01:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr14], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '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 > $expr12), output: [t1.v1, t1.v2, t1._row_id] } + └─StreamDynamicFilter { predicate: (t1.v1 > $expr14), 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 $expr12], watermark_columns: [(now + '00:30:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:30:00':Interval) as $expr14], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '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/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 265087683371f..818424cf199b4 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -53,7 +53,7 @@ select -generate_series(x,x,x) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Neg(Generate($0, $0, $0)) as $expr24] } + └─BatchProject { exprs: [Neg(Generate($0, $0, $0)) as $expr26] } └─BatchProjectSet { select_list: [$0, $1, Generate($0, $0, $0)] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions @@ -98,7 +98,7 @@ └─StreamTopN { order: "[projected_row_id ASC]", limit: 1, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[projected_row_id ASC]", limit: 1, offset: 0, group_key: [3] } - └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as $expr3] } └─StreamProjectSet { select_list: [Unnest($0), $1] } └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 9444c038a90d8..648dd521e5c63 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 = 529 } + | └─StreamShare { id = 609 } | └─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 = 529 } + └─StreamShare { id = 609 } └─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 = 754 } + | └─StreamShare { id = 842 } | └─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 = 754 } + └─StreamShare { id = 842 } └─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"] } diff --git a/src/frontend/planner_test/tests/testdata/shared_views.yaml b/src/frontend/planner_test/tests/testdata/shared_views.yaml index 206a89e9e22fb..9790b081c3d16 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -23,21 +23,21 @@ └─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), $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] } + └─StreamHashJoin { type: Inner, predicate: $expr86 = $expr95, output: [$expr86, $expr95, $expr96, 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] } + | └─StreamProject { exprs: [(t1.x + t1.y) as $expr94, 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($expr87) } - └─StreamProject { exprs: [(t1.x * $expr86) as $expr87, (t1.y * $expr86) as $expr88, t1._row_id, t1._row_id, t1.x, $expr86] } + └─StreamExchange { dist: HashShard($expr95) } + └─StreamProject { exprs: [(t1.x * $expr86) as $expr95, (t1.y * $expr86) as $expr96, 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($expr86) } └─StreamProject { exprs: [$expr86, t1._row_id] } └─StreamShare { id = 207 } - └─StreamProject { exprs: [(t1.x + t1.y) as $expr86, t1._row_id] } + └─StreamProject { exprs: [(t1.x + t1.y) as $expr94, 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/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index c5ffc5cb34f78..5931d6a8eace5 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -134,8 +134,8 @@ └─StreamProject { exprs: [max(max(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v)] } - └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v)] } + └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 @@ -147,17 +147,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v)] } + StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v)] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr1], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [count, max(max(t_v))], primary key: [], value indices: [0, 1], distribution key: [] } - Table 2 { columns: [$expr1, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr1, count, max(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr3, count, max(t_v)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1], primary key: [], value indices: [0], distribution key: [] } - id: extreme_on_AO before: @@ -393,8 +393,8 @@ └─StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v)), sum0(count(t.v))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v), count(t.v)] } - └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } + └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v), count(t.v)] } + └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 @@ -406,17 +406,17 @@ StreamExchange Single from 1 Fragment 1 - StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v), count(t.v)] } + StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v), count(t.v)] } result table: 3, state tables: [2] - StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } + StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } Chain { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } Upstream BatchPlanNode - Table 0 { columns: [max(t_v), $expr1], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [max(t_v), $expr3], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } Table 1 { columns: [count, max(max(t_v)), sum0(count(t_v))], primary key: [], value indices: [0, 1, 2], distribution key: [] } - Table 2 { columns: [$expr1, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } - Table 3 { columns: [$expr1, count, max(t_v), count(t_v)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } + Table 2 { columns: [$expr3, t_v, t__row_id], primary key: [$0 ASC, $1 DESC, $2 ASC], value indices: [1, 2], distribution key: [2], vnode column idx: 0 } + Table 3 { columns: [$expr3, count, max(t_v), count(t_v)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [], vnode column idx: 0 } Table 4294967294 { columns: [a1, a2], primary key: [], value indices: [0, 1], distribution key: [] } - id: extreme_count_on_AO before: diff --git a/src/frontend/planner_test/tests/testdata/subquery.yaml b/src/frontend/planner_test/tests/testdata/subquery.yaml index 7044e20acb7f4..6a01105dcab65 100644 --- a/src/frontend/planner_test/tests/testdata/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/subquery.yaml @@ -188,7 +188,7 @@ └─LogicalScan { table: pg_user, columns: [pg_user.usesysid, pg_user.name, pg_user.usecreatedb, pg_user.usesuper, pg_user.passwd] } batch_plan: | BatchExchange { order: [pg_namespace.nspname ASC, pg_class.relname ASC], dist: Single } - └─BatchProject { exprs: [pg_namespace.nspname, pg_class.relname, Case((pg_class.relkind = 'r':Varchar), 'table':Varchar, (pg_class.relkind = 'v':Varchar), 'view':Varchar, (pg_class.relkind = 'm':Varchar), 'materialized view':Varchar, (pg_class.relkind = 'i':Varchar), 'index':Varchar, (pg_class.relkind = 'S':Varchar), 'sequence':Varchar, (pg_class.relkind = 's':Varchar), 'special':Varchar, (pg_class.relkind = 't':Varchar), 'TOAST table':Varchar, (pg_class.relkind = 'f':Varchar), 'foreign table':Varchar, (pg_class.relkind = 'p':Varchar), 'partitioned table':Varchar, (pg_class.relkind = 'I':Varchar), 'partitioned index':Varchar) as $expr23, pg_user.name] } + └─BatchProject { exprs: [pg_namespace.nspname, pg_class.relname, Case((pg_class.relkind = 'r':Varchar), 'table':Varchar, (pg_class.relkind = 'v':Varchar), 'view':Varchar, (pg_class.relkind = 'm':Varchar), 'materialized view':Varchar, (pg_class.relkind = 'i':Varchar), 'index':Varchar, (pg_class.relkind = 'S':Varchar), 'sequence':Varchar, (pg_class.relkind = 's':Varchar), 'special':Varchar, (pg_class.relkind = 't':Varchar), 'TOAST table':Varchar, (pg_class.relkind = 'f':Varchar), 'foreign table':Varchar, (pg_class.relkind = 'p':Varchar), 'partitioned table':Varchar, (pg_class.relkind = 'I':Varchar), 'partitioned index':Varchar) as $expr25, pg_user.name] } └─BatchSort { order: [pg_namespace.nspname ASC, pg_class.relname ASC] } └─BatchHashJoin { type: LeftOuter, predicate: pg_class.relowner = pg_user.usesysid, output: [pg_class.relname, pg_class.relkind, pg_namespace.nspname, pg_user.name] } ├─BatchExchange { order: [], dist: HashShard(pg_class.relowner) } diff --git a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index 0b53fee2b34d4..385323133d317 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -31,4 +31,4 @@ - sql: | select current_timestamp > now() - interval '10 minutes'; batch_plan: | - BatchValues { rows: [[Now('2021-04-01 00:00:00+00:00':Timestamptz)]] } + BatchValues { rows: [[true:Boolean]] } diff --git a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml index e0af04db818b5..0005353b436d6 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: ($expr12 > now), output: [t1.ts, $expr12, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, (t1.ts + '01:00:00':Interval) as $expr12, t1._row_id] } + └─StreamDynamicFilter { predicate: ($expr14 > now), output: [t1.ts, $expr14, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr14, 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: ($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] } + └─StreamDynamicFilter { predicate: ($expr14 > now), output: [t1.ts, t1.time_to_live, $expr14, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal)))) as $expr14, 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: ($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] } + └─StreamDynamicFilter { predicate: ($expr27 > $expr28), output: [t1.ts, t1.additional_time_to_live, $expr27, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.additional_time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal)))) as $expr27, 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 $expr24], watermark_columns: [(now - '00:15:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval) as $expr28], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '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 < $expr24), output: [t1.ts, t1._row_id] } - ├─StreamDynamicFilter { predicate: (t1.ts >= $expr23), output: [t1.ts, t1._row_id] } + └─StreamDynamicFilter { predicate: (t1.ts < $expr28), output: [t1.ts, t1._row_id] } + ├─StreamDynamicFilter { predicate: (t1.ts >= $expr27), 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 $expr23], watermark_columns: [(now - '02:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval) as $expr27], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval)] } | └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '01:00:00':Interval) as $expr24], watermark_columns: [(now - '01:00:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval) as $expr28], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '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 678147aeeadda..4bb3ae4384dd8 100644 --- a/src/frontend/planner_test/tests/testdata/time_window.yaml +++ b/src/frontend/planner_test/tests/testdata/time_window.yaml @@ -8,7 +8,7 @@ └─LogicalScan { table: t1, columns: [t1.id, t1.created_at, t1._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t1.id, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr47, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr48] } + └─BatchProject { exprs: [t1.id, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr51, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr52] } └─BatchScan { table: t1, columns: [t1.id, t1.created_at], distribution: SomeShard } - sql: | create materialized view t as select * from s; @@ -130,14 +130,14 @@ └─LogicalScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr23] } + └─BatchProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr25] } └─BatchHashAgg { group_key: [t.v1, window_end], aggs: [sum(t.v3), count(t.v3)] } └─BatchHopWindow { time_col: t.v2, slide: 00:01:00, size: 00:10:00, output: [t.v1, t.v3, window_end] } └─BatchExchange { order: [], dist: HashShard(t.v1) } └─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 $expr47] } + └─StreamProject { exprs: [t.v1, window_end, (sum(t.v3) / count(t.v3)::Float64) as $expr51] } └─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] } @@ -155,12 +155,12 @@ └─LogicalScan { table: t1, columns: [t1.id, t1.v1, t1.created_at, t1._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t1.id, t1.v1, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr47, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr48] } + └─BatchProject { exprs: [t1.id, t1.v1, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr51, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr52] } └─BatchFilter { predicate: (t1.v1 >= 10:Int32) } └─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 $expr95, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr96, t1._row_id] } + └─StreamProject { exprs: [t1.id, t1.v1, t1.created_at, TumbleStart(t1.created_at, '3 days':Interval) as $expr103, (TumbleStart(t1.created_at, '3 days':Interval) + '3 days':Interval) as $expr104, 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: | @@ -192,5 +192,5 @@ └─LogicalShare { id = 2 } └─LogicalValues { rows: [['2020-01-01 12:00:00':Timestamp]], schema: Schema { fields: [*VALUES*_0.column_0:Timestamp] } } batch_plan: | - BatchProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr47, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr48] } + BatchProject { exprs: [*VALUES*_0.column_0, TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) as $expr51, (TumbleStart(*VALUES*_0.column_0, '00:00:10':Interval) + '00:00:10':Interval) as $expr52] } └─BatchValues { rows: [['2020-01-01 12:00:00':Timestamp]] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index f231c409d8a44..cfbe8c611209d 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -131,38 +131,38 @@ └─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($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] } + └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr213), sum($expr214), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr215, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr216, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr217, 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($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] } + └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr213), sum($expr214), 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 $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)) } + └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr213, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr214, lineitem.l_discount] } + └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─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($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] } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr343), sum($expr344), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr345, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr346, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr347, count] } + └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr343), sum($expr344), 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 $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)) } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr343, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr344, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], 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($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] } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr343), sum($expr344), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr345, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr346, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr347, count] } + StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr343), sum($expr344), 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 $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)) } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr343, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr344, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } 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($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 0 { columns: [lineitem_l_returnflag, lineitem_l_linestatus, count, sum(lineitem_l_quantity), sum(lineitem_l_extendedprice), sum($expr343), sum($expr344), 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: @@ -301,7 +301,7 @@ └─StreamTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr3] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_supplycost, part.p_partkey, min(partsupp.ps_supplycost)] } @@ -363,7 +363,7 @@ Fragment 1 StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } state table: 1 - StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } + StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr3] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } left table: 2, right table 4, left degree table: 3, right degree table: 5, StreamExchange Hash([8]) from 2 @@ -479,8 +479,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } - Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr1], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 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, 18], distribution key: [16], vnode column idx: 18 } + Table 0 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr3], primary key: [$0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18], distribution key: [] } + Table 1 { columns: [supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), nation_n_regionkey, region_r_regionkey, $expr3], primary key: [$18 ASC, $0 DESC, $2 ASC, $1 ASC, $3 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, 18], distribution key: [16], vnode column idx: 18 } Table 2 { columns: [part_p_partkey, part_p_mfgr, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_acctbal, supplier_s_comment, nation_n_name, nation_n_regionkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, partsupp_ps_supplycost, part_p_partkey_0, min(partsupp_ps_supplycost)], primary key: [$8 ASC, $0 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $15 ASC, $14 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16], distribution key: [8] } Table 3 { columns: [nation_n_regionkey, part_p_partkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, part_p_partkey_0, partsupp_ps_supplycost, min(partsupp_ps_supplycost), _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 4 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -570,13 +570,13 @@ | └─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($expr41) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + BatchTopN { order: "[sum($expr43) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } └─BatchExchange { order: [], dist: Single } - └─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)] } + └─BatchTopN { order: "[sum($expr43) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + └─BatchProject { exprs: [lineitem.l_orderkey, sum($expr43), orders.o_orderdate, orders.o_shippriority] } + └─BatchHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr43)] } └─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 $expr41] } + └─BatchProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr43] } └─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($expr64), orders.o_orderdate, orders.o_shippriority] } - └─StreamTopN { order: "[sum($expr64) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr70), orders.o_orderdate, orders.o_shippriority] } + └─StreamTopN { order: "[sum($expr70) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } └─StreamExchange { dist: Single } - └─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)] } + └─StreamGroupTopN { order: "[sum($expr70) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr70), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr71] } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr70), orders.o_orderdate, orders.o_shippriority] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr70)] } └─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 $expr64, 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 $expr70, 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($expr64), orders.o_orderdate, orders.o_shippriority] } - StreamTopN { order: "[sum($expr64) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr70), orders.o_orderdate, orders.o_shippriority] } + StreamTopN { order: "[sum($expr70) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr64) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + StreamGroupTopN { order: "[sum($expr70) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } state table: 1 - 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)] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr70), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr71] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr70), orders.o_orderdate, orders.o_shippriority] } + StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr70)] } 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 $expr64, 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 $expr70, 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($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 0 { columns: [lineitem_l_orderkey, sum($expr70), orders_o_orderdate, orders_o_shippriority, $expr71], 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($expr70), orders_o_orderdate, orders_o_shippriority, $expr71], 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($expr70)], 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] } @@ -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($expr41) DESC], dist: Single } - └─BatchSort { order: [sum($expr41) DESC] } - └─BatchHashAgg { group_key: [nation.n_name], aggs: [sum($expr41)] } + BatchExchange { order: [sum($expr43) DESC], dist: Single } + └─BatchSort { order: [sum($expr43) DESC] } + └─BatchHashAgg { group_key: [nation.n_name], aggs: [sum($expr43)] } └─BatchExchange { order: [], dist: HashShard(nation.n_name) } - └─BatchProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr41] } + └─BatchProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr43] } └─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($expr64)] } - └─StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr64)] } + └─StreamProject { exprs: [nation.n_name, sum($expr68)] } + └─StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr68)] } └─StreamExchange { dist: HashShard(nation.n_name) } - └─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] } + └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr68, 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($expr64)] } - StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr64)] } + StreamProject { exprs: [nation.n_name, sum($expr68)] } + StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr68)] } 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 $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] } + StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr68, 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($expr64)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [nation_n_name, count, sum($expr68)], 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] } @@ -1016,39 +1016,39 @@ └─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($expr42))] } + BatchSimpleAgg { aggs: [sum(sum($expr44))] } └─BatchExchange { order: [], dist: Single } - └─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) } + └─BatchSimpleAgg { aggs: [sum($expr44)] } + └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr44] } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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($expr66))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr66))] } + └─StreamProject { exprs: [sum(sum($expr70))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr70))] } └─StreamExchange { dist: Single } - └─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) } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr70)] } + └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr70, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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($expr66))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr66))] } + StreamProject { exprs: [sum(sum($expr70))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr70))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - 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) } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr70)] } + StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr70, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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($expr66))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr70))], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [revenue], primary key: [], value indices: [0], distribution key: [] } - id: tpch_q7 before: @@ -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, $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] } + BatchExchange { order: [nation.n_name ASC, nation.n_name ASC, $expr89 ASC], dist: Single } + └─BatchSort { order: [nation.n_name ASC, nation.n_name ASC, $expr89 ASC] } + └─BatchHashAgg { group_key: [nation.n_name, nation.n_name, $expr89], aggs: [sum($expr90)] } + └─BatchExchange { order: [], dist: HashShard(nation.n_name, nation.n_name, $expr89) } + └─BatchProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr89, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr90] } └─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, $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] } + └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr143, sum($expr144)] } + └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr143], aggs: [count, sum($expr144)] } + └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr143) } + └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr143, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr144, 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, $expr135, sum($expr136)] } - StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr135], aggs: [count, sum($expr136)] } + StreamProject { exprs: [nation.n_name, nation.n_name, $expr143, sum($expr144)] } + StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr143], aggs: [count, sum($expr144)] } 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 $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] } + StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr143, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr144, 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, $expr135, count, sum($expr136)], 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, $expr143, count, sum($expr144)], 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] } @@ -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: [$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] } + BatchExchange { order: [$expr175 ASC], dist: Single } + └─BatchProject { exprs: [$expr175, RoundDigit((sum($expr176) / sum($expr177)), 6:Int32) as $expr178] } + └─BatchSort { order: [$expr175 ASC] } + └─BatchHashAgg { group_key: [$expr175], aggs: [sum($expr176), sum($expr177)] } + └─BatchExchange { order: [], dist: HashShard($expr175) } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr175, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr176, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr177] } └─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: [$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] } + └─StreamProject { exprs: [$expr279, RoundDigit((sum($expr280) / sum($expr281)), 6:Int32) as $expr282] } + └─StreamHashAgg { group_key: [$expr279], aggs: [count, sum($expr280), sum($expr281)] } + └─StreamExchange { dist: HashShard($expr279) } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr279, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr280, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr281, 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: [$expr262, RoundDigit((sum($expr263) / sum($expr264)), 6:Int32) as $expr266] } - StreamHashAgg { group_key: [$expr262], aggs: [count, sum($expr263), sum($expr264)] } + StreamProject { exprs: [$expr279, RoundDigit((sum($expr280) / sum($expr281)), 6:Int32) as $expr282] } + StreamHashAgg { group_key: [$expr279], aggs: [count, sum($expr280), sum($expr281)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - 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] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr279, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr280, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr281, 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: [$expr262, count, sum($expr263), sum($expr264)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [$expr279, count, sum($expr280), sum($expr281)], 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] } @@ -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, $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] } + BatchExchange { order: [nation.n_name ASC, $expr131 DESC], dist: Single } + └─BatchProject { exprs: [nation.n_name, $expr131, RoundDigit(sum($expr132), 2:Int32) as $expr133] } + └─BatchSort { order: [nation.n_name ASC, $expr131 DESC] } + └─BatchHashAgg { group_key: [nation.n_name, $expr131], aggs: [sum($expr132)] } + └─BatchExchange { order: [], dist: HashShard(nation.n_name, $expr131) } + └─BatchProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr131, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr132] } └─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, $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] } + └─StreamProject { exprs: [nation.n_name, $expr209, RoundDigit(sum($expr210), 2:Int32) as $expr211] } + └─StreamHashAgg { group_key: [nation.n_name, $expr209], aggs: [count, sum($expr210)] } + └─StreamExchange { dist: HashShard(nation.n_name, $expr209) } + └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr209, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr210, 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, $expr196, RoundDigit(sum($expr197), 2:Int32) as $expr199] } - StreamHashAgg { group_key: [nation.n_name, $expr196], aggs: [count, sum($expr197)] } + StreamProject { exprs: [nation.n_name, $expr209, RoundDigit(sum($expr210), 2:Int32) as $expr211] } + StreamHashAgg { group_key: [nation.n_name, $expr209], aggs: [count, sum($expr210)] } 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 $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] } + StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr209, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr210, 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, $expr196, count, sum($expr197)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [nation_n_name, $expr209, count, sum($expr210)], 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] } @@ -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($expr41) DESC]", limit: 20, offset: 0 } + BatchTopN { order: "[sum($expr43) DESC]", limit: 20, offset: 0 } └─BatchExchange { order: [], dist: Single } - └─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)] } + └─BatchTopN { order: "[sum($expr43) DESC]", limit: 20, offset: 0 } + └─BatchProject { exprs: [customer.c_custkey, customer.c_name, sum($expr43), 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($expr43)] } └─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 $expr41] } + └─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 $expr43] } └─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($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 } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr70), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └─StreamTopN { order: "[sum($expr70) DESC]", limit: 20, offset: 0 } └─StreamExchange { dist: Single } - └─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)] } + └─StreamGroupTopN { order: "[sum($expr70) DESC]", limit: 20, offset: 0, group_key: [8] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr70), 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 $expr71] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr70), 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($expr70)] } └─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 $expr64, 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 $expr70, 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($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 } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr70), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamTopN { order: "[sum($expr70) DESC]", limit: 20, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr64) DESC]", limit: 20, offset: 0, group_key: [8] } + StreamGroupTopN { order: "[sum($expr70) DESC]", limit: 20, offset: 0, group_key: [8] } state table: 1 - 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)] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr70), 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 $expr71] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr70), 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($expr70)] } 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 $expr64, 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 $expr70, 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($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 0 { columns: [customer_c_custkey, customer_c_name, sum($expr70), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr71], 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($expr70), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr71], 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($expr70)], 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] } @@ -2014,22 +2014,22 @@ | └─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($expr121) DESC] } - └─BatchNestedLoopJoin { type: Inner, predicate: (sum($expr121) > $expr123), output: [partsupp.ps_partkey, sum($expr121)] } + BatchSort { order: [sum($expr127) DESC] } + └─BatchNestedLoopJoin { type: Inner, predicate: (sum($expr127) > $expr129), output: [partsupp.ps_partkey, sum($expr127)] } ├─BatchExchange { order: [], dist: Single } - | └─BatchHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr121), sum($expr121)] } + | └─BatchHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr127), sum($expr127)] } | └─BatchExchange { order: [], dist: HashShard(partsupp.ps_partkey) } - | └─BatchProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr121] } + | └─BatchProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr127] } | └─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($expr122)) * 0.0001000000:Decimal) as $expr123] } - └─BatchSimpleAgg { aggs: [sum(sum($expr122))] } + └─BatchProject { exprs: [(sum(sum($expr128)) * 0.0001000000:Decimal) as $expr129] } + └─BatchSimpleAgg { aggs: [sum(sum($expr128))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum($expr122)] } - └─BatchProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr122] } + └─BatchSimpleAgg { aggs: [sum($expr128)] } + └─BatchProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr128] } └─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($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)] } + └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr203)] } + └─StreamDynamicFilter { predicate: (sum($expr203) > $expr205), output: [partsupp.ps_partkey, sum($expr203), sum($expr203)] } + ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr203), sum($expr203)] } + | └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr203), sum($expr203)] } | └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - | └─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] } + | └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr203, 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($expr191)) * 0.0001000000:Decimal) as $expr193] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr191))] } + └─StreamProject { exprs: [(sum(sum($expr204)) * 0.0001000000:Decimal) as $expr205] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr204))] } └─StreamExchange { dist: Single } - └─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] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr204)] } + └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr204, 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($expr190)] } - StreamDynamicFilter { predicate: (sum($expr190) > $expr193), output: [partsupp.ps_partkey, sum($expr190), sum($expr190)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr203)] } + StreamDynamicFilter { predicate: (sum($expr203) > $expr205), output: [partsupp.ps_partkey, sum($expr203), sum($expr203)] } left table: 0, right table 1 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr190), sum($expr190)] } - StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr190), sum($expr190)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr203), sum($expr203)] } + StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr203), sum($expr203)] } 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 $expr190, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr203, 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($expr191)) * 0.0001000000:Decimal) as $expr193] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr191))] } + StreamProject { exprs: [(sum(sum($expr204)) * 0.0001000000:Decimal) as $expr205] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr204))] } result table: 11, state tables: [] StreamExchange Single from 7 Fragment 7 - 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] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr204)] } + StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr204, 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($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 0 { columns: [partsupp_ps_partkey, sum($expr203), sum($expr203)_0], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 1 { columns: [$expr205], primary key: [], value indices: [0], distribution key: [] } + Table 2 { columns: [partsupp_ps_partkey, count, sum($expr203), sum($expr203)_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($expr191))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [count, sum(sum($expr204))], 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] } @@ -2221,9 +2221,9 @@ batch_plan: | BatchExchange { order: [lineitem.l_shipmode ASC], dist: Single } └─BatchSort { order: [lineitem.l_shipmode ASC] } - └─BatchHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr81), sum($expr82)] } + └─BatchHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr85), sum($expr86)] } └─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 $expr81, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr82] } + └─BatchProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr85, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr86] } └─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($expr127), sum($expr128)] } - └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr127), sum($expr128)] } + └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr135), sum($expr136)] } + └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr135), sum($expr136)] } └─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 $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] } + └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr135, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr136, 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($expr127), sum($expr128)] } - StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr127), sum($expr128)] } + StreamProject { exprs: [lineitem.l_shipmode, sum($expr135), sum($expr136)] } + StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr135), sum($expr136)] } 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 $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] } + StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr135, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr136, 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($expr127), sum($expr128)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [lineitem_l_shipmode, count, sum($expr135), sum($expr136)], 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] } @@ -2415,11 +2415,11 @@ ├─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($expr121))) / sum(sum($expr122))) as $expr123] } - └─BatchSimpleAgg { aggs: [sum(sum($expr121)), sum(sum($expr122))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum($expr127))) / sum(sum($expr128))) as $expr129] } + └─BatchSimpleAgg { aggs: [sum(sum($expr127)), sum(sum($expr128))] } └─BatchExchange { order: [], dist: Single } - └─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] } + └─BatchSimpleAgg { aggs: [sum($expr127), sum($expr128)] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr127, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr128] } └─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($expr190))) / sum(sum($expr191))) as $expr193] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr190)), sum(sum($expr191))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr203))) / sum(sum($expr204))) as $expr205] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr203)), sum(sum($expr204))] } └─StreamExchange { dist: Single } - └─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] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr203), sum($expr204)] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr203, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr204, 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($expr190))) / sum(sum($expr191))) as $expr193] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr190)), sum(sum($expr191))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum($expr203))) / sum(sum($expr204))) as $expr205] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr203)), sum(sum($expr204))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - 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] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr203), sum($expr204)] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr203, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr204, 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($expr190)), sum(sum($expr191))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr203)), sum(sum($expr204))], 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] } @@ -2546,68 +2546,68 @@ batch_plan: | BatchExchange { order: [supplier.s_suppkey ASC], dist: Single } └─BatchSort { order: [supplier.s_suppkey ASC] } - └─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)] } + └─BatchHashJoin { type: Inner, predicate: sum($expr77) = max(max(sum($expr76))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr77)] } + ├─BatchExchange { order: [], dist: HashShard(sum($expr77)) } + | └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr77)] } | ├─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($expr73)] } + | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr77)] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } - | └─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)) } + | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr77] } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─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($expr72)))) } - └─BatchSimpleAgg { aggs: [max(max(sum($expr72)))] } + └─BatchExchange { order: [], dist: HashShard(max(max(sum($expr76)))) } + └─BatchSimpleAgg { aggs: [max(max(sum($expr76)))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [max(sum($expr72))] } - └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr72)] } + └─BatchSimpleAgg { aggs: [max(sum($expr76))] } + └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr76)] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } - └─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)) } + └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr76] } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─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($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] } + 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] } | ├─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($expr111)] } - | └─StreamShare { id = 907 } - | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } - | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr111)] } + | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } + | └─StreamShare { id = 993 } + | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr121)] } + | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr121)] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - | └─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)) } + | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr121, lineitem.l_orderkey, lineitem.l_linenumber] } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(max(max(sum($expr111)))) } - └─StreamProject { exprs: [max(max(sum($expr111)))] } - └─StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr111)))] } + └─StreamExchange { dist: HashShard(max(max(sum($expr115)))) } + └─StreamProject { exprs: [max(max(sum($expr115)))] } + └─StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr115)))] } └─StreamExchange { dist: Single } - └─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)] } + └─StreamHashAgg { group_key: [$expr122], aggs: [count, max(sum($expr115))] } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115), Vnode(lineitem.l_suppkey) as $expr122] } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } + └─StreamShare { id = 993 } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr121)] } + └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr121)] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - └─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)) } + └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr121, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], 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($expr111)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr111)))] } + 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)))] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: sum($expr111) = max(max(sum($expr111))), output: all } + StreamHashJoin { type: Inner, predicate: sum($expr115) = max(max(sum($expr115))), 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($expr111), 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($expr115), 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($expr111)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } StreamExchange Hash([0]) from 3 Fragment 2 @@ -2616,45 +2616,45 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } - StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr111)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr121)] } + StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr121)] } 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 $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)) } + StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr121, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } 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($expr111)))] } - StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr111)))] } + StreamProject { exprs: [max(max(sum($expr115)))] } + StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr115)))] } result table: 10, state tables: [9] StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr112], aggs: [count, max(sum($expr111))] } + StreamHashAgg { group_key: [$expr122], aggs: [count, max(sum($expr115))] } result table: 12, state tables: [11] - StreamProject { exprs: [lineitem.l_suppkey, sum($expr111), Vnode(lineitem.l_suppkey) as $expr112] } - StreamProject { exprs: [lineitem.l_suppkey, sum($expr111)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr115), Vnode(lineitem.l_suppkey) as $expr122] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr115)] } StreamExchange Hash([0]) from 3 - 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 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 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($expr111)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 6 { columns: [lineitem_l_suppkey, sum($expr115)], 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($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] } + Table 8 { columns: [lineitem_l_suppkey, count, sum($expr121)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 9 { columns: [max(sum($expr115)), $expr122], 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: [$expr122, 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: [$expr122, 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] } - id: tpch_q16 before: - create_tables @@ -2841,18 +2841,18 @@ | └─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 $expr88] } + BatchProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr92] } └─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 < $expr87) } + └─BatchFilter { predicate: (lineitem.l_quantity < $expr91) } └─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 $expr87] } + └─BatchProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr91] } └─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 $expr139] } + └─StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr147] } └─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 < $expr137) } + └─StreamFilter { predicate: (lineitem.l_quantity < $expr146) } └─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 $expr137] } + └─StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr146] } └─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 $expr139] } + StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr147] } 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 < $expr137) } + StreamFilter { predicate: (lineitem.l_quantity < $expr146) } 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 $expr137] } + StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr146] } 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, $expr137], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 3 { columns: [part_p_partkey, $expr146], 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] } @@ -3060,7 +3060,7 @@ └─StreamTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0, group_key: [6] } - └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } + └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr3] } └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } └─StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [count, sum(lineitem.l_quantity)] } └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } @@ -3091,7 +3091,7 @@ Fragment 1 StreamGroupTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0, group_key: [6] } state table: 1 - StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } + StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr3] } StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [count, sum(lineitem.l_quantity)] } result table: 2, state tables: [] @@ -3134,8 +3134,8 @@ Upstream BatchPlanNode - Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } - Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr1], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } + Table 0 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [] } + Table 1 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, sum(lineitem_l_quantity), $expr3], primary key: [$6 ASC, $4 DESC, $3 ASC, $0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2, 3, 4, 5, 6], distribution key: [2], vnode column idx: 6 } Table 2 { columns: [customer_c_name, customer_c_custkey, orders_o_orderkey, orders_o_orderdate, orders_o_totalprice, count, sum(lineitem_l_quantity)], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5, 6], distribution key: [2] } Table 3 { columns: [customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$2 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8], distribution key: [2] } Table 4 { columns: [orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC], value indices: [5], distribution key: [0] } @@ -3205,10 +3205,10 @@ ├─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($expr41))] } + BatchSimpleAgg { aggs: [sum(sum($expr43))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum($expr41)] } - └─BatchProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr41] } + └─BatchSimpleAgg { aggs: [sum($expr43)] } + └─BatchProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr43] } └─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($expr65))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr65))] } + └─StreamProject { exprs: [sum(sum($expr69))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr69))] } └─StreamExchange { dist: Single } - └─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] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr69)] } + └─StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr69, 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($expr65))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr65))] } + StreamProject { exprs: [sum(sum($expr69))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr69))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - 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] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr69)] } + StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr69, 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($expr65))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr69))], 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] } @@ -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: ($expr67 > $expr66) } + └─BatchFilter { predicate: ($expr71 > $expr70) } └─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 $expr67] } + | └─BatchProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr71] } | └─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 $expr66] } + └─BatchProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr70] } └─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: ($expr105 > $expr104) } + └─StreamFilter { predicate: ($expr112 > $expr113) } └─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 $expr105] } + | └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr112] } | └─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 $expr104] } + └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr113] } └─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: ($expr105 > $expr104) } + StreamFilter { predicate: ($expr112 > $expr113) } 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 $expr104] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr113] } 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 $expr105] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr112] } 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, $expr105], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr112], 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, $expr104], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr113], 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] } @@ -3615,7 +3615,7 @@ └─StreamTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } - └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } + └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr3] } └─StreamProject { exprs: [supplier.s_name, count] } └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count, count] } └─StreamExchange { dist: HashShard(supplier.s_name) } @@ -3658,7 +3658,7 @@ Fragment 1 StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } state table: 1 - StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } + StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr3] } StreamProject { exprs: [supplier.s_name, count] } StreamHashAgg { group_key: [supplier.s_name], aggs: [count, count] } result table: 2, state tables: [] @@ -3726,8 +3726,8 @@ Upstream BatchPlanNode - Table 0 { columns: [supplier_s_name, count, $expr1], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } - Table 1 { columns: [supplier_s_name, count, $expr1], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } + Table 0 { columns: [supplier_s_name, count, $expr3], primary key: [$1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [] } + Table 1 { columns: [supplier_s_name, count, $expr3], primary key: [$2 ASC, $1 DESC, $0 ASC], value indices: [0, 1, 2], distribution key: [0], vnode column idx: 2 } Table 2 { columns: [supplier_s_name, count, count_0], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } Table 3 { columns: [supplier_s_name, lineitem_l_orderkey, lineitem_l_suppkey, supplier_s_suppkey, lineitem_l_linenumber, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey], primary key: [$1 ASC, $3 ASC, $4 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7], distribution key: [1] } Table 4 { columns: [lineitem_l_orderkey, supplier_s_suppkey, lineitem_l_linenumber, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC], value indices: [7], distribution key: [0] } @@ -3819,12 +3819,12 @@ └─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: [$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: [$expr88 ASC], dist: Single } + └─BatchSort { order: [$expr88 ASC] } + └─BatchHashAgg { group_key: [$expr88], aggs: [count, sum(customer.c_acctbal)] } + └─BatchExchange { order: [], dist: HashShard($expr88) } + └─BatchProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr88, customer.c_acctbal] } + └─BatchNestedLoopJoin { type: Inner, predicate: (customer.c_acctbal > $expr87), 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 $expr83] } + └─BatchProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr87] } └─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: [$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] } + └─StreamProject { exprs: [$expr140, count, sum(customer.c_acctbal)] } + └─StreamHashAgg { group_key: [$expr140], aggs: [count, count, sum(customer.c_acctbal)] } + └─StreamExchange { dist: HashShard($expr140) } + └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr140, customer.c_acctbal, customer.c_custkey] } + └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr139), 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 $expr131] } + └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))) as $expr139] } └─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/update.yaml b/src/frontend/planner_test/tests/testdata/update.yaml index 17c3d7f54e329..1ad3fc52f2abf 100644 --- a/src/frontend/planner_test/tests/testdata/update.yaml +++ b/src/frontend/planner_test/tests/testdata/update.yaml @@ -55,7 +55,7 @@ └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [t.v1, t.v2, (t.v2 + 1:Int32) as $expr45, (t.v1 - 1:Int32) as $expr46] } + └─BatchProject { exprs: [t.v1, t.v2, (t.v2 + 1:Int32) as $expr49, (t.v1 - 1:Int32) as $expr50] } └─BatchUpdate { table: t, exprs: [($1 + 1:Int32), ($0 - 1:Int32), $2], returning: true } └─BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (t.v1 <> t.v2) } diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index a0237485407cc..3ade3571598c9 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 $expr25, _row_id], watermark_columns: [(v1 - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr27, _row_id], watermark_columns: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '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"] } From 375b87c868f2f7ca915268e32afecfb1cb568e6b Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Mon, 20 Feb 2023 12:08:33 +0800 Subject: [PATCH 18/30] fix --- src/frontend/src/optimizer/mod.rs | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index b3eb94f8075b0..16c916ff45108 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -483,7 +483,7 @@ impl PlanRoot { plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?; if ctx.is_explain_trace() { - ctx.trace("Const eval exprs:"); + ctx.trace("Inline Session Timezone:"); ctx.trace(plan.explain_to_string().unwrap()); } @@ -651,7 +651,7 @@ impl PlanRoot { plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?; if ctx.is_explain_trace() { - ctx.trace("Const eval exprs:"); + ctx.trace("Inline session timezone:"); ctx.trace(plan.explain_to_string().unwrap()); } From d3070b6e1005f7243e3f4416f709dd1b71c2a0ec Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Mon, 20 Feb 2023 12:14:54 +0800 Subject: [PATCH 19/30] minor --- src/frontend/src/optimizer/plan_node/logical_source.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index a4f0a9ba8876a..5497f72588c54 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -22,7 +22,7 @@ use risingwave_common::catalog::{ColumnDesc, Schema}; use risingwave_common::error::Result; use risingwave_connector::source::DataType; -use super::generic::{GenericPlanNode, GenericPlanRef}; +use super::generic::GenericPlanNode; use super::stream_watermark_filter::StreamWatermarkFilter; use super::{ generic, BatchSource, ColPrunable, ExprRewritable, LogicalFilter, LogicalProject, PlanBase, From ecd865a1ef8a4e943d013b4395afb76f4e730dad Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Mon, 20 Feb 2023 12:32:09 +0800 Subject: [PATCH 20/30] minor --- e2e_test/batch/functions/now.slt.part | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index 5adb66c83db2a..7c2eb82f45921 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -14,8 +14,7 @@ insert into t values(now()); query T explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; ---- -BatchProject { exprs: [true:Boolean] } - └─BatchValues { rows: [[]] } +BatchValues { rows: [[true:Boolean]] } statement ok drop table tz From c10eed3ab56d7751d60e3e1411feb853c19f224b Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Mon, 20 Feb 2023 15:37:24 +0800 Subject: [PATCH 21/30] minor --- src/frontend/planner_test/tests/testdata/explain.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 12665fed66d9e..4d3a6b8b4324d 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -42,7 +42,7 @@ LogicalValues { rows: [[1:Int32]], schema: Schema { fields: [1:Int32:Int32] } } - Const eval exprs: + Inline Session Timezone: BatchValues { rows: [[1:Int32]] } From 596068230870d60c4ef007046b643e9a9d70e20d Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Tue, 21 Feb 2023 17:39:06 +0800 Subject: [PATCH 22/30] fix --- e2e_test/source/basic/kafka_batch.slt | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/e2e_test/source/basic/kafka_batch.slt b/e2e_test/source/basic/kafka_batch.slt index 85e18bfad33e4..20aa4ff74d8f4 100644 --- a/e2e_test/source/basic/kafka_batch.slt +++ b/e2e_test/source/basic/kafka_batch.slt @@ -159,10 +159,8 @@ select count(*) from s5 ---- 100 -statement stream err +statement ok drop table s5 ---- - query I select count(*) from s6 From a8e05f75b90ba051161652df10756eb046a06a9f Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 13:15:07 +0800 Subject: [PATCH 23/30] apply --- .../planner_test/tests/testdata/agg.yaml | 14 +- .../tests/testdata/dynamic_filter.yaml | 12 +- .../planner_test/tests/testdata/explain.yaml | 2 +- .../planner_test/tests/testdata/expr.yaml | 34 +-- .../tests/testdata/index_selection.yaml | 2 +- .../planner_test/tests/testdata/insert.yaml | 2 +- .../planner_test/tests/testdata/join.yaml | 4 +- .../tests/testdata/mv_column_name.yaml | 4 +- .../planner_test/tests/testdata/nexmark.yaml | 114 ++++---- .../tests/testdata/nexmark_source.yaml | 126 ++++----- .../planner_test/tests/testdata/order_by.yaml | 16 +- .../tests/testdata/predicate_pushdown.yaml | 4 +- .../tests/testdata/project_set.yaml | 2 +- .../planner_test/tests/testdata/share.yaml | 16 +- .../tests/testdata/shared_views.yaml | 4 +- .../tests/testdata/stream_dist_agg.yaml | 8 +- .../tests/testdata/temporal_filter.yaml | 12 +- .../planner_test/tests/testdata/tpch.yaml | 250 +++++++++--------- .../tests/testdata/watermark.yaml | 2 +- 19 files changed, 314 insertions(+), 314 deletions(-) diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index b87a217fe6c23..51fe6ac199532 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -715,8 +715,8 @@ └─StreamProject { exprs: [min(min(t.v3)), sum(sum(t.v1)), t.v1, t.v3, t.v2] } └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2], aggs: [count, min(min(t.v3)), sum(sum(t.v1))] } └─StreamExchange { dist: HashShard(t.v1, t.v3, t.v2) } - └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2, $expr3], aggs: [count, min(t.v3), sum(t.v1)] } - └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamHashAgg { group_key: [t.v1, t.v3, t.v2, $expr1], aggs: [count, min(t.v3), sum(t.v1)] } + └─StreamProject { exprs: [t.v1, t.v2, t.v3, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: enable two phase aggregation sql: | @@ -734,8 +734,8 @@ └─StreamProject { exprs: [min(min(t.v1)), sum(sum(t.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, min(min(t.v1)), sum(sum(t.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr3], aggs: [count, min(t.v1), sum(t.v2)] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, min(t.v1), sum(t.v2)] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: disable two phase aggregation sql: | @@ -1083,21 +1083,21 @@ └─LogicalProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } batch_plan: | - BatchProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Decimal::Float64, Pow(((sum(sum($expr1))::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 $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } + BatchProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Float64, Pow(((sum(sum($expr1))::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 $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } └─BatchSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1), sum(t.v1), count(t.v1), sum($expr1), sum(t.v1), count(t.v1)] } └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } batch_local_plan: | - BatchProject { exprs: [Case((count(t.v1) <= 1:Int64), null:Decimal::Float64, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / (count(t.v1) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / count(t.v1))::Float64, 0.5:Float64) as $expr3] } + BatchProject { exprs: [Case((count(t.v1) <= 1:Int64), null:Float64, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / (count(t.v1) - 1:Int64))::Float64, 0.5:Float64)) as $expr2, Pow(((sum($expr1)::Decimal - ((sum(t.v1)::Decimal * sum(t.v1)::Decimal) / count(t.v1))) / count(t.v1))::Float64, 0.5:Float64) as $expr3] } └─BatchSimpleAgg { aggs: [sum($expr1), sum(t.v1), count(t.v1), sum($expr1), sum(t.v1), count(t.v1)] } └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [t.v1, (t.v1 * t.v1) as $expr1] } └─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($expr1))::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 $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } + └─StreamProject { exprs: [Case((sum0(count(t.v1)) <= 1:Int64), null:Float64, Pow(((sum(sum($expr1))::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 $expr2, Pow(((sum(sum($expr1))::Decimal - ((sum(sum(t.v1))::Decimal * sum(sum(t.v1))::Decimal) / sum0(count(t.v1)))) / sum0(count(t.v1)))::Float64, 0.5:Float64) as $expr3] } └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1)), sum(sum($expr1)), sum(sum(t.v1)), sum0(count(t.v1))] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1), sum(t.v1), count(t.v1), sum($expr1), sum(t.v1), count(t.v1)] } diff --git a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml index 5f7197bc47471..a7843a952cfc5 100644 --- a/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/dynamic_filter.yaml @@ -21,8 +21,8 @@ └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr3] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: | With Top-1 on inner side @@ -117,8 +117,8 @@ └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr3] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Dynamic filter join on unequal types sql: | @@ -157,8 +157,8 @@ └─StreamProject { exprs: [max(max(t2.v2))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t2.v2))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t2.v2)] } - └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr3] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t2.v2)] } + └─StreamProject { exprs: [t2.v2, t2._row_id, Vnode(t2._row_id) as $expr1] } └─StreamTableScan { table: t2, columns: [t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: Complex expression on RHS of condition will still result in dynamic filter before: diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 12a0af7ae0b1e..2c8a6138f7877 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -63,7 +63,7 @@ "stages": { "0": { "root": { - "plan_node_id": 30, + "plan_node_id": 34, "plan_node_type": "BatchValues", "schema": [ { diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index ecf1898f5bf63..855c1b89c8d47 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -29,7 +29,7 @@ select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean)))))) as $expr1] } + └─BatchProject { exprs: [false:Boolean] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -80,7 +80,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1)))) as $expr1] } + BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1)))) as $expr1] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -165,8 +165,8 @@ - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar) as $expr1] } - └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } + BatchProject { exprs: [0:Int32] } + └─BatchFilter { predicate: false:Boolean } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -174,11 +174,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal) as $expr1] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal) as $expr1] } └─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 $expr1, t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal) as $expr1, 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: | @@ -186,7 +186,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal) as $expr1] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -194,7 +194,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal) as $expr1] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -260,18 +260,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar) as $expr1] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar) as $expr1] } └─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 $expr1, t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar) as $expr1, 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); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar) as $expr1] } + └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -286,11 +286,11 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar) as $expr1] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar) as $expr1] } └─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 $expr1, t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar) as $expr1, 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); @@ -406,7 +406,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr35, Array(2:Int32)))) as $expr1] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat($expr35, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } @@ -429,7 +429,7 @@ └─LogicalProject { exprs: [Array(1:Int32) as $expr1] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat($expr35, Array(2:Int32)))) as $expr1] } + BatchProject { exprs: [All((1:Int32 < ArrayCat($expr35, ARRAY[2]:List { datatype: Int32 }))) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } @@ -460,7 +460,7 @@ └─StreamDynamicFilter { predicate: (t.v1 >= $expr1), 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 $expr1], watermark_columns: [(now - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval)] } └─StreamNow { output: [now] } - name: and of two now expression condition sql: | @@ -532,7 +532,7 @@ select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1) as $expr1] } + └─BatchProject { exprs: [(3:Int32 + t.v1) as $expr1] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index 6f5f87da0036f..fe01e572efff1 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -618,7 +618,7 @@ └─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 $expr3] } + └─StreamProject { exprs: [t1.a, Vnode(t1.a) as $expr1] } └─StreamTableScan { table: t1, columns: [t1.a], pk: [t1.a], dist: UpstreamHashShard(t1.a) } - sql: | create table t1 (a varchar, b int, c int, d int); diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 68260954e6d53..423f8e0e97f7e 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -171,7 +171,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time as $expr1, 11:Int32, 4.5:Decimal::Float32 as $expr2] } + └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/join.yaml b/src/frontend/planner_test/tests/testdata/join.yaml index eec71f23fcdc5..daa293cb1fd10 100644 --- a/src/frontend/planner_test/tests/testdata/join.yaml +++ b/src/frontend/planner_test/tests/testdata/join.yaml @@ -499,7 +499,7 @@ └─BatchExchange { order: [], dist: HashShard(b.x) } └─BatchScan { table: b, columns: [b.x], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [y, z, $expr144(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: [$expr144, a._row_id, b._row_id, a.x, b.x] } + StreamMaterialize { columns: [y, z, $expr156(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: [$expr156, 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 $expr1, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr2, (Coalesce(a.x, b.x) + Coalesce(a.x, b.x)) as $expr3, a._row_id, b._row_id, a.x, b.x] } └─StreamFilter { predicate: ((2:Int32 * Coalesce(a.x, b.x)) < 10:Int32) } @@ -600,7 +600,7 @@ └─BatchExchange { order: [], dist: HashShard(t2.v2) } └─BatchScan { table: t2, columns: [t2.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr25(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr25, v2] } + StreamMaterialize { columns: [v1, v2, t1._row_id(hidden), $expr29(hidden), t2._row_id(hidden)], pk_columns: [t1._row_id, t2._row_id, $expr29, v2] } └─StreamHashJoin { type: Inner, predicate: $expr1 IS NOT DISTINCT FROM t2.v2, output: [t1.v1, t2.v2, t1._row_id, $expr1, t2._row_id] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [t1.v1, t1.v1::Int64 as $expr1, t1._row_id] } 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 ce0c39180cbf7..d11c80d3d0240 100644 --- a/src/frontend/planner_test/tests/testdata/mv_column_name.yaml +++ b/src/frontend/planner_test/tests/testdata/mv_column_name.yaml @@ -57,6 +57,6 @@ └─StreamProject { exprs: [sum0(count), max(max(t.a))] } └─StreamGlobalSimpleAgg { aggs: [count, sum0(count), max(max(t.a))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr3], aggs: [count, count, max(t.a)] } - └─StreamProject { exprs: [t.a, t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, count, max(t.a)] } + └─StreamProject { exprs: [t.a, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.a, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index 493fe58b76b52..1c98f168e4275 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark.yaml @@ -76,7 +76,7 @@ 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 $expr48, bid.date_time, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr52, 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 @@ -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 $expr49] } + StreamProject { exprs: [auction.category, (sum(max(bid.price)) / count(max(bid.price))) as $expr53] } 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,7 +408,7 @@ └─BatchProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, 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), $expr103(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr103, price, max(bid.price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)] } └─StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr1, max(bid.price)] } └─StreamFilter { predicate: (bid.date_time >= $expr2) AND (bid.date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: bid.price = max(bid.price), output: all } @@ -422,10 +422,10 @@ └─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), $expr103(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr103, price, max(bid.price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, bid._row_id(hidden), $expr112(hidden), max(bid.price)(hidden)], pk_columns: [bid._row_id, $expr112, price, max(bid.price)] } materialized table: 4294967294 - StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr103, max(bid.price)] } - StreamFilter { predicate: (bid.date_time >= $expr105) AND (bid.date_time <= $expr103) } + StreamProject { exprs: [bid.auction, bid.price, bid.bidder, bid.date_time, bid._row_id, $expr112, max(bid.price)] } + StreamFilter { predicate: (bid.date_time >= $expr113) AND (bid.date_time <= $expr112) } 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), $expr103, ($expr103 - '00:00:10':Interval) as $expr105] } - StreamAppendOnlyHashAgg { group_key: [$expr103], aggs: [count, max(bid.price)] } + StreamProject { exprs: [max(bid.price), $expr112, ($expr112 - '00:00:10':Interval) as $expr113] } + StreamAppendOnlyHashAgg { group_key: [$expr112], 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 $expr103, bid.price, bid._row_id] } + StreamProject { exprs: [(TumbleStart(bid.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr112, 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), $expr103, $expr105], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(bid_price), $expr103, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } - Table 4 { columns: [$expr103, 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, $expr103, 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), $expr112, $expr113], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(bid_price), $expr112, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 4 { columns: [$expr112, 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, $expr112, 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,7 +502,7 @@ └─BatchProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [id, name, starttime, $expr200(hidden), auction.seller(hidden), $expr197(hidden), $expr198(hidden)], pk_columns: [id, name, starttime, $expr200, auction.seller, $expr197, $expr198] } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216] } └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } | └─StreamProject { exprs: [person.id, person.name, $expr1, $expr2] } @@ -516,38 +516,38 @@ └─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, $expr200(hidden), auction.seller(hidden), $expr197(hidden), $expr198(hidden)], pk_columns: [id, name, starttime, $expr200, auction.seller, $expr197, $expr198] } + StreamMaterialize { columns: [id, name, starttime, $expr214(hidden), auction.seller(hidden), $expr215(hidden), $expr216(hidden)], pk_columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr199 = $expr197 AND $expr200 = $expr198, output: all } + StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr213 = $expr215 AND $expr214 = $expr216, 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, $expr197, $expr198] } - StreamHashAgg { group_key: [auction.seller, $expr197, $expr198], aggs: [count] } + StreamProject { exprs: [auction.seller, $expr215, $expr216] } + StreamHashAgg { group_key: [auction.seller, $expr215, $expr216], aggs: [count] } result table: 5, state tables: [] StreamExchange Hash([0, 1, 2]) from 2 Fragment 1 - StreamProject { exprs: [person.id, person.name, $expr199, $expr200] } - StreamHashAgg { group_key: [person.id, person.name, $expr199, $expr200], aggs: [count] } + StreamProject { exprs: [person.id, person.name, $expr213, $expr214] } + StreamHashAgg { group_key: [person.id, person.name, $expr213, $expr214], aggs: [count] } result table: 4, state tables: [] - StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr199, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr200] } + StreamProject { exprs: [person.id, person.name, TumbleStart(person.date_time, '00:00:10':Interval) as $expr213, (TumbleStart(person.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr214] } 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 $expr197, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr198, auction.id] } + StreamProject { exprs: [auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr215, (TumbleStart(auction.date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr216, 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, $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: [person_id, $expr199, $expr200, 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, $expr197, $expr198], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [auction_seller, $expr197, $expr198, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [person_id, person_name, $expr199, $expr200, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } - Table 5 { columns: [auction_seller, $expr197, $expr198, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4294967294 { columns: [id, name, starttime, $expr200, auction.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] } + Table 0 { columns: [person_id, person_name, $expr213, $expr214], 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, $expr213, $expr214, 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, $expr215, $expr216], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [auction_seller, $expr215, $expr216, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [person_id, person_name, $expr213, $expr214, count], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0] } + Table 5 { columns: [auction_seller, $expr215, $expr216, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4294967294 { columns: [id, name, starttime, $expr214, auction.seller, $expr215, $expr216], 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 @@ -643,7 +643,7 @@ 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 $expr95, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr96, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr103, ToChar(bid.date_time, 'HH:MI':Varchar) as $expr104, 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 @@ -732,7 +732,7 @@ 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 $expr95, 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 $expr96, bid.date_time, bid.extra, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, (0.908:Decimal * bid.price) as $expr103, 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 $expr104, 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 @@ -784,26 +784,26 @@ 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: [$expr48, 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: [$expr48], 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: [$expr52, 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: [$expr52], 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: [$expr48, 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: [$expr48, 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: [$expr52, 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: [$expr52, 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: [[$expr48], [$expr48, bid.bidder], [$expr48, bid.auction]] } - StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr48, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamExpand { column_subsets: [[$expr52], [$expr52, bid.bidder], [$expr52, bid.auction]] } + StreamProject { exprs: [ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr52, 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: [$expr48, 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: [$expr48, 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: [$expr52, 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: [$expr52, 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: @@ -852,27 +852,27 @@ 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, $expr95, max(max($expr96)) 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, $expr95], aggs: [count, max(max($expr96)) 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, $expr103, max(max($expr104)) 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, $expr103], aggs: [count, max(max($expr104)) 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, $expr95, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr96), 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, $expr95, bid.bidder, bid.auction, flag], aggs: [count, max($expr96), 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, $expr103, bid.bidder, bid.bidder, bid.bidder, bid.bidder, bid.auction, bid.auction, bid.auction, bid.auction, flag, max($expr104), 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, $expr103, bid.bidder, bid.auction, flag], aggs: [count, max($expr104), 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, $expr95, $expr96], [bid.channel, $expr95, bid.bidder], [bid.channel, $expr95, bid.auction]] } - StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr95, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr96, bid.price, bid.bidder, bid.auction, bid._row_id] } + StreamExpand { column_subsets: [[bid.channel, $expr103, $expr104], [bid.channel, $expr103, bid.bidder], [bid.channel, $expr103, bid.auction]] } + StreamProject { exprs: [bid.channel, ToChar(bid.date_time, 'yyyy-MM-dd':Varchar) as $expr103, ToChar(bid.date_time, 'HH:mm':Varchar) as $expr104, 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, $expr95, max($expr96), 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, $expr95, count, max(max($expr96)) 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, $expr95, bid_bidder, bid_auction, flag, count, max($expr96), 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, $expr103, max($expr104), 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, $expr103, count, max(max($expr104)) 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, $expr103, bid_bidder, bid_auction, flag, count, max($expr104), 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: @@ -909,18 +909,18 @@ 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, $expr95, 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 $expr97, sum(bid.price)] } - StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr95], 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, $expr104, 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 $expr105, sum(bid.price)] } + StreamAppendOnlyHashAgg { group_key: [bid.auction, $expr104], 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 $expr95, bid.price, bid._row_id] } + StreamProject { exprs: [bid.auction, ToChar(bid.date_time, 'YYYY-MM-DD':Varchar) as $expr104, 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, $expr95, 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, $expr104, 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: @@ -1075,7 +1075,7 @@ 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 $expr142, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr143, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr144, bid._row_id] } + StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.channel, SplitPart(bid.url, '/':Varchar, 4:Int32) as $expr154, SplitPart(bid.url, '/':Varchar, 5:Int32) as $expr155, SplitPart(bid.url, '/':Varchar, 6:Int32) as $expr156, 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 @@ -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) >= $expr49), output: [auction.id, auction.item_name, count(bid.auction), count(bid.auction)] } + StreamDynamicFilter { predicate: (count(bid.auction) >= $expr53), 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 $expr49] } + StreamProject { exprs: [(sum0(count) / count(bid.auction)) as $expr53] } 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: [$expr49], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr53], 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] } @@ -1417,7 +1417,7 @@ └─StreamTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[count(bid.auction) DESC]", limit: 1000, offset: 0, group_key: [3] } - └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr3] } + └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction), Vnode(auction.id) as $expr1] } └─StreamProject { exprs: [auction.id, auction.item_name, count(bid.auction)] } └─StreamHashAgg { group_key: [auction.id, auction.item_name], aggs: [count, count(bid.auction)] } └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } diff --git a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml index 453230650de5c..a1ec0d8bcefd7 100644 --- a/src/frontend/planner_test/tests/testdata/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/nexmark_source.yaml @@ -97,7 +97,7 @@ StreamExchange Hash([4]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr50, date_time, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr54, 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 @@ -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 $expr49] } + StreamProject { exprs: [category, (sum(max(price))::Decimal / count(max(price))) as $expr53] } 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 = 854 } + | └─StreamShare { id = 1066 } | └─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 = 854 } + └─StreamShare { id = 1066 } └─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,13 +474,13 @@ └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, 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), $expr112(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr112, price, max(price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr121(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr121, price, max(price)] } └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1, max(price)] } └─StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } ├─StreamExchange { dist: HashShard(price) } | └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } - | └─StreamShare { id = 503 } + | └─StreamShare { id = 641 } | └─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"] } @@ -489,16 +489,16 @@ └─StreamAppendOnlyHashAgg { group_key: [$expr1], aggs: [count, max(price)] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } - └─StreamShare { id = 503 } + └─StreamShare { id = 641 } └─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), $expr112(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr112, price, max(price)] } + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr121(hidden), max(price)(hidden)], pk_columns: [_row_id, $expr121, price, max(price)] } materialized table: 4294967294 - StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr112, max(price)] } - StreamFilter { predicate: (date_time >= $expr114) AND (date_time <= $expr112) } + StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr121, max(price)] } + StreamFilter { predicate: (date_time >= $expr122) AND (date_time <= $expr121) } 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), $expr112, ($expr112 - '00:00:10':Interval) as $expr114] } - StreamAppendOnlyHashAgg { group_key: [$expr112], aggs: [count, max(price)] } + StreamProject { exprs: [max(price), $expr121, ($expr121 - '00:00:10':Interval) as $expr122] } + StreamAppendOnlyHashAgg { group_key: [$expr121], 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 $expr112, price, _row_id] } + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr121, 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), $expr112, $expr114], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 3 { columns: [max(price), $expr112, _degree], primary key: [$0 ASC, $1 ASC], value indices: [2], distribution key: [0] } + Table 2 { columns: [max(price), $expr121, $expr122], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 3 { columns: [max(price), $expr121, _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: [$expr112, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr112, 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: [$expr121, count, max(price)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 4294967294 { columns: [auction, price, bidder, date_time, _row_id, $expr121, 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,7 +580,7 @@ └─BatchProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr3, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr4] } └─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, $expr208(hidden), seller(hidden), $expr205(hidden), $expr206(hidden)], pk_columns: [id, name, starttime, $expr208, seller, $expr205, $expr206] } + StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224] } └─StreamHashJoin { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } | └─StreamProject { exprs: [id, name, $expr1, $expr2] } @@ -597,43 +597,43 @@ └─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, $expr208(hidden), seller(hidden), $expr205(hidden), $expr206(hidden)], pk_columns: [id, name, starttime, $expr208, seller, $expr205, $expr206] } + StreamMaterialize { columns: [id, name, starttime, $expr222(hidden), seller(hidden), $expr223(hidden), $expr224(hidden)], pk_columns: [id, name, starttime, $expr222, seller, $expr223, $expr224] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: id = seller AND $expr207 = $expr205 AND $expr208 = $expr206, output: all } + StreamHashJoin { type: Inner, predicate: id = seller AND $expr221 = $expr223 AND $expr222 = $expr224, 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, $expr205, $expr206] } - StreamAppendOnlyHashAgg { group_key: [seller, $expr205, $expr206], aggs: [count] } + StreamProject { exprs: [seller, $expr223, $expr224] } + StreamAppendOnlyHashAgg { group_key: [seller, $expr223, $expr224], aggs: [count] } result table: 6, state tables: [] StreamExchange Hash([0, 1, 2]) from 3 Fragment 1 - StreamProject { exprs: [id, name, $expr207, $expr208] } - StreamAppendOnlyHashAgg { group_key: [id, name, $expr207, $expr208], aggs: [count] } + StreamProject { exprs: [id, name, $expr221, $expr222] } + StreamAppendOnlyHashAgg { group_key: [id, name, $expr221, $expr222], 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 $expr207, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr208, _row_id] } + StreamProject { exprs: [id, name, TumbleStart(date_time, '00:00:10':Interval) as $expr221, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr222, _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 $expr205, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr206, _row_id] } + StreamProject { exprs: [seller, TumbleStart(date_time, '00:00:10':Interval) as $expr223, (TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr224, _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, $expr207, $expr208], 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, $expr207, $expr208, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } - Table 2 { columns: [seller, $expr205, $expr206], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } - Table 3 { columns: [seller, $expr205, $expr206, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } - Table 4 { columns: [id, name, $expr207, $expr208, 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, $expr221, $expr222], 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, $expr221, $expr222, name, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC], value indices: [4], distribution key: [0, 1, 2] } + Table 2 { columns: [seller, $expr223, $expr224], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [0, 1, 2], distribution key: [0, 1, 2] } + Table 3 { columns: [seller, $expr223, $expr224, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 4 { columns: [id, name, $expr221, $expr222, 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, $expr205, $expr206, count], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0, 1, 2] } + Table 6 { columns: [seller, $expr223, $expr224, 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, $expr208, seller, $expr205, $expr206], 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, $expr222, seller, $expr223, $expr224], 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 @@ -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 $expr99, ToChar(date_time, 'HH:MI':Varchar) as $expr100, _row_id] } + StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr107, ToChar(date_time, 'HH:MI':Varchar) as $expr108, _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 @@ -833,7 +833,7 @@ StreamExchange Hash([6]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr99, 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 $expr100, date_time, extra, _row_id] } + StreamProject { exprs: [auction, bidder, (0.908:Decimal * price) as $expr107, 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 $expr108, 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"] } @@ -887,26 +887,26 @@ 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: [$expr50, 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: [$expr50], 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: [$expr54, 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: [$expr54], 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: [$expr50, 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: [$expr50, 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: [$expr54, 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: [$expr54, 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: [[$expr50], [$expr50, bidder], [$expr50, auction]] } - StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr50, price, bidder, auction, _row_id] } + StreamExpand { column_subsets: [[$expr54], [$expr54, bidder], [$expr54, auction]] } + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr54, 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: [$expr50, 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: [$expr50, 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: [$expr54, 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: [$expr54, 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 @@ -957,27 +957,27 @@ 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, $expr99, max(max($expr100)) 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, $expr99], aggs: [count, max(max($expr100)) 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, $expr107, max(max($expr108)) 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, $expr107], aggs: [count, max(max($expr108)) 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, $expr99, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr100), 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, $expr99, bidder, auction, flag], aggs: [count, max($expr100), 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, $expr107, bidder, bidder, bidder, bidder, auction, auction, auction, auction, flag, max($expr108), 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, $expr107, bidder, auction, flag], aggs: [count, max($expr108), 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, $expr99, $expr100], [channel, $expr99, bidder], [channel, $expr99, auction]] } - StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr99, ToChar(date_time, 'HH:mm':Varchar) as $expr100, price, bidder, auction, _row_id] } + StreamExpand { column_subsets: [[channel, $expr107, $expr108], [channel, $expr107, bidder], [channel, $expr107, auction]] } + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr107, ToChar(date_time, 'HH:mm':Varchar) as $expr108, 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, $expr99, max($expr100), 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, $expr99, count, max(max($expr100)) 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, $expr99, bidder, auction, flag, count, max($expr100), 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, $expr107, max($expr108), 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, $expr107, count, max(max($expr108)) 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, $expr107, bidder, auction, flag, count, max($expr108), 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 @@ -1016,18 +1016,18 @@ 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, $expr97, 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 $expr99, sum(price)] } - StreamAppendOnlyHashAgg { group_key: [auction, $expr97], 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, $expr106, 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 $expr107, sum(price)] } + StreamAppendOnlyHashAgg { group_key: [auction, $expr106], 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 $expr97, price, _row_id] } + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr106, 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, $expr97, 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, $expr106, 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 @@ -1203,7 +1203,7 @@ StreamExchange Hash([7]) from 1 Fragment 1 - StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr148, SplitPart(url, '/':Varchar, 5:Int32) as $expr149, SplitPart(url, '/':Varchar, 6:Int32) as $expr150, _row_id] } + StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr160, SplitPart(url, '/':Varchar, 5:Int32) as $expr161, SplitPart(url, '/':Varchar, 6:Int32) as $expr162, _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 @@ -1331,7 +1331,7 @@ | | └─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 = 648 } + | └─StreamShare { id = 842 } | └─StreamProject { exprs: [auction, _row_id] } | └─StreamRowIdGen { row_id_index: 7 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "channel", "url", "date_time", "extra", "_row_id"] } @@ -1343,7 +1343,7 @@ └─StreamAppendOnlyHashAgg { group_key: [auction], aggs: [count, count] } └─StreamExchange { dist: HashShard(auction) } └─StreamProject { exprs: [auction, _row_id] } - └─StreamShare { id = 648 } + └─StreamShare { id = 842 } └─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) >= $expr51), output: [id, item_name, count(auction), count(auction)] } + StreamDynamicFilter { predicate: (count(auction) >= $expr55), 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 $expr51] } + StreamProject { exprs: [(sum0(count) / count(auction)) as $expr55] } 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: [$expr51], primary key: [], value indices: [0], distribution key: [] } + Table 1 { columns: [$expr55], 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] } @@ -1589,7 +1589,7 @@ └─StreamTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[count(auction) DESC]", limit: 1000, offset: 0, group_key: [3] } - └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr3] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } └─StreamProject { exprs: [id, item_name, count(auction)] } └─StreamAppendOnlyHashAgg { group_key: [id, item_name], aggs: [count, count(auction)] } └─StreamAppendOnlyHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 5ad7501409dd7..69a693306434b 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [$expr1 ASC], dist: Single } - └─BatchSort { order: [$expr1 ASC] } - └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr1] } + └─BatchExchange { order: [2:Int32 ASC], dist: Single } + └─BatchSort { order: [2:Int32 ASC] } + └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, $expr48(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr48, t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32) as $expr1, t._row_id] } + StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, 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); @@ -85,7 +85,7 @@ └─StreamTopN { order: "[t.v1 DESC]", limit: 5, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[t.v1 DESC]", limit: 5, offset: 0, group_key: [3] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr1] } └─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); @@ -117,7 +117,7 @@ └─StreamTopN { order: "[t.v1 DESC]", limit: 5, offset: 7 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[t.v1 DESC]", limit: 12, offset: 0, group_key: [3] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by expression that would be valid in select list sql: | @@ -133,7 +133,7 @@ └─BatchProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z] } └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [x, y, $expr67(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr67, t.z, t._row_id] } + StreamMaterialize { columns: [x, y, $expr71(hidden), t.z(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [$expr71, t.z, t._row_id] } └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: order by the number of an output column diff --git a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml index 292d5dcbcb214..c98c825c600ff 100644 --- a/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml +++ b/src/frontend/planner_test/tests/testdata/predicate_pushdown.yaml @@ -267,7 +267,7 @@ | └─StreamDynamicFilter { predicate: (t1.v1 > $expr1), 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 $expr1], watermark_columns: [(now + '01:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '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) } @@ -306,7 +306,7 @@ ├─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 $expr1], watermark_columns: [(now + '00:30:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '00:30:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '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/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index 82cc46056bea8..03626a2da6ab6 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -99,7 +99,7 @@ └─StreamTopN { order: "[projected_row_id ASC]", limit: 1, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[projected_row_id ASC]", limit: 1, offset: 0, group_key: [3] } - └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as $expr1] } └─StreamProjectSet { select_list: [Unnest($0), $1] } └─StreamTableScan { table: t, columns: [t.x, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: issue-7812 diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index c55790c91f452..0c8619f888f20 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 = 595 } + | └─StreamShare { id = 797 } | └─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 = 595 } + └─StreamShare { id = 797 } └─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 = 854 } + | └─StreamShare { id = 1066 } | └─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 = 854 } + └─StreamShare { id = 1066 } └─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 = 243 } + | └─StreamShare { id = 331 } | └─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 = 243 } + └─StreamShare { id = 331 } └─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 = 177 } + | └─StreamShare { id = 251 } | └─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 = 177 } + └─StreamShare { id = 251 } └─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 2b4d641ebfafa..4e54ec97e0189 100644 --- a/src/frontend/planner_test/tests/testdata/shared_views.yaml +++ b/src/frontend/planner_test/tests/testdata/shared_views.yaml @@ -26,7 +26,7 @@ └─StreamHashJoin { type: Inner, predicate: $expr1 = $expr2, output: [$expr1, $expr2, $expr3, t1._row_id, t1._row_id, t1._row_id, t1.x, $expr1] } ├─StreamExchange { dist: HashShard($expr1) } | └─StreamProject { exprs: [$expr1, t1._row_id] } - | └─StreamShare { id = 250 } + | └─StreamShare { id = 348 } | └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, 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) } @@ -37,7 +37,7 @@ | └─StreamTableScan { table: t1, columns: [t1.x, t1.y, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [$expr1, t1._row_id] } - └─StreamShare { id = 250 } + └─StreamShare { id = 348 } └─StreamProject { exprs: [(t1.x + t1.y) as $expr1, 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/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml index 5931d6a8eace5..ac044ee5576dd 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -134,8 +134,8 @@ └─StreamProject { exprs: [max(max(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v)] } - └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v)] } + └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 @@ -393,8 +393,8 @@ └─StreamProject { exprs: [max(max(t.v)), sum0(count(t.v))] } └─StreamGlobalSimpleAgg { aggs: [count, max(max(t.v)), sum0(count(t.v))] } └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [$expr3], aggs: [count, max(t.v), count(t.v)] } - └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr3] } + └─StreamHashAgg { group_key: [$expr1], aggs: [count, max(t.v), count(t.v)] } + └─StreamProject { exprs: [t.v, t._row_id, Vnode(t._row_id) as $expr1] } └─StreamTableScan { table: t, columns: [t.v, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } stream_dist_plan: | Fragment 0 diff --git a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml index e840753ab2250..41fdd3835a5f1 100644 --- a/src/frontend/planner_test/tests/testdata/temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/temporal_filter.yaml @@ -7,7 +7,7 @@ StreamMaterialize { columns: [ts, t1._row_id(hidden)], pk_columns: [t1._row_id] } └─StreamProject { exprs: [t1.ts, t1._row_id] } └─StreamDynamicFilter { predicate: ($expr1 > now), output: [t1.ts, $expr1, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, (t1.ts + '01:00:00':Interval) as $expr1, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + '00:00:00':Interval), 'UTC':Varchar) + '01:00:00':Interval) as $expr1, 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] } @@ -19,7 +19,7 @@ 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: ($expr1 > now), output: [t1.ts, t1.time_to_live, $expr1, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (t1.ts + (t1.time_to_live * 1.5:Decimal)) as $expr1, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.time_to_live * 1.5:Decimal)))) as $expr1, 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] } @@ -31,10 +31,10 @@ 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: ($expr1 > $expr2), output: [t1.ts, t1.additional_time_to_live, $expr1, t1._row_id] } - ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (t1.ts + (t1.additional_time_to_live * 1.5:Decimal)) as $expr1, t1._row_id] } + ├─StreamProject { exprs: [t1.ts, t1.additional_time_to_live, (AtTimeZone((AtTimeZone(t1.ts, 'UTC':Varchar) + DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal))), 'UTC':Varchar) + ((t1.additional_time_to_live * 1.5:Decimal) - DateTrunc('day':Varchar, (t1.additional_time_to_live * 1.5:Decimal)))) as $expr1, 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 $expr2], watermark_columns: [(now - '00:15:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval) as $expr2], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:15:00':Interval)] } └─StreamNow { output: [now] } - name: Temporal filter fails without `now()` in lower bound sql: |- @@ -51,8 +51,8 @@ ├─StreamDynamicFilter { predicate: (t1.ts >= $expr1), 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 $expr1], watermark_columns: [(now - '02:00:00':Interval)] } + | └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval) as $expr1], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '02:00:00':Interval)] } | └─StreamNow { output: [now] } └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(now - '01:00:00':Interval) as $expr2], watermark_columns: [(now - '01:00:00':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval) as $expr2], watermark_columns: [(AtTimeZone((AtTimeZone(now, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '01:00:00':Interval)] } └─StreamNow { output: [now] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 6b4ac9c592dce..0656da6e52618 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -136,7 +136,7 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), 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 $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─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] } @@ -144,25 +144,25 @@ └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), 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 $expr1, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], 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($expr335), sum($expr336), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr340, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr341, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr342, count] } - StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr335), sum($expr336), 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($expr358), sum($expr359), RoundDigit((sum(lineitem.l_quantity) / count(lineitem.l_quantity)), 4:Int32) as $expr360, RoundDigit((sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)), 4:Int32) as $expr361, RoundDigit((sum(lineitem.l_discount) / count(lineitem.l_discount)), 4:Int32) as $expr362, count] } + StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr358), sum($expr359), 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 $expr335, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr336, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr358, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)) as $expr359, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } 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($expr335), sum($expr336), 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($expr358), sum($expr359), 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: @@ -301,7 +301,7 @@ └─StreamTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC]", limit: 100, offset: 0, group_key: [18] } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr3] } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey, Vnode(nation.n_regionkey) as $expr1] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, part.p_partkey, partsupp.ps_supplycost, min(partsupp.ps_supplycost), nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND partsupp.ps_supplycost = min(partsupp.ps_supplycost), output: [part.p_partkey, part.p_mfgr, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, nation.n_name, nation.n_regionkey, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, partsupp.ps_supplycost, part.p_partkey, min(partsupp.ps_supplycost)] } @@ -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($expr67), orders.o_orderdate, orders.o_shippriority] } - StreamTopN { order: "[sum($expr67) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority] } + StreamTopN { order: "[sum($expr73) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr67) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } + StreamGroupTopN { order: "[sum($expr73) DESC, orders.o_orderdate ASC]", limit: 10, offset: 0, group_key: [4] } state table: 1 - StreamProject { exprs: [lineitem.l_orderkey, sum($expr67), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr68] } - StreamProject { exprs: [lineitem.l_orderkey, sum($expr67), orders.o_orderdate, orders.o_shippriority] } - StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr67)] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr74] } + StreamProject { exprs: [lineitem.l_orderkey, sum($expr73), orders.o_orderdate, orders.o_shippriority] } + StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [count, sum($expr73)] } 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 $expr67, 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 $expr73, 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($expr67), orders_o_orderdate, orders_o_shippriority, $expr68], 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($expr67), orders_o_orderdate, orders_o_shippriority, $expr68], 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($expr67)], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3, 4], distribution key: [0, 1, 2] } + Table 0 { columns: [lineitem_l_orderkey, sum($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], 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($expr73), orders_o_orderdate, orders_o_shippriority, $expr74], 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($expr73)], 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] } @@ -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($expr67)] } - StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr67)] } + StreamProject { exprs: [nation.n_name, sum($expr71)] } + StreamHashAgg { group_key: [nation.n_name], aggs: [count, sum($expr71)] } 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 $expr67, 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 $expr71, 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($expr67)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 0 { columns: [nation_n_name, count, sum($expr71)], 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] } @@ -1020,7 +1020,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1)] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1] } - └─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) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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: [] } @@ -1029,26 +1029,26 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1)] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr1, 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) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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($expr69))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr69))] } + StreamProject { exprs: [sum(sum($expr73))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr73))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr69)] } - StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr69, 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) } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr73)] } + StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount) as $expr73, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09: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($expr69))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr73))], primary key: [], value indices: [0, 1], distribution key: [] } Table 4294967294 { columns: [revenue], primary key: [], value indices: [0], distribution key: [] } - id: tpch_q7 before: @@ -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, $expr141, sum($expr142)] } - StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr141], aggs: [count, sum($expr142)] } + StreamProject { exprs: [nation.n_name, nation.n_name, $expr149, sum($expr150)] } + StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr149], aggs: [count, sum($expr150)] } 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 $expr141, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr142, 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 $expr149, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr150, 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, $expr141, count, sum($expr142)], 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, $expr149, count, sum($expr150)], 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] } @@ -1354,7 +1354,7 @@ └─BatchSort { order: [$expr1 ASC] } └─BatchHashAgg { group_key: [$expr1], aggs: [sum($expr2), sum($expr3)] } └─BatchExchange { order: [], dist: HashShard($expr1) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3] } └─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] } @@ -1379,7 +1379,7 @@ └─StreamProject { exprs: [$expr1, RoundDigit((sum($expr2) / sum($expr3)), 6:Int32) as $expr4] } └─StreamHashAgg { group_key: [$expr1], aggs: [count, sum($expr2), sum($expr3)] } └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, 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 $expr1, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr2, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr3, 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: [$expr274, RoundDigit((sum($expr275) / sum($expr276)), 6:Int32) as $expr278] } - StreamHashAgg { group_key: [$expr274], aggs: [count, sum($expr275), sum($expr276)] } + StreamProject { exprs: [$expr291, RoundDigit((sum($expr292) / sum($expr293)), 6:Int32) as $expr294] } + StreamHashAgg { group_key: [$expr291], aggs: [count, sum($expr292), sum($expr293)] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr274, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr275, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr276, 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 $expr291, Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr292, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr293, 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: [$expr274, count, sum($expr275), sum($expr276)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [$expr291, count, sum($expr292), sum($expr293)], 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] } @@ -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, $expr205, RoundDigit(sum($expr206), 2:Int32) as $expr208] } - StreamHashAgg { group_key: [nation.n_name, $expr205], aggs: [count, sum($expr206)] } + StreamProject { exprs: [nation.n_name, $expr218, RoundDigit(sum($expr219), 2:Int32) as $expr220] } + StreamHashAgg { group_key: [nation.n_name, $expr218], aggs: [count, sum($expr219)] } 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 $expr205, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr206, 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 $expr218, ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr219, 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, $expr205, count, sum($expr206)], primary key: [$0 ASC, $1 ASC], value indices: [2, 3], distribution key: [0, 1] } + Table 0 { columns: [nation_n_name, $expr218, count, sum($expr219)], 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] } @@ -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($expr67), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - StreamTopN { order: "[sum($expr67) DESC]", limit: 20, offset: 0 } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + StreamTopN { order: "[sum($expr73) DESC]", limit: 20, offset: 0 } state table: 0 StreamExchange Single from 1 Fragment 1 - StreamGroupTopN { order: "[sum($expr67) DESC]", limit: 20, offset: 0, group_key: [8] } + StreamGroupTopN { order: "[sum($expr73) DESC]", limit: 20, offset: 0, group_key: [8] } state table: 1 - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr67), 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 $expr68] } - StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr67), 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($expr67)] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), 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 $expr74] } + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr73), 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($expr73)] } 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 $expr67, 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 $expr73, 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($expr67), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr68], 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($expr67), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr68], 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($expr67)], 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($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], 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($expr73), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr74], 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($expr73)], 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] } @@ -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($expr199)] } - StreamDynamicFilter { predicate: (sum($expr199) > $expr202), output: [partsupp.ps_partkey, sum($expr199), sum($expr199)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr212)] } + StreamDynamicFilter { predicate: (sum($expr212) > $expr214), output: [partsupp.ps_partkey, sum($expr212), sum($expr212)] } left table: 0, right table 1 - StreamProject { exprs: [partsupp.ps_partkey, sum($expr199), sum($expr199)] } - StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr199), sum($expr199)] } + StreamProject { exprs: [partsupp.ps_partkey, sum($expr212), sum($expr212)] } + StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [count, sum($expr212), sum($expr212)] } 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 $expr199, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty) as $expr212, 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($expr200)) * 0.0001000000:Decimal) as $expr202] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr200))] } + StreamProject { exprs: [(sum(sum($expr213)) * 0.0001000000:Decimal) as $expr214] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr213))] } result table: 11, state tables: [] StreamExchange Single from 7 Fragment 7 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr200)] } - StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr200, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr213)] } + StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty) as $expr213, 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($expr199), sum($expr199)_0], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } - Table 1 { columns: [$expr202], primary key: [], value indices: [0], distribution key: [] } - Table 2 { columns: [partsupp_ps_partkey, count, sum($expr199), sum($expr199)_0], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [partsupp_ps_partkey, sum($expr212), sum($expr212)_0], primary key: [$2 ASC, $0 ASC], value indices: [0, 1, 2], distribution key: [0] } + Table 1 { columns: [$expr214], primary key: [], value indices: [0], distribution key: [] } + Table 2 { columns: [partsupp_ps_partkey, count, sum($expr212), sum($expr212)_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($expr200))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [count, sum(sum($expr213))], 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] } @@ -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($expr133), sum($expr134)] } - StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr133), sum($expr134)] } + StreamProject { exprs: [lineitem.l_shipmode, sum($expr141), sum($expr142)] } + StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [count, sum($expr141), sum($expr142)] } 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 $expr133, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr134, 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 $expr141, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr142, 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($expr133), sum($expr134)], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [lineitem_l_shipmode, count, sum($expr141), sum($expr142)], 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] } @@ -2419,7 +2419,7 @@ └─BatchSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum($expr1), sum($expr2)] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } └─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] } @@ -2431,7 +2431,7 @@ └─StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr1)), sum(sum($expr2))] } └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr1), sum($expr2)] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2, 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($expr199))) / sum(sum($expr200))) as $expr202] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr199)), sum(sum($expr200))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum($expr212))) / sum(sum($expr213))) as $expr214] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr212)), sum(sum($expr213))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr199), sum($expr200)] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal) as $expr199, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr200, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr212), sum($expr213)] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal) as $expr212, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr213, 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($expr199)), sum(sum($expr200))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr212)), sum(sum($expr213))], 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] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1)] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─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($expr2)))) } └─BatchSimpleAgg { aggs: [max(max(sum($expr2)))] } @@ -2563,22 +2563,22 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr2)] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr2] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─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($expr117)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr117)))] } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr121)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr121)))] } └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: all } ├─StreamExchange { dist: HashShard(sum($expr1)) } | └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - | └─StreamShare { id = 1033 } + | └─StreamShare { id = 1277 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr1)] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } └─StreamProject { exprs: [max(max(sum($expr1)))] } @@ -2587,27 +2587,27 @@ └─StreamHashAgg { group_key: [$expr2], aggs: [count, max(sum($expr1))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamShare { id = 1033 } + └─StreamShare { id = 1277 } └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr1)] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], 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($expr117)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr117)))] } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum($expr121)))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum($expr121)))] } materialized table: 4294967294 - StreamHashJoin { type: Inner, predicate: sum($expr117) = max(max(sum($expr117))), output: all } + StreamHashJoin { type: Inner, predicate: sum($expr121) = max(max(sum($expr121))), 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($expr117), 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($expr121), 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($expr117)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr121)] } StreamExchange Hash([0]) from 3 Fragment 2 @@ -2616,45 +2616,45 @@ BatchPlanNode Fragment 3 - StreamProject { exprs: [lineitem.l_suppkey, sum($expr117)] } - StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr117)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr127)] } + StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum($expr127)] } 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 $expr117, 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)) } + StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr127, lineitem.l_orderkey, lineitem.l_linenumber] } + StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } 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($expr117)))] } - StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr117)))] } + StreamProject { exprs: [max(max(sum($expr121)))] } + StreamGlobalSimpleAgg { aggs: [count, max(max(sum($expr121)))] } result table: 10, state tables: [9] StreamExchange Single from 6 Fragment 6 - StreamHashAgg { group_key: [$expr118], aggs: [count, max(sum($expr117))] } + StreamHashAgg { group_key: [$expr128], aggs: [count, max(sum($expr121))] } result table: 12, state tables: [11] - StreamProject { exprs: [lineitem.l_suppkey, sum($expr117), Vnode(lineitem.l_suppkey) as $expr118] } - StreamProject { exprs: [lineitem.l_suppkey, sum($expr117)] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr121), Vnode(lineitem.l_suppkey) as $expr128] } + StreamProject { exprs: [lineitem.l_suppkey, sum($expr121)] } StreamExchange Hash([0]) from 3 - Table 0 { columns: [supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr117), 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($expr117), 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($expr117)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } - Table 3 { columns: [max(max(sum($expr117))), _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($expr121), 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($expr121), 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($expr121)))], primary key: [$0 ASC], value indices: [0], distribution key: [0] } + Table 3 { columns: [max(max(sum($expr121))), _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($expr117)], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 6 { columns: [lineitem_l_suppkey, sum($expr121)], 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($expr117)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } - Table 9 { columns: [max(sum($expr117)), $expr118], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } - Table 10 { columns: [count, max(max(sum($expr117)))], primary key: [], value indices: [0, 1], distribution key: [] } - Table 11 { columns: [$expr118, sum($expr117), 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: [$expr118, count, max(sum($expr117))], 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($expr117)))], 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($expr127)], primary key: [$0 ASC], value indices: [1, 2], distribution key: [0] } + Table 9 { columns: [max(sum($expr121)), $expr128], primary key: [$0 DESC, $1 ASC], value indices: [0, 1], distribution key: [] } + Table 10 { columns: [count, max(max(sum($expr121)))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 11 { columns: [$expr128, sum($expr121), 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: [$expr128, count, max(sum($expr121))], 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($expr121)))], 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 @@ -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 $expr145] } + StreamProject { exprs: [RoundDigit((sum(sum(lineitem.l_extendedprice)) / 7.0:Decimal), 16:Int32) as $expr153] } 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 < $expr143) } + StreamFilter { predicate: (lineitem.l_quantity < $expr152) } 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 $expr143] } + StreamProject { exprs: [part.p_partkey, (0.2:Decimal * (sum(lineitem.l_quantity) / count(lineitem.l_quantity))) as $expr152] } 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, $expr143], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } + Table 3 { columns: [part_p_partkey, $expr152], 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] } @@ -3060,7 +3060,7 @@ └─StreamTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[orders.o_totalprice DESC, orders.o_orderdate ASC]", limit: 100, offset: 0, group_key: [6] } - └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr3] } + └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } └─StreamHashAgg { group_key: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice], aggs: [count, sum(lineitem.l_quantity)] } └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, lineitem.l_quantity, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber] } @@ -3234,14 +3234,14 @@ 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($expr72))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum($expr72))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr68)] } - StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr68, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum($expr72)] } + StreamProject { exprs: [(lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) as $expr72, 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($expr68))], primary key: [], value indices: [0, 1], distribution key: [] } + Table 0 { columns: [count, sum(sum($expr72))], 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] } @@ -3435,11 +3435,11 @@ Fragment 4 StreamProject { exprs: [partsupp.ps_suppkey, partsupp.ps_partkey, partsupp.ps_partkey, partsupp.ps_suppkey] } - StreamFilter { predicate: ($expr108 > $expr107) } + StreamFilter { predicate: ($expr115 > $expr116) } 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 $expr107] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr116] } 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 $expr108] } + StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr115] } 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, $expr108], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 8 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr115], 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, $expr107], primary key: [$0 ASC, $1 ASC], value indices: [0, 1, 2], distribution key: [0, 1] } + Table 10 { columns: [partsupp_ps_partkey, partsupp_ps_suppkey, $expr116], 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] } @@ -3615,7 +3615,7 @@ └─StreamTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0 } └─StreamExchange { dist: Single } └─StreamGroupTopN { order: "[count DESC, supplier.s_name ASC]", limit: 100, offset: 0, group_key: [2] } - └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr3] } + └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } └─StreamProject { exprs: [supplier.s_name, count] } └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count, count] } └─StreamExchange { dist: HashShard(supplier.s_name) } diff --git a/src/frontend/planner_test/tests/testdata/watermark.yaml b/src/frontend/planner_test/tests/testdata/watermark.yaml index 041a527e31a4b..fb989822cb6c9 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 $expr1, _row_id], watermark_columns: [(v1 - '00:00:02':Interval)] } + └─StreamProject { exprs: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '00:00:02':Interval) as $expr1, _row_id], watermark_columns: [(AtTimeZone((AtTimeZone(v1, 'UTC':Varchar) - '00:00:00':Interval), 'UTC':Varchar) - '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"] } From e542b1508460e8e214b16b93ab7b25412834154d Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 13:51:02 +0800 Subject: [PATCH 24/30] fix --- src/storage/src/hummock/mod.rs | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/storage/src/hummock/mod.rs b/src/storage/src/hummock/mod.rs index 9f08e3b561ed0..33d4c2c7d3c07 100644 --- a/src/storage/src/hummock/mod.rs +++ b/src/storage/src/hummock/mod.rs @@ -81,7 +81,9 @@ use self::iterator::{BackwardUserIterator, HummockIterator, UserIterator}; pub use self::sstable_store::*; use super::monitor::HummockStateStoreMetrics; use crate::error::StorageResult; -use crate::hummock::backup_reader::{BackupReader, BackupReaderRef}; +#[cfg(any(test, feature = "test"))] +use crate::hummock::backup_reader::BackupReader; +use crate::hummock::backup_reader::BackupReaderRef; use crate::hummock::compactor::CompactorContext; use crate::hummock::event_handler::hummock_event_handler::BufferTracker; use crate::hummock::event_handler::{HummockEvent, HummockEventHandler}; From 68234db1ff376d8387a8627318f3e532cb7df3b1 Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 14:04:04 +0800 Subject: [PATCH 25/30] fix --- e2e_test/streaming/join.slt | 34 +++++++++++++++++++++++ src/expr/src/expr/expr_array_to_string.rs | 8 ++++-- 2 files changed, 40 insertions(+), 2 deletions(-) diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index 3536b30891f78..b418ba47b93b6 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -167,3 +167,37 @@ drop table t9 statement ok drop table t10 + +# Regression test for: +statement ok +create table t1 (event string); + +statement ok +create table t2 (customer_id int, item_id int); + +statement ok +create materialized view v as SELECT event FROM t1 INNER JOIN t2 ON t1.event=concat("event_", array_join(array(t2.customer_id, t2.event), '_', '*')); + +statement ok +insert into t1 values ("event_0_0") ("event_*_0") ("event_1_1"); + +statement ok +insert into t2 values (0, 0) (NULL, 0) (2, 1); + +statement ok +flush; + +query I rowsort +select * from v; +---- +event_0_0 +event_*_0 + +statement ok +drop materialized view v; + +statement ok +drop table t1 + +statement ok +drop table t2 \ No newline at end of file diff --git a/src/expr/src/expr/expr_array_to_string.rs b/src/expr/src/expr/expr_array_to_string.rs index 213e164d5ead6..d5064c64fd585 100644 --- a/src/expr/src/expr/expr_array_to_string.rs +++ b/src/expr/src/expr/expr_array_to_string.rs @@ -175,8 +175,12 @@ impl Expression for ArrayToStringExpression { let array = self.array.eval_row(input)?; let delimiter = self.delimiter.eval_row(input)?; - let result = if let Some(array) = array && let Some(delimiter) = delimiter && let Some(e) = &self.null_string { - let null_string = e.eval_row(input)?; + let result = if let Some(array) = array && let Some(delimiter) = delimiter { + let null_string = if let Some(e) = &self.null_string { + e.eval_row(input)? + } else { + None + }; let mut writer = String::new(); if let Some(null_string) = null_string { self.evaluate_with_nulls(array.as_scalar_ref_impl().into_list(), delimiter.as_utf8(), null_string.as_utf8(), &mut writer); From 305803f0c1c67dc236fc1cb53a0a963055f0133e Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 14:13:52 +0800 Subject: [PATCH 26/30] fix --- e2e_test/streaming/join.slt | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index b418ba47b93b6..21c56cc84b771 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -168,6 +168,9 @@ drop table t9 statement ok drop table t10 +statement ok +flush; + # Regression test for: statement ok create table t1 (event string); @@ -176,13 +179,13 @@ statement ok create table t2 (customer_id int, item_id int); statement ok -create materialized view v as SELECT event FROM t1 INNER JOIN t2 ON t1.event=concat("event_", array_join(array(t2.customer_id, t2.event), '_', '*')); +create materialized view v as SELECT event FROM t1 INNER JOIN t2 ON t1.event=concat('event_', array_join(array[t2.customer_id, t2.item_id], '_', '*')); statement ok -insert into t1 values ("event_0_0") ("event_*_0") ("event_1_1"); +insert into t1 values ('event_0_0'), ('event_*_0'), ('event_1_1'); statement ok -insert into t2 values (0, 0) (NULL, 0) (2, 1); +insert into t2 values (0, 0), (NULL, 0), (2, 1); statement ok flush; @@ -190,8 +193,8 @@ flush; query I rowsort select * from v; ---- -event_0_0 event_*_0 +event_0_0 statement ok drop materialized view v; From de81675df1432dad251a92cfb769a0aa89dc3dbf Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 14:16:13 +0800 Subject: [PATCH 27/30] minor --- e2e_test/streaming/join.slt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index 21c56cc84b771..4c54d6fd81bbe 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -171,7 +171,7 @@ drop table t10 statement ok flush; -# Regression test for: +# Regression test for: https://github.com/risingwavelabs/risingwave/issues/8114 statement ok create table t1 (event string); From b35f4cc0a3f60cb7b2f3286ea6e50a31d681e7fc Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 14:21:46 +0800 Subject: [PATCH 28/30] fix --- e2e_test/streaming/join.slt | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index 4c54d6fd81bbe..575c7809ec4d5 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -173,16 +173,16 @@ flush; # Regression test for: https://github.com/risingwavelabs/risingwave/issues/8114 statement ok -create table t1 (event string); +create table t1 (event string, uid int, item_id int); statement ok -create table t2 (customer_id int, item_id int); +create table t2 (uid int, name string); statement ok -create materialized view v as SELECT event FROM t1 INNER JOIN t2 ON t1.event=concat('event_', array_join(array[t2.customer_id, t2.item_id], '_', '*')); +create materialized view v as SELECT event, name FROM t1 INNER JOIN t2 ON t1.uid=t2.uid AND t1.event=concat('event_', array_join(array[t2.uid, t1.item_id], '_')); statement ok -insert into t1 values ('event_0_0'), ('event_*_0'), ('event_1_1'); +insert into t1 values ('event_0_0'), ('event_0'), ('event_1_1'); statement ok insert into t2 values (0, 0), (NULL, 0), (2, 1); @@ -193,7 +193,7 @@ flush; query I rowsort select * from v; ---- -event_*_0 +event_0 event_0_0 statement ok From 9b3d394293a999ff2301baa99cd390cc9e3bcc0a Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 14:26:11 +0800 Subject: [PATCH 29/30] minor --- e2e_test/streaming/join.slt | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index 575c7809ec4d5..bcf180de81524 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -173,7 +173,7 @@ flush; # Regression test for: https://github.com/risingwavelabs/risingwave/issues/8114 statement ok -create table t1 (event string, uid int, item_id int); +create table t1 (uid int, item_id int, event string); statement ok create table t2 (uid int, name string); @@ -182,10 +182,10 @@ statement ok create materialized view v as SELECT event, name FROM t1 INNER JOIN t2 ON t1.uid=t2.uid AND t1.event=concat('event_', array_join(array[t2.uid, t1.item_id], '_')); statement ok -insert into t1 values ('event_0_0'), ('event_0'), ('event_1_1'); +insert into t1 values (0, 0, 'event_0_0'), (1, NULL, 'event_1'), (2, 3, 'event_2_1'); statement ok -insert into t2 values (0, 0), (NULL, 0), (2, 1); +insert into t2 values (0, 'a'), (1, 'b'), (2, 'c'); statement ok flush; @@ -193,8 +193,8 @@ flush; query I rowsort select * from v; ---- -event_0 -event_0_0 +event_0_0 a +event_1 b statement ok drop materialized view v; From 0c89a4906b42d8c59a915d25a5389c666021a1fb Mon Sep 17 00:00:00 2001 From: jon-chuang Date: Wed, 22 Feb 2023 14:50:19 +0800 Subject: [PATCH 30/30] fix --- e2e_test/streaming/join.slt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index bcf180de81524..8f50fac0eb729 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -173,16 +173,16 @@ flush; # Regression test for: https://github.com/risingwavelabs/risingwave/issues/8114 statement ok -create table t1 (uid int, item_id int, event string); +create table t1 (uid int, item_id int, event string, name string); statement ok create table t2 (uid int, name string); statement ok -create materialized view v as SELECT event, name FROM t1 INNER JOIN t2 ON t1.uid=t2.uid AND t1.event=concat('event_', array_join(array[t2.uid, t1.item_id], '_')); +create materialized view v as SELECT event, t1.name FROM t1 INNER JOIN t2 WHERE t1.name=t2.name AND t1.event=concat('event_', array_join(array[t2.uid, t1.item_id], '_')); statement ok -insert into t1 values (0, 0, 'event_0_0'), (1, NULL, 'event_1'), (2, 3, 'event_2_1'); +insert into t1 values (0, 0, 'event_0_0', 'a'), (1, NULL, 'event_1', 'b'), (2, 3, 'event_2_1', 'c'); statement ok insert into t2 values (0, 'a'), (1, 'b'), (2, 'c');