diff --git a/Cargo.lock b/Cargo.lock index be049fb158fb0..b1f28fdb8185c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -17,6 +17,12 @@ version = "1.0.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f26201604c87b1e01bd3d98f8d5d9a8fcbb815e8cedb41ffccbeb4bf593a35fe" +[[package]] +name = "adler2" +version = "2.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "512761e0bb2578dd7380c6baaa0f4ce03e84f95e960231d1dec8bf4d7d6e2627" + [[package]] name = "adler32" version = "1.2.0" @@ -1181,27 +1187,26 @@ dependencies = [ [[package]] name = "aws-lc-rs" -version = "1.6.2" +version = "1.12.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "df33e4a55b03f8780ba55041bc7be91a2a8ec8c03517b0379d2d6c96d2c30d95" +checksum = "1ea835662a0af02443aa1396d39be523bbf8f11ee6fad20329607c480bea48c3" dependencies = [ "aws-lc-sys", - "mirai-annotations", "paste", "zeroize", ] [[package]] name = "aws-lc-sys" -version = "0.13.3" +version = "0.25.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "37ede3d6e360a48436fee127cb81710834407b1ec0c48a001cc29dec9005f73e" +checksum = "71b2ddd3ada61a305e1d8bb6c005d1eaa7d14d903681edfc400406d523a9b491" dependencies = [ - "bindgen", + "bindgen 0.69.5", + "cc", "cmake", "dunce", "fs_extra", - "libc", "paste", ] @@ -1776,7 +1781,7 @@ dependencies = [ "cc", "cfg-if", "libc", - "miniz_oxide", + "miniz_oxide 0.7.2", "object 0.32.1", "rustc-demangle", ] @@ -1916,14 +1921,14 @@ dependencies = [ [[package]] name = "bindgen" -version = "0.69.4" +version = "0.69.5" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a00dc851838a2120612785d195287475a3ac45514741da670b735818822129a0" +checksum = "271383c67ccabffb7381723dea0672a673f292304fcb45c01cc648c7a8d58088" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "cexpr", "clang-sys", - "itertools 0.12.1", + "itertools 0.10.5", "lazy_static", "lazycell", "log", @@ -1931,12 +1936,30 @@ dependencies = [ "proc-macro2", "quote", "regex", - "rustc-hash", + "rustc-hash 1.1.0", "shlex", "syn 2.0.87", "which", ] +[[package]] +name = "bindgen" +version = "0.71.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "5f58bf3d7db68cfbac37cfc485a8d711e87e064c3d0fe0435b92f7a407f9d6b3" +dependencies = [ + "bitflags 2.8.0", + "cexpr", + "clang-sys", + "itertools 0.10.5", + "proc-macro2", + "quote", + "regex", + "rustc-hash 2.1.0", + "shlex", + "syn 2.0.87", +] + [[package]] name = "bit-set" version = "0.8.0" @@ -1966,9 +1989,9 @@ checksum = "bef38d45163c2f1dde094a7dfd33ccf595c92905c8f8f4fdc18d06fb1037718a" [[package]] name = "bitflags" -version = "2.7.0" +version = "2.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "1be3f42a67d6d345ecd59f675f3f012d6974981560836e938c22b424b85ce1be" +checksum = "8f68f53c83ab957f72c32642f3868eec03eb974d1fb82e453128456482613d36" dependencies = [ "serde", ] @@ -2819,7 +2842,7 @@ dependencies = [ "hashbrown 0.14.5", "log", "regalloc2", - "rustc-hash", + "rustc-hash 1.1.0", "smallvec", "target-lexicon", ] @@ -3082,7 +3105,7 @@ version = "0.27.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f476fe445d41c9e991fd07515a6f463074b782242ccf4a5b7b1d1012e70824df" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "crossterm_winapi", "libc", "parking_lot 0.12.1", @@ -4065,7 +4088,7 @@ dependencies = [ "lazy_static", "mintex", "parking_lot 0.12.1", - "rustc-hash", + "rustc-hash 1.1.0", "serde", "serde_json", "thousands", @@ -4323,13 +4346,14 @@ dependencies = [ [[package]] name = "elasticsearch" -version = "8.15.0-alpha.1" +version = "8.17.0-alpha.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d99147dd351d320432ec103a20804cbb593d3d08fd2eed0ee86f21a0f5137a3c" +checksum = "52be486463ef0b89e45191803db146387d5f594c26a0c8790807bb3e988ec5f6" dependencies = [ "base64 0.22.0", "bytes", "dyn-clone", + "flate2", "lazy_static", "percent-encoding", "reqwest 0.12.4", @@ -4844,13 +4868,13 @@ dependencies = [ [[package]] name = "flate2" -version = "1.0.27" +version = "1.0.35" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c6c98ee8095e9d1dcbf2fcc6d95acccb90d1c81db1e44725c6a984b1dbdfb010" +checksum = "c936bfdafb507ebbf50b8074c54fa31c5be9a1e7e5f467dd659697041407d07c" dependencies = [ "crc32fast", "libz-sys", - "miniz_oxide", + "miniz_oxide 0.8.2", ] [[package]] @@ -4983,7 +5007,7 @@ checksum = "b278703206c79f02ef555b78e5535b6c0831ad2165ab316a5cb4098b3bc28902" dependencies = [ "ahash 0.8.11", "arc-swap", - "bitflags 2.7.0", + "bitflags 2.8.0", "cmsketch", "equivalent", "fastrace", @@ -5324,7 +5348,7 @@ version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "27d12c0aed7f1e24276a241aadc4cb8ea9f83000f34bc062b7cc2d51e3b0fabd" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "debugid", "fxhash", "serde", @@ -6356,7 +6380,7 @@ version = "0.11.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f37dccff2791ab604f9babef0ba14fbe0be30bd368dc541e2b08d07c8aa908f3" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "inotify-sys", "libc", ] @@ -6386,7 +6410,7 @@ version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "bd05e4e63529f3c9c5f5c668c398217f72756ffe48c85266b49692c55accd1f7" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "crossterm 0.25.0", "dyn-clone", "fuzzy-matcher", @@ -6925,7 +6949,7 @@ version = "0.0.1" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "85c833ca1e66078851dba29046874e38f08b2c883700aa29a03ddd3b23814ee8" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "libc", "redox_syscall 0.4.1", ] @@ -6936,7 +6960,7 @@ version = "0.1.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "c0ff37bd590ca25063e35af745c343cb7a0271906fb7b37e4813e8f79f00268d" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "libc", "redox_syscall 0.5.7", ] @@ -6966,9 +6990,9 @@ dependencies = [ [[package]] name = "libz-sys" -version = "1.1.12" +version = "1.1.21" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d97137b25e321a73eef1418d1d5d2eda4d77e12813f8e6dead84bc52c5870a7b" +checksum = "df9b68e50e6e0b26f672573834882eb57759f6db9b3be2ea3c35c91188bb4eaa" dependencies = [ "cc", "libc", @@ -7427,6 +7451,15 @@ dependencies = [ "adler", ] +[[package]] +name = "miniz_oxide" +version = "0.8.2" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "4ffbe83022cedc1d264172192511ae958937694cd57ce297164951b8b3568394" +dependencies = [ + "adler2", +] + [[package]] name = "minstant" version = "0.1.7" @@ -7468,12 +7501,6 @@ dependencies = [ "windows-sys 0.52.0", ] -[[package]] -name = "mirai-annotations" -version = "1.12.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c9be0862c1b3f26a88803c4a49de6889c10e608b3ee9344e6ef5b45fb37ad3d1" - [[package]] name = "mixtrics" version = "0.0.2" @@ -7670,8 +7697,8 @@ checksum = "8a60cb978c0a1d654edcc1460f8d6092dacf21346ed6017d81fb76a23ef5a8de" dependencies = [ "base64 0.21.7", "bigdecimal 0.4.5", - "bindgen", - "bitflags 2.7.0", + "bindgen 0.71.1", + "bitflags 2.8.0", "bitvec", "btoi", "byteorder", @@ -7772,7 +7799,7 @@ version = "0.29.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "71e2746dc3a24dd78b3cfcb7be93368c6de9963d30f43a6a73998a9cf4b17b46" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "cfg-if", "cfg_aliases", "libc", @@ -7828,7 +7855,7 @@ version = "8.0.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "2fee8403b3d66ac7b26aee6e40a897d85dc5ce26f44da36b8b73e987cc52e943" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "filetime", "fsevent-sys", "inotify", @@ -8248,7 +8275,7 @@ version = "0.10.66" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9529f4786b70a3e8c61e11179af17ab6188ad8d0ded78c5529441ed39d4bd9c1" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "cfg-if", "foreign-types", "libc", @@ -9349,7 +9376,7 @@ version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "cc5b72d8145275d844d4b5f6d4e1eef00c8cd889edb6035c21675d1bb1f45c9f" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "hex", "procfs-core", "rustix 0.38.41", @@ -9361,7 +9388,7 @@ version = "0.17.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "239df02d8349b06fc07398a3a1697b06418223b1c7725085e801e7c0fc6a12ec" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "hex", ] @@ -9488,7 +9515,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" dependencies = [ "anyhow", - "itertools 0.12.1", + "itertools 0.10.5", "proc-macro2", "quote", "syn 2.0.87", @@ -9838,7 +9865,7 @@ version = "11.2.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1ab240315c661615f2ee9f0f2cd32d5a7343a84d5ebcccb99d46e6637565e7b0" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", ] [[package]] @@ -9944,7 +9971,7 @@ version = "0.5.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "9b6dfecf2c74bce2466cabf93f6664d6998a69eb21e39f4207930065b27b771f" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", ] [[package]] @@ -9986,7 +10013,7 @@ checksum = "ad156d539c879b7a24a363a2016d77961786e71f48f2e2fc8302a92abd2429a6" dependencies = [ "hashbrown 0.13.2", "log", - "rustc-hash", + "rustc-hash 1.1.0", "slice-group-by", "smallvec", ] @@ -10523,7 +10550,7 @@ dependencies = [ "async-trait", "auto_enums", "auto_impl", - "bitflags 2.7.0", + "bitflags 2.8.0", "byteorder", "bytes", "chrono", @@ -12265,6 +12292,12 @@ version = "1.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "08d43f7aa6b08d49f382cde6a7982047c3426db949b1424bc4b7ec9ae12c6ce2" +[[package]] +name = "rustc-hash" +version = "2.1.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "c7fb8039b3032c191086b10f11f319a6e99e1e82889c5cc6046f515c9db1d497" + [[package]] name = "rustc_version" version = "0.2.3" @@ -12313,7 +12346,7 @@ version = "0.38.41" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "d7f649912bc1495e167a6edee79151c84b1bad49748cb4f1f1167f459f6224f6" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "errno", "itoa", "libc", @@ -13644,7 +13677,7 @@ dependencies = [ "atoi", "base64 0.22.0", "bigdecimal 0.4.5", - "bitflags 2.7.0", + "bitflags 2.8.0", "byteorder", "bytes", "chrono", @@ -13690,7 +13723,7 @@ dependencies = [ "atoi", "base64 0.22.0", "bigdecimal 0.4.5", - "bitflags 2.7.0", + "bitflags 2.8.0", "byteorder", "chrono", "crc", @@ -14018,7 +14051,7 @@ version = "0.27.2" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b858526d22750088a9b3cf2e3c2aacebd5377f13adeec02860c30d09113010a6" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "cap-fs-ext", "cap-std", "fd-lock", @@ -14662,7 +14695,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8437150ab6bbc8c5f0f519e3d5ed4aa883a83dd4cdd3d1b21f9482936046cb97" dependencies = [ "async-compression", - "bitflags 2.7.0", + "bitflags 2.8.0", "bytes", "futures-core", "futures-util", @@ -15334,7 +15367,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "b86fd41e1e26ff6af9451c6a332a5ce5f5283ca51e87d875cdd9a05305598ee3" dependencies = [ "anyhow", - "bitflags 2.7.0", + "bitflags 2.8.0", "cap-fs-ext", "cap-rand", "cap-std", @@ -15463,7 +15496,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "07035cc9a9b41e62d3bb3a3815a66ab87c993c06fe1cf6b2a3f2a18499d937db" dependencies = [ "ahash 0.8.11", - "bitflags 2.7.0", + "bitflags 2.8.0", "hashbrown 0.14.5", "indexmap 2.7.0", "semver 1.0.18", @@ -15842,7 +15875,7 @@ checksum = "29830e5d01c182d24b94092c697aa7ab0ee97d22e78a2bf40ca91eae6ebca5c2" dependencies = [ "anyhow", "async-trait", - "bitflags 2.7.0", + "bitflags 2.8.0", "thiserror 1.0.63", "tracing", "wasmtime", @@ -16263,7 +16296,7 @@ version = "0.36.3" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "f9643b83820c0cd246ecabe5fa454dd04ba4fa67996369466d0747472d337346" dependencies = [ - "bitflags 2.7.0", + "bitflags 2.8.0", "windows-sys 0.52.0", ] diff --git a/ci/scripts/e2e-iceberg-sink-v2-test.sh b/ci/scripts/e2e-iceberg-sink-v2-test.sh index 4825e4ab11b1a..30f48db61b5b3 100755 --- a/ci/scripts/e2e-iceberg-sink-v2-test.sh +++ b/ci/scripts/e2e-iceberg-sink-v2-test.sh @@ -52,6 +52,7 @@ poetry run python main.py -t ./test_case/iceberg_source_position_delete.toml poetry run python main.py -t ./test_case/iceberg_source_all_delete.toml poetry run python main.py -t ./test_case/iceberg_source_explain_for_delete.toml poetry run python main.py -t ./test_case/iceberg_predicate_pushdown.toml +poetry run python main.py -t ./test_case/iceberg_connection.toml echo "--- Running benchmarks" poetry run python main.py -t ./benches/predicate_pushdown.toml diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 19beb4e47dfbe..8fcc035500398 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -160,7 +160,7 @@ steps: plugins: - docker-compose#v5.5.0: *docker-compose - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 12 + timeout_in_minutes: 15 retry: *auto-retry - group: "end-to-end connector test (release)" @@ -440,7 +440,7 @@ steps: plugins: - docker-compose#v5.5.0: *docker-compose - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 15 + timeout_in_minutes: 17 retry: *auto-retry - label: "e2e java-binding test (release)" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 42a70a3a4dedf..19fda2f2d99d5 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -263,7 +263,7 @@ steps: <<: *docker-compose run: sink-test-env - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 17 + timeout_in_minutes: 19 retry: *auto-retry - label: "end-to-end iceberg cdc test" diff --git a/e2e_test/iceberg/test_case/iceberg_connection.slt b/e2e_test/iceberg/test_case/iceberg_connection.slt new file mode 100644 index 0000000000000..abe3236e79af1 --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_connection.slt @@ -0,0 +1,67 @@ +statement ok +set streaming_parallelism=4; + +statement ok +CREATE TABLE s1 (i1 int, i2 varchar, i3 varchar); + +statement ok +CREATE CONNECTION CONN WITH ( + type = 'iceberg', + catalog.name = 'demo', + catalog.type = 'storage', + warehouse.path = 's3a://hummock001/iceberg-data', + s3.endpoint = 'http://127.0.0.1:9301', + s3.region = 'us-east-1', + s3.access.key = 'hummockadmin', + s3.secret.key = 'hummockadmin' +); + +statement ok +CREATE SINK sink1 from s1 WITH ( + connector = 'iceberg', + type = 'upsert', + database.name = 'demo_db', + table.name = 'test_connection_table', + connection = conn, + create_table_if_not_exists = 'true', + commit_checkpoint_interval = 1, + primary_key = 'i1,i2', +); + +statement ok +INSERT INTO s1 (i1, i2, i3) values(1,'1','1'),(2,'2','2'),(3,'3','3'),(4,'4','4'),(5,'5','5'); + +statement ok +flush + +statement ok +CREATE SOURCE iceberg_t1_source +WITH ( + connector = 'iceberg', + connection = conn, + database.name = 'demo_db', + table.name = 'test_connection_table', +); + +sleep 2s + +query I +select * from iceberg_t1_source order by i1 limit 5; +---- +1 1 1 +2 2 2 +3 3 3 +4 4 4 +5 5 5 + +statement ok +DROP SINK sink1; + +statement ok +DROP SOURCE iceberg_t1_source; + +statement ok +DROP TABLE s1 cascade; + +statement ok +DROP CONNECTION conn; diff --git a/e2e_test/iceberg/test_case/iceberg_connection.toml b/e2e_test/iceberg/test_case/iceberg_connection.toml new file mode 100644 index 0000000000000..3c64c0a067a44 --- /dev/null +++ b/e2e_test/iceberg/test_case/iceberg_connection.toml @@ -0,0 +1,11 @@ +init_sqls = [ + 'CREATE SCHEMA IF NOT EXISTS demo_db', + 'DROP TABLE IF EXISTS demo_db.test_connection_table', +] + +slt = 'test_case/iceberg_connection.slt' + +drop_sqls = [ + 'DROP TABLE IF EXISTS demo_db.test_connection_table', + 'DROP SCHEMA IF EXISTS demo_db', +] diff --git a/e2e_test/source_inline/commands.toml b/e2e_test/source_inline/commands.toml index 57d09d8237efa..3300b6e3a5df5 100644 --- a/e2e_test/source_inline/commands.toml +++ b/e2e_test/source_inline/commands.toml @@ -35,6 +35,11 @@ script = ''' set -e if [ -n "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" ]; then + echo "Deleting all Kafka consumer groups..." + rpk group list | tail -n +2 | awk '{print $2}' | while read -r group; do + echo "Deleting Kafka consumer group: $group" + rpk group delete "$group" + done echo "Deleting all Kafka topics..." rpk topic delete -r "*" echo "Deleting all schema registry subjects" diff --git a/e2e_test/source_inline/kafka/alter/add_column_shared.slt b/e2e_test/source_inline/kafka/alter/add_column_shared.slt index bbb03c178fa2f..210f17bad63ab 100644 --- a/e2e_test/source_inline/kafka/alter/add_column_shared.slt +++ b/e2e_test/source_inline/kafka/alter/add_column_shared.slt @@ -48,6 +48,12 @@ select * from mv_before_alter; statement ok alter source s add column v3 varchar; +# Demonstrate definition change. +query T +SELECT SUBSTRING(definition, 1, POSITION(' WITH' IN definition) - 1) FROM rw_sources WHERE name = 's'; +---- +CREATE SOURCE s (v1 INT, v2 CHARACTER VARYING, v3 CHARACTER VARYING) + # New MV will have v3. # Check it should still be shared source diff --git a/e2e_test/source_inline/kafka/avro/alter_source.slt b/e2e_test/source_inline/kafka/avro/alter_source.slt index 8bce7f4efd5cf..bb36e116279e4 100644 --- a/e2e_test/source_inline/kafka/avro/alter_source.slt +++ b/e2e_test/source_inline/kafka/avro/alter_source.slt @@ -52,6 +52,12 @@ No tracking issue yet. Feel free to submit a feature request at https://github.c statement ok alter source s format plain encode avro (schema.registry = '${RISEDEV_SCHEMA_REGISTRY_URL}'); +# Demonstrate definition change. +query T +SELECT SUBSTRING(definition, 1, POSITION(' WITH' IN definition) - 1) FROM rw_sources WHERE name = 's'; +---- +CREATE SOURCE s (foo CHARACTER VARYING, bar INT) + query ?? select * from s ---- diff --git a/e2e_test/source_inline/kafka/consumer_group.slt.serial b/e2e_test/source_inline/kafka/consumer_group.slt.serial index 29f41c82d6168..ed6ce13e3c006 100644 --- a/e2e_test/source_inline/kafka/consumer_group.slt.serial +++ b/e2e_test/source_inline/kafka/consumer_group.slt.serial @@ -58,28 +58,14 @@ EOF # (enable.auto.commit defaults to true, and auto.commit.interval.ms defaults to 5s) sleep 5s -system ok -./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-members ----- -0 - - -# The lag for MV's group is 0. -system ok -./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-lags ----- -0 - - # Test delete consumer group on drop -# my_group: 1 source fragment, 1 backfill fragment, 1 batch query -# TODO: drop backfill fragment on backfill finish +# my_group: 1 source fragment, 1 batch query, (1 backfill fragment's group is already dropped after backfill finished) # We only check my_group to avoid interfering with other tests. system ok retry 3 backoff 5s ./e2e_test/source_inline/kafka/consumer_group.mjs count-groups | grep my_group ---- -my_group: 3 +my_group: 2 statement ok @@ -90,7 +76,7 @@ sleep 1s system ok retry 3 backoff 5s ./e2e_test/source_inline/kafka/consumer_group.mjs count-groups | grep my_group ---- -my_group: 2 +my_group: 1 system ok diff --git a/e2e_test/source_inline/kafka/consumer_group_non_shared.slt.serial b/e2e_test/source_inline/kafka/consumer_group_non_shared.slt.serial index 2009bf53f8968..3111673f0a5e0 100644 --- a/e2e_test/source_inline/kafka/consumer_group_non_shared.slt.serial +++ b/e2e_test/source_inline/kafka/consumer_group_non_shared.slt.serial @@ -52,7 +52,7 @@ c # (enable.auto.commit defaults to true, and auto.commit.interval.ms defaults to 5s) sleep 5s -system ok +system ok retry 3 backoff 5s ./e2e_test/source_inline/kafka/consumer_group.mjs --mv mv list-members ---- 0 diff --git a/e2e_test/source_inline/kafka/protobuf/alter_source_shared.slt b/e2e_test/source_inline/kafka/protobuf/alter_source_shared.slt index 658d4fa95c6a0..19ded681789cb 100644 --- a/e2e_test/source_inline/kafka/protobuf/alter_source_shared.slt +++ b/e2e_test/source_inline/kafka/protobuf/alter_source_shared.slt @@ -20,6 +20,11 @@ FORMAT PLAIN ENCODE PROTOBUF( message = 'test.User' ); +query T +SELECT SUBSTRING(definition, 1, POSITION(' WITH' IN definition) - 1) FROM rw_sources WHERE name = 'src_user'; +---- +CREATE SOURCE src_user (id INT, name CHARACTER VARYING, address CHARACTER VARYING, city CHARACTER VARYING, gender CHARACTER VARYING, sc STRUCT) INCLUDE timestamp + statement ok CREATE MATERIALIZED VIEW mv_user AS SELECT * FROM src_user; @@ -45,6 +50,11 @@ set streaming_use_shared_source to false; statement ok ALTER SOURCE src_user REFRESH SCHEMA; +query T +SELECT SUBSTRING(definition, 1, POSITION(' WITH' IN definition) - 1) FROM rw_sources WHERE name = 'src_user'; +---- +CREATE SOURCE src_user (id INT, name CHARACTER VARYING, address CHARACTER VARYING, city CHARACTER VARYING, gender CHARACTER VARYING, sc STRUCT, age INT) INCLUDE timestamp + # Check it should still be shared source query EXPLAIN CREATE MATERIALIZED VIEW mv_user_more AS SELECT * FROM src_user; diff --git a/e2e_test/source_inline/kafka/shared_source.slt.serial b/e2e_test/source_inline/kafka/shared_source.slt.serial index af6b371d21c49..42d4cf86725ef 100644 --- a/e2e_test/source_inline/kafka/shared_source.slt.serial +++ b/e2e_test/source_inline/kafka/shared_source.slt.serial @@ -189,10 +189,10 @@ select v1, v2 from mv_1; system ok internal_table.mjs --name mv_1 --type sourcebackfill ---- -0,"{""num_consumed_rows"": 2, ""state"": ""Finished"", ""target_offset"": ""0""}" -1,"{""num_consumed_rows"": 2, ""state"": ""Finished"", ""target_offset"": ""0""}" -2,"{""num_consumed_rows"": 3, ""state"": ""Finished"", ""target_offset"": ""1""}" -3,"{""num_consumed_rows"": 4, ""state"": ""Finished"", ""target_offset"": ""2""}" +0,"{""num_consumed_rows"": 1, ""state"": ""Finished"", ""target_offset"": ""0""}" +1,"{""num_consumed_rows"": 1, ""state"": ""Finished"", ""target_offset"": ""0""}" +2,"{""num_consumed_rows"": 2, ""state"": ""Finished"", ""target_offset"": ""1""}" +3,"{""num_consumed_rows"": 3, ""state"": ""Finished"", ""target_offset"": ""2""}" system ok diff --git a/integration_tests/datagen/go.mod b/integration_tests/datagen/go.mod index c5f406e59a4e4..823eee7c17d58 100644 --- a/integration_tests/datagen/go.mod +++ b/integration_tests/datagen/go.mod @@ -72,7 +72,7 @@ require ( go.uber.org/atomic v1.7.0 // indirect golang.org/x/crypto v0.31.0 // indirect golang.org/x/exp v0.0.0-20200224162631-6cc2880d07d6 // indirect - golang.org/x/net v0.23.0 // indirect + golang.org/x/net v0.33.0 // indirect golang.org/x/oauth2 v0.0.0-20210514164344-f6687ab2804c // indirect golang.org/x/sys v0.28.0 // indirect golang.org/x/term v0.27.0 // indirect diff --git a/integration_tests/datagen/go.sum b/integration_tests/datagen/go.sum index cd36d79a3c45a..9e0363868ea80 100644 --- a/integration_tests/datagen/go.sum +++ b/integration_tests/datagen/go.sum @@ -538,8 +538,8 @@ golang.org/x/net v0.0.0-20210726213435-c6fcb2dbf985/go.mod h1:9nx3DQGgdP8bBQD5qx golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220127200216-cd36cc0744dd/go.mod h1:CfG3xpIq0wQ8r1q4Su4UZFWDARRcnwPjda9FqA0JpMk= golang.org/x/net v0.0.0-20220725212005-46097bf591d3/go.mod h1:AaygXjzTFtRAg2ttMY5RMuhpJ3cNnI0XpyFJD1iQRSM= -golang.org/x/net v0.23.0 h1:7EYJ93RZ9vYSZAIb2x3lnuvqO5zneoD6IvWjuhfxjTs= -golang.org/x/net v0.23.0/go.mod h1:JKghWKKOSdJwpW2GEx0Ja7fmaKnMsbu+MWVZTokSYmg= +golang.org/x/net v0.33.0 h1:74SYHlV8BIgHIFC/LrYkOGIwL19eTYXQ5wc6TBuO36I= +golang.org/x/net v0.33.0/go.mod h1:HXLR5J+9DxmrqMwG9qjGCxZ+zKXxBru04zlTvWlWuN4= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= diff --git a/integration_tests/debezium-mysql/docker-compose.yml b/integration_tests/debezium-mysql/docker-compose.yml index 6cb577ac23886..001d0bb99bf23 100644 --- a/integration_tests/debezium-mysql/docker-compose.yml +++ b/integration_tests/debezium-mysql/docker-compose.yml @@ -48,7 +48,7 @@ services: retries: 5 container_name: mysql debezium: - image: debezium-connect + image: debezium/connect:1.9 build: . environment: BOOTSTRAP_SERVERS: message_queue:29092 diff --git a/integration_tests/debezium-postgres/docker-compose.yml b/integration_tests/debezium-postgres/docker-compose.yml index 327cb44d6db7c..3a3c0bbf4d964 100644 --- a/integration_tests/debezium-postgres/docker-compose.yml +++ b/integration_tests/debezium-postgres/docker-compose.yml @@ -48,7 +48,7 @@ services: - "./postgres_prepare.sql:/postgres_prepare.sql" debezium: - image: debezium-connect + image: debezium/connect:1.9 build: . environment: BOOTSTRAP_SERVERS: message_queue:29092 diff --git a/integration_tests/debezium-sqlserver/docker-compose.yml b/integration_tests/debezium-sqlserver/docker-compose.yml index 9d4bbbf0a5bb6..b85403c9a8d38 100644 --- a/integration_tests/debezium-sqlserver/docker-compose.yml +++ b/integration_tests/debezium-sqlserver/docker-compose.yml @@ -48,7 +48,7 @@ services: - ./sqlserver_prepare.sql:/sqlserver_prepare.sql debezium: - image: debezium-connect + image: debezium/connect:1.9 build: . environment: BOOTSTRAP_SERVERS: message_queue:29092 diff --git a/src/batch/executors/src/executor/iceberg_scan.rs b/src/batch/executors/src/executor/iceberg_scan.rs index d50f656c98ac4..9614576af19d5 100644 --- a/src/batch/executors/src/executor/iceberg_scan.rs +++ b/src/batch/executors/src/executor/iceberg_scan.rs @@ -16,8 +16,7 @@ use std::sync::Arc; use futures_async_stream::try_stream; use futures_util::stream::StreamExt; -use iceberg::scan::FileScanTask; -use iceberg::spec::TableMetadata; +use iceberg::spec::TableMetadataRef; use itertools::Itertools; use risingwave_common::array::arrow::IcebergArrowConvert; use risingwave_common::array::{ArrayImpl, DataChunk, I64Array, Utf8Array}; @@ -26,9 +25,7 @@ use risingwave_common::catalog::{ }; use risingwave_common::types::DataType; use risingwave_common_estimate_size::EstimateSize; -use risingwave_connector::source::iceberg::{ - IcebergFileScanTaskJsonStrEnum, IcebergProperties, IcebergSplit, -}; +use risingwave_connector::source::iceberg::{IcebergFileScanTask, IcebergProperties, IcebergSplit}; use risingwave_connector::source::{ConnectorProperties, SplitImpl, SplitMetaData}; use risingwave_connector::WithOptionsSecResolved; use risingwave_pb::batch_plan::plan_node::NodeBody; @@ -38,54 +35,12 @@ use crate::error::BatchError; use crate::executor::Executor; use crate::monitor::BatchMetrics; -pub enum IcebergFileScanTaskEnum { - // The scan task of the data file and the position delete file - Data(Vec), - // The scan task of the equality delete file - EqualityDelete(Vec), - // The scan task of the position delete file - PositionDelete(Vec), -} - -impl IcebergFileScanTaskEnum { - fn from_iceberg_file_scan_task_json_str_enum( - iceberg_file_scan_task_json_str_enum: IcebergFileScanTaskJsonStrEnum, - ) -> Self { - match iceberg_file_scan_task_json_str_enum { - IcebergFileScanTaskJsonStrEnum::Data(data_file_scan_tasks) => { - IcebergFileScanTaskEnum::Data( - data_file_scan_tasks - .into_iter() - .map(|t| t.deserialize()) - .collect(), - ) - } - IcebergFileScanTaskJsonStrEnum::EqualityDelete(equality_delete_file_scan_tasks) => { - IcebergFileScanTaskEnum::EqualityDelete( - equality_delete_file_scan_tasks - .into_iter() - .map(|t| t.deserialize()) - .collect(), - ) - } - IcebergFileScanTaskJsonStrEnum::PositionDelete(position_delete_file_scan_tasks) => { - IcebergFileScanTaskEnum::PositionDelete( - position_delete_file_scan_tasks - .into_iter() - .map(|t| t.deserialize()) - .collect(), - ) - } - } - } -} - pub struct IcebergScanExecutor { iceberg_config: IcebergProperties, #[allow(dead_code)] snapshot_id: Option, - table_meta: TableMetadata, - file_scan_tasks: Option, + table_meta: TableMetadataRef, + file_scan_tasks: Option, batch_size: usize, schema: Schema, identity: String, @@ -112,8 +67,8 @@ impl IcebergScanExecutor { pub fn new( iceberg_config: IcebergProperties, snapshot_id: Option, - table_meta: TableMetadata, - file_scan_tasks: IcebergFileScanTaskEnum, + table_meta: TableMetadataRef, + file_scan_tasks: IcebergFileScanTask, batch_size: usize, schema: Schema, identity: String, @@ -145,11 +100,11 @@ impl IcebergScanExecutor { let table_name = table.identifier().name().to_owned(); let data_file_scan_tasks = match Option::take(&mut self.file_scan_tasks) { - Some(IcebergFileScanTaskEnum::Data(data_file_scan_tasks)) => data_file_scan_tasks, - Some(IcebergFileScanTaskEnum::EqualityDelete(equality_delete_file_scan_tasks)) => { + Some(IcebergFileScanTask::Data(data_file_scan_tasks)) => data_file_scan_tasks, + Some(IcebergFileScanTask::EqualityDelete(equality_delete_file_scan_tasks)) => { equality_delete_file_scan_tasks } - Some(IcebergFileScanTaskEnum::PositionDelete(position_delete_file_scan_tasks)) => { + Some(IcebergFileScanTask::PositionDelete(position_delete_file_scan_tasks)) => { position_delete_file_scan_tasks } None => { @@ -271,13 +226,13 @@ impl BoxedExecutorBuilder for IcebergScanExecutorBuilder { .fields() .iter() .any(|f| f.name == ICEBERG_FILE_PATH_COLUMN_NAME) - && matches!(split.files, IcebergFileScanTaskJsonStrEnum::Data(_)); + && matches!(split.task, IcebergFileScanTask::Data(_)); Ok(Box::new(IcebergScanExecutor::new( iceberg_properties, Some(split.snapshot_id), - split.table_meta.deserialize(), - IcebergFileScanTaskEnum::from_iceberg_file_scan_task_json_str_enum(split.files), + split.table_meta.clone(), + split.task, source.context().get_config().developer.chunk_size, schema, source.plan_node().get_identity().clone(), diff --git a/src/common/src/config.rs b/src/common/src/config.rs index ed5b5835dd327..dec4a024a6f6b 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -533,9 +533,6 @@ pub struct MetaDeveloperConfig { #[serde(default = "default::developer::hummock_time_travel_sst_info_insert_batch_size")] pub hummock_time_travel_sst_info_insert_batch_size: usize, - #[serde(default = "default::developer::hummock_delta_log_delete_batch_size")] - pub hummock_delta_log_delete_batch_size: usize, - #[serde(default = "default::developer::time_travel_vacuum_interval_sec")] pub time_travel_vacuum_interval_sec: u64, @@ -2070,10 +2067,6 @@ pub mod default { 100 } - pub fn hummock_delta_log_delete_batch_size() -> usize { - 512 - } - pub fn time_travel_vacuum_interval_sec() -> u64 { 30 } diff --git a/src/config/example.toml b/src/config/example.toml index 24f78154ed00e..56c7cd1525734 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -93,7 +93,6 @@ meta_actor_cnt_per_worker_parallelism_soft_limit = 100 meta_actor_cnt_per_worker_parallelism_hard_limit = 400 meta_hummock_time_travel_sst_info_fetch_batch_size = 10000 meta_hummock_time_travel_sst_info_insert_batch_size = 100 -meta_hummock_delta_log_delete_batch_size = 512 meta_time_travel_vacuum_interval_sec = 30 meta_hummock_time_travel_epoch_version_insert_batch_size = 1000 meta_hummock_gc_history_insert_batch_size = 1000 diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index 909f90299e855..b4c3ede616af8 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -46,7 +46,7 @@ deltalake = { workspace = true } duration-str = "0.12.0" easy-ext = "1" either = "1" -elasticsearch = { version = "8.15.0-alpha.1", features = ["rustls-tls"] } +elasticsearch = { version = "8.17.0-alpha.1", features = ["rustls-tls"] } enum-as-inner = "0.6" futures = { version = "0.3", default-features = false, features = ["alloc"] } futures-async-stream = { workspace = true } diff --git a/src/connector/src/connector_common/connection.rs b/src/connector/src/connector_common/connection.rs index 9be0c7ac84fe2..75a808a6f2eb9 100644 --- a/src/connector/src/connector_common/connection.rs +++ b/src/connector/src/connector_common/connection.rs @@ -12,22 +12,30 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashMap; use std::time::Duration; +use anyhow::Context; +use opendal::services::{Gcs, S3}; +use opendal::Operator; use rdkafka::consumer::{BaseConsumer, Consumer}; use rdkafka::ClientConfig; +use risingwave_common::bail; use risingwave_common::secret::LocalSecretManager; use risingwave_pb::catalog::PbConnection; use serde_derive::Deserialize; use serde_with::serde_as; use tonic::async_trait; +use url::Url; use with_options::WithOptions; -use crate::connector_common::{AwsAuthProps, KafkaConnectionProps, KafkaPrivateLinkCommon}; +use crate::connector_common::{ + AwsAuthProps, IcebergCommon, KafkaConnectionProps, KafkaPrivateLinkCommon, +}; use crate::error::ConnectorResult; use crate::schema::schema_registry::Client as ConfluentSchemaRegistryClient; use crate::source::kafka::{KafkaContextCommon, RwConsumerContext}; -use crate::{dispatch_connection_impl, ConnectionImpl}; +use crate::{deserialize_optional_bool_from_string, dispatch_connection_impl, ConnectionImpl}; pub const SCHEMA_REGISTRY_CONNECTION_TYPE: &str = "schema_registry"; @@ -109,12 +117,169 @@ impl KafkaConnection { #[serde_as] #[derive(Debug, Clone, PartialEq, Eq, Deserialize, WithOptions)] #[serde(deny_unknown_fields)] -pub struct IcebergConnection {} +pub struct IcebergConnection { + #[serde(rename = "catalog.type")] + pub catalog_type: Option, + #[serde(rename = "s3.region")] + pub region: Option, + #[serde(rename = "s3.endpoint")] + pub endpoint: Option, + #[serde(rename = "s3.access.key")] + pub access_key: Option, + #[serde(rename = "s3.secret.key")] + pub secret_key: Option, + + #[serde(rename = "gcs.credential")] + pub gcs_credential: Option, + + /// Path of iceberg warehouse, only applicable in storage catalog. + #[serde(rename = "warehouse.path")] + pub warehouse_path: Option, + /// Catalog name, can be omitted for storage catalog, but + /// must be set for other catalogs. + #[serde(rename = "catalog.name")] + pub catalog_name: Option, + /// URI of iceberg catalog, only applicable in rest catalog. + #[serde(rename = "catalog.uri")] + pub catalog_uri: Option, + /// Credential for accessing iceberg catalog, only applicable in rest catalog. + /// A credential to exchange for a token in the OAuth2 client credentials flow. + #[serde(rename = "catalog.credential")] + pub credential: Option, + /// token for accessing iceberg catalog, only applicable in rest catalog. + /// A Bearer token which will be used for interaction with the server. + #[serde(rename = "catalog.token")] + pub token: Option, + /// `oauth2-server-uri` for accessing iceberg catalog, only applicable in rest catalog. + /// Token endpoint URI to fetch token from if the Rest Catalog is not the authorization server. + #[serde(rename = "catalog.oauth2-server-uri")] + pub oauth2_server_uri: Option, + /// scope for accessing iceberg catalog, only applicable in rest catalog. + /// Additional scope for OAuth2. + #[serde(rename = "catalog.scope")] + pub scope: Option, + + #[serde( + rename = "s3.path.style.access", + default, + deserialize_with = "deserialize_optional_bool_from_string" + )] + pub path_style_access: Option, + + #[serde(rename = "catalog.jdbc.user")] + pub jdbc_user: Option, + + #[serde(rename = "catalog.jdbc.password")] + pub jdbc_password: Option, +} #[async_trait] impl Connection for IcebergConnection { async fn test_connection(&self) -> ConnectorResult<()> { - todo!() + let info = match &self.warehouse_path { + Some(warehouse_path) => { + let url = Url::parse(warehouse_path); + if url.is_err() + && let Some(catalog_type) = &self.catalog_type + && catalog_type == "rest" + { + // If the warehouse path is not a valid URL, it could be a warehouse name in rest catalog, + // so we allow it to pass here. + None + } else { + let url = + url.with_context(|| format!("Invalid warehouse path: {}", warehouse_path))?; + let bucket = url + .host_str() + .with_context(|| { + format!("Invalid s3 path: {}, bucket is missing", warehouse_path) + })? + .to_owned(); + let root = url.path().trim_start_matches('/').to_owned(); + Some((url.scheme().to_owned(), bucket, root)) + } + } + None => { + if let Some(catalog_type) = &self.catalog_type + && catalog_type == "rest" + { + None + } else { + bail!("`warehouse.path` must be set"); + } + } + }; + + // test storage + if let Some((scheme, bucket, root)) = info { + match scheme.as_str() { + "s3" | "s3a" => { + let mut builder = S3::default(); + if let Some(region) = &self.region { + builder = builder.region(region); + } + if let Some(endpoint) = &self.endpoint { + builder = builder.endpoint(endpoint); + } + if let Some(access_key) = &self.access_key { + builder = builder.access_key_id(access_key); + } + if let Some(secret_key) = &self.secret_key { + builder = builder.secret_access_key(secret_key); + } + builder = builder.root(root.as_str()).bucket(bucket.as_str()); + let op = Operator::new(builder)?.finish(); + op.check().await?; + } + "gs" | "gcs" => { + let mut builder = Gcs::default(); + if let Some(credential) = &self.gcs_credential { + builder = builder.credential(credential); + } + builder = builder.root(root.as_str()).bucket(bucket.as_str()); + let op = Operator::new(builder)?.finish(); + op.check().await?; + } + _ => { + bail!("Unsupported scheme: {}", scheme); + } + } + } + + // test catalog + let iceberg_common = IcebergCommon { + catalog_type: self.catalog_type.clone(), + region: self.region.clone(), + endpoint: self.endpoint.clone(), + access_key: self.access_key.clone(), + secret_key: self.secret_key.clone(), + gcs_credential: self.gcs_credential.clone(), + warehouse_path: self.warehouse_path.clone(), + catalog_name: self.catalog_name.clone(), + catalog_uri: self.catalog_uri.clone(), + credential: self.credential.clone(), + token: self.token.clone(), + oauth2_server_uri: self.oauth2_server_uri.clone(), + scope: self.scope.clone(), + path_style_access: self.path_style_access, + database_name: Some("test_database".to_owned()), + table_name: "test_table".to_owned(), + enable_config_load: Some(false), + }; + + let mut java_map = HashMap::new(); + if let Some(jdbc_user) = &self.jdbc_user { + java_map.insert("jdbc.user".to_owned(), jdbc_user.to_owned()); + } + if let Some(jdbc_password) = &self.jdbc_password { + java_map.insert("jdbc.password".to_owned(), jdbc_password.to_owned()); + } + let catalog = iceberg_common.create_catalog(&java_map).await?; + // test catalog by `table_exists` api + catalog + .table_exists(&iceberg_common.full_table_name()?) + .await?; + Ok(()) } } diff --git a/src/connector/src/connector_common/iceberg/mod.rs b/src/connector/src/connector_common/iceberg/mod.rs index 6964e03a5b99f..c04e20e53c202 100644 --- a/src/connector/src/connector_common/iceberg/mod.rs +++ b/src/connector/src/connector_common/iceberg/mod.rs @@ -19,11 +19,11 @@ use std::collections::HashMap; use std::sync::Arc; use ::iceberg::io::{S3_ACCESS_KEY_ID, S3_ENDPOINT, S3_REGION, S3_SECRET_ACCESS_KEY}; -use ::iceberg::spec::TableMetadata; use ::iceberg::table::Table; use ::iceberg::{Catalog, TableIdent}; use anyhow::{anyhow, Context}; use iceberg::io::{GCS_CREDENTIALS_JSON, GCS_DISABLE_CONFIG_LOAD, S3_DISABLE_CONFIG_LOAD}; +use iceberg::spec::TableMetadataRef; use iceberg_catalog_glue::{AWS_ACCESS_KEY_ID, AWS_REGION_NAME, AWS_SECRET_ACCESS_KEY}; use risingwave_common::bail; use serde_derive::Deserialize; @@ -479,7 +479,7 @@ impl IcebergCommon { pub async fn load_table_with_metadata( &self, - metadata: TableMetadata, + metadata: TableMetadataRef, java_catalog_props: &HashMap, ) -> ConnectorResult { match self.catalog_type() { diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 28f3a756ea5c9..406d678a65ee6 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -196,6 +196,10 @@ pub trait SplitEnumerator: Sized + Send { async fn on_drop_fragments(&mut self, _fragment_ids: Vec) -> Result<()> { Ok(()) } + /// Do some cleanup work when a backfill fragment is finished, e.g., drop Kafka consumer group. + async fn on_finish_backfill(&mut self, _fragment_ids: Vec) -> Result<()> { + Ok(()) + } } pub type SourceContextRef = Arc; @@ -206,6 +210,7 @@ pub type SourceEnumeratorContextRef = Arc; pub trait AnySplitEnumerator: Send { async fn list_splits(&mut self) -> Result>; async fn on_drop_fragments(&mut self, _fragment_ids: Vec) -> Result<()>; + async fn on_finish_backfill(&mut self, _fragment_ids: Vec) -> Result<()>; } #[async_trait] @@ -219,6 +224,10 @@ impl>> AnySplitEnumerator for T { async fn on_drop_fragments(&mut self, _fragment_ids: Vec) -> Result<()> { SplitEnumerator::on_drop_fragments(self, _fragment_ids).await } + + async fn on_finish_backfill(&mut self, _fragment_ids: Vec) -> Result<()> { + SplitEnumerator::on_finish_backfill(self, _fragment_ids).await + } } /// The max size of a chunk yielded by source stream. diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index 42fc791b3a677..9540d601be2e6 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -22,7 +22,7 @@ use async_trait::async_trait; use futures_async_stream::for_await; use iceberg::expr::Predicate as IcebergPredicate; use iceberg::scan::FileScanTask; -use iceberg::spec::TableMetadata; +use iceberg::spec::TableMetadataRef; use iceberg::table::Table; use iceberg::Catalog; use itertools::Itertools; @@ -88,7 +88,7 @@ impl IcebergProperties { pub async fn load_table_with_metadata( &self, - table_meta: TableMetadata, + table_meta: TableMetadataRef, ) -> ConnectorResult
{ let mut java_catalog_props = HashMap::new(); if let Some(jdbc_user) = self.jdbc_user.clone() { @@ -131,39 +131,26 @@ impl IcebergFileScanTaskJsonStr { } } -#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct TableMetadataJsonStr(String); - -impl TableMetadataJsonStr { - pub fn deserialize(&self) -> TableMetadata { - serde_json::from_str(&self.0).unwrap() - } - - pub fn serialize(metadata: &TableMetadata) -> Self { - Self(serde_json::to_string(metadata).unwrap()) - } -} - -#[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub enum IcebergFileScanTaskJsonStrEnum { - Data(Vec), - EqualityDelete(Vec), - PositionDelete(Vec), +#[derive(Debug, Clone, PartialEq, Serialize, Deserialize)] +pub enum IcebergFileScanTask { + Data(Vec), + EqualityDelete(Vec), + PositionDelete(Vec), } -impl IcebergFileScanTaskJsonStrEnum { +impl IcebergFileScanTask { pub fn new_with_scan_type( iceberg_scan_type: IcebergScanType, - data_files: Vec, - equality_delete_files: Vec, - position_delete_files: Vec, + data_files: Vec, + equality_delete_files: Vec, + position_delete_files: Vec, ) -> Self { match iceberg_scan_type { IcebergScanType::EqualityDeleteScan => { - IcebergFileScanTaskJsonStrEnum::EqualityDelete(equality_delete_files) + IcebergFileScanTask::EqualityDelete(equality_delete_files) } - IcebergScanType::DataScan => IcebergFileScanTaskJsonStrEnum::Data(data_files), + IcebergScanType::DataScan => IcebergFileScanTask::Data(data_files), IcebergScanType::PositionDeleteScan => { - IcebergFileScanTaskJsonStrEnum::PositionDelete(position_delete_files) + IcebergFileScanTask::PositionDelete(position_delete_files) } IcebergScanType::Unspecified => unreachable!("Unspecified iceberg scan type"), } @@ -171,18 +158,18 @@ impl IcebergFileScanTaskJsonStrEnum { pub fn add_files( &mut self, - data_file: IcebergFileScanTaskJsonStr, - equality_delete_file: IcebergFileScanTaskJsonStr, - position_delete_file: IcebergFileScanTaskJsonStr, + data_file: FileScanTask, + equality_delete_file: FileScanTask, + position_delete_file: FileScanTask, ) { match self { - IcebergFileScanTaskJsonStrEnum::Data(data_files) => { + IcebergFileScanTask::Data(data_files) => { data_files.push(data_file); } - IcebergFileScanTaskJsonStrEnum::EqualityDelete(equality_delete_files) => { + IcebergFileScanTask::EqualityDelete(equality_delete_files) => { equality_delete_files.push(equality_delete_file); } - IcebergFileScanTaskJsonStrEnum::PositionDelete(position_delete_files) => { + IcebergFileScanTask::PositionDelete(position_delete_files) => { position_delete_files.push(position_delete_file); } } @@ -190,11 +177,11 @@ impl IcebergFileScanTaskJsonStrEnum { pub fn is_empty(&self) -> bool { match self { - IcebergFileScanTaskJsonStrEnum::Data(data_files) => data_files.is_empty(), - IcebergFileScanTaskJsonStrEnum::EqualityDelete(equality_delete_files) => { + IcebergFileScanTask::Data(data_files) => data_files.is_empty(), + IcebergFileScanTask::EqualityDelete(equality_delete_files) => { equality_delete_files.is_empty() } - IcebergFileScanTaskJsonStrEnum::PositionDelete(position_delete_files) => { + IcebergFileScanTask::PositionDelete(position_delete_files) => { position_delete_files.is_empty() } } @@ -205,17 +192,17 @@ impl IcebergFileScanTaskJsonStrEnum { pub struct IcebergSplit { pub split_id: i64, pub snapshot_id: i64, - pub table_meta: TableMetadataJsonStr, - pub files: IcebergFileScanTaskJsonStrEnum, + pub table_meta: TableMetadataRef, + pub task: IcebergFileScanTask, } impl IcebergSplit { - pub fn empty(table_meta: TableMetadataJsonStr, iceberg_scan_type: IcebergScanType) -> Self { + pub fn empty(table_meta: TableMetadataRef, iceberg_scan_type: IcebergScanType) -> Self { Self { split_id: 0, snapshot_id: 0, table_meta, - files: IcebergFileScanTaskJsonStrEnum::new_with_scan_type( + task: IcebergFileScanTask::new_with_scan_type( iceberg_scan_type, vec![], vec![], @@ -333,13 +320,10 @@ impl IcebergSplitEnumerator { } let table = self.config.load_table().await?; let snapshot_id = Self::get_snapshot_id(&table, time_traval_info)?; - let table_meta = TableMetadataJsonStr::serialize(table.metadata()); + let table_meta = table.metadata_ref(); if snapshot_id.is_none() { // If there is no snapshot, we will return a mock `IcebergSplit` with empty files. - return Ok(vec![IcebergSplit::empty( - TableMetadataJsonStr::serialize(table.metadata()), - iceberg_scan_type, - )]); + return Ok(vec![IcebergSplit::empty(table_meta, iceberg_scan_type)]); } let snapshot_id = snapshot_id.unwrap(); @@ -375,14 +359,14 @@ impl IcebergSplitEnumerator { let mut task: FileScanTask = task.map_err(|e| anyhow!(e))?; match task.data_file_content { iceberg::spec::DataContentType::Data => { - data_files.push(IcebergFileScanTaskJsonStr::serialize(&task)); + data_files.push(task); } iceberg::spec::DataContentType::EqualityDeletes => { - equality_delete_files.push(IcebergFileScanTaskJsonStr::serialize(&task)); + equality_delete_files.push(task); } iceberg::spec::DataContentType::PositionDeletes => { task.project_field_ids = Vec::default(); - position_delete_files.push(IcebergFileScanTaskJsonStr::serialize(&task)); + position_delete_files.push(task); } } } @@ -401,7 +385,7 @@ impl IcebergSplitEnumerator { split_id: index as i64, snapshot_id, table_meta: table_meta.clone(), - files: IcebergFileScanTaskJsonStrEnum::new_with_scan_type( + task: IcebergFileScanTask::new_with_scan_type( iceberg_scan_type, data_file, equality_delete_file, @@ -409,12 +393,12 @@ impl IcebergSplitEnumerator { ), }, ) - .filter(|split| !split.files.is_empty()) + .filter(|split| !split.task.is_empty()) .collect_vec(); if splits.is_empty() { return Ok(vec![IcebergSplit::empty( - TableMetadataJsonStr::serialize(table.metadata()), + table.metadata_ref(), iceberg_scan_type, )]); } @@ -472,10 +456,7 @@ impl IcebergSplitEnumerator { Self::all_delete_parameters(&table, snapshot_id).await } - fn split_n_vecs( - vecs: Vec, - split_num: usize, - ) -> Vec> { + fn split_n_vecs(vecs: Vec, split_num: usize) -> Vec> { let split_size = vecs.len() / split_num; let remaining = vecs.len() % split_num; let mut result_vecs = (0..split_num) diff --git a/src/connector/src/source/kafka/enumerator.rs b/src/connector/src/source/kafka/enumerator.rs index 00e3ead9049b9..42045b20ba2e0 100644 --- a/src/connector/src/source/kafka/enumerator.rs +++ b/src/connector/src/source/kafka/enumerator.rs @@ -75,7 +75,33 @@ pub struct KafkaSplitEnumerator { config: rdkafka::ClientConfig, } -impl KafkaSplitEnumerator {} +impl KafkaSplitEnumerator { + async fn drop_consumer_groups(&self, fragment_ids: Vec) -> ConnectorResult<()> { + let admin = SHARED_KAFKA_ADMIN + .try_get_with_by_ref(&self.properties.connection, async { + tracing::info!("build new kafka admin for {}", self.broker_address); + Ok(Arc::new( + build_kafka_admin(&self.config, &self.properties).await?, + )) + }) + .await?; + + let group_ids = fragment_ids + .iter() + .map(|fragment_id| self.properties.group_id(*fragment_id)) + .collect::>(); + let group_ids: Vec<&str> = group_ids.iter().map(|s| s.as_str()).collect(); + let res = admin + .delete_groups(&group_ids, &AdminOptions::default()) + .await?; + tracing::debug!( + topic = self.topic, + ?fragment_ids, + "delete groups result: {res:?}" + ); + Ok(()) + } +} #[async_trait] impl SplitEnumerator for KafkaSplitEnumerator { @@ -178,29 +204,11 @@ impl SplitEnumerator for KafkaSplitEnumerator { } async fn on_drop_fragments(&mut self, fragment_ids: Vec) -> ConnectorResult<()> { - let admin = SHARED_KAFKA_ADMIN - .try_get_with_by_ref(&self.properties.connection, async { - tracing::info!("build new kafka admin for {}", self.broker_address); - Ok(Arc::new( - build_kafka_admin(&self.config, &self.properties).await?, - )) - }) - .await?; + self.drop_consumer_groups(fragment_ids).await + } - let group_ids = fragment_ids - .iter() - .map(|fragment_id| self.properties.group_id(*fragment_id)) - .collect::>(); - let group_ids: Vec<&str> = group_ids.iter().map(|s| s.as_str()).collect(); - let res = admin - .delete_groups(&group_ids, &AdminOptions::default()) - .await?; - tracing::debug!( - topic = self.topic, - ?fragment_ids, - "delete groups result: {res:?}" - ); - Ok(()) + async fn on_finish_backfill(&mut self, fragment_ids: Vec) -> ConnectorResult<()> { + self.drop_consumer_groups(fragment_ids).await } } diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index cc921bd01326f..e0eddb0dee507 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -327,5 +327,6 @@ impl KafkaSplitReader { // yield in the outer loop so that we can always guarantee that some messages are read // every `MAX_CHUNK_SIZE`. } + tracing::info!("kafka reader finished"); } } diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index 80a4de55b6f42..c86f405d78ccd 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -312,8 +312,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchSortAgg { group_key: [mv.v1], aggs: [max(mv.v2)] } - └─BatchExchange { order: [mv.v1 DESC], dist: HashShard(mv.v1) } - └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: UpstreamHashShard(mv.v1) } - sql: | create table t(v1 int, v2 int); select v1, max(v2) from t group by v1 order by v1 desc; @@ -367,8 +366,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchHashAgg { group_key: [mv.v1], aggs: [max(mv.v2)] } - └─BatchExchange { order: [], dist: HashShard(mv.v1) } - └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v1, mv.v2], distribution: UpstreamHashShard(mv.v1) } with_config_map: RW_BATCH_ENABLE_SORT_AGG: 'false' - name: Not use BatchSortAgg, when output requires order diff --git a/src/frontend/planner_test/tests/testdata/output/append_only.yaml b/src/frontend/planner_test/tests/testdata/output/append_only.yaml index d0701675c3617..5c326ecfcbad3 100644 --- a/src/frontend/planner_test/tests/testdata/output/append_only.yaml +++ b/src/frontend/planner_test/tests/testdata/output/append_only.yaml @@ -25,9 +25,10 @@ select v1 from t1 order by v1 limit 3 offset 3; stream_plan: |- StreamMaterialize { columns: [v1, t1._row_id(hidden)], stream_key: [t1._row_id], pk_columns: [v1, t1._row_id], pk_conflict: NoCheck } - └─StreamTopN [append_only] { order: [t1.v1 ASC], limit: 3, offset: 3 } - └─StreamExchange { dist: Single } - └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } + └─StreamExchange { dist: HashShard(t1.v1) } + └─StreamTopN [append_only] { order: [t1.v1 ASC], limit: 3, offset: 3 } + └─StreamExchange { dist: Single } + └─StreamTableScan { table: t1, columns: [t1.v1, t1._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t1._row_id], pk: [_row_id], dist: UpstreamHashShard(t1._row_id) } - sql: | create table t1 (v1 int, v2 int) append only; select max(v1) as max_v1 from t1; diff --git a/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml index 0cef2142c6f9f..7bb8e7fefe867 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_dist_agg.yaml @@ -11,9 +11,10 @@ sql: | select max(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [max(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [max(max(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [max(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } batch_local_plan: |- BatchSimpleAgg { aggs: [max(s.v)] } └─BatchExchange { order: [], dist: Single } @@ -160,7 +161,7 @@ └─BatchProject { exprs: [max(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [max(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } batch_local_plan: |- BatchProject { exprs: [max(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [max(s.v)] } 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 c80253fd16ef3..835a1f9d0d345 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -134,7 +134,7 @@ └─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) } 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) } + └─StreamExchange { dist: HashShard(nation.n_name, supplier.s_name, item.i_id) } └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard($expr2) } │ └─StreamProject { exprs: [item.i_id, item.i_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_i_id, stock.s_w_id, min(stock.s_quantity)] } @@ -182,7 +182,7 @@ Fragment 0 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 } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([8, 9, 10, 11, 12, 13]) from 1 + └── StreamExchange Hash([2, 1, 3]) from 1 Fragment 1 StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_name, item.i_id, item.i_name, supplier.s_address, supplier.s_phone, supplier.s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey] } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } @@ -363,7 +363,7 @@ Table 42 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 8, 9, 10, 11, 12, 13 ], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id, stock.s_w_id, min(stock.s_quantity), $expr2, region.r_regionkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 2, 1, 3 ], read pk prefix len hint: 9 } - id: ch_q3 before: @@ -413,67 +413,71 @@ └─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: |- 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 } - └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } - └─StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } - └─StreamExchange { dist: HashShard(orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_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 AND new_order.no_w_id = orders.o_w_id AND new_order.no_d_id = orders.o_d_id AND new_order.no_o_id = orders.o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND new_order.no_o_id = order_line.ol_o_id AND new_order.no_d_id = order_line.ol_d_id AND new_order.no_w_id = order_line.ol_w_id, output: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, customer.c_w_id, customer.c_d_id, customer.c_id, new_order.no_o_id, new_order.no_w_id, new_order.no_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } - ├─StreamExchange { dist: HashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } - │ └─StreamHashJoin { type: Inner, predicate: customer.c_d_id = new_order.no_d_id AND customer.c_w_id = new_order.no_w_id, output: all } - │ ├─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: IsNotNull(customer.c_w_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } - │ └─StreamExchange { dist: HashShard(new_order.no_d_id, new_order.no_w_id) } - │ └─StreamFilter { predicate: IsNotNull(new_order.no_w_id) AND IsNotNull(new_order.no_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], pk: [no_w_id, no_d_id, no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_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: IsNotNull(orders.o_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } - └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } - └─StreamFilter { predicate: IsNotNull(order_line.ol_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(sum(order_line.ol_amount), orders.o_entry_d) } + └─StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } + └─StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } + └─StreamExchange { dist: HashShard(orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_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 AND new_order.no_w_id = orders.o_w_id AND new_order.no_d_id = orders.o_d_id AND new_order.no_o_id = orders.o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND new_order.no_o_id = order_line.ol_o_id AND new_order.no_d_id = order_line.ol_d_id AND new_order.no_w_id = order_line.ol_w_id, output: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, customer.c_w_id, customer.c_d_id, customer.c_id, new_order.no_o_id, new_order.no_w_id, new_order.no_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } + ├─StreamExchange { dist: HashShard(new_order.no_o_id, new_order.no_d_id, new_order.no_w_id) } + │ └─StreamHashJoin { type: Inner, predicate: customer.c_d_id = new_order.no_d_id AND customer.c_w_id = new_order.no_w_id, output: all } + │ ├─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: IsNotNull(customer.c_w_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } + │ └─StreamExchange { dist: HashShard(new_order.no_d_id, new_order.no_w_id) } + │ └─StreamFilter { predicate: IsNotNull(new_order.no_w_id) AND IsNotNull(new_order.no_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], pk: [no_w_id, no_d_id, no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_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: IsNotNull(orders.o_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id) } + └─StreamFilter { predicate: IsNotNull(order_line.ol_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_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 } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } - └── StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0, 1, 2, 3]) from 1 + └── StreamExchange Hash([3, 4]) from 1 Fragment 1 + StreamProject { exprs: [order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id, sum(order_line.ol_amount), orders.o_entry_d] } + └── StreamHashAgg { group_key: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0, 1, 2, 3]) from 2 + + Fragment 2 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 AND new_order.no_w_id = orders.o_w_id AND new_order.no_d_id = orders.o_d_id AND new_order.no_o_id = orders.o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND new_order.no_o_id = order_line.ol_o_id AND new_order.no_d_id = order_line.ol_d_id AND new_order.no_w_id = order_line.ol_w_id, output: [orders.o_entry_d, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, customer.c_w_id, customer.c_d_id, customer.c_id, new_order.no_o_id, new_order.no_w_id, new_order.no_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([3, 4, 5]) from 2 + ├── StreamExchange Hash([3, 4, 5]) from 3 └── 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 } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } - ├── StreamExchange Hash([0, 1, 2]) from 5 - └── StreamExchange Hash([0, 1, 2]) from 6 + ├── StreamExchange Hash([0, 1, 2]) from 6 + └── StreamExchange Hash([0, 1, 2]) from 7 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: customer.c_d_id = new_order.no_d_id AND customer.c_w_id = new_order.no_w_id, output: all } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([1, 2]) from 3 - └── StreamExchange Hash([1, 2]) from 4 + ├── StreamExchange Hash([1, 2]) from 4 + └── StreamExchange Hash([1, 2]) from 5 - Fragment 3 + Fragment 4 StreamProject { exprs: [customer.c_id, customer.c_d_id, customer.c_w_id] } └── StreamFilter { predicate: IsNotNull(customer.c_w_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { tables: [ StreamScan: 9 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamFilter { predicate: IsNotNull(new_order.no_w_id) AND IsNotNull(new_order.no_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [new_order.no_w_id, new_order.no_d_id, new_order.no_o_id], pk: [no_w_id, no_d_id, no_o_id], dist: UpstreamHashShard(new_order.no_w_id, new_order.no_d_id, new_order.no_o_id) } { tables: [ StreamScan: 10 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: IsNotNull(orders.o_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamFilter { predicate: IsNotNull(order_line.ol_d_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } { tables: [ StreamScan: 16 ] } ├── Upstream @@ -513,7 +517,7 @@ Table 16 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d, _rw_timestamp ], primary key: [ $3 DESC, $4 ASC, $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 4, 0, 2, 1 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ ol_o_id, ol_w_id, ol_d_id, revenue, o_entry_d, _rw_timestamp ], primary key: [ $3 DESC, $4 ASC, $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3, 4 ], read pk prefix len hint: 5 } - id: ch_q4 before: @@ -1743,72 +1747,76 @@ └─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: |- StreamMaterialize { columns: [c_id, c_last, revenue, c_city, c_phone, n_name], stream_key: [c_id, c_last, c_city, c_phone, n_name], pk_columns: [revenue, c_id, c_last, c_city, c_phone, n_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } - └─StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } - └─StreamExchange { dist: HashShard(customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name) } - └─StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, order_line.ol_amount, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, nation.n_nationkey] } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [order_line.ol_amount, customer.c_id, customer.c_last, customer.c_city, customer.c_phone, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ └─StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } - │ └─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: IsNotNull(order_line.ol_w_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_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) } - │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } - └─StreamExchange { dist: HashShard(nation.n_nationkey) } - └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(sum(order_line.ol_amount)) } + └─StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } + └─StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } + └─StreamExchange { dist: HashShard(customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name) } + └─StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, order_line.ol_amount, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, nation.n_nationkey] } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─StreamProject { exprs: [order_line.ol_amount, customer.c_id, customer.c_last, customer.c_city, customer.c_phone, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ └─StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } + │ └─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: IsNotNull(order_line.ol_w_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_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) } + │ └─StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_c_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamExchange { dist: HashShard(nation.n_nationkey) } + └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_id, c_last, revenue, c_city, c_phone, n_name], stream_key: [c_id, c_last, c_city, c_phone, n_name], pk_columns: [revenue, c_id, c_last, c_city, c_phone, n_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } - └── StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0, 1, 2, 3, 5]) from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [customer.c_id, customer.c_last, sum(order_line.ol_amount), customer.c_city, customer.c_phone, nation.n_name] } + └── StreamHashAgg { group_key: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, nation.n_name], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0, 1, 2, 3, 5]) from 2 + + Fragment 2 StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [customer.c_id, customer.c_last, customer.c_city, customer.c_phone, order_line.ol_amount, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, nation.n_nationkey] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([5]) from 2 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([5]) from 3 + └── StreamExchange Hash([0]) from 8 - Fragment 2 + Fragment 3 StreamProject { exprs: [order_line.ol_amount, customer.c_id, customer.c_last, customer.c_city, customer.c_phone, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr1, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } └── StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } └── 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 } ├── tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] - ├── StreamExchange Hash([1, 2, 0, 1, 2]) from 3 - └── StreamExchange Hash([1, 2, 7, 8, 9]) from 4 + ├── StreamExchange Hash([1, 2, 0, 1, 2]) from 4 + └── StreamExchange Hash([1, 2, 7, 8, 9]) from 5 - Fragment 3 + Fragment 4 StreamFilter { predicate: IsNotNull(order_line.ol_w_id) AND IsNotNull(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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 9 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 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] } { tables: [ HashJoinLeft: 10, HashJoinDegreeLeft: 11, HashJoinRight: 12, HashJoinDegreeRight: 13 ] } - ├── StreamExchange Hash([1, 2, 0]) from 5 - └── StreamExchange Hash([1, 2, 3]) from 6 + ├── StreamExchange Hash([1, 2, 0]) from 6 + └── StreamExchange Hash([1, 2, 3]) from 7 - Fragment 5 + Fragment 6 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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { tables: [ StreamScan: 14 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 16 ] } ├── Upstream └── BatchPlanNode @@ -1847,7 +1855,7 @@ Table 16 { columns: [ vnode, n_nationkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_id, c_last, revenue, c_city, c_phone, n_name, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1, 3, 4, 5 ], read pk prefix len hint: 6 } + Table 4294967294 { columns: [ c_id, c_last, revenue, c_city, c_phone, n_name, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 6 } - id: ch_q11 before: @@ -1921,78 +1929,81 @@ └─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) } 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)] } - └─StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } - ├─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } - │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } - │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ └─StreamShare { id: 10 } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_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) } - │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamProject { exprs: [nation.n_nationkey] } - │ │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_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] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } - └─StreamShare { id: 10 } - └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_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) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } + └─StreamExchange { dist: HashShard(sum(stock.s_order_cnt)) } + └─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } + └─StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } + ├─StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } + │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } + │ └─StreamExchange { dist: HashShard(stock.s_i_id) } + │ └─StreamShare { id: 10 } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_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) } + │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamProject { exprs: [nation.n_nationkey] } + │ │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_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] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } + └─StreamShare { id: 10 } + └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_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) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_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 } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } - └── StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } - ├── tables: [ DynamicFilterLeft: 0, DynamicFilterRight: 1 ] - ├── StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } - │ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } - │ ├── tables: [ HashAggState: 2 ] - │ └── StreamExchange Hash([0]) from 1 - └── StreamExchange Broadcast from 7 + └── StreamExchange Hash([1]) from 1 Fragment 1 - StreamNoOp - └── StreamExchange NoShuffle from 2 + StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt)] } + └── StreamDynamicFilter { predicate: ($expr2 > $expr3), output: [stock.s_i_id, sum(stock.s_order_cnt), $expr2] } + ├── tables: [ DynamicFilterLeft: 0, DynamicFilterRight: 1 ] + ├── StreamProject { exprs: [stock.s_i_id, sum(stock.s_order_cnt), sum(stock.s_order_cnt)::Decimal as $expr2] } + │ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [sum(stock.s_order_cnt), count] } { tables: [ HashAggState: 2 ] } + │ └── StreamExchange Hash([0]) from 2 + └── StreamExchange Broadcast from 8 Fragment 2 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Fragment 3 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([2]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - ├── StreamExchange Hash([1]) from 4 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([1]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 4 + Fragment 5 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 11 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } @@ -2000,21 +2011,21 @@ ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 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], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } ├── tables: [ StreamScan: 13 ] ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [(sum(sum(stock.s_order_cnt))::Decimal * 0.005:Decimal) as $expr3] } └── StreamSimpleAgg { aggs: [sum(sum(stock.s_order_cnt)), count] } { tables: [ SimpleAggState: 14 ] } - └── StreamExchange Single from 8 + └── StreamExchange Single from 9 - Fragment 8 + Fragment 9 StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } - └── StreamExchange NoShuffle from 2 + └── StreamExchange NoShuffle from 3 Table 0 { columns: [ stock_s_i_id, sum(stock_s_order_cnt), $expr2, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2046,7 +2057,7 @@ Table 14 { columns: [ sum(sum(stock_s_order_cnt)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ s_i_id, ordercount, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_i_id, ordercount, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - id: ch_q12 before: @@ -2171,43 +2182,47 @@ └─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) } 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] } - └─StreamExchange { dist: HashShard(count(orders.o_id)) } - └─StreamProject { exprs: [customer.c_id, count(orders.o_id)] } - └─StreamHashAgg { group_key: [customer.c_id], aggs: [count(orders.o_id), count] } - └─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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } + └─StreamExchange { dist: HashShard(count, count(orders.o_id)) } + └─StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(orders.o_id)) } + └─StreamProject { exprs: [customer.c_id, count(orders.o_id)] } + └─StreamHashAgg { group_key: [customer.c_id], aggs: [count(orders.o_id), count] } + └─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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_count, custdist], stream_key: [c_count], pk_columns: [custdist, c_count], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([1]) from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamHashAgg { group_key: [count(orders.o_id)], aggs: [count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([1]) from 2 + + Fragment 2 StreamProject { exprs: [customer.c_id, count(orders.o_id)] } └── StreamHashAgg { group_key: [customer.c_id], aggs: [count(orders.o_id), count] } { tables: [ HashAggState: 1 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 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] } ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] - ├── StreamExchange Hash([1, 2, 0]) from 3 - └── StreamExchange Hash([1, 2, 3]) from 4 + ├── StreamExchange Hash([1, 2, 0]) from 4 + └── StreamExchange Hash([1, 2, 3]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } ├── tables: [ StreamScan: 6 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } @@ -2231,7 +2246,7 @@ Table 7 { columns: [ vnode, o_w_id, o_d_id, o_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: ch_q14 before: @@ -2348,7 +2363,7 @@ └─LogicalScan { table: revenue1, columns: [revenue1.total_revenue] } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(supplier.s_suppkey, revenue1.total_revenue, revenue1.supplier_no) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } └─StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } ├─StreamExchange { dist: HashShard(revenue1.total_revenue) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } @@ -2368,7 +2383,7 @@ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no(hidden)], stream_key: [s_suppkey, revenue1.supplier_no, total_revenue], pk_columns: [s_suppkey, revenue1.supplier_no, total_revenue], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([0, 4, 5]) from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: Inner, predicate: revenue1.total_revenue = max(max(revenue1.total_revenue)), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, revenue1.total_revenue, revenue1.supplier_no] } @@ -2443,7 +2458,7 @@ Table 14 { columns: [ vnode, supplier_no, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, revenue1.supplier_no, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - id: ch_q16 before: @@ -2487,58 +2502,62 @@ └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], distribution: UpstreamHashShard(stock.s_w_id, stock.s_i_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)] } - └─StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } - └─StreamExchange { dist: HashShard(item.i_name, $expr2, item.i_price) } - └─StreamProject { exprs: [item.i_name, Substr(item.i_data, 1:Int32, 3:Int32) as $expr2, item.i_price, ((stock.s_w_id * stock.s_i_id) % 10000:Int32) as $expr3, stock.s_w_id, stock.s_i_id, $expr1] } - └─StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: all } - ├─StreamExchange { dist: HashShard($expr1) } - │ └─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], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_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], stream_scan_type: ArrangementBackfill, stream_key: [item.i_id], pk: [i_id], dist: UpstreamHashShard(item.i_id) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamProject { exprs: [supplier.s_suppkey] } - └─StreamFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } - └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(count(distinct $expr3)) } + └─StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } + └─StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } + └─StreamExchange { dist: HashShard(item.i_name, $expr2, item.i_price) } + └─StreamProject { exprs: [item.i_name, Substr(item.i_data, 1:Int32, 3:Int32) as $expr2, item.i_price, ((stock.s_w_id * stock.s_i_id) % 10000:Int32) as $expr3, stock.s_w_id, stock.s_i_id, $expr1] } + └─StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: all } + ├─StreamExchange { dist: HashShard($expr1) } + │ └─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], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_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], stream_scan_type: ArrangementBackfill, stream_key: [item.i_id], pk: [i_id], dist: UpstreamHashShard(item.i_id) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamProject { exprs: [supplier.s_suppkey] } + └─StreamFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } + └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } stream_dist_plan: |+ Fragment 0 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 } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } - └── StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } { tables: [ HashAggState: 0, HashAggDedupForCol3: 1 ] } - └── StreamExchange Hash([0, 1, 2]) from 1 + └── StreamExchange Hash([3]) from 1 Fragment 1 + StreamProject { exprs: [item.i_name, $expr2, item.i_price, count(distinct $expr3)] } + └── StreamHashAgg { group_key: [item.i_name, $expr2, item.i_price], aggs: [count(distinct $expr3), count] } { tables: [ HashAggState: 0, HashAggDedupForCol3: 1 ] } + └── StreamExchange Hash([0, 1, 2]) from 2 + + Fragment 2 StreamProject { exprs: [item.i_name, Substr(item.i_data, 1:Int32, 3:Int32) as $expr2, item.i_price, ((stock.s_w_id * stock.s_i_id) % 10000:Int32) as $expr3, stock.s_w_id, stock.s_i_id, $expr1] } └── StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: all } { tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] } - ├── StreamExchange Hash([5]) from 2 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([5]) from 3 + └── StreamExchange Hash([0]) from 6 - Fragment 2 + Fragment 3 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] } ├── tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } ├── tables: [ StreamScan: 10 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 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], stream_scan_type: ArrangementBackfill, stream_key: [item.i_id], pk: [i_id], dist: UpstreamHashShard(item.i_id) } ├── tables: [ StreamScan: 11 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamProject { exprs: [supplier.s_suppkey] } └── StreamFilter { predicate: Like(supplier.s_comment, '%bad%':Varchar) } └── StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } @@ -2582,7 +2601,7 @@ Table 12 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ i_name, brand, i_price, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ i_name, brand, i_price, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3 ], read pk prefix len hint: 4 } - id: ch_q17 before: @@ -2765,48 +2784,52 @@ └─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: |- 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 } - └─StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } - └─StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } - └─StreamProject { exprs: [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, sum(order_line.ol_amount)] } - └─StreamHashAgg { group_key: [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], aggs: [sum(order_line.ol_amount), count] } - └─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: [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, order_line.ol_amount, customer.c_w_id, customer.c_d_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) } - │ └─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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_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, orders.o_ol_cnt], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(sum(order_line.ol_amount), orders.o_entry_d) } + └─StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } + └─StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } + └─StreamProject { exprs: [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, sum(order_line.ol_amount)] } + └─StreamHashAgg { group_key: [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], aggs: [sum(order_line.ol_amount), count] } + └─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: [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, order_line.ol_amount, customer.c_w_id, customer.c_d_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) } + │ └─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], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_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, orders.o_ol_cnt], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_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 } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } - └── StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } - └── StreamProject { exprs: [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, sum(order_line.ol_amount)] } - └── StreamHashAgg { group_key: [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], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } - └── 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: [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, order_line.ol_amount, customer.c_w_id, customer.c_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([2, 3, 4]) from 1 - └── StreamExchange Hash([0, 1, 2]) from 4 + └── StreamExchange Hash([5, 2]) from 1 Fragment 1 - 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] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([1, 2, 0]) from 2 - └── StreamExchange Hash([1, 2, 3]) from 3 + StreamProject { exprs: [customer.c_last, customer.c_id, orders.o_entry_d, orders.o_ol_cnt, sum(order_line.ol_amount), sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id] } + └── StreamFilter { predicate: (sum(order_line.ol_amount) > 200:Decimal) } + └── StreamProject { exprs: [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, sum(order_line.ol_amount)] } + └── StreamHashAgg { group_key: [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], aggs: [sum(order_line.ol_amount), count] } { tables: [ HashAggState: 0 ] } + └── 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: [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, order_line.ol_amount, customer.c_w_id, customer.c_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] + ├── StreamExchange Hash([2, 3, 4]) from 2 + └── StreamExchange Hash([0, 1, 2]) from 5 Fragment 2 + 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] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } + ├── StreamExchange Hash([1, 2, 0]) from 3 + └── StreamExchange Hash([1, 2, 3]) from 4 + + Fragment 3 StreamTableScan { table: customer, columns: [customer.c_id, customer.c_d_id, customer.c_w_id, customer.c_last], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_w_id, customer.c_d_id, customer.c_id], pk: [c_w_id, c_d_id, c_id], dist: UpstreamHashShard(customer.c_w_id, customer.c_d_id, customer.c_id) } { tables: [ StreamScan: 9 ] } ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 10 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 11 ] ├── Upstream @@ -2836,7 +2859,7 @@ Table 11 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id, _rw_timestamp ], primary key: [ $5 DESC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6, 7, 8 ], read pk prefix len hint: 8 } + Table 4294967294 { columns: [ c_last, o_id, o_entry_d, o_ol_cnt, sum, sum(order_line.ol_amount), orders.o_id, orders.o_d_id, orders.o_w_id, _rw_timestamp ], primary key: [ $5 DESC, $2 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 5, 2 ], read pk prefix len hint: 8 } - id: ch_q19 before: @@ -2988,7 +3011,7 @@ └─BatchScan { table: order_line, columns: [order_line.ol_i_id, order_line.ol_quantity, order_line.ol_delivery_d], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) } + └─StreamExchange { dist: HashShard(supplier.s_name) } └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } @@ -3019,7 +3042,7 @@ Fragment 0 StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, supplier.s_nationkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([2, 3]) from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } @@ -3122,7 +3145,7 @@ Table 21 { columns: [ vnode, i_id, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2, 3 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - id: ch_q21 before: @@ -3183,102 +3206,106 @@ └─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 } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(supplier.s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND 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 > order_line.ol_delivery_d), output: [supplier.s_name, stock.s_w_id, stock.s_i_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_nationkey, $expr1, order_line.ol_w_id] } - ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } - │ └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [supplier.s_name, 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, order_line.ol_number, $expr1, supplier.s_suppkey, supplier.s_nationkey] } - │ ├─StreamExchange { dist: HashShard($expr1) } - │ │ └─StreamProject { 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, stock.s_w_id, stock.s_i_id, order_line.ol_number] } - │ │ └─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: IsNotNull(stock.s_w_id) } - │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_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) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } + └─StreamExchange { dist: HashShard(count, supplier.s_name) } + └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(supplier.s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND 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 > order_line.ol_delivery_d), output: [supplier.s_name, stock.s_w_id, stock.s_i_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_nationkey, $expr1, order_line.ol_w_id] } + ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_w_id, order_line.ol_d_id) } + │ └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [supplier.s_name, 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, order_line.ol_number, $expr1, supplier.s_suppkey, supplier.s_nationkey] } + │ ├─StreamExchange { dist: HashShard($expr1) } + │ │ └─StreamProject { 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, stock.s_w_id, stock.s_i_id, order_line.ol_number] } + │ │ └─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: IsNotNull(stock.s_w_id) } + │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(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], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_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) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_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 } ├── tables: [ Materialize: 4294967294 ] - └── StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0]) from 2 + + Fragment 2 StreamHashJoin { type: LeftAnti, predicate: order_line.ol_o_id = order_line.ol_o_id AND 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 > order_line.ol_delivery_d), output: [supplier.s_name, stock.s_w_id, stock.s_i_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_nationkey, $expr1, order_line.ol_w_id] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([1, 3, 2]) from 2 - └── StreamExchange Hash([0, 1, 2]) from 11 + ├── StreamExchange Hash([1, 3, 2]) from 3 + └── StreamExchange Hash([0, 1, 2]) from 12 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [supplier.s_name, 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, order_line.ol_number, $expr1, supplier.s_suppkey, supplier.s_nationkey] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([4]) from 3 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([4]) from 4 + └── StreamExchange Hash([0]) from 9 - Fragment 3 + Fragment 4 StreamProject { 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, stock.s_w_id, stock.s_i_id, order_line.ol_number] } └── 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] } { tables: [ HashJoinLeft: 9, HashJoinDegreeLeft: 10, HashJoinRight: 11, HashJoinDegreeRight: 12 ] } - ├── StreamExchange Hash([1, 0, 1]) from 4 - └── StreamExchange Hash([2, 3, 5]) from 5 + ├── StreamExchange Hash([1, 0, 1]) from 5 + └── StreamExchange Hash([2, 3, 5]) from 6 - Fragment 4 + Fragment 5 StreamFilter { predicate: IsNotNull(stock.s_w_id) } └── StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], stream_scan_type: ArrangementBackfill, stream_key: [stock.s_w_id, stock.s_i_id], pk: [s_w_id, s_i_id], dist: UpstreamHashShard(stock.s_w_id, stock.s_i_id) } { tables: [ StreamScan: 13 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + 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_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 } { tables: [ HashJoinLeft: 14, HashJoinDegreeLeft: 15, HashJoinRight: 16, HashJoinDegreeRight: 17 ] } - ├── StreamExchange Hash([0, 1, 2]) from 6 - └── StreamExchange Hash([0, 1, 2]) from 7 + ├── StreamExchange Hash([0, 1, 2]) from 7 + └── StreamExchange Hash([0, 1, 2]) from 8 - Fragment 6 + Fragment 7 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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 18 ] ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: orders, columns: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_w_id, orders.o_d_id, orders.o_id], pk: [o_w_id, o_d_id, o_id], dist: UpstreamHashShard(orders.o_w_id, orders.o_d_id, orders.o_id) } { tables: [ StreamScan: 19 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { tables: [ HashJoinLeft: 20, HashJoinDegreeLeft: 21, HashJoinRight: 22, HashJoinDegreeRight: 23 ] } - ├── StreamExchange Hash([2]) from 9 - └── StreamExchange Hash([0]) from 10 + ├── StreamExchange Hash([2]) from 10 + └── StreamExchange Hash([0]) from 11 - Fragment 9 + Fragment 10 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 24 ] } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'CHINA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 25 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 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], stream_scan_type: ArrangementBackfill, stream_key: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], pk: [ol_w_id, ol_d_id, ol_o_id, ol_number], dist: UpstreamHashShard(order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number) } ├── tables: [ StreamScan: 26 ] @@ -3339,7 +3366,7 @@ Table 26 { columns: [ vnode, ol_w_id, ol_d_id, ol_o_id, ol_number, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: ch_q22 before: diff --git a/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml b/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml index 2b0c10cf89b3d..1d74ed20b82f5 100644 --- a/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml +++ b/src/frontend/planner_test/tests/testdata/output/functional_dependency.yaml @@ -14,12 +14,13 @@ └─BatchScan { table: t1, columns: [t1.id, t1.i], limit: 2, distribution: UpstreamHashShard(t1.id) } stream_plan: |- StreamMaterialize { columns: [id, i], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.id, t1.i] } - └─StreamTopN { order: [t1.id ASC], limit: 2, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } - └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } + └─StreamExchange { dist: HashShard(t1.id) } + └─StreamProject { exprs: [t1.id, t1.i] } + └─StreamTopN { order: [t1.id ASC], limit: 2, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } + └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } - name: test functional dependency for order key pruning (order by - suffix fd) sql: | create table t1 (id int primary key, i int); @@ -35,12 +36,13 @@ └─BatchScan { table: t1, columns: [t1.id, t1.i], distribution: UpstreamHashShard(t1.id) } stream_plan: |- StreamMaterialize { columns: [id, i], stream_key: [id], pk_columns: [i, id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.id, t1.i] } - └─StreamTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } - └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } + └─StreamExchange { dist: HashShard(t1.i, t1.id) } + └─StreamProject { exprs: [t1.id, t1.i] } + └─StreamTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t1.i ASC, t1.id ASC], limit: 2, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t1.id, t1.i, Vnode(t1.id) as _vnode] } + └─StreamTableScan { table: t1, columns: [t1.id, t1.i], stream_scan_type: ArrangementBackfill, stream_key: [t1.id], pk: [id], dist: UpstreamHashShard(t1.id) } - name: test functional dependency for order key pruning on singleton sql: | create table t1 (id int primary key, i int); @@ -54,8 +56,9 @@ └─BatchSort { order: [v.cnt ASC] } └─BatchScan { table: v, columns: [v.cnt], distribution: Single } stream_plan: |- - StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamTableScan { table: v, columns: [v.cnt], stream_scan_type: ArrangementBackfill, stream_key: [], pk: [], dist: Single } + StreamMaterialize { columns: [cnt], stream_key: [], pk_columns: [cnt], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(v.cnt) } + └─StreamTableScan { table: v, columns: [v.cnt], stream_scan_type: ArrangementBackfill, stream_key: [], pk: [], dist: Single } - name: test functional dependency for order key pruning (index) sql: | create table t1 (v1 int, v2 int); diff --git a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml index 349c5f7d89012..209457976b58d 100644 --- a/src/frontend/planner_test/tests/testdata/output/index_selection.yaml +++ b/src/frontend/planner_test/tests/testdata/output/index_selection.yaml @@ -616,12 +616,13 @@ select * from t1 order by a limit 1; stream_plan: |- StreamMaterialize { columns: [a], stream_key: [], pk_columns: [a], pk_conflict: NoCheck } - └─StreamProject { exprs: [t1.a] } - └─StreamTopN { order: [t1.a ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t1.a ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t1.a, Vnode(t1.a) as _vnode] } - └─StreamTableScan { table: t1, columns: [t1.a], stream_scan_type: ArrangementBackfill, stream_key: [t1.a], pk: [a], dist: UpstreamHashShard(t1.a) } + └─StreamExchange { dist: HashShard(t1.a) } + └─StreamProject { exprs: [t1.a] } + └─StreamTopN { order: [t1.a ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t1.a ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t1.a, Vnode(t1.a) as _vnode] } + └─StreamTableScan { table: t1, columns: [t1.a], stream_scan_type: ArrangementBackfill, stream_key: [t1.a], pk: [a], dist: UpstreamHashShard(t1.a) } - sql: | create table t1 (a varchar, b int, c int, d int); create index idx on t1(a); diff --git a/src/frontend/planner_test/tests/testdata/output/join.yaml b/src/frontend/planner_test/tests/testdata/output/join.yaml index aaa401e469d16..8696857df85c1 100644 --- a/src/frontend/planner_test/tests/testdata/output/join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/join.yaml @@ -530,7 +530,7 @@ └─BatchScan { table: b, columns: [b.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [y, z, $expr2(hidden), a._row_id(hidden), b._row_id(hidden), a.x(hidden), b.x(hidden)], stream_key: [a._row_id, b._row_id, a.x, b.x], pk_columns: [$expr2, a._row_id, b._row_id, a.x, b.x], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(a._row_id, b._row_id, a.x, b.x) } + └─StreamExchange { dist: HashShard($expr2) } └─StreamProject { exprs: [(2:Int32 * $expr1) as $expr3, $expr2, $expr2, a._row_id, b._row_id, a.x, b.x] } └─StreamProject { exprs: [a.x, b.x, $expr1, ($expr1 + $expr1) as $expr2, a._row_id, b._row_id] } └─StreamProject { exprs: [a.x, b.x, Coalesce(a.x, b.x) as $expr1, a._row_id, b._row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/limit.yaml b/src/frontend/planner_test/tests/testdata/output/limit.yaml index a7b5c0d7a7f48..29ab9e6a12f7a 100644 --- a/src/frontend/planner_test/tests/testdata/output/limit.yaml +++ b/src/frontend/planner_test/tests/testdata/output/limit.yaml @@ -99,8 +99,9 @@ └─BatchValues { rows: [[1:Int32]] } stream_plan: |- StreamMaterialize { columns: [c, _row_id(hidden)], stream_key: [], pk_columns: [c], pk_conflict: NoCheck } - └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 1, offset: 0 } - └─StreamValues { rows: [[1:Int32, 0:Int64]] } + └─StreamExchange { dist: HashShard(1:Int32) } + └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 1, offset: 0 } + └─StreamValues { rows: [[1:Int32, 0:Int64]] } - sql: | select 1 c union all select 1 c limit 10 batch_plan: |- @@ -117,8 +118,9 @@ └─BatchValues { rows: [[1:Int32], [1:Int32]] } stream_plan: |- StreamMaterialize { columns: [c, _row_id(hidden)], stream_key: [_row_id], pk_columns: [c, _row_id], pk_conflict: NoCheck } - └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 10, offset: 0 } - └─StreamValues { rows: [[1:Int32, 0:Int64], [1:Int32, 1:Int64]] } + └─StreamExchange { dist: HashShard(1:Int32) } + └─StreamTopN [append_only] { order: [1:Int32 ASC], limit: 10, offset: 0 } + └─StreamValues { rows: [[1:Int32, 0:Int64], [1:Int32, 1:Int64]] } - sql: | create table t (a int); select count(*) from t limit 1; diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index 36e8c13b2c19f..6c19f66eba1a0 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -2271,42 +2271,46 @@ └─BatchScan { table: bid, columns: [bid.auction], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } - └─StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } - └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } - └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } - ├─StreamExchange { dist: HashShard(auction.id) } - │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], stream_scan_type: ArrangementBackfill, stream_key: [auction.id], pk: [id], dist: UpstreamHashShard(auction.id) } - └─StreamExchange { dist: HashShard(bid.auction) } - └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } + └─StreamExchange { dist: HashShard(count(bid.auction)) } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + └─StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } + └─StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } + └─StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } + ├─StreamExchange { dist: HashShard(auction.id) } + │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], stream_scan_type: ArrangementBackfill, stream_key: [auction.id], pk: [id], dist: UpstreamHashShard(auction.id) } + └─StreamExchange { dist: HashShard(bid.auction) } + └─StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id], pk_columns: [bid_count, auction_id], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } - └── StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction)] } + └── StreamTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(bid.auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [auction.id, internal_last_seen_value(auction.item_name), count(bid.auction), Vnode(auction.id) as $expr1] } └── StreamHashAgg { group_key: [auction.id], aggs: [internal_last_seen_value(auction.item_name), count(bid.auction), count] } ├── tables: [ HashAggState: 2 ] └── StreamHashJoin { type: Inner, predicate: auction.id = bid.auction, output: all } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 - Fragment 2 + Fragment 3 StreamTableScan { table: auction, columns: [auction.id, auction.item_name], stream_scan_type: ArrangementBackfill, stream_key: [auction.id], pk: [id], dist: UpstreamHashShard(auction.id) } ├── tables: [ StreamScan: 7 ] ├── Upstream └── BatchPlanNode - Fragment 3 + Fragment 4 StreamTableScan { table: bid, columns: [bid.auction, bid._row_id], stream_scan_type: ArrangementBackfill, stream_key: [bid._row_id], pk: [_row_id], dist: UpstreamHashShard(bid._row_id) } ├── tables: [ StreamScan: 8 ] ├── Upstream @@ -2362,7 +2366,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 2 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml index 472bb56b2c12e..fefc517062486 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -2228,41 +2228,45 @@ └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [id, item_name, count(auction)] } - └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } - └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } - └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamExchange { dist: HashShard(count(auction)) } + └─StreamProject { exprs: [id, item_name, count(auction)] } + └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [id, item_name, count(auction)] } - └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [id, item_name, count(auction)] } + └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 10 } └── StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } { tables: [ Source: 7 ] } - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } { tables: [ Source: 8 ] } @@ -2309,7 +2313,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 3 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml index 469d94ca87ff3..8ab8fdee21d1e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml @@ -2254,43 +2254,47 @@ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [id, item_name, count(auction)] } - └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } - └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } - └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(id) } - │ └─StreamRowIdGen { row_id_index: 13 } - │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(auction) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count(auction)) } + └─StreamProject { exprs: [id, item_name, count(auction)] } + └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [id, item_name, count(auction)] } - └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [id, item_name, count(auction)] } + └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 3 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 - Fragment 2 + Fragment 3 StreamRowIdGen { row_id_index: 13 } └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } ├── tables: [ SourceBackfill: 7 ] └── Upstream - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 10 } └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 8 ] } └── Upstream @@ -2338,7 +2342,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 3 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml index 0b15386274989..8ec6f1f3920db 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_temporal_filter.yaml @@ -1960,73 +1960,77 @@ LIMIT 1000; stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, $expr3, count($expr5)] } - └─StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count($expr5) DESC], limit: 1000, offset: 0, group_key: [$expr6] } - └─StreamProject { exprs: [$expr2, $expr3, count($expr5), Vnode($expr2) as $expr6] } - └─StreamHashAgg { group_key: [$expr2, $expr3], aggs: [count($expr5), count] } - └─StreamHashJoin { type: Inner, predicate: $expr2 = $expr5, output: [$expr2, $expr3, $expr5, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr5) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } - └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [[$expr1]], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } - ├─StreamFilter { predicate: (event_type = 2:Int32) } - │ └─StreamShare { id: 5 } - │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [[$expr4]] } - └─StreamNow { output: [now] } + └─StreamExchange { dist: HashShard(count($expr5)) } + └─StreamProject { exprs: [$expr2, $expr3, count($expr5)] } + └─StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count($expr5) DESC], limit: 1000, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [$expr2, $expr3, count($expr5), Vnode($expr2) as $expr6] } + └─StreamHashAgg { group_key: [$expr2, $expr3], aggs: [count($expr5), count] } + └─StreamHashJoin { type: Inner, predicate: $expr2 = $expr5, output: [$expr2, $expr3, $expr5, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr5) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } + └─StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [[$expr1]], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├─StreamFilter { predicate: (event_type = 2:Int32) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [[$expr4]] } + └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [$expr2, $expr3, count($expr5)] } - └── StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [$expr2, $expr3, count($expr5)] } + └── StreamTopN { order: [count($expr5) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count($expr5) DESC], limit: 1000, offset: 0, group_key: [$expr6] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [$expr2, $expr3, count($expr5), Vnode($expr2) as $expr6] } └── StreamHashAgg { group_key: [$expr2, $expr3], aggs: [count($expr5), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin { type: Inner, predicate: $expr2 = $expr5, output: [$expr2, $expr3, $expr5, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 5 - Fragment 2 + Fragment 3 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, auction, bid, $expr1, _row_id], output_watermarks: [[$expr1]] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [[$expr1]] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 7 ] } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(bid, 0:Int32) as $expr5, _row_id] } └── StreamDynamicFilter { predicate: ($expr1 > $expr4), output_watermarks: [[$expr1]], output: [event_type, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } ├── tables: [ DynamicFilterLeft: 8, DynamicFilterRight: 9 ] ├── StreamFilter { predicate: (event_type = 2:Int32) } - │ └── StreamExchange NoShuffle from 3 - └── StreamExchange Broadcast from 5 + │ └── StreamExchange NoShuffle from 4 + └── StreamExchange Broadcast from 6 - Fragment 5 + Fragment 6 StreamProject { exprs: [SubtractWithTimeZone(now, '00:05:00':Interval, 'UTC':Varchar) as $expr4], output_watermarks: [[$expr4]] } └── StreamNow { output: [now] } { tables: [ Now: 10 ] } @@ -2072,7 +2076,7 @@ ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] - ├── distribution key: [] + ├── distribution key: [ 2 ] └── read pk prefix len hint: 3 - id: nexmark_q106 diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml index b655102926a77..ecec45533601e 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -2417,56 +2417,60 @@ └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_plan: |- StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, $expr3, count($expr4)] } - └─StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count($expr4) DESC], limit: 1000, offset: 0, group_key: [$expr5] } - └─StreamProject { exprs: [$expr2, $expr3, count($expr4), Vnode($expr2) as $expr5] } - └─StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [count($expr4), count] } - └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr4, output: [$expr2, $expr3, $expr4, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } - │ └─StreamFilter { predicate: (event_type = 1:Int32) } - │ └─StreamShare { id: 6 } - │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - │ └─StreamRowIdGen { row_id_index: 5 } - │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } - │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } - └─StreamExchange { dist: HashShard($expr4) } - └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } - └─StreamFilter { predicate: (event_type = 2:Int32) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [event_type, auction, bid, _row_id] } - └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } - └─StreamRowIdGen { row_id_index: 5 } - └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } - └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } - └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard(count($expr4)) } + └─StreamProject { exprs: [$expr2, $expr3, count($expr4)] } + └─StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count($expr4) DESC], limit: 1000, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr2, $expr3, count($expr4), Vnode($expr2) as $expr5] } + └─StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [count($expr4), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr4, output: [$expr2, $expr3, $expr4, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard($expr2) } + │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } + │ └─StreamFilter { predicate: (event_type = 1:Int32) } + │ └─StreamShare { id: 6 } + │ └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + │ └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + │ └─StreamRowIdGen { row_id_index: 5 } + │ └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } + │ └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + │ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } + └─StreamExchange { dist: HashShard($expr4) } + └─StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } + └─StreamFilter { predicate: (event_type = 2:Int32) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [event_type, auction, bid, _row_id] } + └─StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamWatermarkFilter { watermark_descs: [Desc { column: $expr1, expr: ($expr1 - '00:00:04':Interval) }], output_watermarks: [[$expr1]] } + └─StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } + └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [$expr2, $expr3, count($expr4)] } - └── StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [$expr2, $expr3, count($expr4)] } + └── StreamTopN { order: [count($expr4) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count($expr4) DESC], limit: 1000, offset: 0, group_key: [$expr5] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [$expr2, $expr3, count($expr4), Vnode($expr2) as $expr5] } └── StreamHashAgg [append_only] { group_key: [$expr2, $expr3], aggs: [count($expr4), count] } { tables: [ HashAggState: 2 ] } └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr4, output: [$expr2, $expr3, $expr4, _row_id, _row_id] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 5 - Fragment 2 + Fragment 3 StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } └── StreamFilter { predicate: (event_type = 1:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamProject { exprs: [event_type, auction, bid, _row_id] } └── StreamFilter { predicate: ((event_type = 1:Int32) OR (event_type = 2:Int32)) } └── StreamRowIdGen { row_id_index: 5 } @@ -2474,10 +2478,10 @@ └── StreamProject { exprs: [event_type, person, auction, bid, Case((event_type = 0:Int32), Field(person, 6:Int32), (event_type = 1:Int32), Field(auction, 5:Int32), Field(bid, 5:Int32)) as $expr1, _row_id] } └── StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } { tables: [ Source: 8 ] } - Fragment 4 + Fragment 5 StreamProject { exprs: [Field(bid, 0:Int32) as $expr4, _row_id] } └── StreamFilter { predicate: (event_type = 2:Int32) } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ $expr2, $expr3, count($expr4), $expr5, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 0 } @@ -2503,7 +2507,7 @@ Table 8 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 2 ], read pk prefix len hint: 3 } eowc_stream_error: |- Not supported: Emit-On-Window-Close mode requires a watermark column in GROUP BY. diff --git a/src/frontend/planner_test/tests/testdata/output/order_by.yaml b/src/frontend/planner_test/tests/testdata/output/order_by.yaml index 16f55c5fd981b..a04e18dd46a39 100644 --- a/src/frontend/planner_test/tests/testdata/output/order_by.yaml +++ b/src/frontend/planner_test/tests/testdata/output/order_by.yaml @@ -9,7 +9,8 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [v1, t._row_id], pk_conflict: NoCheck } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: output names are not qualified after table names sql: | create table t (v1 bigint, v2 double precision); @@ -65,8 +66,9 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, 2:Int32(hidden), t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [2:Int32, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(2:Int32) } + └─StreamProject { exprs: [t.v1, t.v2, 2:Int32, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); select * from t order by v; @@ -85,12 +87,13 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [v1, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } - └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t.v1 DESC], limit: 5, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } + └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t.v1 DESC], limit: 5, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - sql: | create table t (v1 bigint, v2 double precision); select * from t limit 3 offset 4; @@ -117,12 +120,13 @@ └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [v1, v2, t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [v1, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } - └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 7 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [t.v1 DESC], limit: 12, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } - └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(t.v1) } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id] } + └─StreamTopN { order: [t.v1 DESC], limit: 5, offset: 7 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [t.v1 DESC], limit: 12, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [t.v1, t.v2, t._row_id, Vnode(t._row_id) as _vnode] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: order by expression that would be valid in select list sql: | create table t (x int, y int, z int); @@ -138,8 +142,9 @@ └─BatchScan { table: t, columns: [t.x, t.y, t.z], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y, $expr1(hidden), t.z(hidden), t._row_id(hidden)], stream_key: [t._row_id], pk_columns: [$expr1, t.z, t._row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } - └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard($expr1, t.z) } + └─StreamProject { exprs: [t.x, t.y, (t.x + t.y) as $expr1, t.z, t._row_id] } + └─StreamTableScan { table: t, columns: [t.x, t.y, t.z, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: order by the number of an output column sql: | create table t (x int, y int); @@ -175,7 +180,7 @@ select * from mv order by v asc; batch_plan: |- BatchExchange { order: [mv.v ASC], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - name: BatchSort needed, when input is sorted in wrong order sql: | create table t(v int); @@ -184,7 +189,7 @@ batch_plan: |- BatchExchange { order: [mv.v DESC], dist: Single } └─BatchSort { order: [mv.v DESC] } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - name: No BatchSort needed, when input is already sorted sql: | create table t(v int); @@ -192,7 +197,7 @@ select * from mv order by v desc; batch_plan: |- BatchExchange { order: [mv.v DESC], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - name: BatchSort needed, because our BatchScan can not get a ordered result when scan from ranges sql: | create table t(v int); @@ -201,7 +206,7 @@ batch_plan: |- BatchExchange { order: [mv.v ASC], dist: Single } └─BatchSort { order: [mv.v ASC] } - └─BatchScan { table: mv, columns: [mv.v], scan_ranges: [mv.v = Int32(1), mv.v = Int32(2)], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], scan_ranges: [mv.v = Int32(1), mv.v = Int32(2)], distribution: UpstreamHashShard(mv.v) } - name: BatchSort needed, when input is sorted in wrong order sql: | create table t(v int); @@ -210,7 +215,7 @@ batch_plan: |- BatchExchange { order: [mv.v ASC], dist: Single } └─BatchSort { order: [mv.v ASC] } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - sql: | CREATE TABLE test (a INTEGER, b INTEGER); SELECT b % 2 AS f, SUM(a) FROM test GROUP BY b % 2 ORDER BY f; diff --git a/src/frontend/planner_test/tests/testdata/output/project_set.yaml b/src/frontend/planner_test/tests/testdata/output/project_set.yaml index c4102e164d7e8..d54b3d8cff947 100644 --- a/src/frontend/planner_test/tests/testdata/output/project_set.yaml +++ b/src/frontend/planner_test/tests/testdata/output/project_set.yaml @@ -98,13 +98,14 @@ └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [projected_row_id(hidden), unnest, t._row_id(hidden)], stream_key: [], pk_columns: [projected_row_id], pk_conflict: NoCheck } - └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id] } - └─StreamTopN { order: [projected_row_id ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [projected_row_id ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as _vnode] } - └─StreamProjectSet { select_list: [Unnest($0), $1] } - └─StreamTableScan { table: t, columns: [t.x, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } + └─StreamExchange { dist: HashShard(projected_row_id) } + └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id] } + └─StreamTopN { order: [projected_row_id ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [projected_row_id ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [projected_row_id, Unnest($0), t._row_id, Vnode(t._row_id) as _vnode] } + └─StreamProjectSet { select_list: [Unnest($0), $1] } + └─StreamTableScan { table: t, columns: [t.x, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } - name: issue-7812 sql: | -- projected_row_id should be excluded from distinct diff --git a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml index e175558038c81..8e79bb6c62d0f 100644 --- a/src/frontend/planner_test/tests/testdata/output/range_scan.yaml +++ b/src/frontend/planner_test/tests/testdata/output/range_scan.yaml @@ -240,21 +240,21 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (orders_count_by_user_ordered.user_id = 42:Int32) } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - before: - create_table_and_mv_ordered sql: | SELECT * FROM orders_count_by_user_ordered WHERE user_id > 42 AND orders_count = 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10) AND orders_count_by_user_ordered.user_id > Int32(42)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10) AND orders_count_by_user_ordered.user_id > Int32(42)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - before: - create_table_and_mv_ordered sql: | SELECT * FROM orders_count_by_user_ordered WHERE orders_count = 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge mutiple upper bound before: - create_table_and_mv_ordered @@ -262,7 +262,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count < 10 and orders_count < 30 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge include and exclude upper bound of same value before: - create_table_and_mv_ordered @@ -270,7 +270,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count < 10 and orders_count <= 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count < Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge mutiple lower bound before: - create_table_and_mv_ordered @@ -278,7 +278,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count > 10 and orders_count > 30 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: merge include and exclude lower bound of same value before: - create_table_and_mv_ordered @@ -286,7 +286,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count > 10 and orders_count >= 10 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count > Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: invalid range before: - create_table_and_mv_ordered @@ -300,7 +300,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count > 20 and orders_count < 30 and orders_count = 25 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(25)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(25)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: invalid range of merging cmp and eq condition before: - create_table_and_mv_ordered @@ -314,7 +314,7 @@ SELECT * FROM orders_count_by_user_ordered WHERE orders_count in (10,20,30,40) and orders_count <30 batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10), orders_count_by_user_ordered.orders_count = Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date) } + └─BatchScan { table: orders_count_by_user_ordered, columns: [orders_count_by_user_ordered.user_id, orders_count_by_user_ordered.date, orders_count_by_user_ordered.orders_count], scan_ranges: [orders_count_by_user_ordered.orders_count = Int64(10), orders_count_by_user_ordered.orders_count = Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_ordered.orders_count) } - name: invalid range of merging cmp and const-in condition before: - create_table_and_mv_ordered @@ -521,7 +521,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id > 20:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -537,7 +537,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id >= 10:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id >= Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id >= Int64(10)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -553,7 +553,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id > 20:Int32)) OR (orders_count_by_user_desc.user_id = 15:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id > Int64(20)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -569,7 +569,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((((orders_count_by_user_desc.user_id < 10:Int32) AND (orders_count_by_user_desc.user_id > 1:Int32)) OR ((orders_count_by_user_desc.user_id > 20:Int32) AND (orders_count_by_user_desc.user_id < 30:Int32))) OR ((orders_count_by_user_desc.user_id >= 30:Int32) AND (orders_count_by_user_desc.user_id < 40:Int32))) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20) AND orders_count_by_user_desc.user_id < Int64(30), orders_count_by_user_desc.user_id >= Int64(30) AND orders_count_by_user_desc.user_id < Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id > Int64(20) AND orders_count_by_user_desc.user_id < Int64(30), orders_count_by_user_desc.user_id >= Int64(30) AND orders_count_by_user_desc.user_id < Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - name: When OR clauses contain overlapping conditions, we can merge serveral scan_range and pushdown. before: - create_table_and_mv @@ -586,7 +586,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((((orders_count_by_user_desc.user_id < 10:Int32) AND (orders_count_by_user_desc.user_id > 1:Int32)) OR ((orders_count_by_user_desc.user_id > 20:Int32) AND (orders_count_by_user_desc.user_id <= 30:Int32))) OR (((orders_count_by_user_desc.user_id >= 30:Int32) AND (orders_count_by_user_desc.user_id < 40:Int32)) OR (orders_count_by_user_desc.user_id = 15:Int32))) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id >= Int64(20) AND orders_count_by_user_desc.user_id <= Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id > Int64(1) AND orders_count_by_user_desc.user_id < Int64(10), orders_count_by_user_desc.user_id = Int64(15), orders_count_by_user_desc.user_id >= Int64(20) AND orders_count_by_user_desc.user_id <= Int64(40)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -602,7 +602,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (((orders_count_by_user_desc.user_id < 10:Int32) OR (orders_count_by_user_desc.user_id > 30:Int32)) OR ((orders_count_by_user_desc.user_id > 5:Int32) AND (orders_count_by_user_desc.user_id < 15:Int32))) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id <= Int64(15), orders_count_by_user_desc.user_id > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], scan_ranges: [orders_count_by_user_desc.user_id <= Int64(15), orders_count_by_user_desc.user_id > Int64(30)], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - name: When OR clauses contain overlapping conditions, we cannot push down if it results in a full table scan. before: - create_table_and_mv @@ -619,7 +619,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 20:Int32) OR (orders_count_by_user_desc.user_id > 10:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } - before: - create_table_and_mv sql: | @@ -635,4 +635,4 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: ((orders_count_by_user_desc.user_id < 20:Int32) OR (orders_count_by_user_desc.user_id <> 10:Int32)) } - └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id, orders_count_by_user_desc.date) } + └─BatchScan { table: orders_count_by_user_desc, columns: [orders_count_by_user_desc.user_id, orders_count_by_user_desc.date, orders_count_by_user_desc.orders_count], distribution: UpstreamHashShard(orders_count_by_user_desc.user_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/row_filter.yaml b/src/frontend/planner_test/tests/testdata/output/row_filter.yaml index 5de2e79815476..3820983f004d6 100644 --- a/src/frontend/planner_test/tests/testdata/output/row_filter.yaml +++ b/src/frontend/planner_test/tests/testdata/output/row_filter.yaml @@ -39,4 +39,4 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchFilter { predicate: (Row(mv1.v1, mv1.v2, mv1.v3) > '(1,3,1)':Struct(StructType { field_names: [], field_types: [Int32, Int32, Int32] })) } - └─BatchScan { table: mv1, columns: [mv1.v1, mv1.v2, mv1.v3], scan_ranges: [(mv1.v1, mv1.v2) >= (Int32(1), Int32(3))], distribution: SomeShard } + └─BatchScan { table: mv1, columns: [mv1.v1, mv1.v2, mv1.v3], scan_ranges: [(mv1.v1, mv1.v2) >= (Int32(1), Int32(3))], distribution: UpstreamHashShard(mv1.v1, mv1.v2, mv1.v3) } diff --git a/src/frontend/planner_test/tests/testdata/output/singleton.yaml b/src/frontend/planner_test/tests/testdata/output/singleton.yaml index b9ff208e484e6..0710b0d88f0d7 100644 --- a/src/frontend/planner_test/tests/testdata/output/singleton.yaml +++ b/src/frontend/planner_test/tests/testdata/output/singleton.yaml @@ -10,10 +10,10 @@ select v from mv; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: Single } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } batch_local_plan: |- BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - id: select_from_singleton_mv_join before: - create_singleton_mv @@ -21,17 +21,12 @@ select mv1.v from mv mv1, mv mv2 where mv1.v = mv2.v; batch_plan: |- BatchExchange { order: [], dist: Single } - └─BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: HashShard(mv.v) } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: Single } - └─BatchExchange { order: [], dist: HashShard(mv.v) } - └─BatchScan { table: mv, columns: [mv.v], distribution: Single } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } batch_local_plan: |- - BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: Single } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } - id: select_from_singleton_mv_join_top_n before: - create_singleton_mv @@ -43,17 +38,12 @@ └─BatchExchange { order: [], dist: Single } └─BatchTopN { order: [mv.v ASC], limit: 10, offset: 0 } └─BatchProject { exprs: [mv.v, mv.v] } - └─BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: HashShard(mv.v) } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: Single } - └─BatchExchange { order: [], dist: HashShard(mv.v) } - └─BatchScan { table: mv, columns: [mv.v], distribution: Single } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } batch_local_plan: |- BatchProject { exprs: [mv.v] } └─BatchTopN { order: [mv.v ASC], limit: 10, offset: 0 } └─BatchProject { exprs: [mv.v, mv.v] } - └─BatchHashJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v] } - ├─BatchExchange { order: [], dist: Single } - │ └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchLookupJoin { type: Inner, predicate: mv.v = mv.v, output: [mv.v], lookup table: mv } └─BatchExchange { order: [], dist: Single } - └─BatchScan { table: mv, columns: [mv.v], distribution: SomeShard } + └─BatchScan { table: mv, columns: [mv.v], distribution: UpstreamHashShard(mv.v) } diff --git a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml index 7a0ac41a4b85b..7ff0a08172cb2 100644 --- a/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/stream_dist_agg.yaml @@ -12,29 +12,50 @@ sql: | select max(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [max(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [max(max(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [max(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [max(s.v)] } - └─StreamSimpleAgg { aggs: [max(s.v), count] } - └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamProject { exprs: [max(max(s.v))] } + └─StreamSimpleAgg { aggs: [max(max(s.v)), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(s.v), count] } + └─StreamProject { exprs: [s.v, s.o, s.t._row_id, Vnode(s.o) as _vnode] } + └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [max(s.v)] } - └── StreamSimpleAgg { aggs: [max(s.v), count] } { tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] } - └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 2 ] - ├── Upstream - └── BatchPlanNode + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [max(max(s.v))] } + └── StreamSimpleAgg { aggs: [max(max(s.v)), count] } + ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] + └── StreamExchange Single from 1 - Table 0 { columns: [ s_v, s_t__row_id, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Fragment 1 + StreamHashAgg { group_key: [_vnode], aggs: [max(s.v), count] } { tables: [ HashAggState: 3, HashAggCall0: 2 ] } + └── StreamProject { exprs: [s.v, s.o, s.t._row_id, Vnode(s.o) as _vnode] } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 4 ] + ├── Upstream + └── BatchPlanNode + + Table 0 { columns: [ max(s_v), _vnode, _rw_timestamp ], primary key: [ $0 DESC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ max(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 1 { columns: [ max(max(s_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 + ├── columns: [ _vnode, s_v, s_t__row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 3 { columns: [ _vnode, max(s_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 4 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] ├── primary key: [ $0 ASC ] ├── value indices: [ 1, 2, 3, 4 ] @@ -50,25 +71,34 @@ sql: | select sum(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [sum(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [sum(sum(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [sum(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [sum(s.v)] } - └─StreamSimpleAgg { aggs: [sum(s.v), count] } - └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamProject { exprs: [sum(sum(s.v))] } + └─StreamSimpleAgg { aggs: [sum(sum(s.v)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(s.v)] } + └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [sum(s.v)] } - └── StreamSimpleAgg { aggs: [sum(s.v), count] } { tables: [ SimpleAggState: 0 ] } - └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 1 ] - ├── Upstream - └── BatchPlanNode + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [sum(sum(s.v))] } + └── StreamSimpleAgg { aggs: [sum(sum(s.v)), count] } + ├── tables: [ SimpleAggState: 0 ] + └── StreamExchange Single from 1 - Table 0 { columns: [ sum(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Fragment 1 + StreamStatelessSimpleAgg { aggs: [sum(s.v)] } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 1 ] + ├── Upstream + └── BatchPlanNode + + Table 0 { columns: [ sum(sum(s_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] @@ -86,25 +116,34 @@ sql: | select count(v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [count(s.v)] } - └─BatchScan { table: s, columns: [s.v], distribution: Single } + BatchSimpleAgg { aggs: [sum0(count(s.v))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [count(s.v)] } + └─BatchScan { table: s, columns: [s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(s.v)] } - └─StreamSimpleAgg { aggs: [count(s.v), count] } - └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamProject { exprs: [sum0(count(s.v))] } + └─StreamSimpleAgg { aggs: [sum0(count(s.v)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [count(s.v)] } + └─StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } - └── StreamProject { exprs: [count(s.v)] } - └── StreamSimpleAgg { aggs: [count(s.v), count] } { tables: [ SimpleAggState: 0 ] } - └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 1 ] - ├── Upstream - └── BatchPlanNode + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [sum0(count(s.v))] } + └── StreamSimpleAgg { aggs: [sum0(count(s.v)), count] } + ├── tables: [ SimpleAggState: 0 ] + └── StreamExchange Single from 1 + + Fragment 1 + StreamStatelessSimpleAgg { aggs: [count(s.v)] } + └── StreamTableScan { table: s, columns: [s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 1 ] + ├── Upstream + └── BatchPlanNode - Table 0 { columns: [ count(s_v), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ sum0(count(s_v)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 1 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] @@ -122,26 +161,32 @@ sql: | select string_agg(s, ',' order by v) as a1 from S; batch_plan: |- - BatchExchange { order: [], dist: Single } - └─BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + BatchSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } + └─BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [s.s, ',':Varchar, s.v] } - └─BatchScan { table: s, columns: [s.v, s.s], distribution: Single } + └─BatchScan { table: s, columns: [s.v, s.s], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } └─StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } - └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } - └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamExchange { dist: Single } + └─StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } + └─StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + StreamMaterialize { columns: [a1], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] └── StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.v ASC))] } - └── StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } { tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] } - └── StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } - └── StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } - ├── tables: [ StreamScan: 2 ] - ├── Upstream - └── BatchPlanNode + └── StreamSimpleAgg { aggs: [string_agg(s.s, ',':Varchar order_by(s.v ASC)), count] } + ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] + └── StreamExchange Single from 1 + + Fragment 1 + StreamProject { exprs: [s.s, ',':Varchar, s.v, s.t._row_id] } + └── StreamTableScan { table: s, columns: [s.v, s.s, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } + ├── tables: [ StreamScan: 2 ] + ├── Upstream + └── BatchPlanNode Table 0 ├── columns: [ s_v, s_t__row_id, s_s, ',':Varchar, _rw_timestamp ] @@ -150,12 +195,7 @@ ├── distribution key: [] └── read pk prefix len hint: 0 - Table 1 - ├── columns: [ string_agg(s_s, ',':Varchar order_by(s_v ASC)), count, _rw_timestamp ] - ├── primary key: [] - ├── value indices: [ 0, 1 ] - ├── distribution key: [] - └── read pk prefix len hint: 0 + Table 1 { columns: [ string_agg(s_s, ',':Varchar order_by(s_v ASC)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } Table 2 ├── columns: [ vnode, o, t__row_id, backfill_finished, row_count, _rw_timestamp ] @@ -966,13 +1006,13 @@ └─BatchProject { exprs: [max(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [max(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [max(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [max(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } - └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -983,17 +1023,12 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 2 ] ├── Upstream └── BatchPlanNode - Table 0 - ├── columns: [ s_k, s_v, s_t__row_id, _rw_timestamp ] - ├── primary key: [ $0 ASC, $1 DESC, $2 ASC ] - ├── value indices: [ 1, 2 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 + Table 0 { columns: [ s_k, s_v, s_t__row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 { columns: [ s_k, max(s_v), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1135,13 +1170,13 @@ └─BatchProject { exprs: [sum(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [sum(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [sum(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } - └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -1151,7 +1186,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 1 ] ├── Upstream └── BatchPlanNode @@ -1296,13 +1331,13 @@ └─BatchProject { exprs: [count(s.v)] } └─BatchHashAgg { group_key: [s.k], aggs: [count(s.v)] } └─BatchExchange { order: [], dist: HashShard(s.k) } - └─BatchScan { table: s, columns: [s.k, s.v], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.v], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [count(s.v), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [count(s.v), count] } └─StreamExchange { dist: HashShard(s.k) } - └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -1312,7 +1347,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + StreamTableScan { table: s, columns: [s.k, s.v, s.o, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 1 ] ├── Upstream └── BatchPlanNode @@ -1484,14 +1519,14 @@ └─BatchHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC))] } └─BatchExchange { order: [], dist: HashShard(s.k) } └─BatchProject { exprs: [s.k, s.s, ',':Varchar, s.o] } - └─BatchScan { table: s, columns: [s.k, s.o, s.s], distribution: Single } + └─BatchScan { table: s, columns: [s.k, s.o, s.s], distribution: UpstreamHashShard(s.o) } stream_plan: |- StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } └─StreamProject { exprs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), s.k] } └─StreamHashAgg { group_key: [s.k], aggs: [string_agg(s.s, ',':Varchar order_by(s.o ASC)), count] } └─StreamExchange { dist: HashShard(s.k) } └─StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } - └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └─StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [a1, s.k(hidden)], stream_key: [s.k], pk_columns: [s.k], pk_conflict: NoCheck } @@ -1503,7 +1538,7 @@ Fragment 1 StreamProject { exprs: [s.k, s.s, ',':Varchar, s.o, s.t._row_id] } - └── StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: Single } + └── StreamTableScan { table: s, columns: [s.k, s.o, s.s, s.t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [s.t._row_id], pk: [o, t._row_id], dist: UpstreamHashShard(s.o) } ├── tables: [ StreamScan: 2 ] ├── Upstream └── BatchPlanNode diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index a87b116a658c2..6d92ab8947741 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -620,17 +620,16 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } - └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, row_number, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, row_number, integers._row_id] } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 2 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); @@ -679,17 +678,16 @@ └─BatchScan { table: integers, columns: [integers.i], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [i, col, integers._row_id(hidden), integers.i(hidden)], stream_key: [integers._row_id, i], pk_columns: [i, integers._row_id], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(integers.i, integers._row_id) } - └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } - ├─StreamExchange { dist: HashShard(integers.i) } - │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } - └─StreamProject { exprs: [integers.i, sum, integers._row_id] } - └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } - └─StreamExchange { dist: HashShard(integers.i) } - └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } - └─StreamFilter { predicate: IsNotNull(integers.i) } - └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: integers.i IS NOT DISTINCT FROM integers.i, output: [integers.i, sum, integers._row_id, integers.i] } + ├─StreamExchange { dist: HashShard(integers.i) } + │ └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } + └─StreamGroupTopN { order: [integers.i ASC], limit: 1, offset: 0, group_key: [integers.i] } + └─StreamProject { exprs: [integers.i, sum, integers._row_id] } + └─StreamOverWindow { window_functions: [sum(integers.i) OVER(PARTITION BY integers.i ORDER BY integers.i ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(integers.i) } + └─StreamProject { exprs: [integers.i, integers.i, integers._row_id] } + └─StreamFilter { predicate: IsNotNull(integers.i) } + └─StreamTableScan { table: integers, columns: [integers.i, integers._row_id], stream_scan_type: ArrangementBackfill, stream_key: [integers._row_id], pk: [_row_id], dist: UpstreamHashShard(integers._row_id) } - name: test over window subquery 4 (with nested loop join so cannot be transformed into a stream plan) sql: | CREATE TABLE integers(i INTEGER); diff --git a/src/frontend/planner_test/tests/testdata/output/topn.yaml b/src/frontend/planner_test/tests/testdata/output/topn.yaml index 42a7e61cda5a4..21e4d5b48fc5b 100644 --- a/src/frontend/planner_test/tests/testdata/output/topn.yaml +++ b/src/frontend/planner_test/tests/testdata/output/topn.yaml @@ -25,7 +25,7 @@ BatchTopN { order: [t1_mv.a DESC], limit: 50, offset: 50 } └─BatchExchange { order: [], dist: Single } └─BatchLimit { limit: 100, offset: 0 } - └─BatchScan { table: t1_mv, columns: [t1_mv.pk, t1_mv.a, t1_mv.b, t1_mv.c, t1_mv.d], limit: 100, distribution: SomeShard } + └─BatchScan { table: t1_mv, columns: [t1_mv.pk, t1_mv.a, t1_mv.b, t1_mv.c, t1_mv.d], limit: 100, distribution: UpstreamHashShard(t1_mv.a) } - sql: | WITH c1(k, v) AS ( VALUES diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index e90bf2b747ccc..7e6ab186f7dc0 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -146,32 +146,36 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } - └─StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal) as $expr3, (sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)::Decimal) as $expr4, (sum(lineitem.l_discount) / count(lineitem.l_discount)::Decimal) as $expr5, count, Vnode(lineitem.l_returnflag, lineitem.l_linestatus) as $expr6] } - └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } - └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } - └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, $expr1, ($expr1 * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamProject { exprs: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └─StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal) as $expr3, (sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)::Decimal) as $expr4, (sum(lineitem.l_discount) / count(lineitem.l_discount)::Decimal) as $expr5, count, Vnode(lineitem.l_returnflag, lineitem.l_linestatus) as $expr6] } + └─StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } + └─StreamExchange { dist: HashShard(lineitem.l_returnflag, lineitem.l_linestatus) } + └─StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, $expr1, ($expr1 * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } - └── StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0, 1]) from 1 Fragment 1 + StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └── StreamTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [lineitem.l_returnflag ASC, lineitem.l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), (sum(lineitem.l_quantity) / count(lineitem.l_quantity)::Decimal) as $expr3, (sum(lineitem.l_extendedprice) / count(lineitem.l_extendedprice)::Decimal) as $expr4, (sum(lineitem.l_discount) / count(lineitem.l_discount)::Decimal) as $expr5, count, Vnode(lineitem.l_returnflag, lineitem.l_linestatus) as $expr6] } └── StreamHashAgg { group_key: [lineitem.l_returnflag, lineitem.l_linestatus], aggs: [sum(lineitem.l_quantity), sum(lineitem.l_extendedprice), sum($expr1), sum($expr2), count(lineitem.l_quantity), count(lineitem.l_extendedprice), sum(lineitem.l_discount), count(lineitem.l_discount), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1]) from 2 + └── StreamExchange Hash([0, 1]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [lineitem.l_returnflag, lineitem.l_linestatus, lineitem.l_quantity, lineitem.l_extendedprice, $expr1, ($expr1 * (1:Decimal + lineitem.l_tax)) as $expr2, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamProject { exprs: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_tax, lineitem.l_returnflag, lineitem.l_linestatus, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate <= '1998-09-21 00:00:00':Timestamp) } @@ -188,7 +192,7 @@ Table 3 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: tpch_q2 before: @@ -312,152 +316,156 @@ └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_plan: |- StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost)(hidden), partsupp.ps_partkey(hidden), region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), partsupp.ps_partkey#1(hidden)], stream_key: [p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } - └─StreamTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey, Vnode(partsupp.ps_partkey, min(partsupp.ps_supplycost)) as _vnode] } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost AND partsupp.ps_partkey = partsupp.ps_partkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } - ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, min(partsupp.ps_supplycost)) } - │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } - │ ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ │ └─StreamProject { exprs: [part.p_partkey, part.p_mfgr] } - │ │ └─StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } - │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } - │ └─StreamProject { exprs: [partsupp.ps_partkey, min(partsupp.ps_supplycost)] } - │ └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [min(partsupp.ps_supplycost), count] } - │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } - │ │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } - │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_supplycost) } - └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [nation.n_name, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_suppkey] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } - │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } - │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - └─StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } - └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + └─StreamExchange { dist: HashShard(supplier.s_acctbal, nation.n_name, supplier.s_name, part.p_partkey) } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } + └─StreamTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey, Vnode(partsupp.ps_partkey, min(partsupp.ps_supplycost)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost AND partsupp.ps_partkey = partsupp.ps_partkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } + ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, min(partsupp.ps_supplycost)) } + │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } + │ ├─StreamExchange { dist: HashShard(part.p_partkey) } + │ │ └─StreamProject { exprs: [part.p_partkey, part.p_mfgr] } + │ │ └─StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } + │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + │ └─StreamProject { exprs: [partsupp.ps_partkey, min(partsupp.ps_supplycost)] } + │ └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [min(partsupp.ps_supplycost), count] } + │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } + │ │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } + │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } + │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_supplycost) } + └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [nation.n_name, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_suppkey] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } + │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ │ └─StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + └─StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } + └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost)(hidden), partsupp.ps_partkey(hidden), region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), partsupp.ps_partkey#1(hidden)], stream_key: [p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, min(partsupp.ps_supplycost), partsupp.ps_partkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } - └── StreamTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0, 2, 1, 3]) from 1 Fragment 1 + StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } + └── StreamTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey, Vnode(partsupp.ps_partkey, min(partsupp.ps_supplycost)) as _vnode] } └── StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost AND partsupp.ps_partkey = partsupp.ps_partkey, output: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey] } { tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] } - ├── StreamExchange Hash([2, 3]) from 2 - └── StreamExchange Hash([6, 7]) from 11 + ├── StreamExchange Hash([2, 3]) from 3 + └── StreamExchange Hash([6, 7]) from 12 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } { tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] } - ├── StreamExchange Hash([0]) from 3 + ├── StreamExchange Hash([0]) from 4 └── StreamProject { exprs: [partsupp.ps_partkey, min(partsupp.ps_supplycost)] } └── StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [min(partsupp.ps_supplycost), count] } { tables: [ HashAggState: 12, HashAggCall0: 11 ] } - └── StreamExchange Hash([1]) from 4 + └── StreamExchange Hash([1]) from 5 - Fragment 3 + Fragment 4 StreamProject { exprs: [part.p_partkey, part.p_mfgr] } └── StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_mfgr, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 10 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] } - ├── StreamExchange Hash([2]) from 5 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([2]) from 6 + └── StreamExchange Hash([0]) from 9 - Fragment 5 + Fragment 6 StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } { tables: [ HashJoinLeft: 17, HashJoinDegreeLeft: 18, HashJoinRight: 19, HashJoinDegreeRight: 20 ] } - ├── StreamExchange Hash([1]) from 6 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([1]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 6 + Fragment 7 StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 21 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 22 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { tables: [ HashJoinLeft: 23, HashJoinDegreeLeft: 24, HashJoinRight: 25, HashJoinDegreeRight: 26 ] } - ├── StreamExchange Hash([0]) from 9 - └── StreamExchange Hash([1]) from 10 + ├── StreamExchange Hash([0]) from 10 + └── StreamExchange Hash([1]) from 11 - Fragment 9 + Fragment 10 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 28 ] } ├── Upstream └── BatchPlanNode - Fragment 11 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [nation.n_name, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_suppkey] } { tables: [ HashJoinLeft: 29, HashJoinDegreeLeft: 30, HashJoinRight: 31, HashJoinDegreeRight: 32 ] } - ├── StreamExchange Hash([1]) from 12 - └── StreamExchange Hash([1]) from 17 - Fragment 12 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 33, HashJoinDegreeLeft: 34, HashJoinRight: 35, HashJoinDegreeRight: 36 ] } - ├── StreamExchange Hash([0]) from 13 - └── StreamExchange Hash([3]) from 16 + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [nation.n_name, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, partsupp.ps_partkey, partsupp.ps_supplycost, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_suppkey] } { tables: [ HashJoinLeft: 29, HashJoinDegreeLeft: 30, HashJoinRight: 31, HashJoinDegreeRight: 32 ] } + ├── StreamExchange Hash([1]) from 13 + └── StreamExchange Hash([1]) from 18 Fragment 13 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } { tables: [ HashJoinLeft: 37, HashJoinDegreeLeft: 38, HashJoinRight: 39, HashJoinDegreeRight: 40 ] } + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_acctbal, supplier.s_comment, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 33, HashJoinDegreeLeft: 34, HashJoinRight: 35, HashJoinDegreeRight: 36 ] } ├── StreamExchange Hash([0]) from 14 - └── StreamExchange Hash([2]) from 15 + └── StreamExchange Hash([3]) from 17 Fragment 14 + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } { tables: [ HashJoinLeft: 37, HashJoinDegreeLeft: 38, HashJoinRight: 39, HashJoinDegreeRight: 40 ] } + ├── StreamExchange Hash([0]) from 15 + └── StreamExchange Hash([2]) from 16 + + Fragment 15 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'AFRICA':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 41 ] } ├── Upstream └── BatchPlanNode - Fragment 15 + Fragment 16 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 42 ] } ├── Upstream └── BatchPlanNode - Fragment 16 + Fragment 17 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey, supplier.s_phone, supplier.s_acctbal, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 43 ] } ├── Upstream └── BatchPlanNode - Fragment 17 + Fragment 18 StreamFilter { predicate: IsNotNull(partsupp.ps_partkey) } └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 44 ] } ├── Upstream @@ -553,7 +561,7 @@ Table 44 { columns: [ vnode, ps_partkey, ps_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey#1, _rw_timestamp ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [], read pk prefix len hint: 9 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, min(partsupp.ps_supplycost), partsupp.ps_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, partsupp.ps_partkey#1, _rw_timestamp ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $10 ASC, $11 ASC, $12 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 0, 2, 1, 3 ], read pk prefix len hint: 9 } - id: tpch_q3 before: @@ -627,70 +635,74 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } - └─StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } - └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } - │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ │ └─StreamProject { exprs: [customer.c_custkey] } - │ │ └─StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - │ └─StreamExchange { dist: HashShard(orders.o_custkey) } - │ └─StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(sum($expr1), orders.o_orderdate) } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } + └─StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) } + └─StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } + │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ │ └─StreamProject { exprs: [customer.c_custkey] } + │ │ └─StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + │ └─StreamExchange { dist: HashShard(orders.o_custkey) } + │ └─StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } - └── StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1, 2]) from 1 Fragment 1 + StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority] } + └── StreamTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC, orders.o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [lineitem.l_orderkey, sum($expr1), orders.o_orderdate, orders.o_shippriority, Vnode(lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority) as $expr2] } └── StreamHashAgg { group_key: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority], aggs: [sum($expr1), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamExchange Hash([0, 1, 2]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [lineitem.l_orderkey, orders.o_orderdate, orders.o_shippriority, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, orders.o_shippriority, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, customer.c_custkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [orders.o_orderkey, orders.o_orderdate, orders.o_shippriority, customer.c_custkey] } ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([1]) from 6 - Fragment 4 + Fragment 5 StreamProject { exprs: [customer.c_custkey] } └── StreamFilter { predicate: (customer.c_mktsegment = 'FURNITURE':Varchar) } └── StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_mktsegment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: (orders.o_orderdate < '1995-03-29':Date) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate, orders.o_shippriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── tables: [ StreamScan: 12 ] ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate > '1995-03-29':Date) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -726,7 +738,7 @@ Table 13 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ l_orderkey, revenue, o_orderdate, o_shippriority, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ l_orderkey, revenue, o_orderdate, o_shippriority, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2 ], read pk prefix len hint: 4 } - id: tpch_q4 before: @@ -788,43 +800,47 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [orders.o_orderpriority, count] } - └─StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [orders.o_orderpriority, count, Vnode(orders.o_orderpriority) as _vnode] } - └─StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } - └─StreamExchange { dist: HashShard(orders.o_orderpriority) } - └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } - │ └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderpriority) } + └─StreamProject { exprs: [orders.o_orderpriority, count] } + └─StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [orders.o_orderpriority, count, Vnode(orders.o_orderpriority) as _vnode] } + └─StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } + └─StreamExchange { dist: HashShard(orders.o_orderpriority) } + └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } + │ └─StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [orders.o_orderpriority, count] } - └── StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [orders.o_orderpriority, count] } + └── StreamTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [orders.o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [orders.o_orderpriority, count, Vnode(orders.o_orderpriority) as _vnode] } └── StreamHashAgg { group_key: [orders.o_orderpriority], aggs: [count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, orders.o_orderkey] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamProject { exprs: [orders.o_orderkey, orders.o_orderpriority] } └── StreamFilter { predicate: (orders.o_orderdate >= '1997-07-01':Date) AND (orders.o_orderdate < '1997-10-01 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } @@ -832,7 +848,7 @@ ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_commitdate < lineitem.l_receiptdate) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -858,7 +874,7 @@ Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ o_orderpriority, order_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_orderpriority, order_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q5 before: @@ -949,116 +965,120 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } - └─StreamProject { exprs: [nation.n_name, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [nation.n_name, sum($expr1), Vnode(nation.n_name) as $expr2] } - └─StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(nation.n_name) } - └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey] } - └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey AND nation.n_nationkey = customer.c_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey, customer.c_nationkey] } - ├─StreamExchange { dist: HashShard(nation.n_nationkey, nation.n_nationkey) } - │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } - │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ └─StreamFilter { predicate: (region.r_name = 'MIDDLE EAST':Varchar) } - │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ └─StreamFilter { predicate: IsNotNull(nation.n_nationkey) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard(customer.c_nationkey, supplier.s_nationkey) } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND customer.c_nationkey = supplier.s_nationkey, output: [customer.c_nationkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey, customer.c_nationkey) } - │ └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } - │ ├─StreamExchange { dist: HashShard(orders.o_custkey) } - │ │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - │ │ └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - │ └─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey, supplier.s_nationkey) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } - ├─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [nation.n_name, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [nation.n_name, sum($expr1), Vnode(nation.n_name) as $expr2] } + └─StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(nation.n_name) } + └─StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey] } + └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey AND nation.n_nationkey = customer.c_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey, customer.c_nationkey] } + ├─StreamExchange { dist: HashShard(nation.n_nationkey, nation.n_nationkey) } + │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } + │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ └─StreamFilter { predicate: (region.r_name = 'MIDDLE EAST':Varchar) } + │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(nation.n_nationkey) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(customer.c_nationkey, supplier.s_nationkey) } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND customer.c_nationkey = supplier.s_nationkey, output: [customer.c_nationkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey, customer.c_nationkey) } + │ └─StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } + │ ├─StreamExchange { dist: HashShard(orders.o_custkey) } + │ │ └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } + │ │ └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + │ └─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey, supplier.s_nationkey) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } + ├─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [nation.n_name, sum($expr1)] } - └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1]) from 1 Fragment 1 + StreamProject { exprs: [nation.n_name, sum($expr1)] } + └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [nation.n_name, sum($expr1), Vnode(nation.n_name) as $expr2] } └── StreamHashAgg { group_key: [nation.n_name], aggs: [sum($expr1), count] } ├── tables: [ HashAggState: 2 ] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey] } └── StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey AND nation.n_nationkey = customer.c_nationkey, output: [lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_linenumber, lineitem.l_suppkey, customer.c_nationkey] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0, 0]) from 3 - └── StreamExchange Hash([0, 3]) from 6 + ├── StreamExchange Hash([0, 0]) from 4 + └── StreamExchange Hash([0, 3]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, nation.n_name, region.r_regionkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([2]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([2]) from 6 - Fragment 4 + Fragment 5 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'MIDDLE EAST':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: IsNotNull(nation.n_nationkey) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 12 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey AND customer.c_nationkey = supplier.s_nationkey, output: [customer.c_nationkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, orders.o_orderkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_suppkey] } ├── tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] - ├── StreamExchange Hash([0, 1]) from 7 - └── StreamExchange Hash([0, 3]) from 10 + ├── StreamExchange Hash([0, 1]) from 8 + └── StreamExchange Hash([0, 3]) from 11 - Fragment 7 + Fragment 8 StreamHashJoin { type: Inner, predicate: orders.o_custkey = customer.c_custkey, output: [orders.o_orderkey, customer.c_nationkey, orders.o_custkey, customer.c_custkey] } { tables: [ HashJoinLeft: 17, HashJoinDegreeLeft: 18, HashJoinRight: 19, HashJoinDegreeRight: 20 ] } - ├── StreamExchange Hash([1]) from 8 - └── StreamExchange Hash([0]) from 9 + ├── StreamExchange Hash([1]) from 9 + └── StreamExchange Hash([0]) from 10 - Fragment 8 + Fragment 9 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1995-01-01 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 21 ] } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 22 ] } ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamHashJoin { type: Inner, predicate: lineitem.l_suppkey = supplier.s_suppkey, output: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, supplier.s_nationkey, lineitem.l_linenumber, lineitem.l_suppkey, supplier.s_suppkey] } { tables: [ HashJoinLeft: 23, HashJoinDegreeLeft: 24, HashJoinRight: 25, HashJoinDegreeRight: 26 ] } - ├── StreamExchange Hash([1]) from 11 - └── StreamExchange Hash([0]) from 12 + ├── StreamExchange Hash([1]) from 12 + └── StreamExchange Hash([0]) from 13 - Fragment 11 + Fragment 12 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 27 ] ├── Upstream └── BatchPlanNode - Fragment 12 + Fragment 13 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 28 ] } ├── Upstream └── BatchPlanNode @@ -1126,7 +1146,7 @@ Table 28 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: tpch_q6 before: @@ -1290,107 +1310,111 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, nation.n_name, $expr1) as $expr3] } - └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } - └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, customer.c_custkey] } - └─StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: all } - ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_nationkey, supplier.s_suppkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } - │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(orders.o_orderkey) } - └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, nation.n_name, $expr1) as $expr3] } + └─StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(nation.n_name, nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, customer.c_custkey] } + └─StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: all } + ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_nationkey, supplier.s_suppkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } + │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderkey) } + └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } - └── StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 + StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2)] } + └── StreamTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [nation.n_name ASC, nation.n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [nation.n_name, nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, nation.n_name, $expr1) as $expr3] } └── StreamHashAgg { group_key: [nation.n_name, nation.n_name, $expr1], aggs: [sum($expr2), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1, 2]) from 2 + └── StreamExchange Hash([0, 1, 2]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [nation.n_name, nation.n_name, Extract('YEAR':Varchar, lineitem.l_shipdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_nationkey, supplier.s_suppkey, lineitem.l_orderkey, lineitem.l_linenumber, nation.n_nationkey, customer.c_custkey] } └── StreamFilter { predicate: (((nation.n_name = 'ROMANIA':Varchar) AND (nation.n_name = 'IRAN':Varchar)) OR ((nation.n_name = 'IRAN':Varchar) AND (nation.n_name = 'ROMANIA':Varchar))) } └── StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: all } { tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] } - ├── StreamExchange Hash([1]) from 3 - └── StreamExchange Hash([1]) from 8 + ├── StreamExchange Hash([1]) from 4 + └── StreamExchange Hash([1]) from 9 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, nation.n_nationkey, supplier.s_suppkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - ├── StreamExchange Hash([1]) from 4 - └── StreamExchange Hash([1]) from 7 + ├── StreamExchange Hash([1]) from 5 + └── StreamExchange Hash([1]) from 8 - Fragment 4 + Fragment 5 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } - ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([1]) from 6 + ├── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([1]) from 7 - Fragment 5 + Fragment 6 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 16 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 17 ] ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [nation.n_name, orders.o_orderkey, nation.n_nationkey, customer.c_custkey] } { tables: [ HashJoinLeft: 18, HashJoinDegreeLeft: 19, HashJoinRight: 20, HashJoinDegreeRight: 21 ] } - ├── StreamExchange Hash([1]) from 9 - └── StreamExchange Hash([1]) from 12 + ├── StreamExchange Hash([1]) from 10 + └── StreamExchange Hash([1]) from 13 - Fragment 9 + Fragment 10 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 22, HashJoinDegreeLeft: 23, HashJoinRight: 24, HashJoinDegreeRight: 25 ] } - ├── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([1]) from 11 + ├── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([1]) from 12 - Fragment 10 + Fragment 11 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 26 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 12 + Fragment 13 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 28 ] } ├── Upstream └── BatchPlanNode @@ -1453,7 +1477,7 @@ Table 28 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ supp_nation, cust_nation, l_year, revenue, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - id: tpch_q8 before: @@ -1570,146 +1594,150 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr1, $expr4] } - └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } - └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } - └─StreamHashAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [$expr1, Case((nation.n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, region.r_regionkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } - │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } - │ │ │ └─StreamProject { exprs: [region.r_regionkey] } - │ │ │ └─StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } - │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } - │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } - │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ │ └─StreamProject { exprs: [part.p_partkey] } - │ │ └─StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } - │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_partkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(orders.o_orderkey) } - └─StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, $expr4] } + └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } + └─StreamHashAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, Case((nation.n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, region.r_regionkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } + │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } + │ │ │ └─StreamProject { exprs: [region.r_regionkey] } + │ │ │ └─StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } + │ │ │ └─StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } + │ │ └─StreamExchange { dist: HashShard(nation.n_regionkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } + │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(part.p_partkey) } + │ │ └─StreamProject { exprs: [part.p_partkey] } + │ │ └─StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } + │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_partkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderkey) } + └─StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [$expr1, $expr4] } - └── StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [$expr1, $expr4] } + └── StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } └── StreamHashAgg { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } ├── tables: [ HashAggState: 2 ] - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [$expr1, Case((nation.n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderdate, nation.n_name, region.r_regionkey, nation.n_nationkey, customer.c_custkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([3]) from 8 - - Fragment 3 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 7 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([3]) from 9 Fragment 4 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [customer.c_custkey, region.r_regionkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([1]) from 6 + └── StreamExchange Hash([1]) from 8 Fragment 5 + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [nation.n_nationkey, region.r_regionkey] } { tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] } + ├── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([1]) from 7 + + Fragment 6 StreamProject { exprs: [region.r_regionkey] } └── StreamFilter { predicate: (region.r_name = 'ASIA':Varchar) } └── StreamTableScan { table: region, columns: [region.r_regionkey, region.r_name], stream_scan_type: ArrangementBackfill, stream_key: [region.r_regionkey], pk: [r_regionkey], dist: UpstreamHashShard(region.r_regionkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_regionkey], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 16 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 17 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [nation.n_name, lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, orders.o_orderdate, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } ├── tables: [ HashJoinLeft: 18, HashJoinDegreeLeft: 19, HashJoinRight: 20, HashJoinDegreeRight: 21 ] - ├── StreamExchange Hash([1]) from 9 - └── StreamExchange Hash([0]) from 16 + ├── StreamExchange Hash([1]) from 10 + └── StreamExchange Hash([0]) from 17 - Fragment 9 + Fragment 10 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 22, HashJoinDegreeLeft: 23, HashJoinRight: 24, HashJoinDegreeRight: 25 ] - ├── StreamExchange Hash([1]) from 10 - └── StreamExchange Hash([1]) from 13 + ├── StreamExchange Hash([1]) from 11 + └── StreamExchange Hash([1]) from 14 - Fragment 10 + Fragment 11 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 26, HashJoinDegreeLeft: 27, HashJoinRight: 28, HashJoinDegreeRight: 29 ] } - ├── StreamExchange Hash([0]) from 11 - └── StreamExchange Hash([1]) from 12 + ├── StreamExchange Hash([0]) from 12 + └── StreamExchange Hash([1]) from 13 - Fragment 11 + Fragment 12 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 30 ] } ├── Upstream └── BatchPlanNode - Fragment 12 + Fragment 13 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 31 ] } ├── Upstream └── BatchPlanNode - Fragment 13 + Fragment 14 StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, part.p_partkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 32, HashJoinDegreeLeft: 33, HashJoinRight: 34, HashJoinDegreeRight: 35 ] } - ├── StreamExchange Hash([0]) from 14 - └── StreamExchange Hash([1]) from 15 + ├── StreamExchange Hash([0]) from 15 + └── StreamExchange Hash([1]) from 16 - Fragment 14 + Fragment 15 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: (part.p_type = 'PROMO ANODIZED STEEL':Varchar) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_type], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 36 ] } ├── Upstream └── BatchPlanNode - Fragment 15 + Fragment 16 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 37 ] ├── Upstream └── BatchPlanNode - Fragment 16 + Fragment 17 StreamFilter { predicate: (orders.o_orderdate >= '1995-01-01':Date) AND (orders.o_orderdate <= '1996-12-31':Date) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 38 ] } ├── Upstream @@ -1798,7 +1826,7 @@ Table 38 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ o_year, mkt_share, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ o_year, mkt_share, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q9 before: @@ -1894,106 +1922,110 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, $expr1) as $expr3] } - └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } - └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, partsupp.ps_suppkey, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber, partsupp.ps_partkey] } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey AND partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = supplier.s_suppkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, partsupp.ps_suppkey, partsupp.ps_partkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } - │ ├─StreamExchange { dist: HashShard(part.p_partkey) } - │ │ └─StreamProject { exprs: [part.p_partkey] } - │ │ └─StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } - │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } - │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamFilter { predicate: IsNotNull(partsupp.ps_suppkey) } - │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, $expr1) as $expr3] } + └─StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(nation.n_name, $expr1) } + └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, partsupp.ps_suppkey, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber, partsupp.ps_partkey] } + └─StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey AND partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = supplier.s_suppkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, partsupp.ps_suppkey, partsupp.ps_partkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } + │ ├─StreamExchange { dist: HashShard(part.p_partkey) } + │ │ └─StreamProject { exprs: [part.p_partkey] } + │ │ └─StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } + │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamFilter { predicate: IsNotNull(partsupp.ps_suppkey) } + │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } - └── StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0, 1]) from 1 Fragment 1 + StreamProject { exprs: [nation.n_name, $expr1, sum($expr2)] } + └── StreamTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [nation.n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [nation.n_name, $expr1, sum($expr2), Vnode(nation.n_name, $expr1) as $expr3] } └── StreamHashAgg { group_key: [nation.n_name, $expr1], aggs: [sum($expr2), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0, 1]) from 2 + └── StreamExchange Hash([0, 1]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_orderdate) as $expr1, ((lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) - (partsupp.ps_supplycost * lineitem.l_quantity)) as $expr2, part.p_partkey, partsupp.ps_suppkey, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber, partsupp.ps_partkey] } └── StreamHashJoin { type: Inner, predicate: part.p_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey AND partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = supplier.s_suppkey, output: [lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, partsupp.ps_supplycost, orders.o_orderdate, nation.n_name, part.p_partkey, partsupp.ps_suppkey, partsupp.ps_partkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([2]) from 3 + ├── StreamExchange Hash([2]) from 4 └── StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [nation.n_name, supplier.s_suppkey, orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, nation.n_nationkey, orders.o_orderkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] } - ├── StreamExchange Hash([1]) from 6 - └── StreamExchange Hash([2]) from 9 + ├── StreamExchange Hash([1]) from 7 + └── StreamExchange Hash([2]) from 10 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: all } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 4 + Fragment 5 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, '%yellow%':Varchar) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: IsNotNull(partsupp.ps_suppkey) } └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } { tables: [ StreamScan: 12 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [nation.n_name, supplier.s_suppkey, nation.n_nationkey] } { tables: [ HashJoinLeft: 17, HashJoinDegreeLeft: 18, HashJoinRight: 19, HashJoinDegreeRight: 20 ] } - ├── StreamExchange Hash([0]) from 7 - └── StreamExchange Hash([1]) from 8 + ├── StreamExchange Hash([0]) from 8 + └── StreamExchange Hash([1]) from 9 - Fragment 7 + Fragment 8 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 21 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 22 ] } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderdate, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 23, HashJoinDegreeLeft: 24, HashJoinRight: 25, HashJoinDegreeRight: 26 ] } - ├── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + ├── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 10 + Fragment 11 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamFilter { predicate: IsNotNull(lineitem.l_partkey) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { tables: [ StreamScan: 28 ] } ├── Upstream @@ -2057,7 +2089,7 @@ Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ nation, o_year, sum_profit, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ nation, o_year, sum_profit, _rw_timestamp ], primary key: [ $0 ASC, $1 DESC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - id: tpch_q10 before: @@ -2145,70 +2177,74 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_returnflag], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey) as $expr2] } - └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } - └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } - │ └─StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(orders.o_orderkey) } - └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey) as $expr2] } + └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } + └─StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ └─StreamExchange { dist: HashShard(customer.c_nationkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } + │ └─StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_orderkey) } + └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } + └─StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } - └── StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([2]) from 1 Fragment 1 + StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment] } + └── StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [customer.c_custkey, customer.c_name, sum($expr1), customer.c_acctbal, nation.n_name, customer.c_address, customer.c_phone, customer.c_comment, Vnode(customer.c_custkey) as $expr2] } └── StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment], aggs: [sum($expr1), count] } { tables: [ HashAggState: 2 ] } └── StreamProject { exprs: [customer.c_custkey, customer.c_name, customer.c_acctbal, customer.c_phone, nation.n_name, customer.c_address, customer.c_comment, (lineitem.l_extendedprice * (1.00:Decimal - lineitem.l_discount)) as $expr1, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, lineitem.l_extendedprice, lineitem.l_discount, nation.n_name, nation.n_nationkey, lineitem.l_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] - ├── StreamExchange Hash([1]) from 2 - └── StreamExchange Hash([2]) from 5 + ├── StreamExchange Hash([1]) from 3 + └── StreamExchange Hash([2]) from 6 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: nation.n_nationkey = customer.c_nationkey, output: [nation.n_name, customer.c_custkey, customer.c_name, customer.c_address, customer.c_phone, customer.c_acctbal, customer.c_comment, nation.n_nationkey] } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([3]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([3]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name, customer.c_address, customer.c_nationkey, customer.c_phone, customer.c_acctbal, customer.c_comment], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── tables: [ StreamScan: 12 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamHashJoin { type: Inner, predicate: lineitem.l_orderkey = orders.o_orderkey, output: [lineitem.l_extendedprice, lineitem.l_discount, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, orders.o_orderkey] } { tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] } - ├── StreamExchange Hash([0]) from 6 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([0]) from 7 + └── StreamExchange Hash([0]) from 8 - Fragment 6 + Fragment 7 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_returnflag = 'R':Varchar) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_linenumber, lineitem.l_returnflag], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -2216,7 +2252,7 @@ ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: (orders.o_orderdate >= '1994-01-01':Date) AND (orders.o_orderdate < '1994-04-01 00:00:00':Timestamp) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 18 ] } @@ -2272,7 +2308,7 @@ Table 18 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [], read pk prefix len hint: 8 } + Table 4294967294 { columns: [ c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment, _rw_timestamp ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 2 ], read pk prefix len hint: 8 } - id: tpch_q11 before: @@ -2371,55 +2407,59 @@ └─BatchScan { table: partsupp, columns: [partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } - └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1), Vnode(partsupp.ps_partkey) as _vnode] } - └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } - ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } - │ └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_nationkey] } - │ └─StreamShare { id: 9 } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, nation.n_nationkey] } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } - │ │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } - └─StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } - └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } - └─StreamShare { id: 9 } - └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, nation.n_nationkey] } - ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } - │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - └─StreamExchange { dist: HashShard(nation.n_nationkey) } - └─StreamProject { exprs: [nation.n_nationkey] } - └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } - └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1), Vnode(partsupp.ps_partkey) as _vnode] } + └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [partsupp.ps_partkey, sum($expr1)] } + ├─StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } + │ └─StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_nationkey] } + │ └─StreamShare { id: 9 } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, nation.n_nationkey] } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } + │ │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + └─StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } + └─StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } + └─StreamShare { id: 9 } + └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, nation.n_nationkey] } + ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } + │ ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(nation.n_nationkey) } + └─StreamProject { exprs: [nation.n_nationkey] } + └─StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } + └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } - └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1]) from 1 Fragment 1 + StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } + └── StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [partsupp.ps_partkey, sum($expr1), Vnode(partsupp.ps_partkey) as _vnode] } @@ -2428,52 +2468,52 @@ ├── StreamProject { exprs: [partsupp.ps_partkey, sum($expr1)] } │ └── StreamHashAgg { group_key: [partsupp.ps_partkey], aggs: [sum($expr1), count] } │ ├── tables: [ HashAggState: 4 ] - │ └── StreamExchange Hash([0]) from 2 - └── StreamExchange Broadcast from 8 + │ └── StreamExchange Hash([0]) from 3 + └── StreamExchange Broadcast from 9 - Fragment 2 + Fragment 3 StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_nationkey] } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, nation.n_nationkey] } ├── tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] - ├── StreamExchange Hash([3]) from 4 - └── StreamExchange Hash([0]) from 7 + ├── StreamExchange Hash([3]) from 5 + └── StreamExchange Hash([0]) from 8 - Fragment 4 + Fragment 5 StreamHashJoin { type: Inner, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, supplier.s_nationkey, partsupp.ps_suppkey, supplier.s_suppkey] } ├── tables: [ HashJoinLeft: 9, HashJoinDegreeLeft: 10, HashJoinRight: 11, HashJoinDegreeRight: 12 ] - ├── StreamExchange Hash([1]) from 5 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([1]) from 6 + └── StreamExchange Hash([0]) from 7 - Fragment 5 + Fragment 6 StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty, partsupp.ps_supplycost], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── tables: [ StreamScan: 13 ] ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 14 ] } ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'ARGENTINA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } └── StreamSimpleAgg { aggs: [sum(sum($expr2)), count] } { tables: [ SimpleAggState: 16 ] } - └── StreamExchange Single from 9 + └── StreamExchange Single from 10 - Fragment 9 + Fragment 10 StreamStatelessSimpleAgg { aggs: [sum($expr2)] } └── StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } - └── StreamExchange NoShuffle from 3 + └── StreamExchange NoShuffle from 4 Table 0 { columns: [ partsupp_ps_partkey, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } @@ -2509,7 +2549,7 @@ Table 16 { columns: [ sum(sum($expr2)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } - Table 4294967294 { columns: [ ps_partkey, value, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ ps_partkey, value, _rw_timestamp ], primary key: [ $1 DESC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - id: tpch_q12 before: @@ -2576,47 +2616,51 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } - └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } - └─StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2), Vnode(lineitem.l_shipmode) as $expr3] } - └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } - └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } - └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_linenumber] } - └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } - ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } - └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } + └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } + └─StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2), Vnode(lineitem.l_shipmode) as $expr3] } + └─StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } + └─StreamExchange { dist: HashShard(lineitem.l_shipmode) } + └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_linenumber] } + └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } + └─StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } - └── StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2)] } + └── StreamTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [lineitem.l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [lineitem.l_shipmode, sum($expr1), sum($expr2), Vnode(lineitem.l_shipmode) as $expr3] } └── StreamHashAgg { group_key: [lineitem.l_shipmode], aggs: [sum($expr1), sum($expr2), count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_orderpriority <> '1-URGENT':Varchar) AND (orders.o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, orders.o_orderkey, lineitem.l_linenumber] } └── StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [orders.o_orderpriority, lineitem.l_shipmode, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } { tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderpriority], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 7 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber] } └── StreamFilter { predicate: In(lineitem.l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (lineitem.l_commitdate < lineitem.l_receiptdate) AND (lineitem.l_shipdate < lineitem.l_commitdate) AND (lineitem.l_receiptdate >= '1994-01-01':Date) AND (lineitem.l_receiptdate < '1995-01-01 00:00:00':Timestamp) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_shipmode, lineitem.l_linenumber, lineitem.l_shipdate, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -2642,7 +2686,7 @@ Table 8 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ l_shipmode, high_line_count, low_line_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ l_shipmode, high_line_count, low_line_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q13 before: @@ -2703,52 +2747,56 @@ └─BatchScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], distribution: UpstreamHashShard(orders.o_orderkey) } stream_plan: |- StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(orders.o_orderkey), count] } - └─StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [count(orders.o_orderkey), count, Vnode(count(orders.o_orderkey)) as _vnode] } - └─StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } - └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } - └─StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } - └─StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } - └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } - ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ └─StreamTableScan { table: customer, columns: [customer.c_custkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - └─StreamExchange { dist: HashShard(orders.o_custkey) } - └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } - └─StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } - └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: HashShard(count, count(orders.o_orderkey)) } + └─StreamProject { exprs: [count(orders.o_orderkey), count] } + └─StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [count(orders.o_orderkey), count, Vnode(count(orders.o_orderkey)) as _vnode] } + └─StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(orders.o_orderkey)) } + └─StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } + └─StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } + └─StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } + ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ └─StreamTableScan { table: customer, columns: [customer.c_custkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard(orders.o_custkey) } + └─StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } + └─StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } + └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [count(orders.o_orderkey), count] } - └── StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamProject { exprs: [count(orders.o_orderkey), count] } + └── StreamTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count DESC, count(orders.o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [count(orders.o_orderkey), count, Vnode(count(orders.o_orderkey)) as _vnode] } └── StreamHashAgg { group_key: [count(orders.o_orderkey)], aggs: [count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([1]) from 2 + └── StreamExchange Hash([1]) from 3 - Fragment 2 + Fragment 3 StreamProject { exprs: [customer.c_custkey, count(orders.o_orderkey)] } └── StreamHashAgg { group_key: [customer.c_custkey], aggs: [count(orders.o_orderkey), count] } { tables: [ HashAggState: 3 ] } └── StreamHashJoin { type: LeftOuter, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, orders.o_orderkey] } ├── tables: [ HashJoinLeft: 4, HashJoinDegreeLeft: 5, HashJoinRight: 6, HashJoinDegreeRight: 7 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([1]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([1]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: customer, columns: [customer.c_custkey], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } ├── tables: [ StreamScan: 8 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [orders.o_orderkey, orders.o_custkey] } └── StreamFilter { predicate: Not(Like(orders.o_comment, '%:1%:2%':Varchar)) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_comment], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } @@ -2776,7 +2824,7 @@ Table 9 { columns: [ vnode, o_orderkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ c_count, custdist, _rw_timestamp ], primary key: [ $1 DESC, $0 DESC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: tpch_q14 before: @@ -2953,70 +3001,74 @@ └─LogicalScan { table: lineitem, output_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount], required_columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate], predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < ('1993-01-01':Date + '3 mons':Interval)) } stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } - └─StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as _vnode] } - └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } - ├─StreamExchange { dist: HashShard(sum($expr1)) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } - │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamShare { id: 7 } - │ └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - │ └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } - └─StreamProject { exprs: [max(max(sum($expr1)))] } - └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as _vnode] } - └─StreamShare { id: 7 } - └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } - └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } + └─StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } + ├─StreamExchange { dist: HashShard(sum($expr1)) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } + │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamShare { id: 7 } + │ └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } + │ └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } + └─StreamProject { exprs: [max(max(sum($expr1)))] } + └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as _vnode] } + └─StreamShare { id: 7 } + └─StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } + └─StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + └─StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } - └── StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } + └── StreamTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as _vnode] } └── StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1)] } ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] - ├── StreamExchange Hash([4]) from 2 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([4]) from 3 + └── StreamExchange Hash([0]) from 7 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey] } ├── tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange NoShuffle from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange NoShuffle from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 10 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [lineitem.l_suppkey, sum($expr1)] } └── StreamHashAgg { group_key: [lineitem.l_suppkey], aggs: [sum($expr1), count] } { tables: [ HashAggState: 11 ] } - └── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 5 + Fragment 6 StreamProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1993-01-01':Date) AND (lineitem.l_shipdate < '1993-04-01 00:00:00':Timestamp) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -3024,15 +3076,15 @@ ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [max(max(sum($expr1)))] } └── StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } { tables: [ SimpleAggState: 14, SimpleAggCall0: 13 ] } - └── StreamExchange Single from 7 + └── StreamExchange Single from 8 - Fragment 7 + Fragment 8 StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } { tables: [ HashAggState: 16, HashAggCall0: 15 ] } └── StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as _vnode] } - └── StreamExchange NoShuffle from 4 + └── StreamExchange NoShuffle from 5 Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), _vnode, _rw_timestamp ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 0 } @@ -3068,7 +3120,7 @@ Table 16 { columns: [ _vnode, max(sum($expr1)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q16 before: @@ -3141,67 +3193,71 @@ └─BatchScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], distribution: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } stream_plan: |- StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } - └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } - └─StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey), Vnode(part.p_brand, part.p_type, part.p_size) as $expr1] } - └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } - └─StreamExchange { dist: HashShard(part.p_brand, part.p_type, part.p_size) } - └─StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey] } - ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey, part.p_partkey] } - │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ └─StreamExchange { dist: HashShard(part.p_partkey) } - │ └─StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } - │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } - └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamProject { exprs: [supplier.s_suppkey] } - └─StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } - └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + └─StreamExchange { dist: HashShard(count(distinct partsupp.ps_suppkey), part.p_brand, part.p_type, part.p_size) } + └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } + └─StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey), Vnode(part.p_brand, part.p_type, part.p_size) as $expr1] } + └─StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } + └─StreamExchange { dist: HashShard(part.p_brand, part.p_type, part.p_size) } + └─StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey] } + ├─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey, part.p_partkey] } + │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ └─StreamExchange { dist: HashShard(part.p_partkey) } + │ └─StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } + │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + └─StreamExchange { dist: HashShard(supplier.s_suppkey) } + └─StreamProject { exprs: [supplier.s_suppkey] } + └─StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } + └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } - └── StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } - ├── tables: [ TopN: 0 ] - └── StreamExchange Single from 1 + └── StreamExchange Hash([3, 0, 1, 2]) from 1 Fragment 1 + StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey)] } + └── StreamTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0 } + ├── tables: [ TopN: 0 ] + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count(distinct partsupp.ps_suppkey) DESC, part.p_brand ASC, part.p_type ASC, part.p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [part.p_brand, part.p_type, part.p_size, count(distinct partsupp.ps_suppkey), Vnode(part.p_brand, part.p_type, part.p_size) as $expr1] } └── StreamHashAgg { group_key: [part.p_brand, part.p_type, part.p_size], aggs: [count(distinct partsupp.ps_suppkey), count] } ├── tables: [ HashAggState: 2, HashAggDedupForCol0: 3 ] - └── StreamExchange Hash([1, 2, 3]) from 2 + └── StreamExchange Hash([1, 2, 3]) from 3 - Fragment 2 + Fragment 3 StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey] } ├── tables: [ HashJoinLeft: 4, HashJoinDegreeLeft: 5, HashJoinRight: 6, HashJoinDegreeRight: 7 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([0]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([0]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, part.p_brand, part.p_type, part.p_size, partsupp.ps_partkey, part.p_partkey] } ├── tables: [ HashJoinLeft: 8, HashJoinDegreeLeft: 9, HashJoinRight: 10, HashJoinDegreeRight: 11 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([0]) from 6 - Fragment 4 + Fragment 5 StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── tables: [ StreamScan: 12 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamFilter { predicate: (part.p_brand <> 'Brand#45':Varchar) AND Not(Like(part.p_type, 'SMALL PLATED%':Varchar)) AND In(part.p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_brand, part.p_type, part.p_size], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 13 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamProject { exprs: [supplier.s_suppkey] } └── StreamFilter { predicate: Like(supplier.s_comment, '%Customer%Complaints%':Varchar) } └── StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_comment], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { tables: [ StreamScan: 14 ] } @@ -3244,7 +3300,7 @@ Table 14 { columns: [ vnode, s_suppkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ p_brand, p_type, p_size, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ p_brand, p_type, p_size, supplier_cnt, _rw_timestamp ], primary key: [ $3 DESC, $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3, 0, 1, 2 ], read pk prefix len hint: 4 } - id: tpch_q17 before: @@ -3490,37 +3546,41 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } - └─StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } - └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate], aggs: [sum(lineitem.l_quantity), count] } - └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_linenumber] } - ├─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } - │ │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - │ │ └─StreamExchange { dist: HashShard(orders.o_custkey) } - │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamProject { exprs: [lineitem.l_orderkey] } - └─StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Decimal) } - └─StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } - └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(orders.o_totalprice, orders.o_orderdate) } + └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } + └─StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate], aggs: [sum(lineitem.l_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_linenumber] } + ├─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } + │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } + │ │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + │ │ └─StreamExchange { dist: HashShard(orders.o_custkey) } + │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamProject { exprs: [lineitem.l_orderkey] } + └─StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Decimal) } + └─StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } + └─StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } - └── StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([4, 3]) from 1 Fragment 1 + StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity)] } + └── StreamTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } └── StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate], aggs: [sum(lineitem.l_quantity), count] } { tables: [ HashAggState: 2 ] } @@ -3528,38 +3588,38 @@ ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] ├── StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } │ ├── tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] - │ ├── StreamExchange Hash([2]) from 2 - │ └── StreamExchange Hash([0]) from 5 + │ ├── StreamExchange Hash([2]) from 3 + │ └── StreamExchange Hash([0]) from 6 └── StreamProject { exprs: [lineitem.l_orderkey] } └── StreamFilter { predicate: (sum(lineitem.l_quantity) > 1:Decimal) } └── StreamProject { exprs: [lineitem.l_orderkey, sum(lineitem.l_quantity)] } └── StreamHashAgg { group_key: [lineitem.l_orderkey], aggs: [sum(lineitem.l_quantity), count] } { tables: [ HashAggState: 18 ] } - └── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([0]) from 7 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate] } ├── tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([1]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([1]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_name], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } { tables: [ StreamScan: 15 ] } ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_custkey, orders.o_totalprice, orders.o_orderdate], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } ├── tables: [ StreamScan: 16 ] ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 17 ] ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_quantity, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } ├── tables: [ StreamScan: 19 ] ├── Upstream @@ -3621,7 +3681,7 @@ Table 19 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity, _rw_timestamp ], primary key: [ $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity, _rw_timestamp ], primary key: [ $4 DESC, $3 ASC, $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4, 3 ], read pk prefix len hint: 5 } - id: tpch_q19 before: @@ -3810,96 +3870,100 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - └─StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [supplier.s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as _vnode] } - └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } - │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ └─StreamProject { exprs: [nation.n_nationkey] } - │ └─StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } - │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - └─StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey] } - ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, lineitem.l_suppkey] } - │ └─StreamFilter { predicate: ($expr1 > $expr2) } - │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey, output: all } - │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } - │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } - │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } - │ └─StreamHashAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity), count] } - │ └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } - │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } - │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(part.p_partkey) } - └─StreamProject { exprs: [part.p_partkey] } - └─StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } - └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } + └─StreamExchange { dist: HashShard(supplier.s_name) } + └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + └─StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [supplier.s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as _vnode] } + └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ └─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ └─StreamProject { exprs: [nation.n_nationkey] } + │ └─StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } + │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey] } + ├─StreamExchange { dist: HashShard(partsupp.ps_partkey) } + │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, lineitem.l_suppkey] } + │ └─StreamFilter { predicate: ($expr1 > $expr2) } + │ └─StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey, output: all } + │ ├─StreamExchange { dist: HashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ │ └─StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } + │ │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } + │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } + │ └─StreamHashAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity), count] } + │ └─StreamExchange { dist: HashShard(lineitem.l_partkey, lineitem.l_suppkey) } + │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } + │ └─StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(part.p_partkey) } + └─StreamProject { exprs: [part.p_partkey] } + └─StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } + └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } - └── StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 + StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } + └── StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [supplier.s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } { tables: [ GroupTopN: 1 ] } └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as _vnode] } └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] - ├── StreamExchange Hash([0]) from 2 - └── StreamExchange Hash([0]) from 5 + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 6 - Fragment 2 + Fragment 3 StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } ├── tables: [ HashJoinLeft: 6, HashJoinDegreeLeft: 7, HashJoinRight: 8, HashJoinDegreeRight: 9 ] - ├── StreamExchange Hash([3]) from 3 - └── StreamExchange Hash([0]) from 4 + ├── StreamExchange Hash([3]) from 4 + └── StreamExchange Hash([0]) from 5 - Fragment 3 + Fragment 4 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 10 ] ├── Upstream └── BatchPlanNode - Fragment 4 + Fragment 5 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 11 ] } ├── Upstream └── BatchPlanNode - Fragment 5 + Fragment 6 StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey] } { tables: [ HashJoinLeft: 12, HashJoinDegreeLeft: 13, HashJoinRight: 14, HashJoinDegreeRight: 15 ] } - ├── StreamExchange Hash([0]) from 6 - └── StreamExchange Hash([0]) from 9 + ├── StreamExchange Hash([0]) from 7 + └── StreamExchange Hash([0]) from 10 - Fragment 6 + Fragment 7 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, lineitem.l_partkey, lineitem.l_suppkey] } └── StreamFilter { predicate: ($expr1 > $expr2) } └── StreamHashJoin { type: Inner, predicate: partsupp.ps_partkey = lineitem.l_partkey AND partsupp.ps_suppkey = lineitem.l_suppkey, output: all } { tables: [ HashJoinLeft: 16, HashJoinDegreeLeft: 17, HashJoinRight: 18, HashJoinDegreeRight: 19 ] } - ├── StreamExchange Hash([0, 1]) from 7 + ├── StreamExchange Hash([0, 1]) from 8 └── StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, (0.5:Decimal * sum(lineitem.l_quantity)) as $expr2] } └── StreamHashAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity), count] } { tables: [ HashAggState: 21 ] } - └── StreamExchange Hash([0, 1]) from 8 + └── StreamExchange Hash([0, 1]) from 9 - Fragment 7 + Fragment 8 StreamProject { exprs: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty::Decimal as $expr1] } └── StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_availqty], stream_scan_type: ArrangementBackfill, stream_key: [partsupp.ps_partkey, partsupp.ps_suppkey], pk: [ps_partkey, ps_suppkey], dist: UpstreamHashShard(partsupp.ps_partkey, partsupp.ps_suppkey) } ├── tables: [ StreamScan: 20 ] ├── Upstream └── BatchPlanNode - Fragment 8 + Fragment 9 StreamProject { exprs: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -3907,7 +3971,7 @@ ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamProject { exprs: [part.p_partkey] } └── StreamFilter { predicate: Like(part.p_name, 'forest%':Varchar) } └── StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], stream_scan_type: ArrangementBackfill, stream_key: [part.p_partkey], pk: [p_partkey], dist: UpstreamHashShard(part.p_partkey) } { tables: [ StreamScan: 23 ] } @@ -3962,7 +4026,7 @@ Table 23 { columns: [ vnode, p_partkey, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - id: tpch_q21 before: @@ -4071,102 +4135,106 @@ └─BatchScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_commitdate, lineitem.l_receiptdate], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_name, count] } - └─StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as _vnode] } - └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(supplier.s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber, lineitem.l_orderkey] } - ├─StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } - │ ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } - │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } - │ │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } - │ │ │ │ └─StreamProject { exprs: [nation.n_nationkey] } - │ │ │ │ └─StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } - │ │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } - │ │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } - │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } - │ │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } - │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } - │ │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } - │ │ │ └─StreamProject { exprs: [orders.o_orderkey] } - │ │ │ └─StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } - │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - │ │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } - │ │ └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - │ │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } - └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } - └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } - └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } - └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(count, supplier.s_name) } + └─StreamProject { exprs: [supplier.s_name, count] } + └─StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as _vnode] } + └─StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(supplier.s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber, lineitem.l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } + │ ├─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } + │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } + │ │ │ └─StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } + │ │ │ ├─StreamExchange { dist: HashShard(nation.n_nationkey) } + │ │ │ │ └─StreamProject { exprs: [nation.n_nationkey] } + │ │ │ │ └─StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } + │ │ │ │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } + │ │ │ └─StreamExchange { dist: HashShard(supplier.s_nationkey) } + │ │ │ └─StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + │ │ └─StreamExchange { dist: HashShard(lineitem.l_suppkey) } + │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } + │ │ ├─StreamExchange { dist: HashShard(orders.o_orderkey) } + │ │ │ └─StreamProject { exprs: [orders.o_orderkey] } + │ │ │ └─StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } + │ │ │ └─StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + │ │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ │ └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } + │ │ └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } + │ │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + │ └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + │ └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + └─StreamExchange { dist: HashShard(lineitem.l_orderkey) } + └─StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } + └─StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } + └─StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [supplier.s_name, count] } - └── StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } - └── StreamExchange Single from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamProject { exprs: [supplier.s_name, count] } + └── StreamTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 2 + + Fragment 2 StreamGroupTopN { order: [count DESC, supplier.s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } ├── tables: [ GroupTopN: 1 ] └── StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as _vnode] } └── StreamHashAgg { group_key: [supplier.s_name], aggs: [count] } { tables: [ HashAggState: 2 ] } - └── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 - Fragment 2 + Fragment 3 StreamHashJoin { type: LeftAnti, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: [supplier.s_name, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber, lineitem.l_orderkey] } ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] ├── StreamHashJoin { type: LeftSemi, predicate: lineitem.l_orderkey = lineitem.l_orderkey AND (lineitem.l_suppkey <> lineitem.l_suppkey), output: all } { tables: [ HashJoinLeft: 7, HashJoinDegreeLeft: 8, HashJoinRight: 9, HashJoinDegreeRight: 10 ] } - │ ├── StreamExchange Hash([1]) from 3 - │ └── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + │ ├── StreamExchange Hash([1]) from 4 + │ └── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 3 + Fragment 4 StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = lineitem.l_suppkey, output: [supplier.s_name, lineitem.l_orderkey, lineitem.l_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([1]) from 7 - - Fragment 4 - StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } { tables: [ HashJoinLeft: 15, HashJoinDegreeLeft: 16, HashJoinRight: 17, HashJoinDegreeRight: 18 ] } ├── StreamExchange Hash([0]) from 5 - └── StreamExchange Hash([2]) from 6 + └── StreamExchange Hash([1]) from 8 Fragment 5 + StreamHashJoin { type: Inner, predicate: nation.n_nationkey = supplier.s_nationkey, output: [supplier.s_suppkey, supplier.s_name, nation.n_nationkey] } { tables: [ HashJoinLeft: 15, HashJoinDegreeLeft: 16, HashJoinRight: 17, HashJoinDegreeRight: 18 ] } + ├── StreamExchange Hash([0]) from 6 + └── StreamExchange Hash([2]) from 7 + + Fragment 6 StreamProject { exprs: [nation.n_nationkey] } └── StreamFilter { predicate: (nation.n_name = 'GERMANY':Varchar) } └── StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], stream_scan_type: ArrangementBackfill, stream_key: [nation.n_nationkey], pk: [n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } { tables: [ StreamScan: 19 ] } ├── Upstream └── BatchPlanNode - Fragment 6 + Fragment 7 StreamTableScan { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_nationkey], stream_scan_type: ArrangementBackfill, stream_key: [supplier.s_suppkey], pk: [s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } ├── tables: [ StreamScan: 20 ] ├── Upstream └── BatchPlanNode - Fragment 7 + Fragment 8 StreamHashJoin { type: Inner, predicate: orders.o_orderkey = lineitem.l_orderkey, output: [lineitem.l_orderkey, lineitem.l_suppkey, orders.o_orderkey, lineitem.l_linenumber] } ├── tables: [ HashJoinLeft: 21, HashJoinDegreeLeft: 22, HashJoinRight: 23, HashJoinDegreeRight: 24 ] - ├── StreamExchange Hash([0]) from 8 - └── StreamExchange Hash([0]) from 9 + ├── StreamExchange Hash([0]) from 9 + └── StreamExchange Hash([0]) from 10 - Fragment 8 + Fragment 9 StreamProject { exprs: [orders.o_orderkey] } └── StreamFilter { predicate: (orders.o_orderstatus = 'F':Varchar) } └── StreamTableScan { table: orders, columns: [orders.o_orderkey, orders.o_orderstatus], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } { tables: [ StreamScan: 25 ] } ├── Upstream └── BatchPlanNode - Fragment 9 + Fragment 10 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -4174,12 +4242,12 @@ ├── Upstream └── BatchPlanNode - Fragment 10 + Fragment 11 StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { tables: [ StreamScan: 27 ] } ├── Upstream └── BatchPlanNode - Fragment 11 + Fragment 12 StreamProject { exprs: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber] } └── StreamFilter { predicate: (lineitem.l_receiptdate > lineitem.l_commitdate) } └── StreamTableScan { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_linenumber, lineitem.l_commitdate, lineitem.l_receiptdate], stream_scan_type: ArrangementBackfill, stream_key: [lineitem.l_orderkey, lineitem.l_linenumber], pk: [l_orderkey, l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } @@ -4245,7 +4313,7 @@ Table 28 { columns: [ vnode, l_orderkey, l_linenumber, backfill_finished, row_count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } - id: tpch_q22 before: @@ -4342,26 +4410,27 @@ └─BatchScan { table: customer, columns: [customer.c_acctbal, customer.c_phone], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [cntrycode, numcust, totacctbal], stream_key: [], pk_columns: [cntrycode], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal)] } - └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal), Vnode($expr2) as _vnode] } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(customer.c_acctbal)] } - └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr2, customer.c_acctbal, customer.c_custkey] } - └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr1), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } - ├─StreamHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } - │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } - │ │ └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } - │ └─StreamExchange { dist: HashShard(orders.o_custkey) } - │ └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))::Decimal) as $expr1] } - └─StreamSimpleAgg { aggs: [sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } - └─StreamProject { exprs: [customer.c_acctbal, customer.c_custkey] } - └─StreamFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─StreamTableScan { table: customer, columns: [customer.c_acctbal, customer.c_custkey, customer.c_phone], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal)] } + └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [$expr2, count, sum(customer.c_acctbal), Vnode($expr2) as _vnode] } + └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(customer.c_acctbal)] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [Substr(customer.c_phone, 1:Int32, 2:Int32) as $expr2, customer.c_acctbal, customer.c_custkey] } + └─StreamDynamicFilter { predicate: (customer.c_acctbal > $expr1), output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } + ├─StreamHashJoin { type: LeftAnti, predicate: customer.c_custkey = orders.o_custkey, output: [customer.c_phone, customer.c_acctbal, customer.c_custkey] } + │ ├─StreamExchange { dist: HashShard(customer.c_custkey) } + │ │ └─StreamFilter { predicate: In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamTableScan { table: customer, columns: [customer.c_custkey, customer.c_phone, customer.c_acctbal], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } + │ └─StreamExchange { dist: HashShard(orders.o_custkey) } + │ └─StreamTableScan { table: orders, columns: [orders.o_custkey, orders.o_orderkey], stream_scan_type: ArrangementBackfill, stream_key: [orders.o_orderkey], pk: [o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(customer.c_acctbal)) / sum0(count(customer.c_acctbal))::Decimal) as $expr1] } + └─StreamSimpleAgg { aggs: [sum(sum(customer.c_acctbal)), sum0(count(customer.c_acctbal)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(customer.c_acctbal), count(customer.c_acctbal)] } + └─StreamProject { exprs: [customer.c_acctbal, customer.c_custkey] } + └─StreamFilter { predicate: (customer.c_acctbal > 0.00:Decimal) AND In(Substr(customer.c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamTableScan { table: customer, columns: [customer.c_acctbal, customer.c_custkey, customer.c_phone], stream_scan_type: ArrangementBackfill, stream_key: [customer.c_custkey], pk: [c_custkey], dist: UpstreamHashShard(customer.c_custkey) } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml index b1f9c6d5dc88c..fc3a1fc7f5e33 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml @@ -145,18 +145,19 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } - └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } - └─StreamTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } - └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), (sum(l_quantity) / count(l_quantity)::Decimal) as $expr3, (sum(l_extendedprice) / count(l_extendedprice)::Decimal) as $expr4, (sum(l_discount) / count(l_discount)::Decimal) as $expr5, count, Vnode(l_returnflag, l_linestatus) as $expr6] } - └─StreamHashAgg [append_only] { group_key: [l_returnflag, l_linestatus], aggs: [sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count] } - └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } - └─StreamProject { exprs: [l_returnflag, l_linestatus, l_quantity, l_extendedprice, $expr1, ($expr1 * (1:Decimal + l_tax)) as $expr2, l_discount, _row_id] } - └─StreamProject { exprs: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, (l_extendedprice * (1:Decimal - l_discount)) as $expr1] } - └─StreamFilter { predicate: (l_shipdate <= '1998-09-21 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └─StreamTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), (sum(l_quantity) / count(l_quantity)::Decimal) as $expr3, (sum(l_extendedprice) / count(l_extendedprice)::Decimal) as $expr4, (sum(l_discount) / count(l_discount)::Decimal) as $expr5, count, Vnode(l_returnflag, l_linestatus) as $expr6] } + └─StreamHashAgg [append_only] { group_key: [l_returnflag, l_linestatus], aggs: [sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count] } + └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } + └─StreamProject { exprs: [l_returnflag, l_linestatus, l_quantity, l_extendedprice, $expr1, ($expr1 * (1:Decimal + l_tax)) as $expr2, l_discount, _row_id] } + └─StreamProject { exprs: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, (l_extendedprice * (1:Decimal - l_discount)) as $expr1] } + └─StreamFilter { predicate: (l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q2 before: - create_tables @@ -207,77 +208,78 @@ LIMIT 100; stream_plan: |- StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } - └─StreamTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey, Vnode(ps_partkey, min(ps_supplycost)) as _vnode] } - └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } - ├─StreamExchange { dist: HashShard(ps_partkey, min(ps_supplycost)) } - │ └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id] } - │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ └─StreamFilter { predicate: (p_size = 4:Int32) AND Like(p_type, '%TIN':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamProject { exprs: [ps_partkey, min(ps_supplycost)] } - │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [min(ps_supplycost), count] } - │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_supplycost, ps_partkey, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } - │ ├─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ │ └─StreamShare { id: 6 } - │ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamShare { id: 10 } - │ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ └─StreamShare { id: 17 } - │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } - │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ └─StreamShare { id: 22 } - │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(ps_partkey, ps_supplycost) } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = ps_suppkey, output: [n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, s_suppkey, _row_id] } - ├─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ │ └─StreamShare { id: 17 } - │ │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } - │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } - │ │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 22 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamShare { id: 10 } - │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(ps_suppkey) } - └─StreamFilter { predicate: IsNotNull(ps_partkey) } - └─StreamShare { id: 6 } - └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } - └─StreamRowIdGen { row_id_index: 8 } - └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_acctbal, n_name, s_name, p_partkey) } + └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } + └─StreamTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey, Vnode(ps_partkey, min(ps_supplycost)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_partkey, min(ps_supplycost)) } + │ └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: (p_size = 4:Int32) AND Like(p_type, '%TIN':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamProject { exprs: [ps_partkey, min(ps_supplycost)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [min(ps_supplycost), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_supplycost, ps_partkey, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamShare { id: 6 } + │ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamShare { id: 10 } + │ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamShare { id: 17 } + │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamShare { id: 22 } + │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_partkey, ps_supplycost) } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = ps_suppkey, output: [n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, s_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamShare { id: 17 } + │ │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 22 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamShare { id: 10 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamFilter { predicate: IsNotNull(ps_partkey) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + └─StreamRowIdGen { row_id_index: 8 } + └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q3 before: - create_tables @@ -307,29 +309,30 @@ LIMIT 10; stream_plan: |- StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority] } - └─StreamTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority, Vnode(l_orderkey, o_orderdate, o_shippriority) as $expr2] } - └─StreamHashAgg [append_only] { group_key: [l_orderkey, o_orderdate, o_shippriority], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(l_orderkey, o_orderdate, o_shippriority) } - └─StreamProject { exprs: [l_orderkey, o_orderdate, o_shippriority, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, c_custkey, _row_id, o_orderkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, o_shippriority, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, c_custkey, o_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [o_orderkey, o_orderdate, o_shippriority, _row_id, c_custkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(c_custkey) } - │ │ └─StreamFilter { predicate: (c_mktsegment = 'FURNITURE':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 11 } - │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(o_custkey) } - │ └─StreamFilter { predicate: (o_orderdate < '1995-03-29':Date) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamFilter { predicate: (l_shipdate > '1995-03-29':Date) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1), o_orderdate) } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority] } + └─StreamTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority, Vnode(l_orderkey, o_orderdate, o_shippriority) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey, o_orderdate, o_shippriority], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_orderkey, o_orderdate, o_shippriority) } + └─StreamProject { exprs: [l_orderkey, o_orderdate, o_shippriority, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, c_custkey, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, o_shippriority, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, c_custkey, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [o_orderkey, o_orderdate, o_shippriority, _row_id, c_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: (c_mktsegment = 'FURNITURE':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamFilter { predicate: (o_orderdate < '1995-03-29':Date) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: (l_shipdate > '1995-03-29':Date) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q4 before: - create_tables @@ -358,23 +361,24 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } - └─StreamProject { exprs: [o_orderpriority, count] } - └─StreamTopN { order: [o_orderpriority ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [o_orderpriority, count, Vnode(o_orderpriority) as _vnode] } - └─StreamHashAgg { group_key: [o_orderpriority], aggs: [count] } - └─StreamExchange { dist: HashShard(o_orderpriority) } - └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, _row_id, o_orderkey] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamFilter { predicate: (o_orderdate >= '1997-07-01':Date) AND (o_orderdate < '1997-10-01 00:00:00':Timestamp) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamProject { exprs: [l_orderkey, _row_id] } - └─StreamFilter { predicate: (l_commitdate < l_receiptdate) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderpriority) } + └─StreamProject { exprs: [o_orderpriority, count] } + └─StreamTopN { order: [o_orderpriority ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [o_orderpriority, count, Vnode(o_orderpriority) as _vnode] } + └─StreamHashAgg { group_key: [o_orderpriority], aggs: [count] } + └─StreamExchange { dist: HashShard(o_orderpriority) } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamFilter { predicate: (o_orderdate >= '1997-07-01':Date) AND (o_orderdate < '1997-10-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, _row_id] } + └─StreamFilter { predicate: (l_commitdate < l_receiptdate) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q5 before: - create_tables @@ -406,44 +410,45 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [n_name, sum($expr1), Vnode(n_name) as $expr2] } - └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(n_name) } - └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } - ├─StreamExchange { dist: HashShard(n_nationkey, n_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ └─StreamFilter { predicate: (r_name = 'MIDDLE EAST':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } - ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(o_custkey) } - │ │ └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1995-01-01 00:00:00':Timestamp) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } - ├─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(s_suppkey) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [n_name, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [n_name, sum($expr1), Vnode(n_name) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(n_name) } + └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } + ├─StreamExchange { dist: HashShard(n_nationkey, n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamFilter { predicate: (r_name = 'MIDDLE EAST':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } + ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1995-01-01 00:00:00':Timestamp) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q6 before: - create_tables @@ -513,47 +518,48 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2), Vnode(n_name, n_name, $expr1) as $expr3] } - └─StreamHashAgg [append_only] { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } - └─StreamProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, n_nationkey, _row_id, s_suppkey, _row_id, _row_id, n_nationkey, _row_id, c_custkey, l_orderkey] } - └─StreamFilter { predicate: (((n_name = 'ROMANIA':Varchar) AND (n_name = 'IRAN':Varchar)) OR ((n_name = 'IRAN':Varchar) AND (n_name = 'ROMANIA':Varchar))) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: all } - ├─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _row_id, n_nationkey, s_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ └─StreamShare { id: 3 } - │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamFilter { predicate: (l_shipdate >= '1983-01-01':Date) AND (l_shipdate <= '2000-12-31':Date) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_orderkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [n_name, o_orderkey, _row_id, _row_id, n_nationkey, c_custkey, _row_id] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, _row_id, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamShare { id: 3 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_nationkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2), Vnode(n_name, n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } + └─StreamProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, n_nationkey, _row_id, s_suppkey, _row_id, _row_id, n_nationkey, _row_id, c_custkey, l_orderkey] } + └─StreamFilter { predicate: (((n_name = 'ROMANIA':Varchar) AND (n_name = 'IRAN':Varchar)) OR ((n_name = 'IRAN':Varchar) AND (n_name = 'ROMANIA':Varchar))) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: all } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _row_id, n_nationkey, s_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1983-01-01':Date) AND (l_shipdate <= '2000-12-31':Date) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [n_name, o_orderkey, _row_id, _row_id, n_nationkey, c_custkey, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamShare { id: 3 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q8 before: - create_tables @@ -598,59 +604,60 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr1, $expr4] } - └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } - └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } - └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } - └─StreamExchange { dist: HashShard($expr1) } - └─StreamProject { exprs: [$expr1, Case((n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } - └─StreamProject { exprs: [Extract('YEAR':Varchar, o_orderdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [l_extendedprice, l_discount, o_orderdate, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, c_custkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [c_custkey, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ │ └─StreamFilter { predicate: (r_name = 'ASIA':Varchar) } - │ │ │ └─StreamRowIdGen { row_id_index: 6 } - │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ │ └─StreamShare { id: 6 } - │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_nationkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, l_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, p_partkey] } - │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ └─StreamShare { id: 6 } - │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } - │ │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ └─StreamFilter { predicate: (p_type = 'PROMO ANODIZED STEEL':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_partkey) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_orderkey) } - └─StreamFilter { predicate: (o_orderdate >= '1995-01-01':Date) AND (o_orderdate <= '1996-12-31':Date) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, $expr4] } + └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } + └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, Case((n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, o_orderdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [l_extendedprice, l_discount, o_orderdate, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, c_custkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [c_custkey, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamFilter { predicate: (r_name = 'ASIA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 6 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, p_partkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 6 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: (p_type = 'PROMO ANODIZED STEEL':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1995-01-01':Date) AND (o_orderdate <= '1996-12-31':Date) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q9 before: - create_tables @@ -690,43 +697,44 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, $expr1, sum($expr2)] } - └─StreamTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [n_name, $expr1, sum($expr2), Vnode(n_name, $expr1) as $expr3] } - └─StreamHashAgg [append_only] { group_key: [n_name, $expr1], aggs: [sum($expr2), count] } - └─StreamExchange { dist: HashShard(n_name, $expr1) } - └─StreamProject { exprs: [n_name, Extract('YEAR':Varchar, o_orderdate) as $expr1, ((l_extendedprice * (1:Decimal - l_discount)) - (ps_supplycost * l_quantity)) as $expr2, _row_id, _row_id, p_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, ps_suppkey, ps_partkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey AND ps_suppkey = l_suppkey AND ps_partkey = l_partkey AND ps_suppkey = s_suppkey, output: [l_quantity, l_extendedprice, l_discount, ps_supplycost, o_orderdate, n_name, _row_id, _row_id, p_partkey, ps_suppkey, ps_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } - ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id] } - │ ├─StreamExchange { dist: HashShard(p_partkey) } - │ │ └─StreamFilter { predicate: Like(p_name, '%yellow%':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ └─StreamFilter { predicate: IsNotNull(ps_suppkey) } - │ └─StreamRowIdGen { row_id_index: 8 } - │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey] } - ├─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_suppkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamFilter { predicate: IsNotNull(l_partkey) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_name, $expr1) } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2), Vnode(n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, $expr1) } + └─StreamProject { exprs: [n_name, Extract('YEAR':Varchar, o_orderdate) as $expr1, ((l_extendedprice * (1:Decimal - l_discount)) - (ps_supplycost * l_quantity)) as $expr2, _row_id, _row_id, p_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, ps_suppkey, ps_partkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey AND ps_suppkey = l_suppkey AND ps_partkey = l_partkey AND ps_suppkey = s_suppkey, output: [l_quantity, l_extendedprice, l_discount, ps_supplycost, o_orderdate, n_name, _row_id, _row_id, p_partkey, ps_suppkey, ps_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: Like(p_name, '%yellow%':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamFilter { predicate: IsNotNull(ps_suppkey) } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: IsNotNull(l_partkey) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q10 before: - create_tables @@ -765,32 +773,33 @@ LIMIT 20; stream_plan: |- StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } - └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } - └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment, Vnode(c_custkey) as $expr2] } - └─StreamHashAgg [append_only] { group_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], aggs: [sum($expr1), count] } - └─StreamProject { exprs: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment, (l_extendedprice * (1.00:Decimal - l_discount)) as $expr1, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, l_extendedprice, l_discount, n_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, _row_id, n_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(c_nationkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [l_extendedprice, l_discount, o_custkey, _row_id, l_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamFilter { predicate: (l_returnflag = 'R':Varchar) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_orderkey) } - └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1994-04-01 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment, Vnode(c_custkey) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], aggs: [sum($expr1), count] } + └─StreamProject { exprs: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment, (l_extendedprice * (1.00:Decimal - l_discount)) as $expr1, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, l_extendedprice, l_discount, n_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [l_extendedprice, l_discount, o_custkey, _row_id, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamFilter { predicate: (l_returnflag = 'R':Varchar) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1994-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q11 before: - create_tables @@ -826,50 +835,51 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } - └─StreamProject { exprs: [ps_partkey, sum($expr1)] } - └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [ps_partkey, sum($expr1), Vnode(ps_partkey) as _vnode] } - └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [ps_partkey, sum($expr1)] } - ├─StreamProject { exprs: [ps_partkey, sum($expr1)] } - │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(ps_partkey) } - │ └─StreamProject { exprs: [ps_partkey, (ps_supplycost * ps_availqty::Decimal) as $expr1, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } - │ └─StreamShare { id: 11 } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_nationkey) } - │ └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } - └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr2)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } - └─StreamProject { exprs: [(ps_supplycost * ps_availqty::Decimal) as $expr2, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } - └─StreamShare { id: 11 } - └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } - ├─StreamExchange { dist: HashShard(s_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamRowIdGen { row_id_index: 10 } - │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(n_nationkey) } - └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [ps_partkey, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [ps_partkey, sum($expr1), Vnode(ps_partkey) as _vnode] } + └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [ps_partkey, sum($expr1)] } + ├─StreamProject { exprs: [ps_partkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, (ps_supplycost * ps_availqty::Decimal) as $expr1, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + │ └─StreamShare { id: 11 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } + └─StreamProject { exprs: [(ps_supplycost * ps_availqty::Decimal) as $expr2, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + └─StreamShare { id: 11 } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_nationkey) } + └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q12 before: - create_tables @@ -905,22 +915,23 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } - └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2)] } - └─StreamTopN { order: [l_shipmode ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } - └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2), Vnode(l_shipmode) as $expr3] } - └─StreamHashAgg [append_only] { group_key: [l_shipmode], aggs: [sum($expr1), sum($expr2), count] } - └─StreamExchange { dist: HashShard(l_shipmode) } - └─StreamProject { exprs: [l_shipmode, Case(((o_orderpriority = '1-URGENT':Varchar) OR (o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((o_orderpriority <> '1-URGENT':Varchar) AND (o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, _row_id, _row_id, o_orderkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, l_shipmode, _row_id, o_orderkey, _row_id] } - ├─StreamExchange { dist: HashShard(o_orderkey) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamFilter { predicate: In(l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (l_commitdate < l_receiptdate) AND (l_shipdate < l_commitdate) AND (l_receiptdate >= '1994-01-01':Date) AND (l_receiptdate < '1995-01-01 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_shipmode) } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2)] } + └─StreamTopN { order: [l_shipmode ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2), Vnode(l_shipmode) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [l_shipmode], aggs: [sum($expr1), sum($expr2), count] } + └─StreamExchange { dist: HashShard(l_shipmode) } + └─StreamProject { exprs: [l_shipmode, Case(((o_orderpriority = '1-URGENT':Varchar) OR (o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((o_orderpriority <> '1-URGENT':Varchar) AND (o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, _row_id, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, l_shipmode, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: In(l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (l_commitdate < l_receiptdate) AND (l_shipdate < l_commitdate) AND (l_receiptdate >= '1994-01-01':Date) AND (l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q13 before: - create_tables @@ -948,24 +959,25 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } - └─StreamProject { exprs: [count(o_orderkey), count] } - └─StreamTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [count(o_orderkey), count, Vnode(count(o_orderkey)) as _vnode] } - └─StreamHashAgg { group_key: [count(o_orderkey)], aggs: [count] } - └─StreamExchange { dist: HashShard(count(o_orderkey)) } - └─StreamProject { exprs: [c_custkey, count(o_orderkey)] } - └─StreamHashAgg { group_key: [c_custkey], aggs: [count(o_orderkey), count] } - └─StreamHashJoin { type: LeftOuter, predicate: c_custkey = o_custkey, output: [c_custkey, o_orderkey, _row_id, _row_id] } - ├─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamRowIdGen { row_id_index: 11 } - │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(o_custkey) } - └─StreamProject { exprs: [o_orderkey, o_custkey, _row_id] } - └─StreamFilter { predicate: Not(Like(o_comment, '%:1%:2%':Varchar)) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count, count(o_orderkey)) } + └─StreamProject { exprs: [count(o_orderkey), count] } + └─StreamTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [count(o_orderkey), count, Vnode(count(o_orderkey)) as _vnode] } + └─StreamHashAgg { group_key: [count(o_orderkey)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(o_orderkey)) } + └─StreamProject { exprs: [c_custkey, count(o_orderkey)] } + └─StreamHashAgg { group_key: [c_custkey], aggs: [count(o_orderkey), count] } + └─StreamHashJoin { type: LeftOuter, predicate: c_custkey = o_custkey, output: [c_custkey, o_orderkey, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamProject { exprs: [o_orderkey, o_custkey, _row_id] } + └─StreamFilter { predicate: Not(Like(o_comment, '%:1%:2%':Varchar)) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q14 before: - create_tables @@ -1036,39 +1048,40 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, _row_id(hidden)], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } - └─StreamTopN { order: [s_suppkey ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, Vnode(sum($expr1)) as _vnode] } - └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } - ├─StreamExchange { dist: HashShard(sum($expr1)) } - │ └─StreamHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, l_suppkey] } - │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamShare { id: 9 } - │ └─StreamProject { exprs: [l_suppkey, sum($expr1)] } - │ └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } - │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } - │ └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } - └─StreamProject { exprs: [max(max(sum($expr1)))] } - └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } - └─StreamExchange { dist: Single } - └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } - └─StreamProject { exprs: [l_suppkey, sum($expr1), Vnode(l_suppkey) as _vnode] } - └─StreamShare { id: 9 } - └─StreamProject { exprs: [l_suppkey, sum($expr1)] } - └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(l_suppkey) } - └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } - └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + └─StreamTopN { order: [s_suppkey ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, Vnode(sum($expr1)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + ├─StreamExchange { dist: HashShard(sum($expr1)) } + │ └─StreamHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, l_suppkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + │ └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } + └─StreamProject { exprs: [max(max(sum($expr1)))] } + └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } + └─StreamProject { exprs: [l_suppkey, sum($expr1), Vnode(l_suppkey) as _vnode] } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q16 before: - create_tables @@ -1106,28 +1119,29 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } - └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey)] } - └─StreamTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey), Vnode(p_brand, p_type, p_size) as $expr1] } - └─StreamHashAgg { group_key: [p_brand, p_type, p_size], aggs: [count(distinct ps_suppkey), count] } - └─StreamExchange { dist: HashShard(p_brand, p_type, p_size) } - └─StreamHashJoin { type: LeftAnti, predicate: ps_suppkey = s_suppkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, _row_id, ps_partkey] } - ├─StreamExchange { dist: HashShard(ps_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_partkey = p_partkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, ps_partkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(ps_partkey) } - │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(p_partkey) } - │ └─StreamFilter { predicate: (p_brand <> 'Brand#45':Varchar) AND Not(Like(p_type, 'SMALL PLATED%':Varchar)) AND In(p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(s_suppkey) } - └─StreamProject { exprs: [s_suppkey, _row_id] } - └─StreamFilter { predicate: Like(s_comment, '%Customer%Complaints%':Varchar) } - └─StreamRowIdGen { row_id_index: 10 } - └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count(distinct ps_suppkey), p_brand, p_type, p_size) } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey)] } + └─StreamTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey), Vnode(p_brand, p_type, p_size) as $expr1] } + └─StreamHashAgg { group_key: [p_brand, p_type, p_size], aggs: [count(distinct ps_suppkey), count] } + └─StreamExchange { dist: HashShard(p_brand, p_type, p_size) } + └─StreamHashJoin { type: LeftAnti, predicate: ps_suppkey = s_suppkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_partkey = p_partkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, ps_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamFilter { predicate: (p_brand <> 'Brand#45':Varchar) AND Not(Like(p_type, 'SMALL PLATED%':Varchar)) AND In(p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamProject { exprs: [s_suppkey, _row_id] } + └─StreamFilter { predicate: Like(s_comment, '%Customer%Complaints%':Varchar) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q17 before: - create_tables @@ -1216,36 +1230,37 @@ LIMIT 100; stream_plan: |- StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } - └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)] } - └─StreamTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } - └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity), Vnode(o_orderkey) as $expr1] } - └─StreamHashAgg { group_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], aggs: [sum(l_quantity), count] } - └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: all } - ├─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, l_quantity, _row_id, _row_id, _row_id] } - │ ├─StreamExchange { dist: HashShard(o_orderkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, _row_id, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(c_custkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 11 } - │ │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(o_custkey) } - │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamShare { id: 9 } - │ └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamProject { exprs: [l_orderkey] } - └─StreamFilter { predicate: (sum(l_quantity) > 1:Decimal) } - └─StreamProject { exprs: [l_orderkey, sum(l_quantity)] } - └─StreamHashAgg [append_only] { group_key: [l_orderkey], aggs: [sum(l_quantity), count] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamShare { id: 9 } - └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_totalprice, o_orderdate) } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)] } + └─StreamTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity), Vnode(o_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], aggs: [sum(l_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: all } + ├─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, l_quantity, _row_id, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [l_orderkey] } + └─StreamFilter { predicate: (sum(l_quantity) > 1:Decimal) } + └─StreamProject { exprs: [l_orderkey, sum(l_quantity)] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey], aggs: [sum(l_quantity), count] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q19 before: - create_tables @@ -1308,42 +1323,43 @@ sql: "select\n\ts_name,\n\ts_address\nfrom\n\tsupplier,\n\tnation\nwhere\n\ts_suppkey in (\n\t\tselect\n\t\t\tps_suppkey\n\t\tfrom\n\t\t\tpartsupp,\n\t\t\t(\n\t\t\t\tselect\n\t\t\t\t\tl_partkey agg_partkey,\n\t\t\t\t\tl_suppkey agg_suppkey,\n\t\t\t\t\t0.5 * sum(l_quantity) AS agg_quantity\n\t\t\t\tfrom\n\t\t\t\t\tlineitem\n\t\t\t\twhere\n\t\t\t\t\tl_shipdate >= date '1994-01-01'\n\t\t\t\t\tand l_shipdate < date '1994-01-01' + interval '1' year\n\t\t\t\tgroup by\n\t\t\t\t\tl_partkey,\n\t\t\t\t\tl_suppkey\n\t\t\t) agg_lineitem\n\t\twhere\n\t\t\tagg_partkey = ps_partkey\n\t\t\tand agg_suppkey = ps_suppkey\n\t\t\tand ps_partkey in (\n\t\t\t\tselect\n\t\t\t\t\tp_partkey\n\t\t\t\tfrom\n\t\t\t\t\tpart\n\t\t\t\twhere\n\t\t\t\t\tp_name like 'forest%'\n\t\t\t)\n\t\t\tand ps_availqty > agg_quantity\n\t)\n\tand s_nationkey = n_nationkey\n\tand n_name = 'KENYA'\norder by\n\ts_name\nLIMIT 1;\n" stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } - └─StreamTopN { order: [s_name ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey, Vnode(s_suppkey) as _vnode] } - └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } - ├─StreamExchange { dist: HashShard(s_suppkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(s_nationkey) } - │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(n_nationkey) } - │ └─StreamFilter { predicate: (n_name = 'KENYA':Varchar) } - │ └─StreamRowIdGen { row_id_index: 7 } - │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(ps_suppkey) } - └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_suppkey, _row_id, ps_partkey] } - ├─StreamExchange { dist: HashShard(ps_partkey) } - │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, _row_id, l_partkey, l_suppkey] } - │ └─StreamFilter { predicate: ($expr1 > $expr2) } - │ └─StreamHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey, output: all } - │ ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } - │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 8 } - │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamProject { exprs: [l_partkey, l_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } - │ └─StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } - │ └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } - │ └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(p_partkey) } - └─StreamProject { exprs: [p_partkey, _row_id] } - └─StreamFilter { predicate: Like(p_name, 'forest%':Varchar) } - └─StreamRowIdGen { row_id_index: 12 } - └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + └─StreamTopN { order: [s_name ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey, Vnode(s_suppkey) as _vnode] } + └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'KENYA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_suppkey, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, _row_id, l_partkey, l_suppkey] } + │ └─StreamFilter { predicate: ($expr1 > $expr2) } + │ └─StreamHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey, output: all } + │ ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamProject { exprs: [l_partkey, l_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } + │ └─StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } + │ └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamProject { exprs: [p_partkey, _row_id] } + └─StreamFilter { predicate: Like(p_name, 'forest%':Varchar) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q21 before: - create_tables @@ -1390,51 +1406,52 @@ LIMIT 100; stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [s_name, count] } - └─StreamTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [s_name, count, Vnode(s_name) as _vnode] } - └─StreamHashAgg { group_key: [s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } - ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } - │ ├─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } - │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } - │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ │ └─StreamFilter { predicate: (n_name = 'GERMANY':Varchar) } - │ │ │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 10 } - │ │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } - │ │ │ └─StreamFilter { predicate: (o_orderstatus = 'F':Varchar) } - │ │ │ └─StreamRowIdGen { row_id_index: 12 } - │ │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ └─StreamShare { id: 13 } - │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 19 } - │ │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - │ └─StreamShare { id: 13 } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ └─StreamRowIdGen { row_id_index: 19 } - │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └─StreamShare { id: 13 } - └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - └─StreamRowIdGen { row_id_index: 19 } - └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(count, s_name) } + └─StreamProject { exprs: [s_name, count] } + └─StreamTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, count, Vnode(s_name) as _vnode] } + └─StreamHashAgg { group_key: [s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + │ ├─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } + │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ │ └─StreamFilter { predicate: (n_name = 'GERMANY':Varchar) } + │ │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ │ └─StreamFilter { predicate: (o_orderstatus = 'F':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + │ │ └─StreamShare { id: 13 } + │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 19 } + │ │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + │ └─StreamShare { id: 13 } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + └─StreamShare { id: 13 } + └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - id: tpch_q22 before: - create_tables @@ -1479,35 +1496,36 @@ LIMIT 1; stream_plan: |- StreamMaterialize { columns: [cntrycode, numcust, totacctbal], stream_key: [], pk_columns: [cntrycode], pk_conflict: NoCheck } - └─StreamProject { exprs: [$expr2, count, sum(c_acctbal)] } - └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } - └─StreamExchange { dist: Single } - └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } - └─StreamProject { exprs: [$expr2, count, sum(c_acctbal), Vnode($expr2) as _vnode] } - └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(c_acctbal)] } - └─StreamExchange { dist: HashShard($expr2) } - └─StreamProject { exprs: [Substr(c_phone, 1:Int32, 2:Int32) as $expr2, c_acctbal, _row_id, c_custkey] } - └─StreamDynamicFilter { predicate: (c_acctbal > $expr1), output: [c_phone, c_acctbal, _row_id, c_custkey] } - ├─StreamHashJoin { type: LeftAnti, predicate: c_custkey = o_custkey, output: [c_phone, c_acctbal, _row_id, c_custkey] } - │ ├─StreamExchange { dist: HashShard(c_custkey) } - │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - │ │ └─StreamShare { id: 4 } - │ │ └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } - │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - │ │ └─StreamRowIdGen { row_id_index: 11 } - │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - │ └─StreamExchange { dist: HashShard(o_custkey) } - │ └─StreamProject { exprs: [o_custkey, _row_id] } - │ └─StreamRowIdGen { row_id_index: 12 } - │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } - └─StreamExchange { dist: Broadcast } - └─StreamProject { exprs: [(sum(sum(c_acctbal)) / sum0(count(c_acctbal))::Decimal) as $expr1] } - └─StreamSimpleAgg [append_only] { aggs: [sum(sum(c_acctbal)), sum0(count(c_acctbal)), count] } - └─StreamExchange { dist: Single } - └─StreamStatelessSimpleAgg { aggs: [sum(c_acctbal), count(c_acctbal)] } - └─StreamFilter { predicate: (c_acctbal > 0.00:Decimal) AND In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─StreamShare { id: 4 } - └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } - └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } - └─StreamRowIdGen { row_id_index: 11 } - └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal)] } + └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal), Vnode($expr2) as _vnode] } + └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(c_acctbal)] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [Substr(c_phone, 1:Int32, 2:Int32) as $expr2, c_acctbal, _row_id, c_custkey] } + └─StreamDynamicFilter { predicate: (c_acctbal > $expr1), output: [c_phone, c_acctbal, _row_id, c_custkey] } + ├─StreamHashJoin { type: LeftAnti, predicate: c_custkey = o_custkey, output: [c_phone, c_acctbal, _row_id, c_custkey] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamShare { id: 4 } + │ │ └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamProject { exprs: [o_custkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(c_acctbal)) / sum0(count(c_acctbal))::Decimal) as $expr1] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum(c_acctbal)), sum0(count(c_acctbal)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(c_acctbal), count(c_acctbal)] } + └─StreamFilter { predicate: (c_acctbal > 0.00:Decimal) AND In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamShare { id: 4 } + └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamRowIdGen { row_id_index: 11 } + └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml index 772ff95b58f9e..0c5612ba8dc3a 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -209,7 +209,7 @@ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(p_partkey, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey) } + └─StreamExchange { dist: HashShard(s_acctbal, n_name, s_name, p_partkey) } └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } ├─StreamExchange { dist: HashShard(ps_partkey, min(ps_supplycost)) } │ └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id] } @@ -275,7 +275,7 @@ Fragment 0 StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([3, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17]) from 1 + └── StreamExchange Hash([0, 2, 1, 3]) from 1 Fragment 1 StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } @@ -451,7 +451,7 @@ Table 37 { columns: [ n_regionkey, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, _rw_timestamp ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], read pk prefix len hint: 14 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, _rw_timestamp ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 0, 2, 1, 3 ], read pk prefix len hint: 14 } - id: tpch_q5 before: @@ -537,96 +537,100 @@ └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [n_name, revenue], stream_key: [n_name], pk_columns: [revenue, n_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [n_name, sum($expr1)] } - └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } - └─StreamExchange { dist: HashShard(n_name) } - └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } - └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } - ├─StreamExchange { dist: HashShard(n_nationkey, n_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(r_regionkey) } - │ │ └─StreamRowIdGen { row_id_index: 3 } - │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(n_regionkey) } - │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } - │ └─StreamRowIdGen { row_id_index: 4 } - │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } - ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } - │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } - │ ├─StreamExchange { dist: HashShard(o_custkey) } - │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(c_custkey) } - │ └─StreamRowIdGen { row_id_index: 8 } - │ └─StreamSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } - └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } - ├─StreamExchange { dist: HashShard(l_suppkey) } - │ └─StreamRowIdGen { row_id_index: 16 } - │ └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } - └─StreamExchange { dist: HashShard(s_suppkey) } - └─StreamRowIdGen { row_id_index: 7 } - └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + └─StreamExchange { dist: HashShard(sum($expr1)) } + └─StreamProject { exprs: [n_name, sum($expr1)] } + └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(n_name) } + └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } + ├─StreamExchange { dist: HashShard(n_nationkey, n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamRowIdGen { row_id_index: 3 } + │ │ └─StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 4 } + │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } + ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamRowIdGen { row_id_index: 16 } + │ └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [n_name, revenue], stream_key: [n_name], pk_columns: [revenue, n_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamProject { exprs: [n_name, sum($expr1)] } - └── StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([1]) from 1 Fragment 1 + StreamProject { exprs: [n_name, sum($expr1)] } + └── StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0]) from 2 + + Fragment 2 StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } └── StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] - ├── StreamExchange Hash([0, 0]) from 2 - └── StreamExchange Hash([0, 3]) from 5 + ├── StreamExchange Hash([0, 0]) from 3 + └── StreamExchange Hash([0, 3]) from 6 - Fragment 2 + Fragment 3 StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } { tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] } - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([2]) from 4 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([2]) from 5 - Fragment 3 + Fragment 4 StreamRowIdGen { row_id_index: 3 } └── StreamSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id] } { tables: [ Source: 9 ] } - Fragment 4 + Fragment 5 StreamFilter { predicate: IsNotNull(n_nationkey) } └── StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { tables: [ Source: 10 ] } - Fragment 5 + Fragment 6 StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } ├── tables: [ HashJoinLeft: 11, HashJoinDegreeLeft: 12, HashJoinRight: 13, HashJoinDegreeRight: 14 ] - ├── StreamExchange Hash([0, 1]) from 6 - └── StreamExchange Hash([0, 3]) from 9 + ├── StreamExchange Hash([0, 1]) from 7 + └── StreamExchange Hash([0, 3]) from 10 - Fragment 6 + Fragment 7 StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } { tables: [ HashJoinLeft: 15, HashJoinDegreeLeft: 16, HashJoinRight: 17, HashJoinDegreeRight: 18 ] } - ├── StreamExchange Hash([1]) from 7 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([1]) from 8 + └── StreamExchange Hash([0]) from 9 - Fragment 7 + Fragment 8 StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } { tables: [ Source: 19 ] } - Fragment 8 + Fragment 9 StreamRowIdGen { row_id_index: 8 } └── StreamSource { source: customer, columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _row_id] } { tables: [ Source: 20 ] } - Fragment 9 + Fragment 10 StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } { tables: [ HashJoinLeft: 21, HashJoinDegreeLeft: 22, HashJoinRight: 23, HashJoinDegreeRight: 24 ] } - ├── StreamExchange Hash([2]) from 10 - └── StreamExchange Hash([0]) from 11 + ├── StreamExchange Hash([2]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 10 + Fragment 11 StreamRowIdGen { row_id_index: 16 } └── StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } { tables: [ Source: 25 ] } - Fragment 11 + Fragment 12 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { tables: [ Source: 26 ] } @@ -704,7 +708,7 @@ Table 26 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ n_name, revenue, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 2 } - id: tpch_q7 before: @@ -1676,7 +1680,7 @@ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(_row_id, _row_id, s_nationkey, s_suppkey) } + └─StreamExchange { dist: HashShard(s_name) } └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } ├─StreamExchange { dist: HashShard(s_suppkey) } │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } @@ -1709,7 +1713,7 @@ Fragment 0 StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [_row_id, _row_id#1, s_nationkey, s_suppkey], pk_columns: [s_name, _row_id, _row_id#1, s_nationkey, s_suppkey], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamExchange Hash([2, 3, 4, 5]) from 1 + └── StreamExchange Hash([0]) from 1 Fragment 1 StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } @@ -1802,7 +1806,7 @@ Table 21 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2, 3, 4, 5 ], read pk prefix len hint: 5 } + Table 4294967294 { columns: [ s_name, s_address, _row_id, _row_id#1, s_nationkey, s_suppkey, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 5 } - id: tpch_q21 before: @@ -1912,108 +1916,112 @@ └─BatchSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_plan: |- StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } - └─StreamHashAgg { group_key: [s_name], aggs: [count] } - └─StreamExchange { dist: HashShard(s_name) } - └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } - ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } - │ ├─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } - │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } - │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } - │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } - │ │ │ │ └─StreamRowIdGen { row_id_index: 4 } - │ │ │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } - │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 7 } - │ │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(l_suppkey) } - │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } - │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } - │ │ │ └─StreamRowIdGen { row_id_index: 9 } - │ │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } - │ │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - │ │ └─StreamShare { id: 11 } - │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ │ └─StreamRowIdGen { row_id_index: 16 } - │ │ └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } - │ └─StreamExchange { dist: HashShard(l_orderkey) } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - │ └─StreamShare { id: 11 } - │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - │ └─StreamRowIdGen { row_id_index: 16 } - │ └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } - └─StreamExchange { dist: HashShard(l_orderkey) } - └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └─StreamShare { id: 11 } - └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } - └─StreamRowIdGen { row_id_index: 16 } - └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + └─StreamExchange { dist: HashShard(count, s_name) } + └─StreamHashAgg { group_key: [s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + │ ├─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } + │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ │ └─StreamRowIdGen { row_id_index: 4 } + │ │ │ │ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } + │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 9 } + │ │ │ └─StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + │ │ └─StreamShare { id: 11 } + │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 16 } + │ │ └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + │ └─StreamShare { id: 11 } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ └─StreamRowIdGen { row_id_index: 16 } + │ └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + └─StreamShare { id: 11 } + └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + └─StreamRowIdGen { row_id_index: 16 } + └─StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } stream_dist_plan: |+ Fragment 0 StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } ├── tables: [ Materialize: 4294967294 ] - └── StreamHashAgg { group_key: [s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } - └── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([1, 0]) from 1 Fragment 1 + StreamHashAgg { group_key: [s_name], aggs: [count] } { tables: [ HashAggState: 0 ] } + └── StreamExchange Hash([0]) from 2 + + Fragment 2 StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } ├── tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] ├── StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } │ ├── tables: [ HashJoinLeft: 5, HashJoinDegreeLeft: 6, HashJoinRight: 7, HashJoinDegreeRight: 8 ] - │ ├── StreamExchange Hash([1]) from 2 - │ └── StreamExchange Hash([0]) from 10 - └── StreamExchange Hash([0]) from 11 + │ ├── StreamExchange Hash([1]) from 3 + │ └── StreamExchange Hash([0]) from 11 + └── StreamExchange Hash([0]) from 12 - Fragment 2 + Fragment 3 StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } ├── tables: [ HashJoinLeft: 9, HashJoinDegreeLeft: 10, HashJoinRight: 11, HashJoinDegreeRight: 12 ] - ├── StreamExchange Hash([0]) from 3 - └── StreamExchange Hash([1]) from 6 + ├── StreamExchange Hash([0]) from 4 + └── StreamExchange Hash([1]) from 7 - Fragment 3 + Fragment 4 StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } ├── tables: [ HashJoinLeft: 13, HashJoinDegreeLeft: 14, HashJoinRight: 15, HashJoinDegreeRight: 16 ] - ├── StreamExchange Hash([0]) from 4 - └── StreamExchange Hash([3]) from 5 + ├── StreamExchange Hash([0]) from 5 + └── StreamExchange Hash([3]) from 6 - Fragment 4 + Fragment 5 StreamRowIdGen { row_id_index: 4 } └── StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } { tables: [ Source: 17 ] } - Fragment 5 + Fragment 6 StreamRowIdGen { row_id_index: 7 } └── StreamSource { source: supplier, columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } { tables: [ Source: 18 ] } - Fragment 6 + Fragment 7 StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } ├── tables: [ HashJoinLeft: 19, HashJoinDegreeLeft: 20, HashJoinRight: 21, HashJoinDegreeRight: 22 ] - ├── StreamExchange Hash([0]) from 7 - └── StreamExchange Hash([0]) from 8 + ├── StreamExchange Hash([0]) from 8 + └── StreamExchange Hash([0]) from 9 - Fragment 7 + Fragment 8 StreamRowIdGen { row_id_index: 9 } └── StreamSource { source: orders, columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _row_id] } { tables: [ Source: 23 ] } - Fragment 8 + Fragment 9 StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └── StreamExchange NoShuffle from 9 + └── StreamExchange NoShuffle from 10 - Fragment 9 + Fragment 10 StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } └── StreamRowIdGen { row_id_index: 16 } └── StreamSource { source: lineitem, columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _row_id] } └── tables: [ Source: 24 ] - Fragment 10 + Fragment 11 StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } - └── StreamExchange NoShuffle from 9 + └── StreamExchange NoShuffle from 10 - Fragment 11 + Fragment 12 StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } └── StreamFilter { predicate: (l_receiptdate > l_commitdate) } - └── StreamExchange NoShuffle from 9 + └── StreamExchange NoShuffle from 10 Table 0 { columns: [ s_name, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2090,5 +2098,5 @@ Table 24 { columns: [ partition_id, offset_info, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 4294967294 { columns: [ s_name, numwait, _rw_timestamp ], primary key: [ $1 DESC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } diff --git a/src/frontend/src/catalog/source_catalog.rs b/src/frontend/src/catalog/source_catalog.rs index bf02092c1cc0a..c4856e329d605 100644 --- a/src/frontend/src/catalog/source_catalog.rs +++ b/src/frontend/src/catalog/source_catalog.rs @@ -142,6 +142,15 @@ impl SourceCatalog { self.create_sql_ast() } + + /// Fills the `definition` field with the purified SQL definition. + /// + /// There's no need to call this method for correctness because we automatically purify the + /// SQL definition at the time of querying. However, this helps to maintain more accurate + /// `definition` field in the catalog when directly inspected for debugging purposes. + pub fn fill_purified_create_sql(&mut self) { + self.definition = self.create_sql_purified(); + } } impl From<&PbSource> for SourceCatalog { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs index e5276c4f4f6bc..a4b3eb18387d8 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_relation_info.rs @@ -174,7 +174,7 @@ async fn read_relation_info(reader: &SysCatalogReaderImpl) -> Result unreachable!(), } // update version catalog.version += 1; + catalog.fill_purified_create_sql(); let catalog_writer = session.catalog_writer()?; if catalog.info.is_shared() { @@ -148,27 +144,6 @@ pub async fn handle_alter_source_column( Ok(PgResponse::empty_result(StatementType::ALTER_SOURCE)) } -/// `alter_definition_add_column` adds a new column to the definition of the relation. -#[inline(always)] -pub fn alter_definition_add_column(definition: &str, column: ColumnDef) -> Result { - let ast = Parser::parse_sql(definition).expect("failed to parse relation definition"); - let mut stmt = ast - .into_iter() - .exactly_one() - .expect("should contains only one statement"); - - match &mut stmt { - Statement::CreateSource { - stmt: CreateSourceStatement { columns, .. }, - } => { - columns.push(column); - } - _ => unreachable!(), - } - - Ok(stmt.to_string()) -} - #[cfg(test)] pub mod tests { use std::collections::BTreeMap; diff --git a/src/frontend/src/handler/alter_source_with_sr.rs b/src/frontend/src/handler/alter_source_with_sr.rs index e90a250e34bfa..472ce8772b99b 100644 --- a/src/frontend/src/handler/alter_source_with_sr.rs +++ b/src/frontend/src/handler/alter_source_with_sr.rs @@ -14,7 +14,6 @@ use std::sync::Arc; -use anyhow::Context; use either::Either; use itertools::Itertools; use pgwire::pg_response::StatementType; @@ -28,7 +27,6 @@ use risingwave_sqlparser::ast::{ CompatibleFormatEncode, CreateSourceStatement, Encode, Format, FormatEncodeOptions, ObjectName, SqlOption, Statement, }; -use risingwave_sqlparser::parser::Parser; use super::create_source::{ generate_stream_graph_for_source, schema_has_schema_registry, validate_compatibility, @@ -198,10 +196,7 @@ pub async fn refresh_sr_and_get_columns_diff( } fn get_format_encode_from_source(source: &SourceCatalog) -> Result { - let [stmt]: [_; 1] = Parser::parse_sql(&source.definition) - .context("unable to parse original source definition")? - .try_into() - .unwrap(); + let stmt = source.create_sql_ast()?; let Statement::CreateSource { stmt: CreateSourceStatement { format_encode, .. }, } = stmt @@ -263,8 +258,10 @@ pub async fn handle_alter_source_with_sr( source.info = source_info; source.columns.extend(added_columns); - source.definition = - alter_definition_format_encode(&source.definition, format_encode.row_options.clone())?; + source.definition = alter_definition_format_encode( + source.create_sql_ast_purified()?, + format_encode.row_options.clone(), + )?; let (format_encode_options, format_encode_secret_ref) = resolve_secret_ref_in_with_options( WithOptions::try_from(format_encode.row_options())?, @@ -313,15 +310,9 @@ pub async fn handle_alter_source_with_sr( /// Apply the new `format_encode_options` to the source/table definition. pub fn alter_definition_format_encode( - definition: &str, + mut stmt: Statement, format_encode_options: Vec, ) -> Result { - let ast = Parser::parse_sql(definition).expect("failed to parse relation definition"); - let mut stmt = ast - .into_iter() - .exactly_one() - .expect("should contain only one statement"); - match &mut stmt { Statement::CreateSource { stmt: CreateSourceStatement { format_encode, .. }, @@ -391,6 +382,9 @@ pub mod tests { .clone() }; + let source = get_source(); + expect_test::expect!["CREATE SOURCE src (id INT, country STRUCT
, zipcode CHARACTER VARYING>, zipcode BIGINT, rate REAL) WITH (connector = 'kafka', topic = 'test-topic', properties.bootstrap.server = 'localhost:29092') FORMAT PLAIN ENCODE PROTOBUF (message = '.test.TestRecord', schema.location = 'file://')"].assert_eq(&source.create_sql_purified().replace(proto_file.path().to_str().unwrap(), "")); + let sql = format!( r#"ALTER SOURCE src FORMAT UPSERT ENCODE PROTOBUF ( message = '.test.TestRecord', @@ -434,10 +428,6 @@ pub mod tests { .unwrap(); assert_eq!(name_column.column_desc.data_type, DataType::Varchar); - let altered_sql = format!( - r#"CREATE SOURCE src WITH (connector = 'kafka', topic = 'test-topic', properties.bootstrap.server = 'localhost:29092') FORMAT PLAIN ENCODE PROTOBUF (message = '.test.TestRecordExt', schema.location = 'file://{}')"#, - proto_file.path().to_str().unwrap() - ); - assert_eq!(altered_sql, altered_source.definition); + expect_test::expect!["CREATE SOURCE src (id INT, country STRUCT
, zipcode CHARACTER VARYING>, zipcode BIGINT, rate REAL, name CHARACTER VARYING) WITH (connector = 'kafka', topic = 'test-topic', properties.bootstrap.server = 'localhost:29092') FORMAT PLAIN ENCODE PROTOBUF (message = '.test.TestRecordExt', schema.location = 'file://')"].assert_eq(&altered_source.create_sql_purified().replace(proto_file.path().to_str().unwrap(), "")); } } diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 1710223fed3e1..06f9e80f6e4f8 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -128,6 +128,7 @@ pub fn gen_create_mv_plan_bound( } let mut plan_root = Planner::new_for_stream(context).plan_query(query)?; + plan_root.set_req_dist_as_same_as_req_order(); if let Some(col_names) = col_names { for name in &col_names { check_valid_column_name(name)?; diff --git a/src/frontend/src/handler/drop_database.rs b/src/frontend/src/handler/drop_database.rs index 490ead2ebe2b8..ae5fab5443049 100644 --- a/src/frontend/src/handler/drop_database.rs +++ b/src/frontend/src/handler/drop_database.rs @@ -30,9 +30,10 @@ pub async fn handle_drop_database( let catalog_reader = session.env().catalog_reader(); let database_name = Binder::resolve_database_name(database_name)?; if session.database() == database_name { - return Err( - ErrorCode::InternalError("cannot drop the currently open database".to_owned()).into(), - ); + return Err(ErrorCode::PermissionDenied( + "cannot drop the currently open database".to_owned(), + ) + .into()); } if mode.is_some() { return Err(ErrorCode::BindError("Drop database not support drop mode".to_owned()).into()); diff --git a/src/frontend/src/handler/drop_user.rs b/src/frontend/src/handler/drop_user.rs index b18d397a73f5d..f2d89f6b5fa61 100644 --- a/src/frontend/src/handler/drop_user.rs +++ b/src/frontend/src/handler/drop_user.rs @@ -34,12 +34,42 @@ pub async fn handle_drop_user( let user_name = Binder::resolve_user_name(user_name)?; let user_info_reader = session.env().user_info_reader(); - let user_id = user_info_reader + let user_info = user_info_reader .read_guard() .get_user_by_name(&user_name) - .map(|u| u.id); - match user_id { - Some(user_id) => { + .map(|u| (u.id, u.is_super)); + match user_info { + Some((user_id, is_super)) => { + if session.user_id() == user_id { + return Err(ErrorCode::PermissionDenied( + "current user cannot be dropped".to_owned(), + ) + .into()); + } + if let Some(current_user) = user_info_reader + .read_guard() + .get_user_by_name(&session.user_name()) + { + if !current_user.is_super { + if is_super { + return Err(ErrorCode::PermissionDenied( + "must be superuser to drop superusers".to_owned(), + ) + .into()); + } + if !current_user.can_create_user { + return Err(ErrorCode::PermissionDenied( + "permission denied to drop user".to_owned(), + ) + .into()); + } + } + } else { + return Err( + ErrorCode::PermissionDenied("Session user is invalid".to_owned()).into(), + ); + } + let user_info_writer = session.user_info_writer()?; user_info_writer.drop_user(user_id).await?; } diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index df241d76135a8..d60f435648b3a 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -192,6 +192,18 @@ impl PlanRoot { Ok(()) } + pub fn set_req_dist_as_same_as_req_order(&mut self) { + if self.required_order.len() != 0 { + let dist = self + .required_order + .column_orders + .iter() + .map(|o| o.column_index) + .collect_vec(); + self.required_dist = RequiredDist::hash_shard(&dist) + } + } + /// Get the plan root's schema, only including the fields to be output. pub fn schema(&self) -> Schema { // The schema can be derived from the `out_fields` and `out_names`, so we don't maintain it diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index a13a8f0a313e4..6b9aa1ca43bef 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -180,24 +180,33 @@ impl StreamMaterialize { user_distributed_by } TableType::MaterializedView => { - assert_matches!(user_distributed_by, RequiredDist::Any); - // ensure the same pk will not shuffle to different node - let required_dist = - RequiredDist::shard_by_key(input.schema().len(), input.expect_stream_key()); - - // If the input is a stream join, enforce the stream key as the materialized - // view distribution key to avoid slow backfilling caused by - // data skew of the dimension table join key. - // See for more information. - let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join)) - || matches!(input.as_stream_temporal_join(), Some(_join)) - || matches!(input.as_stream_delta_join(), Some(_join)); - - if is_stream_join { - return Ok(required_dist.enforce(input, &Order::any())); + match user_distributed_by { + RequiredDist::PhysicalDist(Distribution::HashShard(_)) => { + user_distributed_by + } + RequiredDist::Any => { + // ensure the same pk will not shuffle to different node + let required_dist = RequiredDist::shard_by_key( + input.schema().len(), + input.expect_stream_key(), + ); + + // If the input is a stream join, enforce the stream key as the materialized + // view distribution key to avoid slow backfilling caused by + // data skew of the dimension table join key. + // See for more information. + let is_stream_join = matches!(input.as_stream_hash_join(), Some(_join)) + || matches!(input.as_stream_temporal_join(), Some(_join)) + || matches!(input.as_stream_delta_join(), Some(_join)); + + if is_stream_join { + return Ok(required_dist.enforce(input, &Order::any())); + } + + required_dist + } + _ => unreachable!("{:?}", user_distributed_by), } - - required_dist } TableType::Index => { assert_matches!( diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index ac1e200c9367c..efbf36b0e386c 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -374,10 +374,6 @@ pub fn start( .meta .developer .hummock_time_travel_sst_info_insert_batch_size, - hummock_delta_log_delete_batch_size: config - .meta - .developer - .hummock_delta_log_delete_batch_size, hummock_time_travel_epoch_version_insert_batch_size: config .meta .developer diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index c470207a2462b..bde42304d58bd 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -463,7 +463,7 @@ fn timestamp_now_sec() -> u64 { fn meta_node_info(host: &str, started_at: Option) -> PbWorkerNode { PbWorkerNode { id: META_NODE_ID, - r#type: WorkerType::Meta as _, + r#type: PbWorkerType::Meta.into(), host: HostAddr::try_from(host) .as_ref() .map(HostAddr::to_protobuf) diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index 334fd6fceead9..5ba75e75382a5 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -28,7 +28,7 @@ use risingwave_hummock_sdk::{ get_object_id_from_path, get_sst_data_path, HummockSstableObjectId, OBJECT_SUFFIX, }; use risingwave_meta_model::hummock_sequence::HUMMOCK_NOW; -use risingwave_meta_model::{hummock_gc_history, hummock_sequence}; +use risingwave_meta_model::{hummock_gc_history, hummock_sequence, hummock_version_delta}; use risingwave_meta_model_migration::OnConflict; use risingwave_object_store::object::{ObjectMetadataIter, ObjectStoreRef}; use risingwave_pb::stream_service::GetMinUncommittedSstIdRequest; @@ -37,10 +37,8 @@ use sea_orm::{ActiveValue, ColumnTrait, EntityTrait, QueryFilter, Set}; use crate::backup_restore::BackupManagerRef; use crate::hummock::error::{Error, Result}; -use crate::hummock::manager::commit_multi_var; use crate::hummock::HummockManager; use crate::manager::MetadataManager; -use crate::model::BTreeMapTransaction; use crate::MetaResult; pub(crate) struct GcManager { @@ -175,44 +173,36 @@ impl GcManager { } impl HummockManager { - /// Deletes at most `batch_size` deltas. + /// Deletes version deltas. /// - /// Returns (number of deleted deltas, number of remain `deltas_to_delete`). - pub async fn delete_version_deltas(&self, batch_size: usize) -> Result<(usize, usize)> { + /// Returns number of deleted deltas + pub async fn delete_version_deltas(&self) -> Result { let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); let context_info = self.context_info.read().await; - let deltas_to_delete_count = versioning - .hummock_version_deltas - .range(..=versioning.checkpoint.version.id) - .count(); // If there is any safe point, skip this to ensure meta backup has required delta logs to // replay version. if !context_info.version_safe_points.is_empty() { - return Ok((0, deltas_to_delete_count)); + return Ok(0); } - let batch = versioning + // The context_info lock must be held to prevent any potential metadata backup. + // The lock order requires version lock to be held as well. + let version_id = versioning.checkpoint.version.id; + let res = hummock_version_delta::Entity::delete_many() + .filter(hummock_version_delta::Column::Id.lte(version_id.to_u64())) + .exec(&self.env.meta_store_ref().conn) + .await?; + tracing::debug!(rows_affected = res.rows_affected, "Deleted version deltas"); + versioning .hummock_version_deltas - .range(..=versioning.checkpoint.version.id) - .map(|(k, _)| *k) - .take(batch_size) - .collect_vec(); - let mut hummock_version_deltas = - BTreeMapTransaction::new(&mut versioning.hummock_version_deltas); - if batch.is_empty() { - return Ok((0, 0)); - } - for delta_id in &batch { - hummock_version_deltas.remove(*delta_id); - } - commit_multi_var!(self.meta_store_ref(), hummock_version_deltas)?; + .retain(|id, _| *id > version_id); #[cfg(test)] { drop(context_info); drop(versioning_guard); self.check_state_consistency().await; } - Ok((batch.len(), deltas_to_delete_count - batch.len())) + Ok(res.rows_affected as usize) } /// Filters by Hummock version and Writes GC history. @@ -464,26 +454,6 @@ impl HummockManager { Ok(()) } - /// Deletes stale Hummock metadata. - /// - /// Returns number of deleted deltas - pub async fn delete_metadata(&self) -> MetaResult { - let batch_size = self.env.opts.hummock_delta_log_delete_batch_size; - let mut total_deleted = 0; - loop { - if total_deleted != 0 && self.env.opts.vacuum_spin_interval_ms != 0 { - tokio::time::sleep(Duration::from_millis(self.env.opts.vacuum_spin_interval_ms)) - .await; - } - let (deleted, remain) = self.delete_version_deltas(batch_size).await?; - total_deleted += deleted; - if total_deleted == 0 || remain < batch_size { - break; - } - } - Ok(total_deleted) - } - pub async fn delete_time_travel_metadata(&self) -> MetaResult<()> { let current_epoch_time = Epoch::now().physical_time(); let epoch_watermark = Epoch::from_physical_time( diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 046f0228012f0..0f6027df9f038 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -566,23 +566,14 @@ async fn test_hummock_manager_basic() { init_version_id + commit_log_count + register_log_count, ); } - assert_eq!( - hummock_manager - .delete_version_deltas(usize::MAX) - .await - .unwrap(), - (0, 0) - ); + assert_eq!(hummock_manager.delete_version_deltas().await.unwrap(), 0); assert_eq!( hummock_manager.create_version_checkpoint(1).await.unwrap(), commit_log_count + register_log_count ); assert_eq!( - hummock_manager - .delete_version_deltas(usize::MAX) - .await - .unwrap(), - ((commit_log_count + register_log_count) as usize, 0) + hummock_manager.delete_version_deltas().await.unwrap(), + (commit_log_count + register_log_count) as usize ); hummock_manager .unpin_version_before(context_id_1, HummockVersionId::MAX) @@ -2563,7 +2554,7 @@ async fn test_vacuum() { hummock_manager.clone(), context_id, )); - assert_eq!(hummock_manager.delete_metadata().await.unwrap(), 0); + assert_eq!(hummock_manager.delete_version_deltas().await.unwrap(), 0); hummock_manager.pin_version(context_id).await.unwrap(); let compaction_group_id = StaticCompactionGroupId::StateDefault.into(); let sst_infos = add_test_tables( @@ -2572,10 +2563,10 @@ async fn test_vacuum() { compaction_group_id, ) .await; - assert_eq!(hummock_manager.delete_metadata().await.unwrap(), 0); + assert_eq!(hummock_manager.delete_version_deltas().await.unwrap(), 0); hummock_manager.create_version_checkpoint(1).await.unwrap(); - assert_eq!(hummock_manager.delete_metadata().await.unwrap(), 6); - assert_eq!(hummock_manager.delete_metadata().await.unwrap(), 0); + assert_eq!(hummock_manager.delete_version_deltas().await.unwrap(), 6); + assert_eq!(hummock_manager.delete_version_deltas().await.unwrap(), 0); hummock_manager .unpin_version_before(context_id, HummockVersionId::MAX) diff --git a/src/meta/src/hummock/mod.rs b/src/meta/src/hummock/mod.rs index db6c9e87d38e4..5c07ce76c911a 100644 --- a/src/meta/src/hummock/mod.rs +++ b/src/meta/src/hummock/mod.rs @@ -83,7 +83,7 @@ pub fn start_vacuum_metadata_loop( return; } } - if let Err(err) = hummock_manager.delete_metadata().await { + if let Err(err) = hummock_manager.delete_version_deltas().await { tracing::warn!(error = %err.as_report(), "Vacuum metadata error"); } } diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index eab30bcce222a..cac22a550e7e3 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -116,7 +116,6 @@ pub struct MetaOpts { pub hummock_time_travel_snapshot_interval: u64, pub hummock_time_travel_sst_info_fetch_batch_size: usize, pub hummock_time_travel_sst_info_insert_batch_size: usize, - pub hummock_delta_log_delete_batch_size: usize, pub hummock_time_travel_epoch_version_insert_batch_size: usize, pub hummock_gc_history_insert_batch_size: usize, pub hummock_time_travel_filter_out_objects_batch_size: usize, @@ -280,7 +279,6 @@ impl MetaOpts { hummock_time_travel_snapshot_interval: 0, hummock_time_travel_sst_info_fetch_batch_size: 10_000, hummock_time_travel_sst_info_insert_batch_size: 10, - hummock_delta_log_delete_batch_size: 1000, hummock_time_travel_epoch_version_insert_batch_size: 1000, hummock_gc_history_insert_batch_size: 1000, hummock_time_travel_filter_out_objects_batch_size: 1000, diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 992f5f2610038..976206477f8c9 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -977,6 +977,7 @@ impl DdlController { .await?; if aborted { tracing::warn!(id = job_id, "aborted streaming job"); + // FIXME: might also need other cleanup here if let Some(source_id) = source_id { self.source_manager .apply_source_change(SourceChange::DropSource { diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 3c883903232ba..367482e831fb4 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -105,6 +105,7 @@ impl SourceManagerCore { pub fn apply_source_change(&mut self, source_change: SourceChange) { let mut added_source_fragments = Default::default(); let mut added_backfill_fragments = Default::default(); + let mut finished_backfill_fragments = Default::default(); let mut split_assignment = Default::default(); let mut dropped_actors = Default::default(); let mut fragment_replacements = Default::default(); @@ -121,6 +122,11 @@ impl SourceManagerCore { added_backfill_fragments = added_backfill_fragments_; split_assignment = split_assignment_; } + SourceChange::CreateJobFinished { + finished_backfill_fragments: finished_backfill_fragments_, + } => { + finished_backfill_fragments = finished_backfill_fragments_; + } SourceChange::SplitChange(split_assignment_) => { split_assignment = split_assignment_; } @@ -188,6 +194,16 @@ impl SourceManagerCore { .extend(fragments); } + for (source_id, fragments) in finished_backfill_fragments { + let handle = self.managed_sources.get(&source_id).unwrap_or_else(|| { + panic!( + "source {} not found when adding backfill fragments {:?}", + source_id, fragments + ); + }); + handle.finish_backfill(fragments.iter().map(|(id, _up_id)| *id).collect()); + } + for (_, actor_splits) in split_assignment { for (actor_id, splits) in actor_splits { // override previous splits info @@ -477,12 +493,21 @@ impl SourceManager { } pub enum SourceChange { - /// `CREATE SOURCE` (shared), or `CREATE MV` + /// `CREATE SOURCE` (shared), or `CREATE MV`. + /// This is applied after the job is successfully created (`post_collect` barrier). CreateJob { added_source_fragments: HashMap>, + /// (`source_id`, -> (`source_backfill_fragment_id`, `upstream_source_fragment_id`)) added_backfill_fragments: HashMap>, split_assignment: SplitAssignment, }, + /// `CREATE SOURCE` (shared), or `CREATE MV` is _finished_ (backfill is done). + /// This is applied after `wait_streaming_job_finished`. + /// XXX: Should we merge `CreateJob` into this? + CreateJobFinished { + /// (`source_id`, -> (`source_backfill_fragment_id`, `upstream_source_fragment_id`)) + finished_backfill_fragments: HashMap>, + }, SplitChange(SplitAssignment), /// `DROP SOURCE` or `DROP MV` DropSource { diff --git a/src/meta/src/stream/source_manager/worker.rs b/src/meta/src/stream/source_manager/worker.rs index 9c77803c63832..d870a72a3cb23 100644 --- a/src/meta/src/stream/source_manager/worker.rs +++ b/src/meta/src/stream/source_manager/worker.rs @@ -241,6 +241,12 @@ impl ConnectorSourceWorker { tracing::warn!(error = %e.as_report(), "error happened when drop fragment"); } } + SourceWorkerCommand::FinishBackfill(fragment_ids) => { + if let Err(e) = self.finish_backfill(fragment_ids).await { + // when error happens, we just log it and ignore + tracing::warn!(error = %e.as_report(), "error happened when finish backfill"); + } + } SourceWorkerCommand::Terminate => { return; } @@ -287,6 +293,11 @@ impl ConnectorSourceWorker { self.enumerator.on_drop_fragments(fragment_ids).await?; Ok(()) } + + async fn finish_backfill(&mut self, fragment_ids: Vec) -> MetaResult<()> { + self.enumerator.on_finish_backfill(fragment_ids).await?; + Ok(()) + } } /// Handle for a running [`ConnectorSourceWorker`]. @@ -354,13 +365,23 @@ impl ConnectorSourceWorkerHandle { } pub fn drop_fragments(&self, fragment_ids: Vec) { + tracing::debug!("drop_fragments: {:?}", fragment_ids); if let Err(e) = self.send_command(SourceWorkerCommand::DropFragments(fragment_ids)) { // ignore drop fragment error, just log it tracing::warn!(error = %e.as_report(), "failed to drop fragments"); } } + pub fn finish_backfill(&self, fragment_ids: Vec) { + tracing::debug!("finish_backfill: {:?}", fragment_ids); + if let Err(e) = self.send_command(SourceWorkerCommand::FinishBackfill(fragment_ids)) { + // ignore error, just log it + tracing::warn!(error = %e.as_report(), "failed to finish backfill"); + } + } + pub fn terminate(&self, dropped_fragments: Option>) { + tracing::debug!("terminate: {:?}", dropped_fragments); if let Some(dropped_fragments) = dropped_fragments { self.drop_fragments(dropped_fragments.into_iter().collect()); } @@ -378,6 +399,8 @@ pub enum SourceWorkerCommand { Tick(#[educe(Debug(ignore))] oneshot::Sender>), /// Async command to drop a fragment. DropFragments(Vec), + /// Async command to finish backfill. + FinishBackfill(Vec), /// Terminate the worker task. Terminate, } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index d9e64ab3c660a..e20165fb50078 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -41,7 +41,7 @@ use crate::manager::{ MetaSrvEnv, MetadataManager, NotificationVersion, StreamingJob, StreamingJobType, }; use crate::model::{ActorId, FragmentId, StreamJobFragments, TableParallelism}; -use crate::stream::SourceManagerRef; +use crate::stream::{SourceChange, SourceManagerRef}; use crate::{MetaError, MetaResult}; pub type GlobalStreamManagerRef = Arc; @@ -392,6 +392,10 @@ impl GlobalStreamManager { .await?, ); + let source_change = SourceChange::CreateJobFinished { + finished_backfill_fragments: stream_job_fragments.source_backfill_fragments()?, + }; + let info = CreateStreamingJobCommandInfo { stream_job_fragments, upstream_root_actors, @@ -444,6 +448,7 @@ impl GlobalStreamManager { .metadata_manager .wait_streaming_job_finished(streaming_job.id() as _) .await?; + self.source_manager.apply_source_change(source_change).await; tracing::debug!(?streaming_job, "stream job finish"); Ok(version) } diff --git a/src/storage/hummock_sdk/src/change_log.rs b/src/storage/hummock_sdk/src/change_log.rs index 7976adbc1adb5..9bb4505fb808c 100644 --- a/src/storage/hummock_sdk/src/change_log.rs +++ b/src/storage/hummock_sdk/src/change_log.rs @@ -58,6 +58,16 @@ impl TableChangeLogCommon { .flat_map(|epoch_change_log| epoch_change_log.epochs.iter()) .cloned() } + + pub(crate) fn change_log_into_iter(self) -> impl Iterator> { + self.0.into_iter() + } + + pub(crate) fn change_log_iter_mut( + &mut self, + ) -> impl Iterator> { + self.0.iter_mut() + } } pub type TableChangeLog = TableChangeLogCommon; diff --git a/src/storage/hummock_sdk/src/compact_task.rs b/src/storage/hummock_sdk/src/compact_task.rs index 3dd5187936540..d173f6d252725 100644 --- a/src/storage/hummock_sdk/src/compact_task.rs +++ b/src/storage/hummock_sdk/src/compact_task.rs @@ -115,8 +115,8 @@ impl CompactTask { } impl From for CompactTask { - #[expect(deprecated)] fn from(pb_compact_task: PbCompactTask) -> Self { + #[expect(deprecated)] Self { input_ssts: pb_compact_task .input_ssts @@ -168,8 +168,8 @@ impl From for CompactTask { } impl From<&PbCompactTask> for CompactTask { - #[expect(deprecated)] fn from(pb_compact_task: &PbCompactTask) -> Self { + #[expect(deprecated)] Self { input_ssts: pb_compact_task .input_ssts @@ -221,8 +221,8 @@ impl From<&PbCompactTask> for CompactTask { } impl From for PbCompactTask { - #[expect(deprecated)] fn from(compact_task: CompactTask) -> Self { + #[expect(deprecated)] Self { input_ssts: compact_task .input_ssts @@ -272,8 +272,8 @@ impl From for PbCompactTask { } impl From<&CompactTask> for PbCompactTask { - #[expect(deprecated)] fn from(compact_task: &CompactTask) -> Self { + #[expect(deprecated)] Self { input_ssts: compact_task .input_ssts diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index eb4bb30e69dc3..8ea85ec57dbe3 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -36,7 +36,7 @@ use crate::level::{Level, LevelCommon, Levels, OverlappingLevel}; use crate::sstable_info::SstableInfo; use crate::table_watermark::{ReadTableWatermark, TableWatermarks}; use crate::version::{ - GroupDelta, GroupDeltaCommon, HummockVersion, HummockVersionCommon, HummockVersionDelta, + GroupDelta, GroupDeltaCommon, HummockVersion, HummockVersionCommon, HummockVersionDeltaCommon, HummockVersionStateTableInfo, IntraLevelDelta, IntraLevelDeltaCommon, ObjectIdReader, SstableIdReader, }; @@ -50,7 +50,7 @@ pub struct SstDeltaInfo { pub type BranchedSstInfo = HashMap>; -impl HummockVersion { +impl HummockVersionCommon { pub fn get_compaction_group_levels(&self, compaction_group_id: CompactionGroupId) -> &Levels { self.levels .get(&compaction_group_id) @@ -187,7 +187,7 @@ pub fn safe_epoch_read_table_watermarks_impl( .collect() } -impl HummockVersion { +impl HummockVersionCommon { pub fn count_new_ssts_in_group_split( &self, parent_group_id: CompactionGroupId, @@ -356,7 +356,10 @@ impl HummockVersion { .all(|level| !level.table_infos.is_empty())); } - pub fn build_sst_delta_infos(&self, version_delta: &HummockVersionDelta) -> Vec { + pub fn build_sst_delta_infos( + &self, + version_delta: &HummockVersionDeltaCommon, + ) -> Vec { let mut infos = vec![]; // Skip trivial move delta for refiller @@ -459,7 +462,10 @@ impl HummockVersion { infos } - pub fn apply_version_delta(&mut self, version_delta: &HummockVersionDelta) { + pub fn apply_version_delta( + &mut self, + version_delta: &HummockVersionDeltaCommon, + ) { assert_eq!(self.id, version_delta.prev_id); let (changed_table_info, mut is_commit_epoch) = self.state_table_info.apply_delta( @@ -934,12 +940,6 @@ impl HummockVersionCommon where T: SstableIdReader + ObjectIdReader, { - pub fn get_combined_levels(&self) -> impl Iterator> + '_ { - self.levels - .values() - .flat_map(|level| level.l0.sub_levels.iter().rev().chain(level.levels.iter())) - } - pub fn get_object_ids(&self) -> HashSet { self.get_sst_infos().map(|s| s.object_id()).collect() } @@ -1094,6 +1094,14 @@ impl Levels { } } +impl HummockVersionCommon { + pub fn get_combined_levels(&self) -> impl Iterator> + '_ { + self.levels + .values() + .flat_map(|level| level.l0.sub_levels.iter().rev().chain(level.levels.iter())) + } +} + pub fn build_initial_compaction_group_levels( group_id: CompactionGroupId, compaction_config: &CompactionConfig, diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 4c90e6cae47f1..84cba3804a96c 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -30,7 +30,9 @@ use risingwave_pb::hummock::{ }; use tracing::warn; -use crate::change_log::{ChangeLogDeltaCommon, TableChangeLogCommon}; +use crate::change_log::{ + ChangeLogDeltaCommon, EpochNewChangeLogCommon, TableChangeLog, TableChangeLogCommon, +}; use crate::compaction_group::hummock_version_ext::build_initial_compaction_group_levels; use crate::compaction_group::StaticCompactionGroupId; use crate::level::LevelsCommon; @@ -217,18 +219,20 @@ impl HummockVersionStateTableInfo { } #[derive(Debug, Clone, PartialEq)] -pub struct HummockVersionCommon { +pub struct HummockVersionCommon { pub id: HummockVersionId, pub levels: HashMap>, #[deprecated] pub(crate) max_committed_epoch: u64, pub table_watermarks: HashMap>, - pub table_change_log: HashMap>, + pub table_change_log: HashMap>, pub state_table_info: HummockVersionStateTableInfo, } pub type HummockVersion = HummockVersionCommon; +pub type LocalHummockVersion = HummockVersionCommon; + impl Default for HummockVersion { fn default() -> Self { HummockVersion::from(&PbHummockVersion::default()) @@ -433,13 +437,6 @@ impl HummockVersion { } } - pub fn table_committed_epoch(&self, table_id: TableId) -> Option { - self.state_table_info - .info() - .get(&table_id) - .map(|info| info.committed_epoch) - } - pub fn create_init_version(default_compaction_config: Arc) -> HummockVersion { #[expect(deprecated)] let mut init_version = HummockVersion { @@ -476,10 +473,41 @@ impl HummockVersion { state_table_info_delta: Default::default(), } } + + pub fn split_change_log(mut self) -> (LocalHummockVersion, HashMap) { + let table_change_log = { + let mut table_change_log = HashMap::new(); + for (table_id, log) in &mut self.table_change_log { + let change_log_iter = + log.change_log_iter_mut() + .map(|item| EpochNewChangeLogCommon { + new_value: std::mem::take(&mut item.new_value), + old_value: std::mem::take(&mut item.old_value), + epochs: item.epochs.clone(), + }); + table_change_log.insert(*table_id, TableChangeLogCommon::new(change_log_iter)); + } + + table_change_log + }; + + let local_version = LocalHummockVersion::from(self); + + (local_version, table_change_log) + } +} + +impl HummockVersionCommon { + pub fn table_committed_epoch(&self, table_id: TableId) -> Option { + self.state_table_info + .info() + .get(&table_id) + .map(|info| info.committed_epoch) + } } #[derive(Debug, PartialEq, Clone)] -pub struct HummockVersionDeltaCommon { +pub struct HummockVersionDeltaCommon { pub id: HummockVersionId, pub prev_id: HummockVersionId, pub group_deltas: HashMap>, @@ -488,12 +516,14 @@ pub struct HummockVersionDeltaCommon { pub trivial_move: bool, pub new_table_watermarks: HashMap, pub removed_table_ids: HashSet, - pub change_log_delta: HashMap>, + pub change_log_delta: HashMap>, pub state_table_info_delta: HashMap, } pub type HummockVersionDelta = HummockVersionDeltaCommon; +pub type LocalHummockVersionDelta = HummockVersionDeltaCommon; + impl Default for HummockVersionDelta { fn default() -> Self { HummockVersionDelta::from(&PbHummockVersionDelta::default()) @@ -1095,3 +1125,64 @@ where self.into() } } + +impl From for LocalHummockVersionDelta { + #[expect(deprecated)] + fn from(delta: HummockVersionDelta) -> Self { + Self { + id: delta.id, + prev_id: delta.prev_id, + group_deltas: delta.group_deltas, + max_committed_epoch: delta.max_committed_epoch, + trivial_move: delta.trivial_move, + new_table_watermarks: delta.new_table_watermarks, + removed_table_ids: delta.removed_table_ids, + change_log_delta: delta + .change_log_delta + .into_iter() + .map(|(k, v)| { + ( + k, + ChangeLogDeltaCommon { + truncate_epoch: v.truncate_epoch, + new_log: EpochNewChangeLogCommon { + epochs: v.new_log.epochs, + new_value: Vec::new(), + old_value: Vec::new(), + }, + }, + ) + }) + .collect(), + state_table_info_delta: delta.state_table_info_delta, + } + } +} + +impl From for LocalHummockVersion { + #[expect(deprecated)] + fn from(version: HummockVersion) -> Self { + Self { + id: version.id, + levels: version.levels, + max_committed_epoch: version.max_committed_epoch, + table_watermarks: version.table_watermarks, + table_change_log: version + .table_change_log + .into_iter() + .map(|(k, v)| { + let epoch_new_change_logs: Vec> = v + .change_log_into_iter() + .map(|epoch_new_change_log| EpochNewChangeLogCommon { + epochs: epoch_new_change_log.epochs, + new_value: Vec::new(), + old_value: Vec::new(), + }) + .collect(); + (k, TableChangeLogCommon::new(epoch_new_change_logs)) + }) + .collect(), + state_table_info: version.state_table_info, + } + } +} diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index fb914b226d351..b584b1f55e5be 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -28,6 +28,8 @@ use prometheus::{Histogram, IntGauge}; use risingwave_common::catalog::TableId; use risingwave_common::metrics::UintGauge; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::SstDeltaInfo; +use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::version::{HummockVersionCommon, LocalHummockVersionDelta}; use risingwave_hummock_sdk::{HummockEpoch, SyncResult}; use tokio::spawn; use tokio::sync::mpsc::error::SendError; @@ -508,7 +510,7 @@ impl HummockEventHandler { let mut sst_delta_infos = vec![]; if let Some(new_pinned_version) = Self::resolve_version_update_info( - pinned_version.clone(), + &pinned_version, version_payload, Some(&mut sst_delta_infos), ) { @@ -518,31 +520,57 @@ impl HummockEventHandler { } fn resolve_version_update_info( - pinned_version: PinnedVersion, + pinned_version: &PinnedVersion, version_payload: HummockVersionUpdate, mut sst_delta_infos: Option<&mut Vec>, ) -> Option { - let newly_pinned_version = match version_payload { + match version_payload { HummockVersionUpdate::VersionDeltas(version_deltas) => { - let mut version_to_apply = (*pinned_version).clone(); - for version_delta in &version_deltas { - assert_eq!(version_to_apply.id, version_delta.prev_id); - if let Some(sst_delta_infos) = &mut sst_delta_infos { - sst_delta_infos.extend( - version_to_apply - .build_sst_delta_infos(version_delta) - .into_iter(), - ); + let mut version_to_apply = (**pinned_version).clone(); + { + let mut table_change_log_to_apply_guard = + pinned_version.table_change_log_write_lock(); + for version_delta in version_deltas { + assert_eq!(version_to_apply.id, version_delta.prev_id); + + // apply change-log-delta + { + let mut state_table_info = version_to_apply.state_table_info.clone(); + let (changed_table_info, _is_commit_epoch) = state_table_info + .apply_delta( + &version_delta.state_table_info_delta, + &version_delta.removed_table_ids, + ); + + HummockVersionCommon::::apply_change_log_delta( + &mut *table_change_log_to_apply_guard, + &version_delta.change_log_delta, + &version_delta.removed_table_ids, + &version_delta.state_table_info_delta, + &changed_table_info, + ); + } + + let local_hummock_version_delta = + LocalHummockVersionDelta::from(version_delta); + if let Some(sst_delta_infos) = &mut sst_delta_infos { + sst_delta_infos.extend( + version_to_apply + .build_sst_delta_infos(&local_hummock_version_delta) + .into_iter(), + ); + } + + version_to_apply.apply_version_delta(&local_hummock_version_delta); } - version_to_apply.apply_version_delta(version_delta); } - version_to_apply + pinned_version.new_with_local_version(version_to_apply) } - HummockVersionUpdate::PinnedVersion(version) => *version, - }; - - pinned_version.new_pin_version(newly_pinned_version) + HummockVersionUpdate::PinnedVersion(version) => { + pinned_version.new_pin_version(*version) + } + } } fn apply_version_update( diff --git a/src/storage/src/hummock/local_version/pinned_version.rs b/src/storage/src/hummock/local_version/pinned_version.rs index 25e25938398f3..082879be992ee 100644 --- a/src/storage/src/hummock/local_version/pinned_version.rs +++ b/src/storage/src/hummock/local_version/pinned_version.rs @@ -12,16 +12,19 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::{BTreeMap, HashMap}; use std::iter::empty; use std::ops::Deref; use std::sync::Arc; use std::time::{Duration, Instant}; use auto_enums::auto_enum; +use parking_lot::RwLock; use risingwave_common::catalog::TableId; +use risingwave_hummock_sdk::change_log::TableChangeLogCommon; use risingwave_hummock_sdk::level::{Level, Levels}; -use risingwave_hummock_sdk::version::HummockVersion; +use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::version::{HummockVersion, LocalHummockVersion}; use risingwave_hummock_sdk::{CompactionGroupId, HummockVersionId, INVALID_VERSION_ID}; use risingwave_rpc_client::HummockMetaClient; use thiserror_ext::AsReport; @@ -74,12 +77,13 @@ impl Drop for PinnedVersionGuard { #[derive(Clone)] pub struct PinnedVersion { - version: Arc, + version: Arc, guard: Arc, + table_change_log: Arc>>>, } impl Deref for PinnedVersion { - type Target = HummockVersion; + type Target = LocalHummockVersion; fn deref(&self) -> &Self::Target { &self.version @@ -92,12 +96,14 @@ impl PinnedVersion { pinned_version_manager_tx: UnboundedSender, ) -> Self { let version_id = version.id; + let (local_version, table_id_to_change_logs) = version.split_change_log(); PinnedVersion { - version: Arc::new(version), guard: Arc::new(PinnedVersionGuard::new( version_id, pinned_version_manager_tx, )), + table_change_log: Arc::new(RwLock::new(table_id_to_change_logs)), + version: Arc::new(local_version), } } @@ -111,14 +117,39 @@ impl PinnedVersion { if version.id == self.version.id { return None; } + let version_id = version.id; + let (local_version, table_id_to_change_logs) = version.split_change_log(); + Some(PinnedVersion { + guard: Arc::new(PinnedVersionGuard::new( + version_id, + self.guard.pinned_version_manager_tx.clone(), + )), + table_change_log: Arc::new(RwLock::new(table_id_to_change_logs)), + version: Arc::new(local_version), + }) + } + + /// Create a new `PinnedVersion` with the given `LocalHummockVersion`. Referring to the usage in the `hummock_event_handler`. + pub fn new_with_local_version(&self, version: LocalHummockVersion) -> Option { + assert!( + version.id >= self.version.id, + "pinning a older version {}. Current is {}", + version.id, + self.version.id + ); + if version.id == self.version.id { + return None; + } + let version_id = version.id; Some(PinnedVersion { - version: Arc::new(version), guard: Arc::new(PinnedVersionGuard::new( version_id, self.guard.pinned_version_manager_tx.clone(), )), + table_change_log: self.table_change_log.clone(), + version: Arc::new(version), }) } @@ -159,6 +190,19 @@ impl PinnedVersion { None => empty(), } } + + pub fn table_change_log_read_lock( + &self, + ) -> parking_lot::RwLockReadGuard<'_, HashMap>> { + self.table_change_log.read() + } + + pub fn table_change_log_write_lock( + &self, + ) -> parking_lot::RwLockWriteGuard<'_, HashMap>> + { + self.table_change_log.write() + } } pub(crate) async fn start_pinned_version_worker( diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index 88133f45a56b0..6b231f045e56a 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -28,7 +28,7 @@ use risingwave_hummock_sdk::key::{ }; use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::TableWatermarksIndex; -use risingwave_hummock_sdk::version::HummockVersion; +use risingwave_hummock_sdk::version::{HummockVersion, LocalHummockVersion}; use risingwave_hummock_sdk::{HummockReadEpoch, HummockSstableObjectId, SyncResult}; use risingwave_rpc_client::HummockMetaClient; use thiserror_ext::AsReport; @@ -641,7 +641,7 @@ impl StateStoreReadLog for HummockStorage { async fn next_epoch(&self, epoch: u64, options: NextEpochOptions) -> StorageResult { fn next_epoch( - version: &HummockVersion, + version: &LocalHummockVersion, epoch: u64, table_id: TableId, ) -> HummockResult> { diff --git a/src/storage/src/hummock/store/version.rs b/src/storage/src/hummock/store/version.rs index 8e0c7a589b203..acf7c6503fb7f 100644 --- a/src/storage/src/hummock/store/version.rs +++ b/src/storage/src/hummock/store/version.rs @@ -992,11 +992,15 @@ impl HummockVersionReader { key_range: TableKeyRange, options: ReadLogOptions, ) -> HummockResult { - let change_log = if let Some(change_log) = version.table_change_log.get(&options.table_id) { - change_log.filter_epoch(epoch_range).collect_vec() - } else { - Vec::new() + let change_log = { + let table_change_logs = version.table_change_log_read_lock(); + if let Some(change_log) = table_change_logs.get(&options.table_id) { + change_log.filter_epoch(epoch_range).cloned().collect_vec() + } else { + Vec::new() + } }; + if let Some(max_epoch_change_log) = change_log.last() { let (_, max_epoch) = epoch_range; if !max_epoch_change_log.epochs.contains(&max_epoch) { diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index d960ea97ef859..d8117f86fb5dc 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -636,6 +636,13 @@ impl SourceBackfillExecutorInner { .await?; if self.should_report_finished(&backfill_stage.states) { + // drop the backfill kafka consumers + backfill_stream = select_with_strategy( + input.by_ref().map(Either::Left), + futures::stream::pending().boxed().map(Either::Right), + select_strategy, + ); + self.progress.finish( barrier.epoch, backfill_stage.total_backfilled_rows(), @@ -734,6 +741,7 @@ impl SourceBackfillExecutorInner { } } + std::mem::drop(backfill_stream); let mut states = backfill_stage.states; // Make sure `Finished` state is persisted. self.backfill_state_store.set_states(states.clone()).await?; diff --git a/src/tests/simulation/src/nexmark.rs b/src/tests/simulation/src/nexmark.rs index 7959cf392f232..814f751733ada 100644 --- a/src/tests/simulation/src/nexmark.rs +++ b/src/tests/simulation/src/nexmark.rs @@ -219,7 +219,7 @@ pub mod queries { pub mod q105 { use super::*; pub const CREATE: &str = include_str!("nexmark/q105.sql"); - pub const SELECT: &str = "SELECT * FROM nexmark_q105;"; + pub const SELECT: &str = "SELECT * FROM nexmark_q105 ORDER BY 1,2,3;"; pub const DROP: &str = "DROP MATERIALIZED VIEW nexmark_q105;"; pub const INITIAL_INTERVAL: Duration = DEFAULT_INITIAL_INTERVAL; pub const INITIAL_TIMEOUT: Duration = DEFAULT_INITIAL_TIMEOUT;