diff --git a/src/frontend/planner_test/tests/testdata/array.yaml b/src/frontend/planner_test/tests/testdata/array.yaml index fd26d4a94ac4a..b436cbdff2d4e 100644 --- a/src/frontend/planner_test/tests/testdata/array.yaml +++ b/src/frontend/planner_test/tests/testdata/array.yaml @@ -4,13 +4,13 @@ logical_plan: | LogicalValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Varchar }] } } batch_plan: | - BatchValues { rows: [[ARRAY[foo, bar]:List { datatype: Varchar }]] } + BatchValues { rows: [[Array('foo':Varchar, 'bar':Varchar)]] } - sql: | values (ARRAY[1, 2+3, 4*5+1]); logical_plan: | LogicalValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]], schema: Schema { fields: [*VALUES*_0.column_0:List { datatype: Int32 }] } } batch_plan: | - BatchValues { rows: [[ARRAY[1, 5, 21]:List { datatype: Int32 }]] } + BatchValues { rows: [[Array(1:Int32, (2:Int32 + 3:Int32), ((4:Int32 * 5:Int32) + 1:Int32))]] } - sql: | create table t (v1 int); select (ARRAY[1, v1]) from t; @@ -52,7 +52,7 @@ LogicalProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayCat(Array(66:Int32), Array(123:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[array[66]], array[233]); @@ -60,7 +60,7 @@ LogicalProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[{66}, {233}]:List { datatype: List { datatype: Int32 } }] } + BatchProject { exprs: [ArrayCat(Array(Array(66:Int32)), Array(233:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[66]]); @@ -68,7 +68,7 @@ LogicalProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[{233}, {66}]:List { datatype: List { datatype: Int32 } }] } + BatchProject { exprs: [ArrayCat(Array(233:Int32), Array(Array(66:Int32)))] } └─BatchValues { rows: [[]] } - sql: | select array_cat(array[233], array[array[array[66]]]); @@ -85,7 +85,7 @@ LogicalProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[66, 123]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayAppend(Array(66:Int32), 123:Int32)] } └─BatchValues { rows: [[]] } - sql: | select array_append(123, 234); @@ -102,7 +102,7 @@ LogicalProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [ARRAY[123, 66]:List { datatype: Int32 }] } + BatchProject { exprs: [ArrayPrepend(123:Int32, Array(66:Int32))] } └─BatchValues { rows: [[]] } - sql: | select array_prepend(123, 234); diff --git a/src/frontend/planner_test/tests/testdata/basic_query.yaml b/src/frontend/planner_test/tests/testdata/basic_query.yaml index 51ac43d99d7fd..5b141abcc0d1f 100644 --- a/src/frontend/planner_test/tests/testdata/basic_query.yaml +++ b/src/frontend/planner_test/tests/testdata/basic_query.yaml @@ -1,7 +1,7 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: values (11, 22), (33+(1+2), 44); batch_plan: | - BatchValues { rows: [[11:Int32, 22:Int32], [36:Int32, 44:Int32]] } + BatchValues { rows: [[11:Int32, 22:Int32], [(33:Int32 + (1:Int32 + 2:Int32)), 44:Int32]] } - sql: select * from t binder_error: 'Catalog error: table or source not found: t' - sql: | @@ -22,11 +22,11 @@ select * from t where 1>2 and 1=1 and 3<1 and 4<>1 or 1=1 and 2>=1 and 1<=2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean AND true:Boolean } + └─BatchFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } └─BatchScan { table: t, columns: [], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamFilter { predicate: true:Boolean AND true:Boolean } + └─StreamFilter { predicate: (1:Int32 = 1:Int32) AND ((((1:Int32 > 2:Int32) AND (3:Int32 < 1:Int32)) AND (4:Int32 <> 1:Int32)) OR ((2:Int32 >= 1:Int32) AND (1:Int32 <= 2:Int32))) } └─StreamTableScan { table: t, columns: [t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 int); @@ -130,11 +130,11 @@ - sql: | select * from unnest(Array[1,2,3]); batch_plan: | - BatchTableFunction { Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 }) } + BatchTableFunction { Unnest(Array(1:Int32, 2:Int32, 3:Int32)) } - sql: | select * from unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchTableFunction { Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } }) } + BatchTableFunction { Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32))) } - sql: | create table t1 (x int); select * from t1 where EXISTS(select * where t1.x=1); diff --git a/src/frontend/planner_test/tests/testdata/cast.yaml b/src/frontend/planner_test/tests/testdata/cast.yaml index b9385ce48355e..e122dd0ace778 100644 --- a/src/frontend/planner_test/tests/testdata/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/cast.yaml @@ -18,7 +18,7 @@ sql: | select case when NULL then 1 end; batch_plan: | - BatchProject { exprs: [null:Int32] } + BatchProject { exprs: [Case(null:Boolean, 1:Int32)] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON NULL) sql: | @@ -65,7 +65,7 @@ sql: | select case when 'y' then 1 end; batch_plan: | - BatchProject { exprs: [1:Int32] } + BatchProject { exprs: [Case(true:Boolean, 1:Int32)] } └─BatchValues { rows: [[]] } - name: implicit cast boolean (JOIN ON with literal 'y' of unknown type) sql: | diff --git a/src/frontend/planner_test/tests/testdata/explain.yaml b/src/frontend/planner_test/tests/testdata/explain.yaml index 541b3be817f90..c89502dce3ade 100644 --- a/src/frontend/planner_test/tests/testdata/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/explain.yaml @@ -36,11 +36,6 @@ LogicalProject { exprs: [1:Int32] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } - Const eval exprs: - - BatchProject { exprs: [1:Int32] } - └─BatchValues { rows: [[]] } - To Batch Physical Plan: BatchProject { exprs: [1:Int32] } @@ -56,7 +51,7 @@ "stages": { "0": { "root": { - "plan_node_id": 34, + "plan_node_id": 28, "plan_node_type": "BatchProject", "schema": [ { @@ -69,7 +64,7 @@ ], "children": [ { - "plan_node_id": 32, + "plan_node_id": 26, "plan_node_type": "BatchValues", "schema": [], "children": [], diff --git a/src/frontend/planner_test/tests/testdata/expr.yaml b/src/frontend/planner_test/tests/testdata/expr.yaml index 3f8d2435e0dfd..6ff7530e9f2c3 100644 --- a/src/frontend/planner_test/tests/testdata/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/expr.yaml @@ -19,17 +19,17 @@ - sql: | values(cast(1 as bigint)); batch_plan: | - BatchValues { rows: [[1:Int64]] } + BatchValues { rows: [[1:Int32::Int64]] } - sql: | values(not true); batch_plan: | - BatchValues { rows: [[false:Boolean]] } + BatchValues { rows: [[Not(true:Boolean)]] } - sql: | create table t (); select (((((false is not true) is true) is not false) is false) is not null) is null from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [false:Boolean] } + └─BatchProject { exprs: [IsNull(IsNotNull(IsFalse(IsNotFalse(IsTrue(IsNotTrue(false:Boolean))))))] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: bind between sql: | @@ -65,13 +65,13 @@ sql: | SELECT 1::real in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [true:Boolean] } + BatchProject { exprs: [In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32)] } └─BatchValues { rows: [[]] } - name: not in-list with aligned types sql: | SELECT 1::real not in (3, 1.0, 2); batch_plan: | - BatchProject { exprs: [false:Boolean] } + BatchProject { exprs: [Not(In(1:Int32::Float32, 3:Int32::Float32, 1.0:Decimal::Float32, 2:Int32::Float32))] } └─BatchValues { rows: [[]] } - name: in-list with misaligned types sql: | @@ -82,7 +82,7 @@ create table t (v1 int); SELECT 1 in (3, 0.5*2, min(v1)) from t; batch_plan: | - BatchProject { exprs: [(true:Boolean OR (1:Int32 = min(min(t.v1))))] } + BatchProject { exprs: [(In(1:Int32::Decimal, 3:Int32::Decimal, (0.5:Decimal * 2:Int32)) OR (1:Int32 = min(min(t.v1))))] } └─BatchSimpleAgg { aggs: [min(min(t.v1))] } └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [min(t.v1)] } @@ -94,7 +94,7 @@ SELECT b2 from b where 1 in (3, 1.0, (select min(v1) from t)); batch_plan: | BatchProject { exprs: [b.b2] } - └─BatchFilter { predicate: (true:Boolean OR (1:Int32 = min(min(t.v1)))) } + └─BatchFilter { predicate: (In(1:Int32::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (1:Int32 = min(min(t.v1)))) } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b2], distribution: SomeShard } @@ -108,7 +108,7 @@ create table b (b1 int, b2 int); SELECT b2 from b where exists (select 2 from t where v1 in (3, 1.0, b1)); batch_plan: | - BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } + BatchNestedLoopJoin { type: LeftSemi, predicate: (In(t.v1::Decimal, 3:Int32::Decimal, 1.0:Decimal) OR (t.v1 = b.b1)), output: [b.b2] } ├─BatchExchange { order: [], dist: Single } | └─BatchScan { table: b, columns: [b.b1, b.b2], distribution: SomeShard } └─BatchExchange { order: [], dist: Single } @@ -122,19 +122,19 @@ - sql: | values(round(42.4382, 2)); batch_plan: | - BatchValues { rows: [[42.44:Decimal]] } + BatchValues { rows: [[RoundDigit(42.4382:Decimal, 2:Int32)]] } - sql: | values(round(42.4382)); batch_plan: | - BatchValues { rows: [[42:Decimal]] } + BatchValues { rows: [[Round(42.4382:Decimal)]] } - sql: | values(round(42)); batch_plan: | - BatchValues { rows: [[42:Float64]] } + BatchValues { rows: [[Round(42:Int32::Float64)]] } - sql: | values(round(42, 2)); batch_plan: | - BatchValues { rows: [[42:Decimal]] } + BatchValues { rows: [[RoundDigit(42:Int32::Decimal, 2:Int32)]] } - sql: | values(round(true)); binder_error: |- @@ -144,18 +144,18 @@ -- Single quoted literal can be treated as number without error. values(round('123')); batch_plan: | - BatchValues { rows: [[123:Float64]] } + BatchValues { rows: [[Round(123:Float64)]] } - sql: | -- When it is invalid, PostgreSQL reports error during explain, but we have to wait until execution as of now. #4235 values(round('abc')); - sql: | values(extract(hour from timestamp '2001-02-16 20:38:40')); batch_plan: | - BatchValues { rows: [[20:Decimal]] } + BatchValues { rows: [[Extract('HOUR':Varchar, '2001-02-16 20:38:40':Timestamp)]] } - sql: | values('Postgres' not like 'Post%'); batch_plan: | - BatchValues { rows: [[false:Boolean]] } + BatchValues { rows: [[Not(Like('Postgres':Varchar, 'Post%':Varchar))]] } - sql: | values(1 not like 1.23); binder_error: |- @@ -164,13 +164,13 @@ - sql: | select length(trim(trailing '1' from '12'))+length(trim(leading '2' from '23'))+length(trim(both '3' from '34')); batch_plan: | - BatchProject { exprs: [4:Int32] } + BatchProject { exprs: [((Length(Rtrim('12':Varchar, '1':Varchar)) + Length(Ltrim('23':Varchar, '2':Varchar))) + Length(Trim('34':Varchar, '3':Varchar)))] } └─BatchValues { rows: [[]] } - sql: | select position(replace('1','1','2'),'123') where '12' like '%1'; batch_plan: | - BatchProject { exprs: [0:Int32] } - └─BatchFilter { predicate: false:Boolean } + BatchProject { exprs: [Position(Replace('1':Varchar, '1':Varchar, '2':Varchar), '123':Varchar)] } + └─BatchFilter { predicate: Like('12':Varchar, '%1':Varchar) } └─BatchValues { rows: [[]] } - name: case searched form with else sql: | @@ -178,11 +178,11 @@ select (case when v1=1 then 1 when v1=2 then 2 else 0.0 end) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2:Decimal, 0.0:Decimal), t._row_id] } + └─StreamProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2:Int32::Decimal, 0.0:Decimal), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - name: case searched form without else sql: | @@ -190,7 +190,7 @@ select (case when v1=1 then 1 when v1=2 then 2.1 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2:Int32), 2.1:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case simple form sql: | @@ -198,7 +198,7 @@ select (case v1 when 1 then 1 when 2.0 then 2 else 0.0 end) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Decimal, (t.v1 = 2.0:Decimal), 2:Decimal, 0.0:Decimal)] } + └─BatchProject { exprs: [Case((t.v1 = 1:Int32), 1:Int32::Decimal, (t.v1 = 2.0:Decimal), 2:Int32::Decimal, 0.0:Decimal)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: case misaligned result types sql: | @@ -264,18 +264,18 @@ select concat_ws(v1, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, '1':Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs(t.v1, '1':Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs(t.v1, 1:Int32::Varchar), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 varchar); select concat_ws(v1, 1.2) from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs(t.v1, '1.2':Varchar)] } + └─BatchProject { exprs: [ConcatWs(t.v1, 1.2:Decimal::Varchar)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - sql: | create table t (v1 int); @@ -290,11 +290,11 @@ select concat(v1, v2, v3, 1) as expr from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar)] } + └─BatchProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar)] } └─BatchScan { table: t, columns: [t.v1, t.v2, t.v3], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [expr, t._row_id(hidden)], pk_columns: [t._row_id] } - └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, '1':Varchar), t._row_id] } + └─StreamProject { exprs: [ConcatWs('':Varchar, t.v1, t.v2::Varchar, t.v3::Varchar, 1:Int32::Varchar), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t.v3, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 float); @@ -310,22 +310,22 @@ - sql: | select concat(':', true); batch_plan: | - BatchProject { exprs: [':t':Varchar] } + BatchProject { exprs: [ConcatWs('':Varchar, ':':Varchar, BoolOut(true:Boolean))] } └─BatchValues { rows: [[]] } - sql: | select ':' || true; batch_plan: | - BatchProject { exprs: [':true':Varchar] } + BatchProject { exprs: [ConcatOp(':':Varchar, true:Boolean::Varchar)] } └─BatchValues { rows: [[]] } - sql: | select substr('hello', NULL); batch_plan: | - BatchProject { exprs: [null:Varchar] } + BatchProject { exprs: [Substr('hello':Varchar, null:Int32)] } └─BatchValues { rows: [[]] } - sql: | select substr(NULL, 1); batch_plan: | - BatchProject { exprs: [null:Varchar] } + BatchProject { exprs: [Substr(null:Varchar, 1:Int32)] } └─BatchValues { rows: [[]] } - sql: | select pg_typeof('123'); @@ -419,10 +419,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [Some((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [Some((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } + └─BatchProject { exprs: [Array(1:Int32)] } └─BatchValues { rows: [[]] } - sql: | select 1 < ALL(array[null]::integer[]); @@ -443,10 +443,10 @@ └─LogicalProject { exprs: [Array(1:Int32)] } └─LogicalValues { rows: [[]], schema: Schema { fields: [] } } batch_plan: | - BatchProject { exprs: [All((1:Int32 < ArrayCat(ARRAY[1]:List { datatype: Int32 }, ARRAY[2]:List { datatype: Int32 })))] } + BatchProject { exprs: [All((1:Int32 < ArrayCat(Array(1:Int32), Array(2:Int32))))] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchValues { rows: [[]] } - └─BatchProject { exprs: [ARRAY[1]:List { datatype: Int32 }] } + └─BatchProject { exprs: [Array(1:Int32)] } └─BatchValues { rows: [[]] } - name: now expression sql: | @@ -540,7 +540,7 @@ sql: | select false >= 'LN1O0QP1yi' NOT IN (md5('4SeUPZhUbH')) batch_plan: | - BatchProject { exprs: [false:Boolean] } + BatchProject { exprs: [(false:Boolean >= Not(In('LN1O0QP1yi':Varchar, Md5('4SeUPZhUbH':Varchar))))] } └─BatchValues { rows: [[]] } - name: const_eval of const expr sql: | @@ -548,7 +548,7 @@ select 1 + 2 + v1 from t; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: [(3:Int32 + t.v1)] } + └─BatchProject { exprs: [((1:Int32 + 2:Int32) + t.v1)] } └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } - name: const_eval of division by 0 error sql: select 1 / 0 t1; diff --git a/src/frontend/planner_test/tests/testdata/index_selection.yaml b/src/frontend/planner_test/tests/testdata/index_selection.yaml index c98da2bf8e0bc..af18731a327f8 100644 --- a/src/frontend/planner_test/tests/testdata/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/index_selection.yaml @@ -77,7 +77,7 @@ select a,b from t1 where a in (1,2) and b in (2,3) batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: In(idx1.b, 2:Decimal, 3:Decimal) } + └─BatchFilter { predicate: In(idx1.b, 2:Int32::Decimal, 3:Int32::Decimal) } └─BatchScan { table: idx1, columns: [idx1.a, idx1.b], scan_ranges: [idx1.a = Int32(1) , idx1.a = Int32(2)], distribution: UpstreamHashShard(idx1.a, idx1.b) } - sql: | create table t1 (a int, b numeric, c bigint); @@ -213,13 +213,13 @@ update t1 set c = 3 where a = 1 and b = 2; batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } + └─BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } └─BatchExchange { order: [], dist: Single } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: UpstreamHashShard(idx2.t1._row_id) } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } batch_local_plan: | - BatchUpdate { table: t1, exprs: [$0, $1, 3:Int64, $3] } + BatchUpdate { table: t1, exprs: [$0, $1, 3:Int32::Int64, $3] } └─BatchLookupJoin { type: Inner, predicate: idx2.t1._row_id IS NOT DISTINCT FROM t1._row_id, output: [t1.a, t1.b, t1.c, t1._row_id] } └─BatchExchange { order: [], dist: Single } └─BatchScan { table: idx2, columns: [idx2.t1._row_id], scan_ranges: [idx2.b = Decimal(Normalized(2)) AND idx2.a = Int32(1)], distribution: SomeShard } diff --git a/src/frontend/planner_test/tests/testdata/insert.yaml b/src/frontend/planner_test/tests/testdata/insert.yaml index 985277c9273f7..aa67c803d90a2 100644 --- a/src/frontend/planner_test/tests/testdata/insert.yaml +++ b/src/frontend/planner_test/tests/testdata/insert.yaml @@ -14,7 +14,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [[22.33:Float32, 33:Int32], [44:Float32, 55:Int32]] } + └─BatchValues { rows: [[22.33:Decimal::Float32, 33:Int32], [44:Int32::Float32, 55.0:Decimal::Int32]] } - name: insert values on non-assign-castable types sql: | create table t (v1 real, v2 int); @@ -73,7 +73,7 @@ batch_plan: | BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } - └─BatchValues { rows: [['01:02:03':Time], ['03:04:05':Time]] } + └─BatchValues { rows: [['2020-01-01 01:02:03':Timestamp::Time], ['03:04:05':Time]] } - name: a `VALUES` without insert context may be invalid on its own (compare with above) sql: | create table t (v1 time); @@ -93,7 +93,7 @@ sql: | values (1), (null), (2.3); batch_plan: | - BatchValues { rows: [[1:Decimal], [null:Decimal], [2.3:Decimal]] } + BatchValues { rows: [[1:Int32::Decimal], [null:Decimal], [2.3:Decimal]] } - name: rows of different number of columns sql: | values (1), (2, 3); @@ -115,7 +115,7 @@ BatchExchange { order: [], dist: Single } └─BatchInsert { table: t } └─BatchExchange { order: [], dist: Single } - └─BatchProject { exprs: ['01:02:03':Time, 11:Int32, 4.5:Float32] } + └─BatchProject { exprs: ['2020-01-01 01:02:03':Timestamp::Time, 11:Int32, 4.5:Decimal::Float32] } └─BatchScan { table: t, columns: [], distribution: SomeShard } - name: insert into select with cast error sql: | diff --git a/src/frontend/planner_test/tests/testdata/order_by.yaml b/src/frontend/planner_test/tests/testdata/order_by.yaml index 5a74f834a74f5..4f460213c6791 100644 --- a/src/frontend/planner_test/tests/testdata/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/order_by.yaml @@ -59,13 +59,13 @@ select * from t order by 1+1; batch_plan: | BatchProject { exprs: [t.v1, t.v2] } - └─BatchExchange { order: [2:Int32 ASC], dist: Single } - └─BatchSort { order: [2:Int32 ASC] } - └─BatchProject { exprs: [t.v1, t.v2, 2:Int32] } + └─BatchExchange { order: [(1:Int32 + 1:Int32) ASC], dist: Single } + └─BatchSort { order: [(1:Int32 + 1:Int32) ASC] } + └─BatchProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32)] } └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: | - StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [2:Int32, t._row_id] } - └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } + StreamMaterialize { columns: [v1, v2, (1:Int32 + 1:Int32)(hidden), t._row_id(hidden)], pk_columns: [t._row_id], order_descs: [(1:Int32 + 1:Int32), t._row_id] } + └─StreamProject { exprs: [t.v1, t.v2, (1:Int32 + 1:Int32), t._row_id] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); diff --git a/src/frontend/planner_test/tests/testdata/project_set.yaml b/src/frontend/planner_test/tests/testdata/project_set.yaml index fde593e397a9c..4adbeadc2bc68 100644 --- a/src/frontend/planner_test/tests/testdata/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/project_set.yaml @@ -8,14 +8,14 @@ - sql: | select unnest(Array[1,2,3]); batch_plan: | - BatchProject { exprs: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } - └─BatchProjectSet { select_list: [Unnest(ARRAY[1, 2, 3]:List { datatype: Int32 })] } + BatchProject { exprs: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } + └─BatchProjectSet { select_list: [Unnest(Array(1:Int32, 2:Int32, 3:Int32))] } └─BatchValues { rows: [[]] } - sql: | select unnest(Array[Array[1,2,3], Array[4,5,6]]); batch_plan: | - BatchProject { exprs: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } - └─BatchProjectSet { select_list: [Unnest(ARRAY[{1,2,3}, {4,5,6}]:List { datatype: List { datatype: Int32 } })] } + BatchProject { exprs: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } + └─BatchProjectSet { select_list: [Unnest(Array(Array(1:Int32, 2:Int32, 3:Int32), Array(4:Int32, 5:Int32, 6:Int32)))] } └─BatchValues { rows: [[]] } - sql: | create table t(x int[]); @@ -43,9 +43,9 @@ create table t(x int[]); select unnest(x), unnest(Array[1,2]) from t; batch_plan: | - BatchProject { exprs: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } + BatchProject { exprs: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } └─BatchExchange { order: [], dist: Single } - └─BatchProjectSet { select_list: [Unnest($0), Unnest(ARRAY[1, 2]:List { datatype: Int32 })] } + └─BatchProjectSet { select_list: [Unnest($0), Unnest(Array(1:Int32, 2:Int32))] } └─BatchScan { table: t, columns: [t.x, t._row_id], distribution: UpstreamHashShard(t._row_id) } - name: table functions as parameters of usual functions sql: | diff --git a/src/frontend/planner_test/tests/testdata/range_scan.yaml b/src/frontend/planner_test/tests/testdata/range_scan.yaml index 4c6678d46ccbd..df8dbca94c355 100644 --- a/src/frontend/planner_test/tests/testdata/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/range_scan.yaml @@ -87,7 +87,7 @@ SELECT * FROM orders_count_by_user WHERE user_id > 42 AND date = 1111 AND 2>1 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean AND (orders_count_by_user.date = 1111:Int32) } + └─BatchFilter { predicate: (2:Int32 > 1:Int32) AND (orders_count_by_user.date = 1111:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id > Int64(42)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv @@ -95,7 +95,7 @@ SELECT * FROM orders_count_by_user WHERE date > 1111 AND user_id = 42 AND 5<6 AND date <= 6666 batch_plan: | BatchExchange { order: [], dist: Single } - └─BatchFilter { predicate: true:Boolean } + └─BatchFilter { predicate: (5:Int32 < 6:Int32) } └─BatchScan { table: orders_count_by_user, columns: [orders_count_by_user.user_id, orders_count_by_user.date, orders_count_by_user.orders_count], scan_ranges: [orders_count_by_user.user_id = Int64(42) AND orders_count_by_user.date > Int32(1111) AND orders_count_by_user.date <= Int32(6666)], distribution: UpstreamHashShard(orders_count_by_user.user_id, orders_count_by_user.date) } - before: - create_table_and_mv diff --git a/src/frontend/planner_test/tests/testdata/share.yaml b/src/frontend/planner_test/tests/testdata/share.yaml index 9273d92b13642..d150ff23dcd82 100644 --- a/src/frontend/planner_test/tests/testdata/share.yaml +++ b/src/frontend/planner_test/tests/testdata/share.yaml @@ -48,7 +48,7 @@ | └─StreamProject { exprs: [id, _row_id] } | └─StreamFilter { predicate: (initial_bid = 1:Int32) } | └─StreamProject { exprs: [id, initial_bid, _row_id] } - | └─StreamShare { id = 559 } + | └─StreamShare { id = 519 } | └─StreamProject { exprs: [id, initial_bid, _row_id] } | └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } | └─StreamRowIdGen { row_id_index: 9 } @@ -57,7 +57,7 @@ └─StreamProject { exprs: [id, _row_id] } └─StreamFilter { predicate: (initial_bid = 2:Int32) } └─StreamProject { exprs: [id, initial_bid, _row_id] } - └─StreamShare { id = 559 } + └─StreamShare { id = 519 } └─StreamProject { exprs: [id, initial_bid, _row_id] } └─StreamFilter { predicate: ((initial_bid = 1:Int32) OR (initial_bid = 2:Int32)) } └─StreamRowIdGen { row_id_index: 9 } @@ -125,7 +125,7 @@ | └─StreamExchange { dist: HashShard(auction, window_start) } | └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } | └─StreamProject { exprs: [auction, date_time, _row_id] } - | └─StreamShare { id = 788 } + | └─StreamShare { id = 744 } | └─StreamProject { exprs: [auction, date_time, _row_id] } | └─StreamRowIdGen { row_id_index: 4 } | └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } @@ -137,7 +137,7 @@ └─StreamExchange { dist: HashShard(auction, window_start) } └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } └─StreamProject { exprs: [auction, date_time, _row_id] } - └─StreamShare { id = 788 } + └─StreamShare { id = 744 } └─StreamProject { exprs: [auction, date_time, _row_id] } └─StreamRowIdGen { row_id_index: 4 } └─StreamSource { source: "bid", columns: ["auction", "bidder", "price", "date_time", "_row_id"] } diff --git a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml index db6be75eb1831..c40ca8c25279d 100644 --- a/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml +++ b/src/frontend/planner_test/tests/testdata/sysinfo_funcs.yaml @@ -37,5 +37,5 @@ - sql: | select current_timestamp; batch_plan: | - BatchProject { exprs: ['2021-04-01 00:00:00+00:00':Timestamptz] } + BatchProject { exprs: [Now] } └─BatchValues { rows: [[]] } diff --git a/src/frontend/planner_test/tests/testdata/tpch.yaml b/src/frontend/planner_test/tests/testdata/tpch.yaml index 34d7fa4e6246d..953cdbb3bcfd3 100644 --- a/src/frontend/planner_test/tests/testdata/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/tpch.yaml @@ -136,7 +136,7 @@ └─BatchHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─BatchProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─BatchFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], pk_columns: [l_returnflag, l_linestatus] } @@ -144,7 +144,7 @@ └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [count, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), sum(((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax))), sum(lineitem.l_quantity), count(lineitem.l_quantity), sum(lineitem.l_extendedprice), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -157,7 +157,7 @@ Fragment 1 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), ((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)) * (1:Int32 + lineitem.l_tax)), lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + StreamFilter { predicate: (lineitem.l_shipdate <= ('1998-12-01':Date - '71 days':Interval)) } Chain { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -727,7 +727,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority] } ├─BatchExchange { order: [], dist: HashShard(orders.o_orderkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey] } @@ -741,7 +741,7 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├─StreamExchange { dist: HashShard(orders.o_orderkey) } | └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } @@ -764,7 +764,7 @@ Fragment 2 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < ('1997-07-01':Date + '3 mons':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -859,7 +859,7 @@ | | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], distribution: UpstreamHashShard(customer.c_custkey) } | | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + | | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } | | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } | └─BatchExchange { order: [], dist: HashShard(supplier.s_nationkey) } | └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } @@ -884,7 +884,7 @@ | | | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + | | | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } | | | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | | | └─StreamExchange { dist: HashShard(supplier.s_nationkey) } | | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -943,7 +943,7 @@ Fragment 7 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -1020,7 +1020,7 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─BatchProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount)] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─BatchScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [revenue], pk_columns: [] } @@ -1029,7 +1029,7 @@ └─StreamExchange { dist: Single } └─StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } └─StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -1043,7 +1043,7 @@ Fragment 1 StreamStatelessLocalSimpleAgg { aggs: [count, sum((lineitem.l_extendedprice * lineitem.l_discount))] } StreamProject { exprs: [(lineitem.l_extendedprice * lineitem.l_discount), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (lineitem.l_discount >= 0.07:Decimal) AND (lineitem.l_discount <= 0.09:Decimal) AND (lineitem.l_quantity < 24:Int32) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) AND (lineitem.l_discount >= (0.08:Decimal - 0.01:Decimal)) AND (lineitem.l_discount <= (0.08:Decimal + 0.01:Decimal)) AND (lineitem.l_quantity < 24:Int32) } Chain { table: lineitem, columns: [lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_quantity, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -1350,11 +1350,11 @@ └─LogicalScan { table: region, output_columns: [region.r_regionkey], required_columns: [region.r_regionkey, region.r_name], predicate: (region.r_name = 'ASIA':Varchar) } batch_plan: | BatchExchange { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC], dist: Single } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } └─BatchSort { order: [Extract('YEAR':Varchar, orders.o_orderdate) ASC] } - └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─BatchLookupJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey AND (region.r_name = 'ASIA':Varchar), output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name] } └─BatchExchange { order: [], dist: UpstreamHashShard(nation.n_regionkey) } └─BatchLookupJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey] } @@ -1376,10 +1376,10 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + └─StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(Extract('YEAR':Varchar, orders.o_orderdate)) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } └─StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } ├─StreamExchange { dist: HashShard(nation.n_regionkey) } | └─StreamHashJoin { type: Inner, predicate: customer.c_nationkey = nation.n_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, orders.o_orderdate, nation.n_regionkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, customer.c_nationkey, nation.n_nationkey] } @@ -1418,13 +1418,13 @@ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], pk_columns: [o_year] } materialized table: 4294967294 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } - StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), RoundDigit((sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)) / sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))), 6:Int32)] } + StreamHashAgg { group_key: [Extract('YEAR':Varchar, orders.o_orderdate)], aggs: [count, sum(Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } result table: 0, state tables: [] StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate), Case((nation.n_name = 'IRAN':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, region.r_regionkey, nation.n_regionkey] } StreamHashJoin { type: Inner, predicate: nation.n_regionkey = region.r_regionkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, supplier.s_suppkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, nation.n_nationkey, customer.c_nationkey, nation.n_regionkey, region.r_regionkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([4]) from 2 @@ -1511,7 +1511,7 @@ Upstream BatchPlanNode - Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } + Table 0 { columns: [Extract('YEAR':Varchar, orders_o_orderdate), count, sum(Case((nation_n_name = 'IRAN':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal)), sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)))], primary key: [$0 ASC], value indices: [1, 2, 3], distribution key: [0] } Table 1 { columns: [lineitem_l_extendedprice, lineitem_l_discount, nation_n_name, orders_o_orderdate, nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, customer_c_nationkey, nation_n_nationkey_0], primary key: [$4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC], value indices: [0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17], distribution key: [4] } Table 2 { columns: [nation_n_regionkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, lineitem_l_partkey, supplier_s_suppkey, lineitem_l_suppkey, nation_n_nationkey, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, nation_n_nationkey_0, customer_c_nationkey, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC], value indices: [14], distribution key: [0] } Table 3 { columns: [region_r_regionkey], primary key: [$0 ASC], value indices: [0], distribution key: [0] } @@ -1832,7 +1832,7 @@ | | └─BatchScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], distribution: UpstreamHashShard(customer.c_custkey) } | └─BatchExchange { order: [], dist: HashShard(orders.o_custkey) } | └─BatchProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount] } @@ -1858,7 +1858,7 @@ | | | └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], pk: [customer.c_custkey], dist: UpstreamHashShard(customer.c_custkey) } | | └─StreamExchange { dist: HashShard(orders.o_custkey) } | | └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + | | └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } | | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } | └─StreamExchange { dist: HashShard(nation.n_nationkey) } | └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -1910,7 +1910,7 @@ Fragment 6 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < ('1994-01-01':Date + '3 mons':Interval)) } Chain { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } Upstream BatchPlanNode @@ -2229,7 +2229,7 @@ | └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], distribution: UpstreamHashShard(orders.o_orderkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_orderkey) } └─BatchProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode] } - └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], pk_columns: [l_shipmode] } @@ -2242,7 +2242,7 @@ | └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2267,7 +2267,7 @@ Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2415,27 +2415,27 @@ ├─LogicalScan { table: lineitem, output_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─LogicalScan { table: part, columns: [part.p_partkey, part.p_type] } batch_plan: | - BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + BatchProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─BatchSimpleAgg { aggs: [sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } + └─BatchSimpleAgg { aggs: [sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─BatchProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } └─BatchLookupJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type] } └─BatchExchange { order: [], dist: UpstreamHashShard(lineitem.l_partkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [promo_revenue], pk_columns: [] } - └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + └─StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamExchange { dist: Single } - └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } | └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(part.p_partkey) } └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } @@ -2443,14 +2443,14 @@ Fragment 0 StreamMaterialize { columns: [promo_revenue], pk_columns: [] } materialized table: 4294967294 - StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } - StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } + StreamProject { exprs: [((100.00:Decimal * sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)))) / sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } + StreamGlobalSimpleAgg { aggs: [count, sum(sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal))), sum(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } result table: 0, state tables: [] StreamExchange Single from 1 Fragment 1 - StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + StreamStatelessLocalSimpleAgg { aggs: [count, sum(Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal)), sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } + StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), 0:Int32::Decimal), (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } left table: 1, right table 3, left degree table: 2, right degree table: 4, StreamExchange Hash([0]) from 2 @@ -2458,7 +2458,7 @@ Fragment 2 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < '1995-10-01 00:00:00':Timestamp) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1995-09-01':Date) AND (lineitem.l_shipdate < ('1995-09-01':Date + '1 mon':Interval)) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -2468,7 +2468,7 @@ Upstream BatchPlanNode - Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } + Table 0 { columns: [count, sum(sum(Case(Like(part_p_type, 'PROMO%':Varchar), (lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount)), 0:Int32::Decimal))), sum(sum((lineitem_l_extendedprice * (1:Int32 - lineitem_l_discount))))], primary key: [], value indices: [0, 1, 2], distribution key: [] } Table 1 { columns: [lineitem_l_partkey, lineitem_l_extendedprice, lineitem_l_discount, lineitem_l_orderkey, lineitem_l_linenumber], primary key: [$0 ASC, $3 ASC, $4 ASC], value indices: [0, 1, 2, 3, 4], distribution key: [0] } Table 2 { columns: [lineitem_l_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree], primary key: [$0 ASC, $1 ASC, $2 ASC], value indices: [3], distribution key: [0] } Table 3 { columns: [part_p_partkey, part_p_type], primary key: [$0 ASC], value indices: [0, 1], distribution key: [0] } @@ -2554,7 +2554,7 @@ | └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } | └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } | └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─BatchSimpleAgg { aggs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2563,7 +2563,7 @@ └─BatchHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─BatchExchange { order: [], dist: HashShard(lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))] } - └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden), max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))(hidden)], pk_columns: [s_suppkey, lineitem.l_suppkey, total_revenue, max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2573,12 +2573,12 @@ | ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } | | └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - | └─StreamShare { id = 943 } + | └─StreamShare { id = 900 } | └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } | └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } | └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + | └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } | └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } └─StreamExchange { dist: HashShard(max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))) } └─StreamProject { exprs: [max(max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))))] } @@ -2587,12 +2587,12 @@ └─StreamHashAgg { group_key: [Vnode(lineitem.l_suppkey)], aggs: [count, max(sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))))] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount))), Vnode(lineitem.l_suppkey)] } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } - └─StreamShare { id = 943 } + └─StreamShare { id = 900 } └─StreamProject { exprs: [lineitem.l_suppkey, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [count, sum((lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)))] } └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -2623,7 +2623,7 @@ Fragment 4 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32 - lineitem.l_discount)), lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } Chain { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode @@ -3369,7 +3369,7 @@ | └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─BatchExchange { order: [], dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─BatchProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity] } - └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + └─BatchFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: | StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], pk_columns: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], order_descs: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } @@ -3404,7 +3404,7 @@ | └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [partsupp.ps_partkey, partsupp.ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: | Fragment 0 @@ -3476,7 +3476,7 @@ Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + StreamFilter { predicate: IsNotNull(lineitem.l_partkey) AND IsNotNull(lineitem.l_suppkey) AND (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < ('1994-01-01':Date + '1 year':Interval)) } Chain { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } Upstream BatchPlanNode diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 25dd1c80679ce..c511b80f358f0 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -441,7 +441,6 @@ impl PlanRoot { fn gen_batch_plan(&mut self) -> Result { // Logical optimization let mut plan = self.gen_optimized_logical_plan()?; - let ctx = plan.ctx(); // Convert the dag back to the tree, because we don't support physical dag plan for now. plan = self.optimize_by_rules( @@ -456,12 +455,13 @@ impl PlanRoot { // TODO: SessionTimezone substitution // Const eval of exprs at the last minute - plan = const_eval_exprs(plan)?; + // plan = const_eval_exprs(plan)?; - if ctx.is_explain_trace() { - ctx.trace("Const eval exprs:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + // let ctx = plan.ctx(); + // if ctx.is_explain_trace() { + // ctx.trace("Const eval exprs:"); + // ctx.trace(plan.explain_to_string().unwrap()); + // } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone()); @@ -616,12 +616,12 @@ impl PlanRoot { } // Const eval of exprs at the last minute - plan = const_eval_exprs(plan)?; + // plan = const_eval_exprs(plan)?; - if ctx.is_explain_trace() { - ctx.trace("Const eval exprs:"); - ctx.trace(plan.explain_to_string().unwrap()); - } + // if ctx.is_explain_trace() { + // ctx.trace("Const eval exprs:"); + // ctx.trace(plan.explain_to_string().unwrap()); + // } #[cfg(debug_assertions)] InputRefValidator.validate(plan.clone());