From aaf41e66ad77c896fbcd8d2692e3f5d90f5dcc49 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Mon, 20 Nov 2023 14:52:29 +0800 Subject: [PATCH] Revert "feat: new table stream graph (#12240)" This reverts commit 48bf62e6b59056b1efc5aefa9717e6c18724f546. --- e2e_test/batch/aggregate/jsonb_agg.slt.part | 94 +++---- .../basic/table_with_default_columns.slt.part | 4 +- e2e_test/batch/basic/to_jsonb.slt.part | 32 +-- .../batch/functions/array_transform.slt.part | 2 +- e2e_test/ddl/table/generated_columns.slt.part | 6 +- e2e_test/streaming/distinct_on.slt | 4 +- src/common/src/hash/consistent_hash/vnode.rs | 16 +- .../tests/testdata/output/ch_benchmark.yaml | 258 +++++++++--------- .../tests/testdata/output/create_source.yaml | 31 +-- .../tests/testdata/output/explain.yaml | 20 +- .../testdata/output/generated_columns.yaml | 30 +- .../tests/testdata/output/union.yaml | 2 +- .../tests/testdata/output/watermark.yaml | 14 +- src/frontend/src/handler/create_table.rs | 6 +- src/frontend/src/optimizer/mod.rs | 198 +++----------- .../src/optimizer/plan_node/logical_source.rs | 3 +- .../optimizer/plan_node/stream_row_id_gen.rs | 17 +- .../src/optimizer/plan_node/stream_union.rs | 11 +- src/meta/service/src/ddl_service.rs | 5 - .../integration_tests/scale/no_shuffle.rs | 7 +- 20 files changed, 287 insertions(+), 473 deletions(-) diff --git a/e2e_test/batch/aggregate/jsonb_agg.slt.part b/e2e_test/batch/aggregate/jsonb_agg.slt.part index 97eb4462b8414..fad23dda14c68 100644 --- a/e2e_test/batch/aggregate/jsonb_agg.slt.part +++ b/e2e_test/batch/aggregate/jsonb_agg.slt.part @@ -12,79 +12,79 @@ insert into t values (true, 3, 'aaa', '{}', '2021-01-01 03:00:00', '3 days'); query T -select jsonb_agg(v1 order by v1) from t; +select jsonb_agg(v1) from t; ---- -[false, true, true, null] +[null, false, true, true] query T -select jsonb_agg(v2::int2 order by v2) from t; +select jsonb_agg(v2::int2) from t; ---- -[1, 2, 3, null] +[null, 1, 2, 3] query T -select jsonb_agg(v2::int4 order by v2) from t; +select jsonb_agg(v2::int4) from t; ---- -[1, 2, 3, null] +[null, 1, 2, 3] query T -select jsonb_agg(v2::int8 order by v2) from t; +select jsonb_agg(v2::int8) from t; ---- -[1, 2, 3, null] +[null, 1, 2, 3] query T -select jsonb_agg(v2::float4 order by v2) from t; +select jsonb_agg(v2::float4) from t; ---- -[1.0, 2.0, 3.0, null] +[null, 1.0, 2.0, 3.0] query T -select jsonb_agg(v2::float8 order by v2) from t; +select jsonb_agg(v2::float8) from t; ---- -[1.0, 2.0, 3.0, null] +[null, 1.0, 2.0, 3.0] query T -select jsonb_agg(v2::decimal order by v2) from t; +select jsonb_agg(v2::decimal) from t; ---- -[1.0, 2.0, 3.0, null] +[null, 1.0, 2.0, 3.0] query T -select jsonb_agg(v3 order by v3) from t; +select jsonb_agg(v3) from t; ---- -["aaa", "bbb", "ccc", null] +[null, "ccc", "bbb", "aaa"] query T -select jsonb_agg(v3::bytea order by v3) from t; +select jsonb_agg(v3::bytea) from t; ---- -["\\x616161", "\\x626262", "\\x636363", null] +[null, "\\x636363", "\\x626262", "\\x616161"] query T -select jsonb_agg(v4 order by v4) from t; +select jsonb_agg(v4) from t; ---- -[false, null, {}, null] +[null, null, false, {}] query T -select jsonb_agg(v5::date order by v5) from t; +select jsonb_agg(v5::date) from t; ---- -["2019-01-01", "2020-01-01", "2021-01-01", null] +[null, "2019-01-01", "2020-01-01", "2021-01-01"] query T -select jsonb_agg(v5::time order by v5) from t; +select jsonb_agg(v5::time) from t; ---- -["01:00:00", "02:00:00", "03:00:00", null] +[null, "01:00:00", "02:00:00", "03:00:00"] query T -select jsonb_agg(v5::timestamp order by v5) from t; +select jsonb_agg(v5::timestamp) from t; ---- -["2019-01-01T01:00:00", "2020-01-01T02:00:00", "2021-01-01T03:00:00", null] +[null, "2019-01-01T01:00:00", "2020-01-01T02:00:00", "2021-01-01T03:00:00"] query T -select jsonb_agg(v5::timestamptz order by v5) from t; +select jsonb_agg(v5::timestamptz) from t; ---- -["2019-01-01T01:00:00+00:00", "2020-01-01T02:00:00+00:00", "2021-01-01T03:00:00+00:00", null] +[null, "2019-01-01T01:00:00+00:00", "2020-01-01T02:00:00+00:00", "2021-01-01T03:00:00+00:00"] query T -select jsonb_agg(v6 order by v6) from t; +select jsonb_agg(v6) from t; ---- -["1 day", "2 days", "3 days", null] +[null, "1 day", "2 days", "3 days"] # query T # select jsonb_agg(distinct v1) from t; @@ -97,85 +97,85 @@ select jsonb_agg(v2 order by v3 desc) from t; [null, 1, 2, 3] query T -select jsonb_agg(v2 order by v3) filter (where v3 >= 'bbb') from t; +select jsonb_agg(v2) filter (where v3 >= 'bbb') from t; ---- -[2, 1] +[1, 2] statement error field name must not be null select jsonb_object_agg(v3, v1) from t; query T -select jsonb_object_agg(v3, v1 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v1) filter (where v3 is not null) from t; ---- {"aaa": true, "bbb": true, "ccc": false} query T -select jsonb_object_agg(v3, v2::int2 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::int2) filter (where v3 is not null) from t; ---- {"aaa": 3, "bbb": 2, "ccc": 1} query T -select jsonb_object_agg(v3, v2::int4 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::int4) filter (where v3 is not null) from t; ---- {"aaa": 3, "bbb": 2, "ccc": 1} query T -select jsonb_object_agg(v3, v2::int8 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::int8) filter (where v3 is not null) from t; ---- {"aaa": 3, "bbb": 2, "ccc": 1} query T -select jsonb_object_agg(v3, v2::float4 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::float4) filter (where v3 is not null) from t; ---- {"aaa": 3.0, "bbb": 2.0, "ccc": 1.0} query T -select jsonb_object_agg(v3, v2::float8 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::float8) filter (where v3 is not null) from t; ---- {"aaa": 3.0, "bbb": 2.0, "ccc": 1.0} query T -select jsonb_object_agg(v3, v2::decimal order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v2::decimal) filter (where v3 is not null) from t; ---- {"aaa": 3.0, "bbb": 2.0, "ccc": 1.0} query T -select jsonb_object_agg(v3, v3 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v3) filter (where v3 is not null) from t; ---- {"aaa": "aaa", "bbb": "bbb", "ccc": "ccc"} query T -select jsonb_object_agg(v3, v3::bytea order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v3::bytea) filter (where v3 is not null) from t; ---- {"aaa": "\\x616161", "bbb": "\\x626262", "ccc": "\\x636363"} query T -select jsonb_object_agg(v3, v4 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v4) filter (where v3 is not null) from t; ---- {"aaa": {}, "bbb": false, "ccc": null} query T -select jsonb_object_agg(v3, v5::date order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::date) filter (where v3 is not null) from t; ---- {"aaa": "2021-01-01", "bbb": "2020-01-01", "ccc": "2019-01-01"} query T -select jsonb_object_agg(v3, v5::time order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::time) filter (where v3 is not null) from t; ---- {"aaa": "03:00:00", "bbb": "02:00:00", "ccc": "01:00:00"} query T -select jsonb_object_agg(v3, v5::timestamp order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::timestamp) filter (where v3 is not null) from t; ---- {"aaa": "2021-01-01T03:00:00", "bbb": "2020-01-01T02:00:00", "ccc": "2019-01-01T01:00:00"} query T -select jsonb_object_agg(v3, v5::timestamptz order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v5::timestamptz) filter (where v3 is not null) from t; ---- {"aaa": "2021-01-01T03:00:00+00:00", "bbb": "2020-01-01T02:00:00+00:00", "ccc": "2019-01-01T01:00:00+00:00"} query T -select jsonb_object_agg(v3, v6 order by v3) filter (where v3 is not null) from t; +select jsonb_object_agg(v3, v6) filter (where v3 is not null) from t; ---- {"aaa": "3 days", "bbb": "2 days", "ccc": "1 day"} diff --git a/e2e_test/batch/basic/table_with_default_columns.slt.part b/e2e_test/batch/basic/table_with_default_columns.slt.part index 117850683f6b9..da37cd7787de1 100644 --- a/e2e_test/batch/basic/table_with_default_columns.slt.part +++ b/e2e_test/batch/basic/table_with_default_columns.slt.part @@ -66,7 +66,7 @@ create table t2 (v1 int, v2 int default 1.5); statement ok insert into t2 values (1), (2); -query II rowsort +query II select * from t2; ---- 1 2 @@ -76,7 +76,7 @@ select * from t2; statement ok alter table t2 add column v3 timestamp with time zone default now(); -query IT rowsort +query IT select v1, v3 >= date '2021-01-01' as later_than_2021 from t2; ---- 1 t diff --git a/e2e_test/batch/basic/to_jsonb.slt.part b/e2e_test/batch/basic/to_jsonb.slt.part index 471a4de9f41ae..9cad572daf8d0 100644 --- a/e2e_test/batch/basic/to_jsonb.slt.part +++ b/e2e_test/batch/basic/to_jsonb.slt.part @@ -230,7 +230,7 @@ statement ok INSERT INTO structtype VALUES (row(6, 'v1')), (row(7, 'v2')); query T -SELECT to_jsonb(st) from structtype order by (st).key; +SELECT to_jsonb(st) from structtype; ---- {"key": 6, "val": "v1"} {"key": 7, "val": "v2"} @@ -252,37 +252,37 @@ INSERT INTO alltypes VALUES (false, (SMALLINT '972'), (INT '-235825836'), (BIGINT '842'), (REAL '27'), (FLOAT '675'), (0), DATE '2023-08-25', 'uwAFEeex9Y', TIME '22:12:37', TIMESTAMP '2023-09-01 22:11:40', '2023-09-01 22:12:37Z'::TIMESTAMPTZ, (INTERVAL '28'), CAST(NULL AS STRUCT), ARRAY[(INT '355')], ARRAY['xan6o2VHID', 'MTSy3lzImo', 'UZqnEMW60w'], BYTEA'\xAABBCCDDEEFF', '"value3"'::JSONB); query T -SELECT to_jsonb(row(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18)) from alltypes order by c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18; +SELECT to_jsonb(row(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18)) from alltypes; ---- -{"f1": false, "f10": "22:11:54", "f11": "2023-09-01T22:12:54", "f12": "2023-09-01T21:12:54+00:00", "f13": "-00:01:24", "f14": null, "f15": [246, 120, 154], "f16": ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "f17": "\\x00ff00", "f18": {"key3": ["value3", "value55"]}, "f2": 255, "f3": 1, "f4": 9223372036854775807, "f5": 0.0, "f6": 775.0, "f7": -2147483648.0, "f8": "2023-09-01", "f9": "2498VN2txc"} -{"f1": false, "f10": "22:11:17", "f11": "2023-09-01T22:11:50", "f12": "2023-09-01T21:12:54+00:00", "f13": "-00:01:31", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f17": "\\x0123456789abcdef", "f18": null, "f2": 761, "f3": 966, "f4": 153, "f5": 1023789440.0, "f6": 752.0, "f7": 630.0, "f8": "2023-09-01", "f9": "ySrgeBXDuc"} -{"f1": false, "f10": "22:12:37", "f11": "2023-09-01T22:11:40", "f12": "2023-09-01T22:12:37+00:00", "f13": "00:00:28", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f17": "\\xaabbccddeeff", "f18": "value3", "f2": 972, "f3": -235825836, "f4": 842, "f5": 27.0, "f6": 675.0, "f7": 0.0, "f8": "2023-08-25", "f9": "uwAFEeex9Y"} -{"f1": true, "f10": "22:12:25", "f11": "2023-09-01T21:12:54", "f12": "2023-09-01T21:12:54+00:00", "f13": "24:00:00", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f17": "\\xdeadbeef", "f18": {"key2": "value2"}, "f2": 82, "f3": 216, "f4": 732, "f5": 337.0, "f6": 772.0, "f7": 378.0, "f8": "2023-09-01", "f9": "6nNf6LL2C1"} {"f1": true, "f10": "22:12:54", "f11": "2023-09-01T22:12:42", "f12": "2023-09-01T22:12:42+00:00", "f13": "-00:01:18", "f14": null, "f15": [354, 627], "f16": ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "f17": "\\xdeadbeef", "f18": {"key1": "value1"}, "f2": 786, "f3": 1409922817, "f4": 925, "f5": 536.0, "f6": 782.0, "f7": 487.0, "f8": "2023-09-01", "f9": "IwfwuseZmg"} +{"f1": true, "f10": "22:12:25", "f11": "2023-09-01T21:12:54", "f12": "2023-09-01T21:12:54+00:00", "f13": "24:00:00", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f17": "\\xdeadbeef", "f18": {"key2": "value2"}, "f2": 82, "f3": 216, "f4": 732, "f5": 337.0, "f6": 772.0, "f7": 378.0, "f8": "2023-09-01", "f9": "6nNf6LL2C1"} +{"f1": false, "f10": "22:11:17", "f11": "2023-09-01T22:11:50", "f12": "2023-09-01T21:12:54+00:00", "f13": "-00:01:31", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f17": "\\x0123456789abcdef", "f18": null, "f2": 761, "f3": 966, "f4": 153, "f5": 1023789440.0, "f6": 752.0, "f7": 630.0, "f8": "2023-09-01", "f9": "ySrgeBXDuc"} +{"f1": false, "f10": "22:11:54", "f11": "2023-09-01T22:12:54", "f12": "2023-09-01T21:12:54+00:00", "f13": "-00:01:24", "f14": null, "f15": [246, 120, 154], "f16": ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "f17": "\\x00ff00", "f18": {"key3": ["value3", "value55"]}, "f2": 255, "f3": 1, "f4": 9223372036854775807, "f5": 0.0, "f6": 775.0, "f7": -2147483648.0, "f8": "2023-09-01", "f9": "2498VN2txc"} {"f1": true, "f10": "22:12:54", "f11": "2023-09-01T22:12:02", "f12": "2023-09-01T22:12:02+00:00", "f13": "01:00:00", "f14": null, "f15": [329, 577, 255, 70], "f16": ["1HQloIk7oW", "ixxNgP8vaq", "9CSOsftyRA", "jiqocRdrUC"], "f17": "\\x00ff00", "f18": {"key4": {"inner_key": "value4"}}, "f2": 933, "f3": 915, "f4": 433, "f5": 734.0, "f6": 438.0, "f7": 998512901.0, "f8": "2023-09-01", "f9": "Qgfzps4qkX"} {"f1": true, "f10": "22:12:40", "f11": "2023-09-01T22:11:27", "f12": "2023-09-01T22:12:02+00:00", "f13": "-00:01:00", "f14": null, "f15": [252, 137, 110, 574], "f16": ["t5073iSwvs"], "f17": "\\x80", "f18": null, "f2": 7096, "f3": -1627323193, "f4": 191, "f5": 483.0, "f6": 85.0, "f7": -2147483648.0, "f8": "2023-09-01", "f9": "sLgs9Am1iP"} +{"f1": false, "f10": "22:12:37", "f11": "2023-09-01T22:11:40", "f12": "2023-09-01T22:12:37+00:00", "f13": "00:00:28", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f17": "\\xaabbccddeeff", "f18": "value3", "f2": 972, "f3": -235825836, "f4": 842, "f5": 27.0, "f6": 675.0, "f7": 0.0, "f8": "2023-08-25", "f9": "uwAFEeex9Y"} query T -SELECT jsonb_build_array(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18) from alltypes order by c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18; +SELECT jsonb_build_array(c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18) from alltypes; ---- -[false, 255, 1, 9223372036854775807, 0.0, 775.0, -2147483648.0, "2023-09-01", "2498VN2txc", "22:11:54", "2023-09-01T22:12:54", "2023-09-01T21:12:54+00:00", "-00:01:24", null, [246, 120, 154], ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "\\x00ff00", {"key3": ["value3", "value55"]}] -[false, 761, 966, 153, 1023789440.0, 752.0, 630.0, "2023-09-01", "ySrgeBXDuc", "22:11:17", "2023-09-01T22:11:50", "2023-09-01T21:12:54+00:00", "-00:01:31", null, [77, 718, 401, 874], ["k6N5rUX8p1", "sFRQ1u2ihF"], "\\x0123456789abcdef", null] -[false, 972, -235825836, 842, 27.0, 675.0, 0.0, "2023-08-25", "uwAFEeex9Y", "22:12:37", "2023-09-01T22:11:40", "2023-09-01T22:12:37+00:00", "00:00:28", null, [355], ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "\\xaabbccddeeff", "value3"] -[true, 82, 216, 732, 337.0, 772.0, 378.0, "2023-09-01", "6nNf6LL2C1", "22:12:25", "2023-09-01T21:12:54", "2023-09-01T21:12:54+00:00", "24:00:00", null, [0], ["3NE5ewEx4T"], "\\xdeadbeef", {"key2": "value2"}] [true, 786, 1409922817, 925, 536.0, 782.0, 487.0, "2023-09-01", "IwfwuseZmg", "22:12:54", "2023-09-01T22:12:42", "2023-09-01T22:12:42+00:00", "-00:01:18", null, [354, 627], ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "\\xdeadbeef", {"key1": "value1"}] +[true, 82, 216, 732, 337.0, 772.0, 378.0, "2023-09-01", "6nNf6LL2C1", "22:12:25", "2023-09-01T21:12:54", "2023-09-01T21:12:54+00:00", "24:00:00", null, [0], ["3NE5ewEx4T"], "\\xdeadbeef", {"key2": "value2"}] +[false, 761, 966, 153, 1023789440.0, 752.0, 630.0, "2023-09-01", "ySrgeBXDuc", "22:11:17", "2023-09-01T22:11:50", "2023-09-01T21:12:54+00:00", "-00:01:31", null, [77, 718, 401, 874], ["k6N5rUX8p1", "sFRQ1u2ihF"], "\\x0123456789abcdef", null] +[false, 255, 1, 9223372036854775807, 0.0, 775.0, -2147483648.0, "2023-09-01", "2498VN2txc", "22:11:54", "2023-09-01T22:12:54", "2023-09-01T21:12:54+00:00", "-00:01:24", null, [246, 120, 154], ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "\\x00ff00", {"key3": ["value3", "value55"]}] [true, 933, 915, 433, 734.0, 438.0, 998512901.0, "2023-09-01", "Qgfzps4qkX", "22:12:54", "2023-09-01T22:12:02", "2023-09-01T22:12:02+00:00", "01:00:00", null, [329, 577, 255, 70], ["1HQloIk7oW", "ixxNgP8vaq", "9CSOsftyRA", "jiqocRdrUC"], "\\x00ff00", {"key4": {"inner_key": "value4"}}] [true, 7096, -1627323193, 191, 483.0, 85.0, -2147483648.0, "2023-09-01", "sLgs9Am1iP", "22:12:40", "2023-09-01T22:11:27", "2023-09-01T22:12:02+00:00", "-00:01:00", null, [252, 137, 110, 574], ["t5073iSwvs"], "\\x80", null] +[false, 972, -235825836, 842, 27.0, 675.0, 0.0, "2023-08-25", "uwAFEeex9Y", "22:12:37", "2023-09-01T22:11:40", "2023-09-01T22:12:37+00:00", "00:00:28", null, [355], ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "\\xaabbccddeeff", "value3"] query T -SELECT jsonb_build_object(c1, c1, c2, c2, c3, c3, c4, c4, c5, c5, c6, c6, c7, c7, c8, c8, c9, c9, c10, c10, c11, c11, c12, c12, c13, c13, 'f14', c14, 'f15', c15, 'f16', c16, c17, c17, 'f18', c18) from alltypes order by c1, c2, c3, c4, c5, c6, c7, c8, c9, c10, c11, c12, c13, c14, c15, c16, c17, c18; +SELECT jsonb_build_object(c1, c1, c2, c2, c3, c3, c4, c4, c5, c5, c6, c6, c7, c7, c8, c8, c9, c9, c10, c10, c11, c11, c12, c12, c13, c13, 'f14', c14, 'f15', c15, 'f16', c16, c17, c17, 'f18', c18) from alltypes; ---- -{"-00:01:24": "-00:01:24", "-2147483648": -2147483648.0, "0": 0.0, "1": 1, "2023-09-01": "2023-09-01", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "2023-09-01 22:12:54": "2023-09-01T22:12:54", "22:11:54": "22:11:54", "2498VN2txc": "2498VN2txc", "255": 255, "775": 775.0, "9223372036854775807": 9223372036854775807, "\\x00ff00": "\\x00ff00", "f14": null, "f15": [246, 120, 154], "f16": ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "f18": {"key3": ["value3", "value55"]}, "false": false} -{"-00:01:31": "-00:01:31", "1023789440": 1023789440.0, "153": 153, "2023-09-01": "2023-09-01", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "2023-09-01 22:11:50": "2023-09-01T22:11:50", "22:11:17": "22:11:17", "630": 630.0, "752": 752.0, "761": 761, "966": 966, "\\x0123456789abcdef": "\\x0123456789abcdef", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f18": null, "false": false, "ySrgeBXDuc": "ySrgeBXDuc"} -{"-235825836": -235825836, "0": 0.0, "00:00:28": "00:00:28", "2023-08-25": "2023-08-25", "2023-09-01 22:11:40": "2023-09-01T22:11:40", "2023-09-01 22:12:37+00:00": "2023-09-01T22:12:37+00:00", "22:12:37": "22:12:37", "27": 27.0, "675": 675.0, "842": 842, "972": 972, "\\xaabbccddeeff": "\\xaabbccddeeff", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f18": "value3", "false": false, "uwAFEeex9Y": "uwAFEeex9Y"} -{"2023-09-01": "2023-09-01", "2023-09-01 21:12:54": "2023-09-01T21:12:54", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "216": 216, "22:12:25": "22:12:25", "24:00:00": "24:00:00", "337": 337.0, "378": 378.0, "6nNf6LL2C1": "6nNf6LL2C1", "732": 732, "772": 772.0, "82": 82, "\\xdeadbeef": "\\xdeadbeef", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f18": {"key2": "value2"}, "true": true} {"-00:01:18": "-00:01:18", "1409922817": 1409922817, "2023-09-01": "2023-09-01", "2023-09-01 22:12:42": "2023-09-01T22:12:42", "2023-09-01 22:12:42+00:00": "2023-09-01T22:12:42+00:00", "22:12:54": "22:12:54", "487": 487.0, "536": 536.0, "782": 782.0, "786": 786, "925": 925, "IwfwuseZmg": "IwfwuseZmg", "\\xdeadbeef": "\\xdeadbeef", "f14": null, "f15": [354, 627], "f16": ["yRMgX7pFXW", "r7PAN6KB2b", "NQJbRQoVib"], "f18": {"key1": "value1"}, "true": true} +{"2023-09-01": "2023-09-01", "2023-09-01 21:12:54": "2023-09-01T21:12:54", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "216": 216, "22:12:25": "22:12:25", "24:00:00": "24:00:00", "337": 337.0, "378": 378.0, "6nNf6LL2C1": "6nNf6LL2C1", "732": 732, "772": 772.0, "82": 82, "\\xdeadbeef": "\\xdeadbeef", "f14": null, "f15": [0], "f16": ["3NE5ewEx4T"], "f18": {"key2": "value2"}, "true": true} +{"-00:01:31": "-00:01:31", "1023789440": 1023789440.0, "153": 153, "2023-09-01": "2023-09-01", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "2023-09-01 22:11:50": "2023-09-01T22:11:50", "22:11:17": "22:11:17", "630": 630.0, "752": 752.0, "761": 761, "966": 966, "\\x0123456789abcdef": "\\x0123456789abcdef", "f14": null, "f15": [77, 718, 401, 874], "f16": ["k6N5rUX8p1", "sFRQ1u2ihF"], "f18": null, "false": false, "ySrgeBXDuc": "ySrgeBXDuc"} +{"-00:01:24": "-00:01:24", "-2147483648": -2147483648.0, "0": 0.0, "1": 1, "2023-09-01": "2023-09-01", "2023-09-01 21:12:54+00:00": "2023-09-01T21:12:54+00:00", "2023-09-01 22:12:54": "2023-09-01T22:12:54", "22:11:54": "22:11:54", "2498VN2txc": "2498VN2txc", "255": 255, "775": 775.0, "9223372036854775807": 9223372036854775807, "\\x00ff00": "\\x00ff00", "f14": null, "f15": [246, 120, 154], "f16": ["Rau1Iezv50", "uWtqX1jIP0", "N356wachUq", "tDVFlmtDNk"], "f18": {"key3": ["value3", "value55"]}, "false": false} {"01:00:00": "01:00:00", "2023-09-01": "2023-09-01", "2023-09-01 22:12:02": "2023-09-01T22:12:02", "2023-09-01 22:12:02+00:00": "2023-09-01T22:12:02+00:00", "22:12:54": "22:12:54", "433": 433, "438": 438.0, "734": 734.0, "915": 915, "933": 933, "998512901": 998512901.0, "Qgfzps4qkX": "Qgfzps4qkX", "\\x00ff00": "\\x00ff00", "f14": null, "f15": [329, 577, 255, 70], "f16": ["1HQloIk7oW", "ixxNgP8vaq", "9CSOsftyRA", "jiqocRdrUC"], "f18": {"key4": {"inner_key": "value4"}}, "true": true} {"-00:01:00": "-00:01:00", "-1627323193": -1627323193, "-2147483648": -2147483648.0, "191": 191, "2023-09-01": "2023-09-01", "2023-09-01 22:11:27": "2023-09-01T22:11:27", "2023-09-01 22:12:02+00:00": "2023-09-01T22:12:02+00:00", "22:12:40": "22:12:40", "483": 483.0, "7096": 7096, "85": 85.0, "\\x80": "\\x80", "f14": null, "f15": [252, 137, 110, 574], "f16": ["t5073iSwvs"], "f18": null, "sLgs9Am1iP": "sLgs9Am1iP", "true": true} +{"-235825836": -235825836, "0": 0.0, "00:00:28": "00:00:28", "2023-08-25": "2023-08-25", "2023-09-01 22:11:40": "2023-09-01T22:11:40", "2023-09-01 22:12:37+00:00": "2023-09-01T22:12:37+00:00", "22:12:37": "22:12:37", "27": 27.0, "675": 675.0, "842": 842, "972": 972, "\\xaabbccddeeff": "\\xaabbccddeeff", "f14": null, "f15": [355], "f16": ["xan6o2VHID", "MTSy3lzImo", "UZqnEMW60w"], "f18": "value3", "false": false, "uwAFEeex9Y": "uwAFEeex9Y"} query T SELECT jsonb_build_array(); diff --git a/e2e_test/batch/functions/array_transform.slt.part b/e2e_test/batch/functions/array_transform.slt.part index 231e27a5e0e32..d07a7e5701429 100644 --- a/e2e_test/batch/functions/array_transform.slt.part +++ b/e2e_test/batch/functions/array_transform.slt.part @@ -43,7 +43,7 @@ insert into t values (4, '{1,2,3}'), (5, '{4,5,6,8}'); # this makes sure `x + 1` is not extracted as common sub-expression by accident. See #11766 query TT -select array_transform(arr, |x| x + 1), array_transform(arr, |x| x + 1 + 2) from t order by v; +select array_transform(arr, |x| x + 1), array_transform(arr, |x| x + 1 + 2) from t; ---- {2,3,4} {4,5,6} {5,6,7,9} {7,8,9,11} diff --git a/e2e_test/ddl/table/generated_columns.slt.part b/e2e_test/ddl/table/generated_columns.slt.part index 2271522a47fd9..88294172ac206 100644 --- a/e2e_test/ddl/table/generated_columns.slt.part +++ b/e2e_test/ddl/table/generated_columns.slt.part @@ -8,7 +8,7 @@ insert into t1 (v2) values (1), (2); statement ok flush; -query IIIR rowsort +query IIIR select * from t1; ---- 0 1 2 2.02 @@ -52,7 +52,7 @@ insert into t2 values (1), (2); statement ok flush; -query II rowsort +query II select * from t2; ---- 1 2 @@ -124,7 +124,7 @@ sleep 2s statement ok flush; -query TT rowsort +query TT select v, t >= date '2021-01-01' as later_than_2021 from mv; ---- 1 t diff --git a/e2e_test/streaming/distinct_on.slt b/e2e_test/streaming/distinct_on.slt index 3312ce71e97ac..f00e43eef3ff6 100644 --- a/e2e_test/streaming/distinct_on.slt +++ b/e2e_test/streaming/distinct_on.slt @@ -5,7 +5,7 @@ statement ok create table t1 (k int, v int) append only; statement ok -create materialized view mv1 as select distinct on (k) k + v as sum from t1 order by k, v; +create materialized view mv1 as select distinct on (k) k + v as sum from t1; statement ok insert into t1 values (1,11), (2,22), (3,33), (1,111); @@ -21,7 +21,7 @@ statement ok create table t2 (k int, v int); statement ok -create materialized view mv2 as select distinct on (k) k + v as sum from t2 order by k, v; +create materialized view mv2 as select distinct on (k) k + v as sum from t2; statement ok insert into t2 values (1,11), (2,22), (3,33), (1,111); diff --git a/src/common/src/hash/consistent_hash/vnode.rs b/src/common/src/hash/consistent_hash/vnode.rs index 9c8fe989aaf39..0ee3596c82d8e 100644 --- a/src/common/src/hash/consistent_hash/vnode.rs +++ b/src/common/src/hash/consistent_hash/vnode.rs @@ -119,18 +119,10 @@ impl VirtualNode { if let Ok(idx) = keys.iter().exactly_one() && let ArrayImpl::Serial(serial_array) = &**data_chunk.column_at(*idx) { - return serial_array.iter().enumerate().map(|(idx, serial)| { - if let Some(serial) = serial { - extract_vnode_id_from_row_id(serial.as_row_id()) - } else { - // NOTE: here it will hash the entire row when the `_row_id` is missing, - // which could result in rows from the same chunk being allocated to different chunks. - // This process doesn’t guarantee the order of rows, producing indeterminate results in some cases, - // such as when `distinct on` is used without an `order by`. - let (row, _) = data_chunk.row_at(idx); - row.hash(Crc32FastBuilder).into() - } - } ).collect(); + return serial_array + .iter() + .map(|serial| extract_vnode_id_from_row_id(serial.unwrap().as_row_id())) + .collect(); } data_chunk diff --git a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml index 026f3de33eebc..97e64b0867b8c 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -31,7 +31,7 @@ └─StreamExchange { dist: HashShard(order_line.ol_number) } └─StreamProject { exprs: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id] } └─StreamFilter { predicate: (order_line.ol_delivery_d > '2007-01-02 00:00:00':Timestamp) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [ol_number, sum_qty, sum_amount, avg_qty, avg_amount, count_order], stream_key: [ol_number], pk_columns: [ol_number], pk_conflict: NoCheck } { materialized table: 4294967294 } @@ -42,7 +42,7 @@ Fragment 1 StreamProject { exprs: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_number, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -119,7 +119,7 @@ │ │ └─BatchFilter { predicate: Like(item.i_data, '%b':Varchar) } │ │ └─BatchScan { table: item, columns: [item.i_id, item.i_name, item.i_data], distribution: UpstreamHashShard(item.i_id) } │ └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─BatchHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity)] } └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } └─BatchHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity] } @@ -131,7 +131,7 @@ │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard($expr1) } └─BatchProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), region.r_regionkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } └─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey) } @@ -158,14 +158,14 @@ │ │ └─StreamExchange { dist: HashShard($expr1) } │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } │ │ └─StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } │ ├─StreamExchange { dist: HashShard(item.i_id) } │ │ └─StreamProject { exprs: [item.i_id, item.i_name] } │ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─StreamExchange { dist: HashShard(supplier.s_suppkey) } └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard(region.r_regionkey) } @@ -234,7 +234,7 @@ Fragment 9 StreamProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } └── StreamFilter { predicate: (stock.s_i_id = stock.s_i_id) } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 25 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 25 } ├── Upstream └── BatchPlanNode @@ -246,7 +246,7 @@ └── BatchPlanNode Fragment 11 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 31 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 31 } ├── Upstream └── BatchPlanNode @@ -400,15 +400,15 @@ └─BatchHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, orders.o_entry_d, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = new_order.no_w_id AND orders.o_d_id = new_order.no_d_id AND orders.o_id = new_order.no_o_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], lookup table: new_order } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } │ │ └─BatchProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } │ │ └─BatchFilter { predicate: (customer.c_state >= 'a':Varchar) AND (customer.c_state < 'b':Varchar) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─BatchFilter { predicate: (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -422,17 +422,17 @@ │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id) } │ │ └─StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } │ │ └─StreamFilter { predicate: (customer.c_w_id = customer.c_w_id) AND (customer.c_d_id = customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─StreamExchange { dist: HashShard(new_order.no_d_id, new_order.no_w_id) } │ └─StreamFilter { predicate: (new_order.no_w_id = new_order.no_w_id) AND (new_order.no_d_id = new_order.no_d_id) AND (new_order.no_o_id = new_order.no_o_id) } - │ └─StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } + │ └─StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamFilter { predicate: (orders.o_d_id = orders.o_d_id) AND (orders.o_w_id = orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_w_id = order_line.ol_w_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d], stream_key: [o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_columns: [revenue, o_entry_d, ol_o_id, ol_d_id, ol_w_id], pk_conflict: NoCheck } @@ -463,25 +463,25 @@ Fragment 3 StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } └── StreamFilter { predicate: (customer.c_w_id = customer.c_w_id) AND (customer.c_d_id = customer.c_d_id) AND Like(customer.c_state, 'a%':Varchar) } - └── StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 9 } + └── StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } ├── Upstream └── BatchPlanNode Fragment 4 StreamFilter { predicate: (new_order.no_w_id = new_order.no_w_id) AND (new_order.no_d_id = new_order.no_d_id) AND (new_order.no_o_id = new_order.no_o_id) } - └── StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } { state table: 10 } + └── StreamTableScan { table: new_order, columns: [new_order.no_o_id, new_order.no_d_id, new_order.no_w_id], pk: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], dist: UpstreamHashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } { state table: 10 } ├── Upstream └── BatchPlanNode Fragment 5 StreamFilter { predicate: (orders.o_d_id = orders.o_d_id) AND (orders.o_w_id = orders.o_w_id) AND (orders.o_entry_d > '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 15 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_w_id = order_line.ol_w_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 16 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 16 } ├── Upstream └── BatchPlanNode @@ -549,7 +549,7 @@ └─BatchHashJoin { type: LeftSemi, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d >= orders.o_entry_d), output: [orders.o_ol_cnt] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_w_id, orders.o_d_id) } │ └─BatchFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d] } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d], distribution: SomeShard } @@ -560,10 +560,10 @@ └─StreamHashJoin { type: LeftSemi, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND (order_line.ol_delivery_d >= orders.o_entry_d), output: [orders.o_ol_cnt, orders.o_w_id, orders.o_d_id, orders.o_id] } ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_w_id, orders.o_d_id) } │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_ol_cnt, order_count], stream_key: [o_ol_cnt], pk_columns: [o_ol_cnt], pk_conflict: NoCheck } @@ -585,13 +585,13 @@ Fragment 2 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d < '2032-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 5 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -662,16 +662,16 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2] } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id], lookup table: stock } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_i_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_w_id) } └─BatchHashJoin { type: Inner, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id, output: [customer.c_state, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_state, orders.o_id, orders.o_d_id, orders.o_w_id] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─BatchProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } │ └─BatchFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -686,18 +686,18 @@ │ ├─StreamExchange { dist: HashShard(orders.o_id, customer.c_d_id, customer.c_w_id, orders.o_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_id, orders.o_d_id, orders.o_w_id, customer.c_id] } │ │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ │ └─StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } │ │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, stock.s_w_id) } │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, stock.s_i_id, stock.s_w_id, order_line.ol_number, order_line.ol_i_id] } │ ├─StreamExchange { dist: HashShard(order_line.ol_w_id, order_line.ol_i_id) } │ │ └─StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) } - │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } │ └─StreamExchange { dist: HashShard(stock.s_w_id, stock.s_i_id) } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) } └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_nationkey, nation.n_name, region.r_regionkey] } ├─StreamExchange { dist: HashShard(region.r_regionkey) } @@ -746,14 +746,14 @@ └── StreamExchange Hash([1, 2, 3]) from 5 Fragment 4 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 13 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 13 } ├── Upstream └── BatchPlanNode Fragment 5 StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └── StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 14 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -764,12 +764,12 @@ Fragment 7 StreamFilter { predicate: (order_line.ol_d_id = order_line.ol_d_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 19 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 8 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 20 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 20 } ├── Upstream └── BatchPlanNode @@ -894,7 +894,7 @@ └─StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } └─StreamProject { exprs: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └─StreamFilter { predicate: (order_line.ol_delivery_d >= '1997-01-01 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 100000:Int32) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } @@ -910,7 +910,7 @@ StreamStatelessSimpleAgg { aggs: [sum(order_line.ol_amount)] } └── StreamProject { exprs: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d >= '1997-01-01 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) AND (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 100000:Int32) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 1 ├── Upstream └── BatchPlanNode @@ -980,12 +980,12 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state], lookup table: customer } - └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_c_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_c_id, orders.o_d_id, orders.o_w_id) } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], lookup table: orders } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_supply_w_id AND stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(stock.s_i_id, stock.s_w_id) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount] } └─BatchFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d <= '2032-01-02 00:00:00':Timestamp) } @@ -1007,17 +1007,17 @@ │ │ ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_w_id) } │ │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_supply_w_id AND stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number] } │ │ │ ├─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id) } - │ │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } │ │ │ └─StreamExchange { dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } │ │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number] } │ │ │ └─StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d <= '2032-01-02 00:00:00':Timestamp) } - │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } │ │ └─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id, customer.c_d_id, customer.c_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_c_id = customer.c_id AND orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_c_id, customer.c_id] } │ │ ├─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─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) } └─StreamExchange { dist: HashShard(supplier.s_suppkey) } @@ -1067,14 +1067,14 @@ └── StreamExchange Hash([3, 4]) from 6 Fragment 5 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 17 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d <= '2032-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 18 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -1084,12 +1084,12 @@ └── StreamExchange Hash([1, 2, 0]) from 9 Fragment 8 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 23 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 23 } ├── Upstream └── BatchPlanNode Fragment 9 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 24 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 24 } ├── Upstream └── BatchPlanNode @@ -1246,9 +1246,9 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id AND orders.o_c_id = customer.c_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state], lookup table: customer } - └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_c_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(orders.o_c_id, orders.o_d_id, orders.o_w_id) } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp), output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], lookup table: orders } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(item.i_id, stock.s_w_id) } │ └─BatchHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } @@ -1256,7 +1256,7 @@ │ │ └─BatchScan { table: item, columns: [item.i_id], scan_ranges: [item.i_id < Int32(1000)], distribution: UpstreamHashShard(item.i_id) } │ └─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } │ └─BatchFilter { predicate: (stock.s_i_id < 1000:Int32) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } └─BatchFilter { predicate: (order_line.ol_i_id = order_line.ol_i_id) AND (order_line.ol_i_id < 1000:Int32) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount], distribution: SomeShard } @@ -1277,9 +1277,9 @@ │ │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_c_id = customer.c_id AND orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_c_id, customer.c_id] } │ │ │ ├─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ │ │ │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp) } - │ │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ │ │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ │ └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, item.i_id, order_line.ol_number] } │ │ ├─StreamExchange { dist: HashShard(item.i_id, stock.s_w_id) } @@ -1289,10 +1289,10 @@ │ │ │ │ └─StreamTableScan { table: item, columns: [item.i_id], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } │ │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } │ │ │ └─StreamFilter { predicate: (stock.s_i_id < 1000:Int32) } - │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } │ │ └─StreamExchange { dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } │ │ └─StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_i_id = order_line.ol_i_id) AND (order_line.ol_i_id < 1000:Int32) } - │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_name, supplier.s_nationkey, nation.n_nationkey] } │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -1355,12 +1355,12 @@ Fragment 5 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) AND (orders.o_entry_d <= '2032-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 17 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 17 } ├── Upstream └── BatchPlanNode Fragment 6 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 18 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 18 } ├── Upstream └── BatchPlanNode @@ -1382,13 +1382,13 @@ Fragment 10 StreamFilter { predicate: (stock.s_i_id < 1000:Int32) } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 28 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 28 } ├── Upstream └── BatchPlanNode Fragment 11 StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) AND (order_line.ol_i_id = order_line.ol_i_id) AND (order_line.ol_i_id < 1000:Int32) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 29 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 29 } ├── Upstream └── BatchPlanNode @@ -1562,9 +1562,9 @@ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } └─BatchProject { exprs: [order_line.ol_amount, orders.o_entry_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id, output: [order_line.ol_amount, stock.s_i_id, stock.s_w_id, orders.o_entry_d], lookup table: orders } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchLookupJoin { type: Inner, predicate: order_line.ol_supply_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, stock.s_i_id, stock.s_w_id], lookup table: stock } - └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_supply_w_id, order_line.ol_i_id) } + └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } └─BatchHashJoin { type: Inner, predicate: item.i_id = order_line.ol_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount] } ├─BatchExchange { order: [], dist: HashShard(item.i_id) } │ └─BatchProject { exprs: [item.i_id] } @@ -1589,14 +1589,14 @@ │ │ │ └─StreamFilter { predicate: Like(item.i_data, '%BB':Varchar) } │ │ │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_i_id, order_line.ol_supply_w_id) } │ └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: [orders.o_entry_d, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ └─StreamFilter { predicate: (order_line.ol_i_id = order_line.ol_i_id) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(supplier.s_suppkey) } └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [supplier.s_suppkey, nation.n_name, supplier.s_nationkey, nation.n_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -1647,7 +1647,7 @@ └── BatchPlanNode Fragment 5 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 14 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 14 } ├── Upstream └── BatchPlanNode @@ -1661,13 +1661,13 @@ └── StreamExchange Hash([0, 1, 2]) from 8 Fragment 7 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 19 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 8 StreamFilter { predicate: (order_line.ol_i_id = order_line.ol_i_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_supply_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 20 ├── Upstream └── BatchPlanNode @@ -1788,10 +1788,10 @@ ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─BatchFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -1806,14 +1806,14 @@ │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND order_line.ol_d_id = customer.c_d_id AND order_line.ol_w_id = customer.c_w_id, output: all } │ ├─StreamExchange { dist: HashShard(order_line.ol_d_id, order_line.ol_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ │ └─StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) } - │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } │ └─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d] } │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } │ └─StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─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) } stream_dist_plan: |+ @@ -1849,7 +1849,7 @@ Fragment 3 StreamFilter { predicate: (order_line.ol_w_id = order_line.ol_w_id) AND (order_line.ol_d_id = order_line.ol_d_id) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 9 ├── Upstream └── BatchPlanNode @@ -1864,13 +1864,13 @@ └── StreamExchange Hash([1, 2, 3]) from 6 Fragment 5 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 14 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last, customer.c_city, customer.c_state, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 14 } ├── Upstream └── BatchPlanNode Fragment 6 StreamFilter { predicate: (orders.o_entry_d >= '2007-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 15 } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 15 } ├── Upstream └── BatchPlanNode @@ -1977,7 +1977,7 @@ │ │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } │ └─BatchExchange { order: [], dist: HashShard($expr1) } │ └─BatchProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─BatchProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr4] } └─BatchSimpleAgg { aggs: [sum(sum(stock.s_order_cnt))] } └─BatchExchange { order: [], dist: Single } @@ -1989,7 +1989,7 @@ │ └─BatchScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], distribution: UpstreamHashShard(supplier.s_suppkey) } └─BatchExchange { order: [], dist: HashShard($expr3) } └─BatchProject { exprs: [stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr3] } - └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_plan: |- StreamMaterialize { columns: [s_i_id, ordercount], stream_key: [s_i_id], pk_columns: [ordercount, s_i_id], pk_conflict: NoCheck } └─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } @@ -2009,7 +2009,7 @@ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } │ └─StreamExchange { dist: HashShard($expr1) } │ └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } └─StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } @@ -2027,7 +2027,7 @@ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_i_id, ordercount], stream_key: [s_i_id], pk_columns: [ordercount, s_i_id], pk_conflict: NoCheck } @@ -2082,7 +2082,7 @@ Fragment 6 StreamProject { exprs: [stock.s_i_id, stock.s_order_cnt, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, stock.s_w_id] } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_order_cnt], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } ├── state table: 13 ├── Upstream └── BatchPlanNode @@ -2202,7 +2202,7 @@ └─BatchProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2] } └─BatchHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id AND (orders.o_entry_d <= order_line.ol_delivery_d), output: [orders.o_carrier_id, orders.o_ol_cnt] } ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d], distribution: SomeShard } @@ -2215,10 +2215,10 @@ └─StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─StreamFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_ol_cnt, high_line_count, low_line_count], stream_key: [o_ol_cnt], pk_columns: [o_ol_cnt], pk_conflict: NoCheck } @@ -2238,13 +2238,13 @@ └── StreamExchange Hash([0, 1, 2]) from 3 Fragment 2 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 5 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_carrier_id, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 5 } ├── Upstream └── BatchPlanNode Fragment 3 StreamFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -2294,11 +2294,11 @@ └─BatchExchange { order: [], dist: HashShard(customer.c_id) } └─BatchHashJoin { type: LeftOuter, predicate: customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND customer.c_id = orders.o_c_id, output: [customer.c_id, orders.o_id] } ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } └─BatchProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └─BatchFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } stream_plan: |- StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck } └─StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } @@ -2308,11 +2308,11 @@ └─StreamExchange { dist: HashShard(customer.c_id) } └─StreamHashJoin { type: LeftOuter, predicate: customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id AND customer.c_id = orders.o_c_id, output: [customer.c_id, orders.o_id, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id] } ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } └─StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └─StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck } @@ -2341,14 +2341,14 @@ └── StreamExchange Hash([1, 2, 3]) from 4 Fragment 3 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 6 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 6 } ├── Upstream └── BatchPlanNode Fragment 4 StreamProject { exprs: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id] } └── StreamFilter { predicate: (orders.o_carrier_id > 8:Int32) } - └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └── StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_carrier_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } ├── state table: 7 ├── Upstream └── BatchPlanNode @@ -2412,7 +2412,7 @@ ├─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ └─StreamFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-02 00:00:00':Timestamp) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(item.i_id) } └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } stream_dist_plan: |+ @@ -2440,7 +2440,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d >= '2007-01-02 00:00:00':Timestamp) AND (order_line.ol_delivery_d < '2030-01-02 00:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -2630,7 +2630,7 @@ └─BatchProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchLookupJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND (Not((item.i_data >= 'zz':Varchar)) OR Not((item.i_data < 'z{':Varchar))), output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data], lookup table: item } └─BatchExchange { order: [], dist: UpstreamHashShard(stock.s_i_id) } - └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_plan: |- StreamMaterialize { columns: [i_name, brand, i_price, supplier_cnt], stream_key: [i_name, brand, i_price], pk_columns: [supplier_cnt, i_name, brand, i_price], pk_conflict: NoCheck } └─StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } @@ -2642,7 +2642,7 @@ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id] } │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } │ └─StreamExchange { dist: HashShard(item.i_id) } │ └─StreamFilter { predicate: Not(Like(item.i_data, 'zz%':Varchar)) } │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_name, item.i_price, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } @@ -2678,7 +2678,7 @@ └── StreamExchange Hash([0]) from 4 Fragment 3 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 10 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 10 } ├── Upstream └── BatchPlanNode @@ -2819,7 +2819,7 @@ └─StreamHashJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id, output: all } ├─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_quantity::Decimal as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } └─StreamProject { exprs: [item.i_id, (sum(order_line.ol_quantity)::Decimal / count(order_line.ol_quantity)::Decimal) as $expr2] } └─StreamHashAgg { group_key: [item.i_id], aggs: [sum(order_line.ol_quantity), count(order_line.ol_quantity), count] } └─StreamHashJoin { type: Inner, predicate: item.i_id = order_line.ol_i_id, output: [item.i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } @@ -2828,7 +2828,7 @@ │ └─StreamFilter { predicate: Like(item.i_data, '%b':Varchar) } │ └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } └─StreamExchange { dist: HashShard(order_line.ol_i_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } @@ -2858,7 +2858,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_quantity::Decimal as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -2871,7 +2871,7 @@ └── BatchPlanNode Fragment 4 - StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 12 } + StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 12 } ├── Upstream └── BatchPlanNode @@ -2939,9 +2939,9 @@ ├─BatchExchange { order: [], dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─BatchHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt] } │ ├─BatchExchange { order: [], dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─BatchScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], distribution: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount], distribution: SomeShard } stream_plan: |- @@ -2954,11 +2954,11 @@ ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_id, customer.c_last, orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, orders.o_ol_cnt, customer.c_w_id, customer.c_d_id] } │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount)(hidden), orders.o_id(hidden), orders.o_d_id(hidden), orders.o_w_id(hidden)], stream_key: [o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_entry_d, o_ol_cnt], pk_columns: [sum(order_line.ol_amount), o_entry_d, o_id, c_last, orders.o_id, orders.o_d_id, orders.o_w_id, o_ol_cnt], pk_conflict: NoCheck } @@ -2981,17 +2981,17 @@ └── StreamExchange Hash([1, 2, 3]) from 3 Fragment 2 - StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { state table: 9 } + StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } { state table: 9 } ├── Upstream └── BatchPlanNode Fragment 3 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 10 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d, orders.o_ol_cnt], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 10 } ├── Upstream └── BatchPlanNode Fragment 4 - StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 11 } + StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 11 } ├── Upstream └── BatchPlanNode @@ -3059,7 +3059,7 @@ ├─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ └─StreamFilter { predicate: (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 10:Int32) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(item.i_id) } └─StreamProject { exprs: [item.i_id, item.i_data] } └─StreamFilter { predicate: (item.i_price >= 1:Decimal) AND (item.i_price <= 400000:Decimal) } @@ -3086,7 +3086,7 @@ Fragment 2 StreamProject { exprs: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_quantity >= 1:Int32) AND (order_line.ol_quantity <= 10:Int32) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_w_id, order_line.ol_i_id, order_line.ol_amount, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_quantity], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 5 ├── Upstream └── BatchPlanNode @@ -3166,7 +3166,7 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(stock.s_i_id) } └─BatchHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity] } ├─BatchExchange { order: [], dist: HashShard(stock.s_i_id) } - │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─BatchExchange { order: [], dist: HashShard(order_line.ol_i_id) } └─BatchProject { exprs: [order_line.ol_i_id, order_line.ol_quantity] } └─BatchFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } @@ -3191,11 +3191,11 @@ └─StreamHashJoin { type: LeftSemi, predicate: stock.s_i_id = item.i_id, output: all } ├─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, stock.s_quantity, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } │ └─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } │ └─StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } - │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ └─StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } └─StreamExchange { dist: HashShard(item.i_id) } └─StreamProject { exprs: [item.i_id] } └─StreamFilter { predicate: Like(item.i_data, 'co%':Varchar) } @@ -3244,14 +3244,14 @@ └── StreamExchange Hash([0]) from 8 Fragment 6 - StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 19 } + StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode Fragment 7 StreamProject { exprs: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } - └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └── StreamTableScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_delivery_d], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } ├── state table: 20 ├── Upstream └── BatchPlanNode @@ -3359,9 +3359,9 @@ │ └─BatchExchange { order: [], dist: UpstreamHashShard($expr1) } │ └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = stock.s_w_id AND order_line.ol_i_id = stock.s_i_id, output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, stock.s_i_id, stock.s_w_id], lookup table: stock } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_i_id) } + │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_i_id, order_line.ol_w_id) } │ └─BatchLookupJoin { type: Inner, predicate: order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id AND order_line.ol_o_id = orders.o_id AND (order_line.ol_delivery_d > orders.o_entry_d), output: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d], lookup table: orders } - │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id) } + │ └─BatchExchange { order: [], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } │ └─BatchScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d], distribution: SomeShard } └─BatchExchange { order: [], dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─BatchProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d] } @@ -3378,15 +3378,15 @@ │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_w_id AND stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = orders.o_w_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number] } │ │ ├─StreamExchange { dist: HashShard(stock.s_w_id, stock.s_i_id, stock.s_w_id) } │ │ │ └─StreamFilter { predicate: (stock.s_w_id = stock.s_w_id) } - │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } │ │ └─StreamExchange { dist: HashShard(order_line.ol_w_id, order_line.ol_i_id, orders.o_w_id) } │ │ └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, orders.o_w_id, order_line.ol_number, orders.o_d_id, orders.o_id] } │ │ └─StreamFilter { predicate: (order_line.ol_delivery_d > orders.o_entry_d) } │ │ └─StreamHashJoin { type: Inner, predicate: order_line.ol_o_id = orders.o_id AND order_line.ol_w_id = orders.o_w_id AND order_line.ol_d_id = orders.o_d_id, output: all } │ │ ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + │ │ │ └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } │ │ └─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -3397,7 +3397,7 @@ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } @@ -3430,7 +3430,7 @@ Fragment 4 StreamFilter { predicate: (stock.s_w_id = stock.s_w_id) } - └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { state table: 13 } + └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } { state table: 13 } ├── Upstream └── BatchPlanNode @@ -3442,12 +3442,12 @@ └── StreamExchange Hash([0, 1, 2]) from 7 Fragment 6 - StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 18 } + StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_i_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 18 } ├── Upstream └── BatchPlanNode Fragment 7 - StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { state table: 19 } + StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } { state table: 19 } ├── Upstream └── BatchPlanNode @@ -3470,7 +3470,7 @@ Fragment 11 StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_delivery_d, order_line.ol_number] } - └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { state table: 26 } + └── StreamTableScan { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 26 } ├── Upstream └── BatchPlanNode @@ -3582,7 +3582,7 @@ │ ├─BatchExchange { order: [], dist: HashShard(customer.c_id, customer.c_w_id, customer.c_d_id) } │ │ └─BatchProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance] } │ │ └─BatchFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) } - │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], distribution: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─BatchScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], distribution: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─BatchExchange { order: [], dist: HashShard(orders.o_c_id, orders.o_w_id, orders.o_d_id) } │ └─BatchProject { exprs: [orders.o_c_id, orders.o_w_id, orders.o_d_id] } │ └─BatchScan { table: orders, columns: [orders.o_d_id, orders.o_w_id, orders.o_c_id], distribution: SomeShard } @@ -3604,10 +3604,10 @@ │ ├─StreamExchange { dist: HashShard(customer.c_id, customer.c_w_id, customer.c_d_id) } │ │ └─StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance] } │ │ └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_state, customer.c_balance, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } │ └─StreamExchange { dist: HashShard(orders.o_c_id, orders.o_w_id, orders.o_d_id) } │ └─StreamProject { exprs: [orders.o_c_id, orders.o_w_id, orders.o_d_id, orders.o_id] } - │ └─StreamTableScan { table: orders, columns: [orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + │ └─StreamTableScan { table: orders, columns: [orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_id], pk: [orders.o_w_id, orders.o_d_id, orders.o_id], dist: UpstreamHashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } └─StreamExchange { dist: Broadcast } └─StreamProject { exprs: [(sum(sum(customer.c_balance)) / sum0(count(customer.c_balance))::Decimal) as $expr1] } └─StreamSimpleAgg { aggs: [sum(sum(customer.c_balance)), sum0(count(customer.c_balance)), count] } @@ -3615,4 +3615,4 @@ └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_balance), count(customer.c_balance)] } └─StreamProject { exprs: [customer.c_balance, customer.c_w_id, customer.c_d_id, customer.c_id] } └─StreamFilter { predicate: (customer.c_balance > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 1:Int32), '1':Varchar, '2':Varchar, '3':Varchar, '4':Varchar, '5':Varchar, '6':Varchar, '7':Varchar) } - └─StreamTableScan { table: customer, columns: [customer.c_balance, customer.c_w_id, customer.c_d_id, customer.c_id, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + └─StreamTableScan { table: customer, columns: [customer.c_balance, customer.c_w_id, customer.c_d_id, customer.c_id, customer.c_phone], pk: [customer.c_w_id, customer.c_d_id, customer.c_id], dist: UpstreamHashShard(customer.c_id, customer.c_d_id, customer.c_w_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index 2aeea6f914d2b..fe228592ba35f 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -21,13 +21,10 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = ',', without_header = true); explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamUnion { all: true } - ├─StreamExchange [no_shuffle] { dist: SomeShard } - │ └─StreamSource { source: s0, columns: [v1, v2, _row_id] } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource { source: s0, columns: [v1, v2, _row_id] } - id: csv_delimiter_tab sql: | explain create table s0 (v1 int, v2 varchar) with ( @@ -38,13 +35,10 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamUnion { all: true } - ├─StreamExchange [no_shuffle] { dist: SomeShard } - │ └─StreamSource { source: s0, columns: [v1, v2, _row_id] } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource { source: s0, columns: [v1, v2, _row_id] } - id: create_source_with_cdc_backfill sql: | create source mysql_mydb with ( @@ -83,11 +77,8 @@ ) from mysql_mydb table 'mydb.t1'; explain_output: | StreamMaterialize { columns: [v1, v2], stream_key: [v1], pk_columns: [v1], pk_conflict: Overwrite } - └─StreamUnion { all: true } - ├─StreamExchange { dist: HashShard(mydb.t1.v1) } - │ └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } - └─StreamExchange { dist: HashShard(v1) } - └─StreamDml { columns: [v1, v2] } - └─StreamSource + └─StreamExchange { dist: HashShard(mydb.t1.v1) } + └─StreamDml { columns: [v1, v2] } + └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } with_config_map: CDC_BACKFILL: 'true' diff --git a/src/frontend/planner_test/tests/testdata/output/explain.yaml b/src/frontend/planner_test/tests/testdata/output/explain.yaml index e44431ffcd3d7..a80d7ce274644 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain.yaml @@ -155,19 +155,15 @@ explain create table t (v1 int, v2 varchar); explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamUnion { all: true } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource - sql: | explain create table t (v1 int, v2 varchar) with ( connector = 'kafka', kafka.topic = 'kafka_3_partition_topic', kafka.brokers = '127.0.0.1:1234', kafka.scan.startup.mode='earliest' ) FORMAT PLAIN ENCODE JSON; explain_output: | StreamMaterialize { columns: [v1, v2, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamRowIdGen { row_id_index: 2 } - └─StreamUnion { all: true } - ├─StreamExchange [no_shuffle] { dist: SomeShard } - │ └─StreamSource { source: t, columns: [v1, v2, _row_id] } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamDml { columns: [v1, v2, _row_id] } - └─StreamSource + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamRowIdGen { row_id_index: 2 } + └─StreamDml { columns: [v1, v2, _row_id] } + └─StreamSource { source: t, columns: [v1, v2, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml index e88df797de0c8..a3acaf174d237 100644 --- a/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml +++ b/src/frontend/planner_test/tests/testdata/output/generated_columns.yaml @@ -4,12 +4,11 @@ explain create table t1 (v1 int as v2-1, v2 int, v3 int as v2+1); explain_output: | StreamMaterialize { columns: [v1, v2, v3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamUnion { all: true } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [(v2 - 1:Int32) as $expr1, v2, (v2 + 1:Int32) as $expr2, _row_id] } - └─StreamDml { columns: [v2, _row_id] } - └─StreamSource + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamRowIdGen { row_id_index: 3 } + └─StreamProject { exprs: [(v2 - 1:Int32) as $expr1, v2, (v2 + 1:Int32) as $expr2, _row_id] } + └─StreamDml { columns: [v2, _row_id] } + └─StreamSource - name: source with generated columns sql: | create source s1 (v1 int as v2-1, v2 int, v3 int as v2+1) with (connector = 'kinesis') FORMAT PLAIN ENCODE JSON; @@ -31,12 +30,11 @@ explain create table t1 (proc_time TIMESTAMP AS proctime()); explain_output: | StreamMaterialize { columns: [proc_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: Overwrite, watermark_columns: [proc_time] } - └─StreamRowIdGen { row_id_index: 1 } - └─StreamUnion { all: true, output_watermarks: [$expr1] } - └─StreamExchange { dist: HashShard(_row_id) } - └─StreamProject { exprs: [AtTimeZone(Proctime, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } - └─StreamDml { columns: [_row_id] } - └─StreamSource + └─StreamExchange { dist: HashShard(_row_id) } + └─StreamRowIdGen { row_id_index: 1 } + └─StreamProject { exprs: [AtTimeZone(Proctime, 'UTC':Varchar) as $expr1, _row_id], output_watermarks: [$expr1] } + └─StreamDml { columns: [_row_id] } + └─StreamSource - name: watermark on generated column sql: | explain create table t (v int, w int as v+1, watermark for w as w) append only @@ -44,8 +42,6 @@ StreamMaterialize { columns: [v, w, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [w] } └─StreamRowIdGen { row_id_index: 2 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: $expr1 }], output_watermarks: [$expr1] } - └─StreamUnion { all: true } - └─StreamExchange [no_shuffle] { dist: SomeShard } - └─StreamProject { exprs: [v, (v + 1:Int32) as $expr1, _row_id] } - └─StreamDml { columns: [v, _row_id] } - └─StreamSource + └─StreamProject { exprs: [v, (v + 1:Int32) as $expr1, _row_id] } + └─StreamDml { columns: [v, _row_id] } + └─StreamSource diff --git a/src/frontend/planner_test/tests/testdata/output/union.yaml b/src/frontend/planner_test/tests/testdata/output/union.yaml index 48f45722494ac..725aab00d5a32 100644 --- a/src/frontend/planner_test/tests/testdata/output/union.yaml +++ b/src/frontend/planner_test/tests/testdata/output/union.yaml @@ -596,7 +596,7 @@ Fragment 4 StreamProject { exprs: [t4.a, t4.b, t4.c, 3:Int32] } - └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c], pk: [t4.b, t4.a], dist: UpstreamHashShard(t4.b, t4.a) } + └── StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c], pk: [t4.b, t4.a], dist: UpstreamHashShard(t4.a, t4.b) } ├── state table: 3 ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index cc54951beec4a..d57d41fa76bc3 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -51,12 +51,8 @@ StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamUnion { all: true } - ├─StreamExchange [no_shuffle] { dist: SomeShard } - │ └─StreamSource { source: t, columns: [v1, _row_id] } - └─StreamExchange [no_shuffle] { dist: SomeShard } - └─StreamDml { columns: [v1, _row_id] } - └─StreamSource + └─StreamDml { columns: [v1, _row_id] } + └─StreamSource { source: t, columns: [v1, _row_id] } - name: watermark on append only table without source sql: | explain create table t (v1 timestamp with time zone, watermark for v1 as v1 - INTERVAL '1' SECOND) append only; @@ -64,10 +60,8 @@ StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck, watermark_columns: [v1] } └─StreamRowIdGen { row_id_index: 1 } └─StreamWatermarkFilter { watermark_descs: [Desc { column: v1, expr: (v1 - '00:00:01':Interval) }], output_watermarks: [v1] } - └─StreamUnion { all: true } - └─StreamExchange [no_shuffle] { dist: SomeShard } - └─StreamDml { columns: [v1, _row_id] } - └─StreamSource + └─StreamDml { columns: [v1, _row_id] } + └─StreamSource - name: hash agg sql: | create table t (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 92b5d2f8d5c4e..ab7cedcbbddb1 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -698,8 +698,6 @@ fn gen_table_plan_inner( let connection_id = resolve_privatelink_in_with_option(&mut with_options, &schema_name, &session)?; - let is_external_source = source_info.is_some(); - let source = source_info.map(|source_info| PbSource { id: TableId::placeholder().table_id, schema_id, @@ -778,7 +776,6 @@ fn gen_table_plan_inner( append_only, watermark_descs, version, - is_external_source, )?; let mut table = materialize.table().to_prost(schema_id, database_id); @@ -887,9 +884,8 @@ pub(crate) fn gen_create_table_plan_for_cdc_source( pk_column_ids, None, append_only, - vec![], + vec![], // no watermarks Some(col_id_gen.into_version()), - true, )?; let mut table = materialize.table().to_prost(schema_id, database_id); diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 70d5ae1769c83..7ee62bab1cb6c 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -17,6 +17,7 @@ use std::ops::DerefMut; pub mod plan_node; pub use plan_node::{Explain, PlanRef}; + pub mod property; mod delta_join_solver; @@ -38,9 +39,7 @@ pub use logical_optimization::*; pub use optimizer_context::*; use plan_expr_rewriter::ConstEvalRewriter; use property::Order; -use risingwave_common::catalog::{ - ColumnCatalog, ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, -}; +use risingwave_common::catalog::{ColumnCatalog, ColumnId, ConflictBehavior, Field, Schema}; use risingwave_common::error::{ErrorCode, Result}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::iter_util::ZipEqDebug; @@ -60,10 +59,8 @@ use self::plan_visitor::{has_batch_exchange, CardinalityVisitor}; use self::property::{Cardinality, RequiredDist}; use self::rule::*; use crate::catalog::table_catalog::{TableType, TableVersion}; -use crate::optimizer::plan_node::generic::Union; use crate::optimizer::plan_node::{ - BatchExchange, PlanNodeType, PlanTreeNode, RewriteExprsRecursive, StreamExchange, StreamUnion, - ToStream, + BatchExchange, PlanNodeType, PlanTreeNode, RewriteExprsRecursive, }; use crate::optimizer::plan_visitor::TemporalJoinValidator; use crate::optimizer::property::Distribution; @@ -434,156 +431,29 @@ impl PlanRoot { append_only: bool, watermark_descs: Vec, version: Option, - with_external_source: bool, ) -> Result { - let stream_plan = self.gen_optimized_stream_plan(false)?; - - assert!(!pk_column_ids.is_empty() || row_id_index.is_some()); - - let pk_column_indices = { - let mut id_to_idx = HashMap::new(); + let mut stream_plan = self.gen_optimized_stream_plan(false)?; - columns.iter().enumerate().for_each(|(idx, c)| { - id_to_idx.insert(c.column_id(), idx); - }); - pk_column_ids + // Add DML node. + stream_plan = StreamDml::new( + stream_plan, + append_only, + columns .iter() - .map(|c| id_to_idx.get(c).copied().unwrap()) // pk column id must exist in table columns. - .collect_vec() - }; - - fn inject_project_if_needed(columns: &[ColumnCatalog], node: PlanRef) -> Result { - let exprs = LogicalSource::derive_output_exprs_from_generated_columns(columns)?; - if let Some(exprs) = exprs { - let logical_project = generic::Project::new(exprs, node); - return Ok(StreamProject::new(logical_project).into()); - } - Ok(node) - } - - fn inject_dml_node( - columns: &[ColumnCatalog], - append_only: bool, - stream_plan: PlanRef, - pk_column_indices: &[usize], - kind: PrimaryKeyKind, - column_descs: Vec, - ) -> Result { - let mut dml_node = StreamDml::new(stream_plan, append_only, column_descs).into(); - - // Add generated columns. - dml_node = inject_project_if_needed(columns, dml_node)?; - - dml_node = match kind { - PrimaryKeyKind::UserDefinedPrimaryKey | PrimaryKeyKind::RowIdAsPrimaryKey => { - RequiredDist::hash_shard(pk_column_indices) - .enforce_if_not_satisfies(dml_node, &Order::any())? - } - PrimaryKeyKind::AppendOnly => StreamExchange::new_no_shuffle(dml_node).into(), - }; - - Ok(dml_node) - } - - #[derive(PartialEq, Debug, Copy, Clone)] - enum PrimaryKeyKind { - UserDefinedPrimaryKey, - RowIdAsPrimaryKey, - AppendOnly, - } - - let kind = if append_only { - assert!(row_id_index.is_some()); - PrimaryKeyKind::AppendOnly - } else if let Some(row_id_index) = row_id_index { - assert_eq!( - pk_column_indices.iter().exactly_one().copied().unwrap(), - row_id_index - ); - PrimaryKeyKind::RowIdAsPrimaryKey - } else { - PrimaryKeyKind::UserDefinedPrimaryKey - }; - - let column_descs = columns - .iter() - .filter(|&c| (!c.is_generated())) - .map(|c| c.column_desc.clone()) - .collect(); - - let union_inputs = if with_external_source { - let mut external_source_node = stream_plan; - external_source_node = inject_project_if_needed(&columns, external_source_node)?; - external_source_node = match kind { - PrimaryKeyKind::UserDefinedPrimaryKey => { - RequiredDist::hash_shard(&pk_column_indices) - .enforce_if_not_satisfies(external_source_node, &Order::any())? - } - PrimaryKeyKind::RowIdAsPrimaryKey | PrimaryKeyKind::AppendOnly => { - StreamExchange::new_no_shuffle(external_source_node).into() - } - }; - - let dummy_source_node = LogicalSource::new( - None, - columns.clone(), - row_id_index, - false, - true, - context.clone(), - ) - .and_then(|s| s.to_stream(&mut ToStreamContext::new(false)))?; - - let dml_node = inject_dml_node( - &columns, - append_only, - dummy_source_node, - &pk_column_indices, - kind, - column_descs, - )?; - - vec![external_source_node, dml_node] - } else { - let dml_node = inject_dml_node( - &columns, - append_only, - stream_plan, - &pk_column_indices, - kind, - column_descs, - )?; - - vec![dml_node] - }; - - let dists = union_inputs - .iter() - .map(|input| input.distribution()) - .unique() - .collect_vec(); - - let dist = match &dists[..] { - &[Distribution::SomeShard, Distribution::HashShard(_)] - | &[Distribution::HashShard(_), Distribution::SomeShard] => Distribution::SomeShard, - &[dist @ Distribution::SomeShard] | &[dist @ Distribution::HashShard(_)] => { - dist.clone() - } - _ => { - unreachable!() - } - }; - - let mut stream_plan = StreamUnion::new_with_dist( - Union { - all: true, - inputs: union_inputs, - source_col: None, - }, - dist.clone(), + .filter(|&c| (!c.is_generated())) + .map(|c| c.column_desc.clone()) + .collect(), ) .into(); + // Add generated columns. + let exprs = LogicalSource::derive_output_exprs_from_generated_columns(&columns)?; + if let Some(exprs) = exprs { + let logical_project = generic::Project::new(exprs, stream_plan); + // The project node merges a chunk if it has an ungenerated row id as stream key. + stream_plan = StreamProject::new(logical_project).into(); + } + // Add WatermarkFilter node. if !watermark_descs.is_empty() { stream_plan = StreamWatermarkFilter::new(stream_plan, watermark_descs).into(); @@ -591,19 +461,7 @@ impl PlanRoot { // Add RowIDGen node if needed. if let Some(row_id_index) = row_id_index { - match kind { - PrimaryKeyKind::UserDefinedPrimaryKey => { - unreachable!() - } - PrimaryKeyKind::RowIdAsPrimaryKey | PrimaryKeyKind::AppendOnly => { - stream_plan = StreamRowIdGen::new_with_dist( - stream_plan, - row_id_index, - Distribution::HashShard(vec![row_id_index]), - ) - .into(); - } - } + stream_plan = StreamRowIdGen::new(stream_plan, row_id_index).into(); } let conflict_behavior = match append_only { @@ -611,6 +469,18 @@ impl PlanRoot { false => ConflictBehavior::Overwrite, }; + let pk_column_indices = { + let mut id_to_idx = HashMap::new(); + + columns.iter().enumerate().for_each(|(idx, c)| { + id_to_idx.insert(c.column_id(), idx); + }); + pk_column_ids + .iter() + .map(|c| id_to_idx.get(c).copied().unwrap()) // pk column id must exist in table columns. + .collect_vec() + }; + let table_required_dist = { let mut bitset = FixedBitSet::with_capacity(columns.len()); for idx in &pk_column_indices { @@ -828,7 +698,7 @@ mod tests { let subplan = root.into_subplan(); assert_eq!( subplan.schema(), - &Schema::new(vec![Field::with_name(DataType::Int32, "v1")]) + &Schema::new(vec![Field::with_name(DataType::Int32, "v1"),]) ); } } diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 30c4ee19002ad..542178a830b73 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -45,7 +45,6 @@ use crate::optimizer::plan_node::{ ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamDedup, ToStreamContext, }; -use crate::optimizer::property::Distribution::HashShard; use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; @@ -575,7 +574,7 @@ impl ToStream for LogicalSource { if let Some(row_id_index) = self.core.row_id_index && self.core.gen_row_id { - plan = StreamRowIdGen::new_with_dist(plan, row_id_index, HashShard(vec![row_id_index])).into(); + plan = StreamRowIdGen::new(plan, row_id_index).into(); } Ok(plan) } diff --git a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs index b7634c652a34d..203bf8951493f 100644 --- a/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs +++ b/src/frontend/src/optimizer/plan_node/stream_row_id_gen.rs @@ -30,16 +30,13 @@ pub struct StreamRowIdGen { impl StreamRowIdGen { pub fn new(input: PlanRef, row_id_index: usize) -> Self { - let distribution = input.distribution().clone(); - Self::new_with_dist(input, row_id_index, distribution) - } + let distribution = if input.append_only() { + // remove exchange for append only source + Distribution::HashShard(vec![row_id_index]) + } else { + input.distribution().clone() + }; - /// Create a new `StreamRowIdGen` with a custom distribution. - pub fn new_with_dist( - input: PlanRef, - row_id_index: usize, - distribution: Distribution, - ) -> StreamRowIdGen { let base = PlanBase::new_stream( input.ctx(), input.schema().clone(), @@ -71,7 +68,7 @@ impl PlanTreeNodeUnary for StreamRowIdGen { } fn clone_with_input(&self, input: PlanRef) -> Self { - Self::new_with_dist(input, self.row_id_index, self.distribution().clone()) + Self::new(input, self.row_id_index) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_union.rs b/src/frontend/src/optimizer/plan_node/stream_union.rs index c67884ac10b27..425cdc6914564 100644 --- a/src/frontend/src/optimizer/plan_node/stream_union.rs +++ b/src/frontend/src/optimizer/plan_node/stream_union.rs @@ -26,7 +26,6 @@ use super::utils::{childless_record, watermark_pretty, Distill}; use super::{generic, ExprRewritable, PlanRef}; use crate::optimizer::plan_node::generic::GenericPlanNode; use crate::optimizer::plan_node::{PlanBase, PlanTreeNode, StreamNode}; -use crate::optimizer::property::Distribution; use crate::stream_fragmenter::BuildFragmentGraphState; /// `StreamUnion` implements [`super::LogicalUnion`] @@ -41,12 +40,6 @@ impl StreamUnion { let inputs = &core.inputs; let dist = inputs[0].distribution().clone(); assert!(inputs.iter().all(|input| *input.distribution() == dist)); - Self::new_with_dist(core, dist) - } - - pub fn new_with_dist(core: generic::Union, dist: Distribution) -> Self { - let inputs = &core.inputs; - let watermark_columns = inputs.iter().fold( { let mut bitset = FixedBitSet::with_capacity(core.schema().len()); @@ -63,7 +56,6 @@ impl StreamUnion { inputs.iter().all(|x| x.emit_on_window_close()), watermark_columns, ); - StreamUnion { base, core } } } @@ -86,8 +78,7 @@ impl PlanTreeNode for StreamUnion { fn clone_with_inputs(&self, inputs: &[crate::optimizer::PlanRef]) -> PlanRef { let mut new = self.core.clone(); new.inputs = inputs.to_vec(); - let dist = self.distribution().clone(); - Self::new_with_dist(new, dist).into() + Self::new(new).into() } } diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index e30c430b420cb..3f40f76a6587c 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -868,11 +868,6 @@ fn fill_table_stream_graph_info( for fragment in fragment_graph.fragments.values_mut() { visit_fragment(fragment, |node_body| { if let NodeBody::Source(source_node) = node_body { - if source_node.source_inner.is_none() { - // skip empty source for dml node - return; - } - // If we're creating a table with connector, we should additionally fill its ID first. if let Some(&mut (ref mut source, source_id)) = source_info.as_mut() { source.id = source_id; diff --git a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs index ed65c8d5894e3..6b04503ab2bc5 100644 --- a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs +++ b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs @@ -15,7 +15,7 @@ use anyhow::Result; use itertools::Itertools; use risingwave_simulation::cluster::{Cluster, Configuration}; -use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; +use risingwave_simulation::ctl_ext::predicate::identity_contains; use risingwave_simulation::utils::AssertResult; #[tokio::test] @@ -48,10 +48,7 @@ async fn test_delta_join() -> Result<()> { .await?; assert_eq!(lookup_fragments.len(), 2, "failed to plan delta join"); let union_fragment = cluster - .locate_one_fragment([ - identity_contains("union"), - no_identity_contains("materialize"), // skip union for table - ]) + .locate_one_fragment([identity_contains("union")]) .await?; let mut test_times = 0;