From 88dc35e44554152a88366f6535d7a60b37a4bf9f Mon Sep 17 00:00:00 2001 From: jon-chuang <9093549+jon-chuang@users.noreply.github.com> Date: Thu, 23 Feb 2023 08:09:40 +0800 Subject: [PATCH] feat(frontend): apply `SessionTimezone` and `ConstEvalRewriter` expr rewriters to during `gen_{batch,stream}_plan` (#7761) apply `SessionTimezone` and `ConstEvalRewriter` expr rewriters to during `gen_{batch,stream}_plan` Notes: - wait for https://github.com/risingwavelabs/risingwave/pull/7757 to be merged - wait for https://github.com/risingwavelabs/risingwave/pull/7777 to be merged - wait for https://github.com/risingwavelabs/risingwave/pull/7786 to be merged Approved-By: ice1000 Approved-By: st1page Co-Authored-By: jon-chuang Co-Authored-By: jon-chuang <9093549+jon-chuang@users.noreply.github.com> --- e2e_test/batch/functions/now.slt.part | 9 +- e2e_test/streaming/join.slt | 37 +++ src/expr/src/expr/expr_array_to_string.rs | 8 +- .../planner_test/tests/testdata/agg.yaml | 6 +- .../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 | 10 +- .../planner_test/tests/testdata/expr.yaml | 77 ++--- .../tests/testdata/index_selection.yaml | 6 +- .../planner_test/tests/testdata/insert.yaml | 12 +- .../planner_test/tests/testdata/join.yaml | 4 +- .../planner_test/tests/testdata/nexmark.yaml | 118 +++---- .../tests/testdata/nexmark_source.yaml | 130 ++++---- .../planner_test/tests/testdata/order_by.yaml | 12 +- .../tests/testdata/predicate_pushdown.yaml | 4 +- .../tests/testdata/project_set.yaml | 12 +- .../tests/testdata/range_scan.yaml | 4 +- .../planner_test/tests/testdata/share.yaml | 16 +- .../tests/testdata/shared_views.yaml | 4 +- .../tests/testdata/stream_dist_agg.yaml | 20 +- .../tests/testdata/sysinfo_funcs.yaml | 4 +- .../tests/testdata/temporal_filter.yaml | 12 +- .../planner_test/tests/testdata/tpch.yaml | 298 +++++++++--------- .../tests/testdata/watermark.yaml | 2 +- src/frontend/src/optimizer/mod.rs | 46 ++- .../src/optimizer/optimizer_context.rs | 9 +- .../src/optimizer/plan_node/batch_filter.rs | 6 +- .../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 | 8 +- .../optimizer/plan_node/batch_simple_agg.rs | 4 +- .../src/optimizer/plan_node/batch_sort_agg.rs | 9 +- .../src/optimizer/plan_node/batch_update.rs | 8 +- .../src/optimizer/plan_node/batch_values.rs | 11 +- .../src/optimizer/plan_node/generic/agg.rs | 7 +- .../src/optimizer/plan_node/logical_scan.rs | 12 +- .../src/optimizer/plan_node/logical_source.rs | 3 +- .../src/optimizer/plan_node/stream.rs | 37 +-- .../optimizer/plan_node/stream_delta_join.rs | 7 +- .../src/optimizer/plan_node/stream_filter.rs | 8 +- .../plan_node/stream_global_simple_agg.rs | 3 +- .../optimizer/plan_node/stream_hash_agg.rs | 3 +- .../optimizer/plan_node/stream_hash_join.rs | 7 +- .../plan_node/stream_local_simple_agg.rs | 3 +- .../src/optimizer/plan_node/stream_project.rs | 7 +- .../optimizer/plan_node/stream_table_scan.rs | 3 +- src/storage/src/hummock/mod.rs | 4 +- 50 files changed, 515 insertions(+), 553 deletions(-) diff --git a/e2e_test/batch/functions/now.slt.part b/e2e_test/batch/functions/now.slt.part index 8d3ce7867b3d9..7c2eb82f45921 100644 --- a/e2e_test/batch/functions/now.slt.part +++ b/e2e_test/batch/functions/now.slt.part @@ -11,11 +11,10 @@ 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: [[]] } +query T +explain select now() + interval '1 hour' = now() + interval '30 minutes' + interval '30 minutes' true; +---- +BatchValues { rows: [[true:Boolean]] } statement ok drop table tz diff --git a/e2e_test/streaming/join.slt b/e2e_test/streaming/join.slt index 3536b30891f78..8f50fac0eb729 100644 --- a/e2e_test/streaming/join.slt +++ b/e2e_test/streaming/join.slt @@ -167,3 +167,40 @@ drop table t9 statement ok drop table t10 + +statement ok +flush; + +# Regression test for: https://github.com/risingwavelabs/risingwave/issues/8114 +statement ok +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, 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', 'a'), (1, NULL, 'event_1', 'b'), (2, 3, 'event_2_1', 'c'); + +statement ok +insert into t2 values (0, 'a'), (1, 'b'), (2, 'c'); + +statement ok +flush; + +query I rowsort +select * from v; +---- +event_0_0 a +event_1 b + +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); diff --git a/src/frontend/planner_test/tests/testdata/agg.yaml b/src/frontend/planner_test/tests/testdata/agg.yaml index b19d96a9fa6bc..51fe6ac199532 100644 --- a/src/frontend/planner_test/tests/testdata/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/agg.yaml @@ -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/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index 46c5d783358d9..bc4bb13311ca9 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,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/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 7320d46d79e9a..ae2969499939f 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); @@ -129,11 +129,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 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/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 0aca7036a4366..2c8a6138f7877 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] } } + Inline Session Timezone: + + 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": 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 819bfad027f93..855c1b89c8d47 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)))))) as $expr1] } + └─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 $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)] } @@ -92,7 +92,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 } @@ -106,7 +106,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 } @@ -119,19 +119,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: |- @@ -141,18 +141,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: |- @@ -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 $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); @@ -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($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: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($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:Int32)]] } + └─BatchValues { rows: [[ARRAY[1]:List { datatype: Int32 }]] } - name: now expression sql: | create table t (v1 timestamp with time zone); @@ -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: | @@ -525,14 +525,15 @@ 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 $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; + batch_error: 'Expr error: Division by zero' diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index ca394a435975b..fe01e572efff1 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 170089b02a1b3..423f8e0e97f7e 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); @@ -129,7 +129,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); @@ -149,7 +149,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); @@ -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/nexmark.yaml b/src/frontend/planner_test/tests/testdata/nexmark.yaml index a82c6919cffb8..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] } @@ -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 0633b67e5272c..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] } @@ -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 ea1f2df3b8faa..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); @@ -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 879e6f59690d8..03626a2da6ab6 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 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 c5ffc5cb34f78..ac044ee5576dd 100644 --- a/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/stream_dist_agg.yaml @@ -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: @@ -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/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index c6b3a39fe411d..385323133d317 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -29,6 +29,6 @@ Feature is not yet implemented: Only boolean literals are supported in `current_schemas`. 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; + 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 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 62db385a1f08e..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: @@ -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] } @@ -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] } @@ -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) } @@ -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 @@ -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 @@ -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] } @@ -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) } @@ -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 @@ -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 @@ -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] } @@ -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,19 +2242,19 @@ | └─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 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 @@ -2267,12 +2267,12 @@ 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 - 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,11 +2419,11 @@ └─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] } - └─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: [] } @@ -2431,11 +2431,11 @@ └─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] } - | └─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($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 @@ -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($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] } @@ -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] } @@ -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] } @@ -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 @@ -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 @@ -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 @@ -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] } @@ -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] } 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"] } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 528b8f9cd04d7..c76a93184a01f 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -11,6 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. +use std::ops::DerefMut; pub mod plan_node; pub use plan_node::{Explain, PlanRef}; @@ -486,15 +487,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("Inline Session Timezone:"); + 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()); @@ -648,13 +656,21 @@ impl PlanRoot { ); } + // Inline session timezone + plan = inline_session_timezone_in_exprs(ctx.clone(), plan)?; + + if ctx.is_explain_trace() { + ctx.trace("Inline session timezone:"); + 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()); @@ -784,7 +800,6 @@ impl PlanRoot { } } -#[allow(dead_code)] fn const_eval_exprs(plan: PlanRef) -> Result { let mut const_eval_rewriter = ConstEvalRewriter { error: None }; @@ -795,6 +810,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 66b26f965fec0..481dc87b07f21 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -14,13 +14,13 @@ 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; 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; @@ -176,9 +176,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..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, }; @@ -80,10 +79,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 55cebf0190072..4728f3d0247de 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_agg.rs @@ -122,7 +122,7 @@ impl ToBatchProst for BatchHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .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..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, @@ -89,12 +88,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..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, @@ -114,7 +114,7 @@ impl ToBatchProst for BatchSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .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 1e3056dff4427..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,18 +119,13 @@ impl ToBatchProst for BatchSortAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .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 94bc258e69de3..977353c14068e 100644 --- a/src/frontend/src/optimizer/plan_node/batch_update.rs +++ b/src/frontend/src/optimizer/plan_node/batch_update.rs @@ -19,7 +19,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, @@ -80,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..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, @@ -55,15 +54,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 7c6815107e3fe..8b83c129e54d3 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -606,7 +606,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()), @@ -617,10 +617,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 0989528a6ad4e..219af17c0d0b9 100644 --- a/src/frontend/src/optimizer/plan_node/logical_scan.rs +++ b/src/frontend/src/optimizer/plan_node/logical_scan.rs @@ -610,17 +610,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 1486ae0ea72d4..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, @@ -326,7 +326,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 4ccf758a9c806..c9baaf9f2770d 100644 --- a/src/frontend/src/optimizer/plan_node/stream.rs +++ b/src/frontend/src/optimizer/plan_node/stream.rs @@ -438,7 +438,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) @@ -484,7 +484,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 { @@ -528,11 +528,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,11 +538,7 @@ pub fn to_stream_prost_body( let distinct_dedup_tables = me.infer_distinct_dedup_tables(base, None); ProstNode::GlobalSimpleAgg(SimpleAggNode { - agg_calls: me - .agg_calls - .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) - .collect(), + agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), distribution_key: base .dist .dist_column_indices() @@ -596,7 +588,7 @@ pub fn to_stream_prost_body( .core .agg_calls .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) + .map(PlanAggCall::to_protobuf) .collect(), is_append_only: me.core.input.0.append_only, @@ -662,7 +654,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()), @@ -685,11 +677,7 @@ pub fn to_stream_prost_body( Node::LocalSimpleAgg(me) => { let me = &me.core; ProstNode::LocalSimpleAgg(SimpleAggNode { - agg_calls: me - .agg_calls - .iter() - .map(|x| PlanAggCall::to_protobuf(x, base.ctx())) - .collect(), + agg_calls: me.agg_calls.iter().map(PlanAggCall::to_protobuf).collect(), distribution_key: base .dist .dist_column_indices() @@ -722,16 +710,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..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; @@ -76,12 +75,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 8846868b0ebc6..b659a946e75ea 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; @@ -99,7 +98,7 @@ impl StreamNode for StreamGlobalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .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 b8184bb627428..5adf9a8803386 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; @@ -114,7 +113,7 @@ impl StreamNode for StreamHashAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .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 25fbe75d01733..42eb2da1c057b 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; @@ -96,7 +95,7 @@ impl StreamNode for StreamLocalSimpleAgg { agg_calls: self .agg_calls() .iter() - .map(|x| PlanAggCall::to_protobuf(x, self.base.ctx())) + .map(PlanAggCall::to_protobuf) .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 8658db4889fde..6c1e67e43775b 100644 --- a/src/frontend/src/optimizer/plan_node/stream_project.rs +++ b/src/frontend/src/optimizer/plan_node/stream_project.rs @@ -141,12 +141,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 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 d53d51fa110b3..e89079d03e224 100644 --- a/src/frontend/src/optimizer/plan_node/stream_table_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_table_scan.rs @@ -246,12 +246,13 @@ impl ExprRewritable for StreamTableScan { } fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { - Self::new( + Self::new_with_chain_type( self.logical .rewrite_exprs(r) .as_logical_scan() .unwrap() .clone(), + self.chain_type, ) .into() } 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};