diff --git a/Cargo.lock b/Cargo.lock index e3b68e7f34607..dccda78d5bb4e 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -213,7 +213,7 @@ dependencies = [ "uuid", "xz2", "zerocopy", - "zstd", + "zstd 0.12.4", ] [[package]] @@ -4541,8 +4541,9 @@ dependencies = [ [[package]] name = "madsim-rdkafka" -version = "0.2.22" -source = "git+https://github.com/madsim-rs/madsim.git?rev=fedb1e3#fedb1e3a0a8758650c9e15076941c999150bdb31" +version = "0.3.0+0.34.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "00f9ab2d0545a55e4f209fc72c180a7e7b45a4e7baee7b4994c4628a877c5525" dependencies = [ "async-channel", "async-trait", @@ -5646,7 +5647,7 @@ dependencies = [ "thrift", "tokio", "twox-hash", - "zstd", + "zstd 0.12.4", ] [[package]] @@ -6616,8 +6617,9 @@ dependencies = [ [[package]] name = "rdkafka-sys" -version = "4.3.0+1.9.2" -source = "git+https://github.com/MaterializeInc/rust-rdkafka?rev=8ea07c4#8ea07c4d2b96636ff093e670bc921892aee0d56a" +version = "4.6.0+2.2.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "ad63c279fca41a27c231c450a2d2ad18288032e9cbb159ad16c9d96eba35aaaf" dependencies = [ "cmake", "libc", @@ -8036,7 +8038,7 @@ dependencies = [ "workspace-hack", "xorf", "xxhash-rust", - "zstd", + "zstd 0.13.0", ] [[package]] @@ -11104,7 +11106,16 @@ version = "0.12.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1a27595e173641171fc74a1232b7b1c7a7cb6e18222c11e9dfb9888fa424c53c" dependencies = [ - "zstd-safe", + "zstd-safe 6.0.6", +] + +[[package]] +name = "zstd" +version = "0.13.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "bffb3309596d527cfcba7dfc6ed6052f1d39dfbd7c867aa2e865e4a449c10110" +dependencies = [ + "zstd-safe 7.0.0", ] [[package]] @@ -11117,6 +11128,15 @@ dependencies = [ "zstd-sys", ] +[[package]] +name = "zstd-safe" +version = "7.0.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "43747c7422e2924c11144d5229878b98180ef8b06cca4ab5af37afc8a8d8ea3e" +dependencies = [ + "zstd-sys", +] + [[package]] name = "zstd-sys" version = "2.0.8+zstd.1.5.5" diff --git a/Cargo.toml b/Cargo.toml index 8c7030ed0d619..81d914b416d17 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -96,7 +96,7 @@ aws-types = "0.55" etcd-client = { package = "madsim-etcd-client", version = "0.4" } futures-async-stream = "0.2" hytra = "0.1" -rdkafka = { package = "madsim-rdkafka", git = "https://github.com/madsim-rs/madsim.git", rev = "fedb1e3", features = [ +rdkafka = { package = "madsim-rdkafka", version = "0.3.0", features = [ "cmake-build", ] } hashbrown = { version = "0.14.0", features = [ diff --git a/ci/scripts/s3-source-test.sh b/ci/scripts/s3-source-test.sh index 710ba63b6fd60..9fce76f000e31 100755 --- a/ci/scripts/s3-source-test.sh +++ b/ci/scripts/s3-source-test.sh @@ -30,7 +30,7 @@ cargo make ci-start ci-1cn-1fe echo "--- Run test" python3 -m pip install minio psycopg2-binary -python3 e2e_test/s3/$script.py +python3 e2e_test/s3/$script echo "--- Kill cluster" cargo make ci-kill diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index a0e05a2b43f3d..349f6d8359f2b 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -290,7 +290,24 @@ steps: retry: *auto-retry - label: "S3 source check on AWS (json parser)" - command: "ci/scripts/s3-source-test.sh -p ci-release -s json_file" + command: "ci/scripts/s3-source-test.sh -p ci-release -s run.py" + depends_on: build + plugins: + - seek-oss/aws-sm#v2.3.1: + env: + S3_SOURCE_TEST_CONF: ci_s3_source_test_aws + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + environment: + - S3_SOURCE_TEST_CONF + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 20 + retry: *auto-retry + + - label: "S3 source check on AWS (json parser)" + command: "ci/scripts/s3-source-test.sh -p ci-release -s json_file.py" depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: @@ -307,7 +324,41 @@ steps: retry: *auto-retry - label: "S3 source check on AWS (csv parser)" - command: "ci/scripts/s3-source-test.sh -p ci-release -s run_csv" + command: "ci/scripts/s3-source-test.sh -p ci-release -s run_csv.py" + depends_on: build + plugins: + - seek-oss/aws-sm#v2.3.1: + env: + S3_SOURCE_TEST_CONF: ci_s3_source_test_aws + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + environment: + - S3_SOURCE_TEST_CONF + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 25 + retry: *auto-retry + + - label: "S3_v2 source check on AWS (json parser)" + command: "ci/scripts/s3-source-test.sh -p ci-release -s 'fs_source_v2.py json'" + depends_on: build + plugins: + - seek-oss/aws-sm#v2.3.1: + env: + S3_SOURCE_TEST_CONF: ci_s3_source_test_aws + - docker-compose#v4.9.0: + run: rw-build-env + config: ci/docker-compose.yml + mount-buildkite-agent: true + environment: + - S3_SOURCE_TEST_CONF + - ./ci/plugins/upload-failure-logs + timeout_in_minutes: 25 + retry: *auto-retry + + - label: "S3_v2 source check on AWS (csv parser)" + command: "ci/scripts/s3-source-test.sh -p ci-release -s 'fs_source_v2.py csv_without_header'" depends_on: build plugins: - seek-oss/aws-sm#v2.3.1: diff --git a/docs/developer-guide.md b/docs/developer-guide.md index 7778f2b587029..4ecc756131dff 100644 --- a/docs/developer-guide.md +++ b/docs/developer-guide.md @@ -23,6 +23,7 @@ http://ecotrust-canada.github.io/markdown-toc/ * [Start the playground with RiseDev](#start-the-playground-with-risedev) * [Start the playground with cargo](#start-the-playground-with-cargo) - [Debug playground using vscode](#debug-playground-using-vscode) +- [Use standalone-mode](#use-standalone-mode) - [Develop the dashboard](#develop-the-dashboard) - [Observability components](#observability-components) * [Cluster Control](#cluster-control) @@ -206,6 +207,10 @@ psql -h localhost -p 4566 -d dev -U root To step through risingwave locally with a debugger you can use the `launch.json` and the `tasks.json` provided in `vscode_suggestions`. After adding these files to your local `.vscode` folder you can debug and set breakpoints by launching `Launch 'risingwave p' debug`. +## Use standalone-mode + +Please refer to [README](../src/cmd_all/src/README.md) for more details. + ## Develop the dashboard Currently, RisingWave has two versions of dashboards. You can use RiseDev config to select which version to use. diff --git a/e2e_test/s3/fs_source_v2.py b/e2e_test/s3/fs_source_v2.py new file mode 100644 index 0000000000000..7ac5d91ad4bd7 --- /dev/null +++ b/e2e_test/s3/fs_source_v2.py @@ -0,0 +1,158 @@ +import os +import sys +import csv +import json +import random +import psycopg2 + +from time import sleep +from io import StringIO +from minio import Minio +from functools import partial + +def gen_data(file_num, item_num_per_file): + assert item_num_per_file % 2 == 0, \ + f'item_num_per_file should be even to ensure sum(mark) == 0: {item_num_per_file}' + return [ + [{ + 'id': file_id * item_num_per_file + item_id, + 'name': f'{file_id}_{item_id}', + 'sex': item_id % 2, + 'mark': (-1) ** (item_id % 2), + } for item_id in range(item_num_per_file)] + for file_id in range(file_num) + ] + +def format_json(data): + return [ + '\n'.join([json.dumps(item) for item in file]) + for file in data + ] + +def format_csv(data, with_header): + csv_files = [] + + for file_data in data: + ostream = StringIO() + writer = csv.DictWriter(ostream, fieldnames=file_data[0].keys()) + if with_header: + writer.writeheader() + for item_data in file_data: + writer.writerow(item_data) + # For now paser can only handle \n line seperator, + # and tailing white spaces are not allowed. + # TODO: remove replace and rstrip later + csv_files.append(ostream.getvalue().replace('\r', '').rstrip()) + return csv_files + +def do_test(config, file_num, item_num_per_file, prefix, fmt): + conn = psycopg2.connect( + host="localhost", + port="4566", + user="root", + database="dev" + ) + + # Open a cursor to execute SQL statements + cur = conn.cursor() + + def _table(): + return f's3_test_{fmt}' + + def _encode(): + if fmt == 'json': + return 'JSON' + else: + return f"CSV (delimiter = ',', without_header = {str('without' in fmt).lower()})" + + # Execute a SELECT statement + cur.execute(f'''CREATE TABLE {_table()}( + id int, + name TEXT, + sex int, + mark int, + ) WITH ( + connector = 's3_v2', + match_pattern = '{prefix}*.{fmt}', + s3.region_name = '{config['S3_REGION']}', + s3.bucket_name = '{config['S3_BUCKET']}', + s3.credentials.access = '{config['S3_ACCESS_KEY']}', + s3.credentials.secret = '{config['S3_SECRET_KEY']}', + s3.endpoint_url = 'https://{config['S3_ENDPOINT']}' + ) FORMAT PLAIN ENCODE {_encode()};''') + + total_rows = file_num * item_num_per_file + MAX_RETRIES = 40 + for retry_no in range(MAX_RETRIES): + cur.execute(f'select count(*) from {_table()}') + result = cur.fetchone() + if result[0] == total_rows: + break + print(f"[retry {retry_no}] Now got {result[0]} rows in table, {total_rows} expected, wait 30s") + sleep(30) + + stmt = f'select count(*), sum(id), sum(sex), sum(mark) from {_table()}' + print(f'Execute {stmt}') + cur.execute(stmt) + result = cur.fetchone() + + print('Got:', result) + + def _assert_eq(field, got, expect): + assert got == expect, f'{field} assertion failed: got {got}, expect {expect}.' + + _assert_eq('count(*)', result[0], total_rows) + _assert_eq('sum(id)', result[1], (total_rows - 1) * total_rows / 2) + _assert_eq('sum(sex)', result[2], total_rows / 2) + _assert_eq('sum(mark)', result[3], 0) + + print('Test pass') + + cur.execute(f'drop table {_table()}') + cur.close() + conn.close() + + +if __name__ == "__main__": + FILE_NUM = 4001 + ITEM_NUM_PER_FILE = 2 + data = gen_data(FILE_NUM, ITEM_NUM_PER_FILE) + + fmt = sys.argv[1] + FORMATTER = { + 'json': format_json, + 'csv_with_header': partial(format_csv, with_header=True), + 'csv_without_header': partial(format_csv, with_header=False), + } + assert fmt in FORMATTER, f"Unsupported format: {fmt}" + formatted_files = FORMATTER[fmt](data) + + config = json.loads(os.environ["S3_SOURCE_TEST_CONF"]) + client = Minio( + config["S3_ENDPOINT"], + access_key=config["S3_ACCESS_KEY"], + secret_key=config["S3_SECRET_KEY"], + secure=True, + ) + run_id = str(random.randint(1000, 9999)) + _local = lambda idx: f'data_{idx}.{fmt}' + _s3 = lambda idx: f"{run_id}_data_{idx}.{fmt}" + + # put s3 files + for idx, file_str in enumerate(formatted_files): + with open(_local(idx), "w") as f: + f.write(file_str) + os.fsync(f.fileno()) + + client.fput_object( + config["S3_BUCKET"], + _s3(idx), + _local(idx) + ) + + # do test + do_test(config, FILE_NUM, ITEM_NUM_PER_FILE, run_id, fmt) + + # clean up s3 files + for idx, _ in enumerate(formatted_files): + client.remove_object(config["S3_BUCKET"], _s3(idx)) diff --git a/e2e_test/streaming/aggregate/jsonb_agg.slt b/e2e_test/streaming/aggregate/jsonb_agg.slt new file mode 100644 index 0000000000000..18cb80cc69085 --- /dev/null +++ b/e2e_test/streaming/aggregate/jsonb_agg.slt @@ -0,0 +1,46 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table t(v1 boolean, v2 int, v3 varchar, v4 jsonb); + +statement ok +create materialized view mv_tmp as +select jsonb_agg(v1) as j1 from t; + +statement ok +drop materialized view mv_tmp; + +statement ok +create materialized view mv1 as +select + jsonb_agg(v1 order by v2) as j1, + jsonb_agg(v2 order by v2) as j2, + jsonb_object_agg(v3, v4) as j3 +from t; + +statement ok +insert into t values + (null, 2, 'bbb', null), + (false, 1, 'ccc', 'null'); + +query TTT +select * from mv1; +---- +[false, null] [1, 2] {"bbb": null, "ccc": null} + +statement ok +insert into t values + (true, 0, 'bbb', '999'), + (true, 8, 'ddd', '{"foo": "bar"}'); + +query TTT +select * from mv1; +---- +[true, false, null, true] [0, 1, 2, 8] {"bbb": 999, "ccc": null, "ddd": {"foo": "bar"}} + +statement ok +drop materialized view mv1; + +statement ok +drop table t; diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 149fba8439db1..683a43ef6e9be 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -154,6 +154,21 @@ message StreamSource { optional uint32 rate_limit = 9; } +// copy contents from StreamSource to prevent compatibility issues in the future +message StreamFsFetch { + uint32 source_id = 1; + catalog.Table state_table = 2; + optional uint32 row_id_index = 3; + repeated plan_common.ColumnCatalog columns = 4; + reserved "pk_column_ids"; + reserved 5; + map properties = 6; + catalog.StreamSourceInfo info = 7; + string source_name = 8; + // Streaming rate limit + optional uint32 rate_limit = 9; +} + // The executor only for receiving barrier from the meta service. It always resides in the leaves // of the streaming graph. message BarrierRecvNode {} @@ -164,6 +179,10 @@ message SourceNode { StreamSource source_inner = 1; } +message StreamFsFetchNode { + StreamFsFetch node_inner = 1; +} + message SinkDesc { reserved 4; reserved "columns"; @@ -659,6 +678,7 @@ message StreamNode { NoOpNode no_op = 135; EowcOverWindowNode eowc_over_window = 136; OverWindowNode over_window = 137; + StreamFsFetchNode stream_fs_fetch = 138; } // The id for the operator. This is local per mview. // TODO: should better be a uint32. diff --git a/src/cmd_all/src/README.md b/src/cmd_all/src/README.md index 5748d62e6780c..fbbae4439f97f 100644 --- a/src/cmd_all/src/README.md +++ b/src/cmd_all/src/README.md @@ -8,9 +8,23 @@ This mode is just for local development. It starts with an in-memory etcd store This mode is for production. It provides cli parameters to configure etcd and object store. It will spawn `meta`, `frontend` and `compute` node within a single process. -It will take cli parameters to configure etcd and object store, connector node, and the compactor node. -## Development Notes +You can omit options, and the corresponding node will not be started in the standalone process: -The `cmd_all` module directly calls the entry points functions of `meta`, `frontend` and `compute` modules. -It does so within a single process. \ No newline at end of file +```bash +standalone \ + --meta-opts="..." \ + --frontend-opts="..." + # --compute-opts="..." not provided, so it won't be started. +``` + +### Examples of using standalone mode + +You may run and reference the [demo script](../scripts/e2e-full-standalone-demo.sh), as an example. + +### Internals + +Standalone mode simply passes the options to the corresponding node, and starts them in the same process. + +For example `--meta-opts` is parsed, and then Meta Node's entrypoint, `risingwave_meta::start`, is called with the parsed options. +If any option is missing, the corresponding node will not be started. \ No newline at end of file diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 346dc4d357c1d..9e515ba471967 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -516,7 +516,7 @@ pub struct StorageConfig { pub compactor_max_task_multiplier: f32, /// The percentage of memory available when compactor is deployed separately. - /// total_memory_available_bytes = total_memory_available_bytes * + /// total_memory_available_bytes = system_memory_available_bytes * /// compactor_memory_available_proportion #[serde(default = "default::storage::compactor_memory_available_proportion")] pub compactor_memory_available_proportion: f64, diff --git a/src/common/src/monitor/connection.rs b/src/common/src/monitor/connection.rs index dbe75087dfa21..0c153a3931098 100644 --- a/src/common/src/monitor/connection.rs +++ b/src/common/src/monitor/connection.rs @@ -226,7 +226,7 @@ where } fn call(&mut self, uri: Uri) -> Self::Future { - let endpoint = format!("{:?}:{:?}", uri.host(), uri.port()); + let endpoint = format!("{:?}", uri.host()); let monitor = self.monitor.clone(); self.inner .call(uri) @@ -259,7 +259,7 @@ where result.map(|conn| { let remote_addr = conn.connect_info().remote_addr(); let endpoint = remote_addr - .map(|remote_addr| format!("{}:{}", remote_addr.ip(), remote_addr.port())) + .map(|remote_addr| format!("{}", remote_addr.ip())) .unwrap_or("unknown".to_string()); MonitoredConnection::new(conn, monitor.new_connection_monitor(endpoint)) }) diff --git a/src/common/src/telemetry/mod.rs b/src/common/src/telemetry/mod.rs index 3c25811ca5e3a..65e314d232aeb 100644 --- a/src/common/src/telemetry/mod.rs +++ b/src/common/src/telemetry/mod.rs @@ -22,7 +22,7 @@ use sysinfo::{System, SystemExt}; use crate::util::env_var::env_var_is_true_or; use crate::util::resource_util::cpu::total_cpu_available; -use crate::util::resource_util::memory::{total_memory_available_bytes, total_memory_used_bytes}; +use crate::util::resource_util::memory::{system_memory_available_bytes, total_memory_used_bytes}; /// Url of telemetry backend pub const TELEMETRY_REPORT_URL: &str = "https://telemetry.risingwave.dev/api/v1/report"; @@ -98,7 +98,7 @@ impl SystemData { let mut sys = System::new(); let memory = { - let available = total_memory_available_bytes(); + let available = system_memory_available_bytes(); let used = total_memory_used_bytes(); Memory { available, diff --git a/src/common/src/util/resource_util.rs b/src/common/src/util/resource_util.rs index 52fb9876c0ee5..289ac1c704acb 100644 --- a/src/common/src/util/resource_util.rs +++ b/src/common/src/util/resource_util.rs @@ -18,13 +18,13 @@ pub enum CgroupVersion { V2, } -// Current controllers available in implementation. +/// Current controllers available in implementation. pub enum Controller { Cpu, Memory, } -// Default constant Cgroup paths and hierarchy. +/// Default constant Cgroup paths and hierarchy. const DEFAULT_CGROUP_ROOT_HIERARCYHY: &str = "/sys/fs/cgroup"; const DEFAULT_CGROUP_V2_CONTROLLER_LIST_PATH: &str = "/sys/fs/cgroup/cgroup.controllers"; const DEFAULT_CGROUP_MAX_INDICATOR: &str = "max"; @@ -44,33 +44,33 @@ mod runtime { env::consts::OS.eq(DEFAULT_LINUX_IDENTIFIER) } - // checks if is running in a docker container by checking for docker env file, or if it is - // running in a kubernetes pod. + /// checks if is running in a docker container by checking for docker env file, or if it is + /// running in a kubernetes pod. fn is_running_in_container() -> bool { return env_var_check_if_running_in_container() || docker_env_exists() || is_running_in_kubernetes_pod(); - // checks for existence of docker env file + /// checks for existence of docker env file fn docker_env_exists() -> bool { Path::new(DEFAULT_DOCKER_ENV_PATH).exists() } - // checks for environment + /// checks for environment fn env_var_check_if_running_in_container() -> bool { env::var(DEFAULT_IN_CONTAINER_ENV_VARIABLE).is_ok() } - // checks if it is running in a kubernetes pod + /// checks if it is running in a kubernetes pod fn is_running_in_kubernetes_pod() -> bool { Path::new(DEFAULT_KUBERNETES_SECRETS_PATH).exists() } } - // Given a certain controller, checks if it is enabled. - // For cgroup_v1, existence of directory with controller name is checked in cgroup default root - // hierarchy. e.g if directory "/sys/fs/cgroup"/cpu" exists then CPU controller is enabled. - // For cgroup_v2, check the controller list path for the controller name. + /// Given a certain controller, checks if it is enabled. + /// For cgroup v1, existence of directory with controller name is checked in cgroup default root + /// hierarchy. e.g if directory "/sys/fs/cgroup"/cpu" exists then CPU controller is enabled. + /// For cgroup v2, check the controller list path for the controller name. pub fn is_controller_activated( controller_type: super::Controller, cgroup_version: CgroupVersion, @@ -90,7 +90,7 @@ mod runtime { } } - // If cgroup exists or is enabled in kernel, returnb true, else false. + /// If cgroup exists or is enabled in kernel, returnb true, else false. fn cgroup_exists() -> bool { Path::new(super::DEFAULT_CGROUP_ROOT_HIERARCYHY).is_dir() } @@ -134,20 +134,20 @@ pub mod memory { use super::runtime::get_resource; - // Default paths for memory limtiations and usage for cgroup_v1 and cgroup_v2. + /// Default paths for memory limtiations and usage for cgroup v1 and cgroup v2. const V1_MEMORY_LIMIT_PATH: &str = "/sys/fs/cgroup/memory/memory.limit_in_bytes"; const V1_MEMORY_CURRENT_PATH: &str = "/sys/fs/cgroup/memory/memory.usage_in_bytes"; const V2_MEMORY_LIMIT_PATH: &str = "/sys/fs/cgroup/memory.max"; const V2_MEMORY_CURRENT_PATH: &str = "/sys/fs/cgroup/memory.current"; - // Returns the system memory. + /// Returns the system memory. pub fn get_system_memory() -> usize { let mut sys = System::new(); sys.refresh_memory(); sys.total_memory() as usize } - // Returns the used memory of the system. + /// Returns the used memory of the system. pub fn get_system_memory_used() -> usize { let mut sys = System::new(); sys.refresh_memory(); @@ -187,9 +187,9 @@ pub mod memory { /// /// Basic usage: /// ``` ignore - /// let mem_available = memory::total_memory_available_bytes(); + /// let mem_available = memory::system_memory_available_bytes(); /// ``` - pub fn total_memory_available_bytes() -> usize { + pub fn system_memory_available_bytes() -> usize { get_resource( "memory available", super::Controller::Memory, @@ -198,10 +198,10 @@ pub mod memory { ) } - // Returns the memory limit of a container if running in a container else returns the system - // memory available. - // When the limit is set to max, total_memory_available_bytes() will return default system - // memory. + /// Returns the memory limit of a container if running in a container else returns the system + /// memory available. + /// When the limit is set to max, [`system_memory_available_bytes()`] will return default system + /// memory. fn get_container_memory_limit( cgroup_version: super::CgroupVersion, ) -> Result { @@ -214,8 +214,8 @@ pub mod memory { Ok(std::cmp::min(value, system)) } - // Returns the memory used in a container if running in a container else returns the system - // memory used. + /// Returns the memory used in a container if running in a container else returns the system + /// memory used. fn get_container_memory_used( cgroup_version: super::CgroupVersion, ) -> Result { @@ -235,7 +235,7 @@ pub mod cpu { use super::runtime::get_resource; use super::util::parse_error; - // Default constant Cgroup paths and hierarchy. + /// Default constant Cgroup paths and hierarchy. const V1_CPU_QUOTA_PATH: &str = "/sys/fs/cgroup/cpu/cpu.cfs_quota_us"; const V1_CPU_PERIOD_PATH: &str = "/sys/fs/cgroup/cpu/cpu.cfs_period_us"; const V2_CPU_LIMIT_PATH: &str = "/sys/fs/cgroup/cpu.max"; @@ -264,7 +264,7 @@ pub mod cpu { ) } - // Returns the CPU limit of the container. + /// Returns the CPU limit of the container. fn get_container_cpu_limit( cgroup_version: super::CgroupVersion, ) -> Result { @@ -277,7 +277,7 @@ pub mod cpu { } } - // Returns the total system cpu. + /// Returns the total system cpu. pub fn get_system_cpu() -> f32 { match thread::available_parallelism() { Ok(available_parallelism) => available_parallelism.get() as f32, @@ -285,7 +285,7 @@ pub mod cpu { } } - // Returns the CPU limit when cgroup_V1 is utilised. + /// Returns the CPU limit when cgroup v1 is utilised. pub fn get_cpu_limit_v1( quota_path: &str, period_path: &str, @@ -304,7 +304,7 @@ pub mod cpu { Ok((cpu_quota as f32) / (cpu_period as f32)) } - // Returns the CPU limit when cgroup_V2 is utilised. + /// Returns the CPU limit when cgroup v2 is utilised. pub fn get_cpu_limit_v2(limit_path: &str, max_value: f32) -> Result { let cpu_limit_string = fs_err::read_to_string(limit_path)?; @@ -333,7 +333,7 @@ pub mod cpu { } mod util { - // Parses the filepath and checks for the existence of controller_name in the file. + /// Parses the filepath and checks for the existence of `controller_name` in the file. pub fn parse_controller_enable_file_for_cgroup_v2( file_path: &str, controller_name: &str, @@ -362,7 +362,7 @@ mod util { ) } - // Reads an integer value from a file path. + /// Reads an integer value from a file path. pub fn read_usize(file_path: &str) -> Result { let content = fs_err::read_to_string(file_path)?; let limit_val = content @@ -372,8 +372,8 @@ mod util { Ok(limit_val) } - // Helper function that helps to retrieve value in file, if value is "max", max_value will be - // returned instead. + /// Helper function that helps to retrieve value in file, if value is "max", `max_value` will be + /// returned instead. pub fn read_usize_or_max(file_path: &str, max_value: usize) -> Result { let content = fs_err::read_to_string(file_path)?; if content.trim() == super::DEFAULT_CGROUP_MAX_INDICATOR { diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index cc8e950e42b12..a6aa2082ad141 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -147,6 +147,11 @@ fn visit_stream_node_tables_inner( always!(source.state_table, "Source"); } } + NodeBody::StreamFsFetch(node) => { + if let Some(source) = &mut node.node_inner { + always!(source.state_table, "FsFetch"); + } + } // Sink NodeBody::Sink(node) => { diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 04a1b3ba37045..bdd84ae402746 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -38,9 +38,13 @@ use std::pin::Pin; use clap::{Parser, ValueEnum}; use risingwave_common::config::{AsyncStackTraceOption, MetricLevel, OverrideConfig}; use risingwave_common::util::resource_util::cpu::total_cpu_available; -use risingwave_common::util::resource_util::memory::total_memory_available_bytes; +use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use serde::{Deserialize, Serialize}; +/// If `total_memory_bytes` is not specified, the default memory limit will be set to +/// the system memory limit multiplied by this proportion +const DEFAULT_MEMORY_PROPORTION: f64 = 0.7; + /// Command-line arguments for compute-node. #[derive(Parser, Clone, Debug, OverrideConfig)] #[command( @@ -168,9 +172,9 @@ impl Role { } fn validate_opts(opts: &ComputeNodeOpts) { - let total_memory_available_bytes = total_memory_available_bytes(); - if opts.total_memory_bytes > total_memory_available_bytes { - let error_msg = format!("total_memory_bytes {} is larger than the total memory available bytes {} that can be acquired.", opts.total_memory_bytes, total_memory_available_bytes); + let system_memory_available_bytes = system_memory_available_bytes(); + if opts.total_memory_bytes > system_memory_available_bytes { + let error_msg = format!("total_memory_bytes {} is larger than the total memory available bytes {} that can be acquired.", opts.total_memory_bytes, system_memory_available_bytes); tracing::error!(error_msg); panic!("{}", error_msg); } @@ -224,7 +228,7 @@ pub fn start(opts: ComputeNodeOpts) -> Pin + Send>> } fn default_total_memory_bytes() -> usize { - total_memory_available_bytes() + (system_memory_available_bytes() as f64 * DEFAULT_MEMORY_PROPORTION) as usize } fn default_parallelism() -> usize { diff --git a/src/connector/src/sink/formatter/append_only.rs b/src/connector/src/sink/formatter/append_only.rs index ba7d018cd7fbc..523a52dab91bb 100644 --- a/src/connector/src/sink/formatter/append_only.rs +++ b/src/connector/src/sink/formatter/append_only.rs @@ -19,12 +19,12 @@ use crate::sink::encoder::RowEncoder; use crate::tri; pub struct AppendOnlyFormatter { - key_encoder: KE, + key_encoder: Option, val_encoder: VE, } impl AppendOnlyFormatter { - pub fn new(key_encoder: KE, val_encoder: VE) -> Self { + pub fn new(key_encoder: Option, val_encoder: VE) -> Self { Self { key_encoder, val_encoder, @@ -45,7 +45,10 @@ impl SinkFormatter for AppendOnlyFormatter Some(tri!(key_encoder.encode(row))), + None => None, + }; let event_object = Some(tri!(self.val_encoder.encode(row))); yield Ok((event_key_object, event_object)) diff --git a/src/connector/src/sink/formatter/mod.rs b/src/connector/src/sink/formatter/mod.rs index be153ded9062e..61fe7a830c0a2 100644 --- a/src/connector/src/sink/formatter/mod.rs +++ b/src/connector/src/sink/formatter/mod.rs @@ -35,6 +35,10 @@ pub trait SinkFormatter { type K; type V; + /// * Key may be None so that messages are partitioned using round-robin. + /// For example append-only without `primary_key` (aka `downstream_pk`) set. + /// * Value may be None so that messages with same key are removed during log compaction. + /// For example debezium tombstone event. fn format_chunk( &self, chunk: &StreamChunk, @@ -80,11 +84,13 @@ impl SinkFormatterImpl { match format_desc.format { SinkFormat::AppendOnly => { - let key_encoder = JsonEncoder::new( - schema.clone(), - Some(pk_indices), - TimestampHandlingMode::Milli, - ); + let key_encoder = (!pk_indices.is_empty()).then(|| { + JsonEncoder::new( + schema.clone(), + Some(pk_indices), + TimestampHandlingMode::Milli, + ) + }); let val_encoder = JsonEncoder::new(schema, None, TimestampHandlingMode::Milli); let formatter = AppendOnlyFormatter::new(key_encoder, val_encoder); diff --git a/src/connector/src/sink/kafka.rs b/src/connector/src/sink/kafka.rs index 274be95f9b94a..ef4efe88e43f8 100644 --- a/src/connector/src/sink/kafka.rs +++ b/src/connector/src/sink/kafka.rs @@ -481,14 +481,12 @@ impl<'w> KafkaPayloadWriter<'w> { event_object: Option>, ) -> Result<()> { let topic = self.config.common.topic.clone(); - // here we assume the key part always exists and value part is optional. - // if value is None, we will skip the payload part. - let key_str = event_key_object.unwrap(); - let mut record = FutureRecord::<[u8], [u8]>::to(topic.as_str()).key(&key_str); - let payload; - if let Some(value) = event_object { - payload = value; - record = record.payload(&payload); + let mut record = FutureRecord::<[u8], [u8]>::to(topic.as_str()); + if let Some(key_str) = &event_key_object { + record = record.key(key_str); + } + if let Some(payload) = &event_object { + record = record.payload(payload); } // Send the data but not wait it to finish sinking // Will join all `DeliveryFuture` during commit @@ -737,19 +735,14 @@ mod test { }, ]); - // We do not specify primary key for this schema - let pk_indices = vec![]; let kafka_config = KafkaConfig::from_hashmap(properties)?; // Create the actual sink writer to Kafka let mut sink = KafkaLogSinker::new( kafka_config.clone(), SinkFormatterImpl::AppendOnlyJson(AppendOnlyFormatter::new( - JsonEncoder::new( - schema.clone(), - Some(pk_indices), - TimestampHandlingMode::Milli, - ), + // We do not specify primary key for this schema + None, JsonEncoder::new(schema, None, TimestampHandlingMode::Milli), )), ) diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index b1ad198045bf2..741bc1bb1db14 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -26,7 +26,7 @@ use serde_with::serde_as; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tokio_retry::Retry; -use super::catalog::{SinkFormat, SinkFormatDesc}; +use super::catalog::SinkFormatDesc; use super::SinkParam; use crate::common::KinesisCommon; use crate::dispatch_sink_formatter_impl; @@ -72,11 +72,11 @@ impl Sink for KinesisSink { const SINK_NAME: &'static str = KINESIS_SINK; async fn validate(&self) -> Result<()> { - // For upsert Kafka sink, the primary key must be defined. - if self.format_desc.format != SinkFormat::AppendOnly && self.pk_indices.is_empty() { + // Kinesis requires partition key. There is no builtin support for round-robin as in kafka/pulsar. + // https://docs.aws.amazon.com/kinesis/latest/APIReference/API_PutRecord.html#Streams-PutRecord-request-PartitionKey + if self.pk_indices.is_empty() { return Err(SinkError::Config(anyhow!( - "primary key not defined for {:?} kafka sink (please define in `primary_key` field)", - self.format_desc.format + "kinesis sink requires partition key (please define in `primary_key` field)", ))); } @@ -195,9 +195,12 @@ impl FormattedSink for KinesisSinkPayloadWriter { type V = Vec; async fn write_one(&mut self, k: Option, v: Option) -> Result<()> { - self.put_record(&k.unwrap(), v.unwrap_or_default()) - .await - .map(|_| ()) + self.put_record( + &k.ok_or_else(|| SinkError::Kinesis(anyhow!("no key provided")))?, + v.unwrap_or_default(), + ) + .await + .map(|_| ()) } } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index 0c42bbe3a3663..b694696842746 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -50,8 +50,8 @@ use crate::sink::coordinate::CoordinatedSinkWriter; use crate::sink::encoder::TimestampHandlingMode; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ - DummySinkCommitCoordinator, Result, Sink, SinkCommitCoordinator, SinkError, SinkParam, - SinkWriterParam, + DummySinkCommitCoordinator, Result, Sink, SinkCommitCoordinator, SinkError, SinkMetrics, + SinkParam, SinkWriterParam, }; use crate::ConnectorParams; @@ -106,11 +106,13 @@ impl Sink for RemoteSink { const SINK_NAME: &'static str = R::SINK_NAME; async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { - Ok( - RemoteSinkWriter::new(self.param.clone(), writer_param.connector_params) - .await? - .into_log_sinker(writer_param.sink_metrics), + Ok(RemoteSinkWriter::new( + self.param.clone(), + writer_param.connector_params, + writer_param.sink_metrics.clone(), ) + .await? + .into_log_sinker(writer_param.sink_metrics)) } async fn validate(&self) -> Result<()> { @@ -224,8 +226,12 @@ impl Sink for CoordinatedRemoteSink { "sink needs coordination should not have singleton input" )) })?, - CoordinatedRemoteSinkWriter::new(self.0.param.clone(), writer_param.connector_params) - .await?, + CoordinatedRemoteSinkWriter::new( + self.0.param.clone(), + writer_param.connector_params, + writer_param.sink_metrics.clone(), + ) + .await?, ) .await? .into_log_sinker(writer_param.sink_metrics)) @@ -355,11 +361,16 @@ pub struct RemoteSinkWriterInner { payload_format: SinkPayloadFormat, stream_handle: SinkWriterStreamJniHandle, json_encoder: JsonEncoder, + sink_metrics: SinkMetrics, _phantom: PhantomData<(SM, R)>, } impl RemoteSinkWriterInner { - pub async fn new(param: SinkParam, connector_params: ConnectorParams) -> Result { + pub async fn new( + param: SinkParam, + connector_params: ConnectorParams, + sink_metrics: SinkMetrics, + ) -> Result { let (request_tx, request_rx) = mpsc::channel(DEFAULT_CHANNEL_SIZE); let (response_tx, response_rx) = mpsc::channel(DEFAULT_CHANNEL_SIZE); @@ -440,6 +451,7 @@ impl RemoteSinkWriterInner { stream_handle, payload_format: connector_params.sink_payload_format, json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::String), + sink_metrics, _phantom: PhantomData, }) } @@ -479,6 +491,7 @@ impl RemoteSinkWriterInner { json_encoder: JsonEncoder::new(schema, None, TimestampHandlingMode::String), stream_handle, payload_format: SinkPayloadFormat::Json, + sink_metrics: SinkMetrics::for_test(), _phantom: PhantomData, } } @@ -529,9 +542,13 @@ where type CommitMetadata = SM; async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + let cardinality = chunk.cardinality(); + self.sink_metrics + .connector_sink_rows_received + .inc_by(cardinality as _); let payload = match self.payload_format { SinkPayloadFormat::Json => { - let mut row_ops = Vec::with_capacity(chunk.cardinality()); + let mut row_ops = Vec::with_capacity(cardinality); for (op, row_ref) in chunk.rows() { let map = self.json_encoder.encode(row_ref)?; let row_op = RowOp { diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index cef342b201788..a0c0aee86592e 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; use anyhow::{anyhow, Result}; use async_trait::async_trait; +use aws_sdk_s3::types::Object; use bytes::Bytes; use enum_as_inner::EnumAsInner; use futures::stream::BoxStream; @@ -40,7 +41,9 @@ use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use crate::parser::ParserConfig; pub(crate) use crate::source::common::CommonSplitReader; +use crate::source::filesystem::{FsPageItem, S3Properties, S3_V2_CONNECTOR}; use crate::source::monitor::EnumeratorMetrics; +use crate::source::S3_CONNECTOR; use crate::{ dispatch_source_prop, dispatch_split_impl, for_all_sources, impl_connector_properties, impl_split, match_source_name_str, @@ -48,6 +51,7 @@ use crate::{ const SPLIT_TYPE_FIELD: &str = "split_type"; const SPLIT_INFO_FIELD: &str = "split_info"; +const UPSTREAM_SOURCE_KEY: &str = "connector"; pub trait TryFromHashmap: Sized { fn try_from_hashmap(props: HashMap) -> Result; @@ -296,9 +300,50 @@ pub trait SplitReader: Sized + Send { for_all_sources!(impl_connector_properties); +impl ConnectorProperties { + pub fn is_new_fs_connector_b_tree_map(props: &BTreeMap) -> bool { + props + .get(UPSTREAM_SOURCE_KEY) + .map(|s| s.eq_ignore_ascii_case(S3_V2_CONNECTOR)) + .unwrap_or(false) + } + + pub fn is_new_fs_connector_hash_map(props: &HashMap) -> bool { + props + .get(UPSTREAM_SOURCE_KEY) + .map(|s| s.eq_ignore_ascii_case(S3_V2_CONNECTOR)) + .unwrap_or(false) + } + + pub fn rewrite_upstream_source_key_hash_map(props: &mut HashMap) { + let connector = props.remove(UPSTREAM_SOURCE_KEY).unwrap(); + match connector.as_str() { + S3_V2_CONNECTOR => { + tracing::info!( + "using new fs source, rewrite connector from '{}' to '{}'", + S3_V2_CONNECTOR, + S3_CONNECTOR + ); + props.insert(UPSTREAM_SOURCE_KEY.to_string(), S3_CONNECTOR.to_string()); + } + _ => { + props.insert(UPSTREAM_SOURCE_KEY.to_string(), connector); + } + } + } +} + impl ConnectorProperties { pub fn extract(mut props: HashMap) -> Result { - const UPSTREAM_SOURCE_KEY: &str = "connector"; + if Self::is_new_fs_connector_hash_map(&props) { + _ = props + .remove(UPSTREAM_SOURCE_KEY) + .ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; + return Ok(ConnectorProperties::S3(Box::new( + S3Properties::try_from_hashmap(props)?, + ))); + } + let connector = props .remove(UPSTREAM_SOURCE_KEY) .ok_or_else(|| anyhow!("Must specify 'connector' in WITH clause"))?; @@ -504,6 +549,17 @@ pub trait SplitMetaData: Sized { /// [`None`] and the created source stream will be a pending stream. pub type ConnectorState = Option>; +#[derive(Debug, Clone, Default)] +pub struct FsFilterCtrlCtx; +pub type FsFilterCtrlCtxRef = Arc; + +#[async_trait] +pub trait FsListInner: Sized { + // fixme: better to implement as an Iterator, but the last page still have some contents + async fn get_next_page From<&'a Object>>(&mut self) -> Result<(Vec, bool)>; + fn filter_policy(&self, ctx: &FsFilterCtrlCtx, page_num: usize, item: &FsPageItem) -> bool; +} + #[cfg(test)] mod tests { use maplit::*; diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index d4328289b547f..85d65f40a3479 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -12,7 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. use anyhow::anyhow; -use risingwave_common::types::JsonbVal; +use aws_sdk_s3::types::Object; +use risingwave_common::types::{JsonbVal, Timestamp}; use serde::{Deserialize, Serialize}; use crate::source::{SplitId, SplitMetaData}; @@ -26,6 +27,16 @@ pub struct FsSplit { pub size: usize, } +impl From<&Object> for FsSplit { + fn from(value: &Object) -> Self { + Self { + name: value.key().unwrap().to_owned(), + offset: 0, + size: value.size() as usize, + } + } +} + impl SplitMetaData for FsSplit { fn id(&self) -> SplitId { self.name.as_str().into() @@ -55,3 +66,23 @@ impl FsSplit { } } } + +#[derive(Clone, Debug)] +pub struct FsPageItem { + pub name: String, + pub size: i64, + pub timestamp: Timestamp, +} + +pub type FsPage = Vec; + +impl From<&Object> for FsPageItem { + fn from(value: &Object) -> Self { + let aws_ts = value.last_modified().unwrap(); + Self { + name: value.key().unwrap().to_owned(), + size: value.size(), + timestamp: Timestamp::from_timestamp_uncheck(aws_ts.secs(), aws_ts.subsec_nanos()), + } + } +} diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index 729fb376ecc6e..8f2587384280b 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -16,5 +16,7 @@ pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; mod file_common; pub mod nd_streaming; -pub use file_common::FsSplit; +pub use file_common::{FsPage, FsPageItem, FsSplit}; mod s3; +pub mod s3_v2; +pub const S3_V2_CONNECTOR: &str = "s3_v2"; diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index 6ca19e673aa33..7c16c087d14c6 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -12,20 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Context}; +use anyhow::Context; use async_trait::async_trait; use aws_sdk_s3::client::Client; -use aws_sdk_s3::error::DisplayErrorContext; -use itertools::Itertools; use crate::aws_auth::AwsAuthProps; use crate::aws_utils::{default_conn_config, s3_client}; use crate::source::filesystem::file_common::FsSplit; use crate::source::filesystem::s3::S3Properties; -use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; +use crate::source::{FsListInner, SourceEnumeratorContextRef, SplitEnumerator}; /// Get the prefix from a glob -fn get_prefix(glob: &str) -> String { +pub fn get_prefix(glob: &str) -> String { let mut escaped = false; let mut escaped_filter = false; glob.chars() @@ -59,11 +57,14 @@ fn get_prefix(glob: &str) -> String { #[derive(Debug, Clone)] pub struct S3SplitEnumerator { - bucket_name: String, + pub(crate) bucket_name: String, // prefix is used to reduce the number of objects to be listed - prefix: Option, - matcher: Option, - client: Client, + pub(crate) prefix: Option, + pub(crate) matcher: Option, + pub(crate) client: Client, + + // token get the next page, used when the current page is truncated + pub(crate) next_continuation_token: Option, } #[async_trait] @@ -92,53 +93,26 @@ impl SplitEnumerator for S3SplitEnumerator { matcher, prefix, client: s3_client, + next_continuation_token: None, }) } async fn list_splits(&mut self) -> anyhow::Result> { let mut objects = Vec::new(); - let mut next_continuation_token = None; loop { - let mut req = self - .client - .list_objects_v2() - .bucket(&self.bucket_name) - .set_prefix(self.prefix.clone()); - if let Some(continuation_token) = next_continuation_token.take() { - req = req.continuation_token(continuation_token); - } - let mut res = req - .send() - .await - .map_err(|e| anyhow!(DisplayErrorContext(e)))?; - objects.extend(res.contents.take().unwrap_or_default()); - if res.is_truncated() { - next_continuation_token = Some(res.next_continuation_token.unwrap()) - } else { + let (files, has_finished) = self.get_next_page::().await?; + objects.extend(files); + if has_finished { break; } } - - let matched_objs = objects - .iter() - .filter(|obj| obj.key().is_some()) - .filter(|obj| { - self.matcher - .as_ref() - .map(|m| m.matches(obj.key().unwrap())) - .unwrap_or(true) - }) - .collect_vec(); - - Ok(matched_objs - .into_iter() - .map(|obj| FsSplit::new(obj.key().unwrap().to_owned(), 0, obj.size() as usize)) - .collect_vec()) + Ok(objects) } } #[cfg(test)] mod tests { + use itertools::Itertools; #[test] fn test_get_prefix() { diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 12701087309e0..464a11f99fc09 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -37,7 +37,7 @@ pub struct S3Properties { #[serde(rename = "s3.credentials.secret", default)] pub secret: Option, #[serde(rename = "s3.endpoint_url")] - endpoint_url: Option, + pub endpoint_url: Option, } impl SourceProperties for S3Properties { diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 736e4493d3f55..b1e368a2b409e 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -55,7 +55,7 @@ pub struct S3FileReader { impl S3FileReader { #[try_stream(boxed, ok = Vec, error = anyhow::Error)] - async fn stream_read_object( + pub async fn stream_read_object( client_for_s3: s3_client::Client, bucket_name: String, split: FsSplit, @@ -137,7 +137,7 @@ impl S3FileReader { } } - async fn get_object( + pub async fn get_object( client_for_s3: &s3_client::Client, bucket_name: &str, object_name: &str, diff --git a/src/connector/src/source/filesystem/s3_v2/lister.rs b/src/connector/src/source/filesystem/s3_v2/lister.rs new file mode 100644 index 0000000000000..3c55f23d37f67 --- /dev/null +++ b/src/connector/src/source/filesystem/s3_v2/lister.rs @@ -0,0 +1,66 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use anyhow::anyhow; +use async_trait::async_trait; +use aws_sdk_s3::error::DisplayErrorContext; +use aws_sdk_s3::types::Object; +use itertools::Itertools; + +use crate::source::filesystem::{FsPageItem, S3SplitEnumerator}; +use crate::source::{FsFilterCtrlCtx, FsListInner}; + +#[async_trait] +impl FsListInner for S3SplitEnumerator { + async fn get_next_page From<&'a Object>>( + &mut self, + ) -> anyhow::Result<(Vec, bool)> { + let mut has_finished = false; + let mut req = self + .client + .list_objects_v2() + .bucket(&self.bucket_name) + .set_prefix(self.prefix.clone()); + if let Some(continuation_token) = self.next_continuation_token.take() { + req = req.continuation_token(continuation_token); + } + let mut res = req + .send() + .await + .map_err(|e| anyhow!(DisplayErrorContext(e)))?; + if res.is_truncated() { + self.next_continuation_token = res.next_continuation_token.clone(); + } else { + has_finished = true; + self.next_continuation_token = None; + } + let objects = res.contents.take().unwrap_or_default(); + let matched_objs: Vec = objects + .iter() + .filter(|obj| obj.key().is_some()) + .filter(|obj| { + self.matcher + .as_ref() + .map(|m| m.matches(obj.key().unwrap())) + .unwrap_or(true) + }) + .map(T::from) + .collect_vec(); + Ok((matched_objs, has_finished)) + } + + fn filter_policy(&self, _ctx: &FsFilterCtrlCtx, _page_num: usize, _item: &FsPageItem) -> bool { + true + } +} diff --git a/src/connector/src/source/filesystem/s3_v2/mod.rs b/src/connector/src/source/filesystem/s3_v2/mod.rs new file mode 100644 index 0000000000000..6fab862daca1e --- /dev/null +++ b/src/connector/src/source/filesystem/s3_v2/mod.rs @@ -0,0 +1,15 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +pub mod lister; diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 762af05cd0c96..869b7089ac271 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -39,5 +39,6 @@ pub mod test_source; pub use manager::{SourceColumnDesc, SourceColumnType}; pub use mock_external_table::MockExternalTableReader; +pub use crate::source::filesystem::{S3_CONNECTOR, S3_V2_CONNECTOR}; pub use crate::source::nexmark::NEXMARK_CONNECTOR; pub use crate::source::pulsar::PULSAR_CONNECTOR; diff --git a/src/expr/core/src/aggregate/def.rs b/src/expr/core/src/aggregate/def.rs index 39d4c158c10d7..f71bfd454a415 100644 --- a/src/expr/core/src/aggregate/def.rs +++ b/src/expr/core/src/aggregate/def.rs @@ -308,11 +308,7 @@ pub mod agg_kinds { #[macro_export] macro_rules! unimplemented_in_stream { () => { - AggKind::JsonbAgg - | AggKind::JsonbObjectAgg - | AggKind::PercentileCont - | AggKind::PercentileDisc - | AggKind::Mode + AggKind::PercentileCont | AggKind::PercentileDisc | AggKind::Mode }; } pub use unimplemented_in_stream; diff --git a/src/frontend/planner_test/tests/testdata/input/union.yaml b/src/frontend/planner_test/tests/testdata/input/union.yaml index 2d7a005d12e21..8775d4f9d36f2 100644 --- a/src/frontend/planner_test/tests/testdata/input/union.yaml +++ b/src/frontend/planner_test/tests/testdata/input/union.yaml @@ -53,3 +53,45 @@ expected_outputs: - batch_plan - optimized_logical_plan_for_batch +- name: test merged union stream key (2 columns, row_id + src_col) + sql: | + create table t1 (a int, b numeric, c bigint); + create table t2 (a int, b numeric, c bigint); + create table t3 (a int, b numeric, c bigint); + create table t4 (a int, b numeric, c bigint); + create table t5 (a int, b numeric, c bigint); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- name: test merged union stream key (5 columns, row_id + src_col + a + b + c) + sql: | + create table t1 (a int, b numeric, c bigint, primary key (a)); + create table t2 (a int, b numeric, c bigint, primary key (b)); + create table t3 (a int, b numeric, c bigint, primary key (c)); + create table t4 (a int, b numeric, c bigint); + create table t5 (a int, b numeric, c bigint, primary key (a, b)); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + expected_outputs: + - stream_dist_plan +- name: test merged union stream key (4 columns, row_id + src_col + a + b) + sql: | + create table t1 (a int, b numeric, c bigint, primary key (a)); + create table t2 (a int, b numeric, c bigint, primary key (b)); + create table t3 (a int, b numeric, c bigint); + create table t4 (a int, b numeric, c bigint); + create table t5 (a int, b numeric, c bigint, primary key (a, b)); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + expected_outputs: + - stream_dist_plan +- name: test merged union stream key (3 columns, src_col + a + b) + sql: | + create table t1 (a int, b numeric, c bigint, primary key (a)); + create table t2 (a int, b numeric, c bigint, primary key (b)); + create table t3 (a int, b numeric, c bigint, primary key (b)); + create table t4 (a int, b numeric, c bigint, primary key (b, a)); + create table t5 (a int, b numeric, c bigint, primary key (a, b)); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + expected_outputs: + - stream_dist_plan diff --git a/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml b/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml index 50d0617e68f0d..d2b81cf331334 100644 --- a/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/bushy_join.yaml @@ -12,7 +12,7 @@ └─StreamSimpleAgg { aggs: [sum0(count), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [count] } - └─StreamHashJoin { type: Inner, predicate: t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id, output: [t._row_id, t._row_id, t.id, t._row_id, t._row_id, t.id, t.id, t.id, t._row_id, t._row_id, t.id, t._row_id, t._row_id, t.id, t.id] } + └─StreamHashJoin { type: Inner, predicate: t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id, output: [t._row_id, t._row_id, t.id, t._row_id, t._row_id, t.id, t.id, t.id, t._row_id, t._row_id, t.id, t._row_id, t._row_id, t.id] } ├─StreamExchange { dist: HashShard(t.id) } │ └─StreamHashJoin { type: Inner, predicate: t.id = t.id AND t.id = t.id AND t.id = t.id AND t.id = t.id, output: [t.id, t.id, t.id, t.id, t._row_id, t._row_id, t._row_id, t._row_id] } │ ├─StreamExchange { dist: HashShard(t.id) } 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 4fea4316481e4..6f4f8a673c996 100644 --- a/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/ch_benchmark.yaml @@ -133,17 +133,17 @@ └─BatchProject { exprs: [stock.s_i_id, stock.s_quantity, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } └─BatchScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], distribution: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } stream_plan: |- - StreamMaterialize { columns: [s_suppkey, s_name, n_name, i_id, i_name, s_address, s_phone, s_comment, stock.s_i_id(hidden), stock.s_w_id(hidden), stock.s_i_id#1(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, i_id, stock.s_w_id, stock.s_i_id#1, min(stock.s_quantity), region.r_regionkey, s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, stock.s_i_id#1, min(stock.s_quantity), region.r_regionkey, s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } - └─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, stock.s_i_id, min(stock.s_quantity), $expr2, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } + 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 } + └─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, stock.s_i_id, min(stock.s_quantity)] } + │ └─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)] } │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } │ ├─StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } │ │ └─StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } │ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } - │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, stock.s_w_id] } + │ │ └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } │ │ ├─StreamExchange { dist: HashShard(supplier.s_suppkey) } - │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } + │ │ │ └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } │ │ │ ├─StreamExchange { dist: HashShard(region.r_regionkey) } │ │ │ │ └─StreamProject { exprs: [region.r_regionkey] } │ │ │ │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } @@ -166,7 +166,7 @@ │ └─StreamExchange { dist: HashShard(stock.s_i_id) } │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id, stock.s_quantity], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─StreamExchange { dist: HashShard(supplier.s_suppkey) } - └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } + └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard(region.r_regionkey) } │ └─StreamProject { exprs: [region.r_regionkey] } │ └─StreamFilter { predicate: Like(region.r_name, 'EUROP%':Varchar) } @@ -179,14 +179,14 @@ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } stream_dist_plan: |+ 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), stock.s_i_id#1(hidden), min(stock.s_quantity)(hidden), $expr2(hidden), region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], stream_key: [stock.s_i_id, i_id, stock.s_w_id, stock.s_i_id#1, min(stock.s_quantity), region.r_regionkey, s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr2], pk_columns: [n_name, s_name, i_id, stock.s_i_id, stock.s_w_id, stock.s_i_id#1, min(stock.s_quantity), region.r_regionkey, s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr2], pk_conflict: NoCheck } + 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 } ├── materialized table: 4294967294 - └── 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, stock.s_i_id, min(stock.s_quantity), $expr2, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + └── 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] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([2]) from 1 └── StreamExchange Hash([0]) from 11 Fragment 1 - 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, stock.s_i_id, min(stock.s_quantity)] } + 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)] } └── StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id AND min(stock.s_quantity) = stock.s_quantity AND stock.s_i_id = stock.s_i_id, output: [item.i_id, item.i_name, stock.s_i_id, stock.s_w_id, stock.s_i_id, min(stock.s_quantity)] } { left table: 4, right table: 6, left degree table: 5, right degree table: 7 } ├── StreamProject { exprs: [stock.s_i_id, min(stock.s_quantity)] } │ └── StreamHashAgg { group_key: [stock.s_i_id], aggs: [min(stock.s_quantity), count] } { intermediate state table: 9, state tables: [ 8 ], distinct tables: [] } @@ -196,12 +196,12 @@ └── StreamExchange Hash([0]) from 10 Fragment 2 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } + StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_quantity, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey, stock.s_w_id] } { left table: 10, right table: 12, left degree table: 11, right degree table: 13 } ├── StreamExchange Hash([0]) from 3 └── StreamExchange Hash([2]) from 8 Fragment 3 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 14, right table: 16, left degree table: 15, right degree table: 17 } ├── StreamExchange Hash([0]) from 4 └── StreamExchange Hash([1]) from 5 @@ -247,7 +247,7 @@ └── BatchPlanNode Fragment 11 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 } + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, supplier.s_comment, nation.n_name, region.r_regionkey, supplier.s_nationkey] } { left table: 32, right table: 34, left degree table: 33, right degree table: 35 } ├── StreamExchange Hash([0]) from 12 └── StreamExchange Hash([6]) from 13 @@ -273,13 +273,13 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ item_i_id, item_i_name, $expr2, stock_s_i_id, stock_s_w_id, stock_s_i_id_0, min(stock_s_quantity) ], primary key: [ $2 ASC, $3 ASC, $0 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 0 { columns: [ item_i_id, item_i_name, $expr2, stock_s_i_id, stock_s_w_id, min(stock_s_quantity) ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 1 { columns: [ $expr2, stock_s_i_id, item_i_id, stock_s_w_id, stock_s_i_id_0, min(stock_s_quantity), _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ $expr2, stock_s_i_id, stock_s_w_id, min(stock_s_quantity), _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey ], primary key: [ $0 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 { columns: [ stock_s_i_id, min(stock_s_quantity) ], primary key: [ $0 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 3 } @@ -289,13 +289,13 @@ Table 7 { columns: [ item_i_id, stock_s_quantity, stock_s_i_id, stock_s_w_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 3 } - Table 8 { columns: [ stock_s_i_id, stock_s_quantity, region_r_regionkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, stock_s_w_id ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ stock_s_i_id, stock_s_quantity, region_r_regionkey, supplier_s_suppkey, supplier_s_nationkey, stock_s_w_id ], primary key: [ $0 ASC, $1 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: 1 } Table 9 { columns: [ stock_s_i_id, min(stock_s_quantity), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 10 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 10 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 11 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 11 { columns: [ supplier_s_suppkey, region_r_regionkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 12 { columns: [ stock_s_i_id, stock_s_quantity, $expr1, stock_s_w_id ], primary key: [ $2 ASC, $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 1 } @@ -305,9 +305,9 @@ Table 15 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 16 { columns: [ supplier_s_suppkey, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $1 ASC, $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 16 { columns: [ supplier_s_suppkey, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $1 ASC, $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 17 { columns: [ nation_n_regionkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 17 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 18 { columns: [ vnode, r_regionkey, region_backfill_finished, region_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } @@ -341,9 +341,9 @@ Table 33 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 34 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $6 ASC, $0 ASC, $8 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + Table 34 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, supplier_s_comment, nation_n_name, nation_n_regionkey, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $6 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 6 ], read pk prefix len hint: 1 } - Table 35 { columns: [ nation_n_regionkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 35 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 36 { columns: [ vnode, r_regionkey, region_backfill_finished, region_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } @@ -359,7 +359,7 @@ Table 42 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], 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, stock.s_i_id#1, min(stock.s_quantity), $expr2, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey ], primary key: [ $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $0 ASC, $14 ASC, $15 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 12 ], read pk prefix len hint: 12 } + 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 ], 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: [ 11 ], read pk prefix len hint: 9 } - id: ch_q3 before: @@ -412,7 +412,7 @@ └─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_w_id, new_order.no_d_id, new_order.no_o_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } + └─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) } @@ -441,7 +441,7 @@ └── StreamExchange Hash([0, 1, 2, 3]) 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 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_w_id, new_order.no_d_id, new_order.no_o_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } + 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] } ├── left table: 1 ├── right table: 3 ├── left degree table: 2 @@ -675,10 +675,10 @@ └─StreamProject { exprs: [nation.n_name, sum(order_line.ol_amount)] } └─StreamHashAgg { group_key: [nation.n_name], aggs: [sum(order_line.ol_amount), count] } └─StreamExchange { dist: HashShard(nation.n_name) } - └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey AND $expr2 = supplier.s_nationkey, output: [order_line.ol_amount, nation.n_name, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, stock.s_w_id, stock.s_i_id, order_line.ol_i_id, $expr1, $expr2, region.r_regionkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey AND $expr2 = supplier.s_nationkey, output: [order_line.ol_amount, nation.n_name, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_id, order_line.ol_number, order_line.ol_i_id, orders.o_w_id, orders.o_d_id, $expr1, $expr2, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard($expr1, $expr2) } - │ └─StreamProject { exprs: [order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, stock.s_w_id, stock.s_i_id, order_line.ol_i_id] } - │ └─StreamHashJoin { type: Inner, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND orders.o_w_id = stock.s_w_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id] } + │ └─StreamProject { exprs: [order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_id, order_line.ol_number, order_line.ol_i_id, orders.o_w_id, orders.o_d_id] } + │ └─StreamHashJoin { type: Inner, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND orders.o_w_id = stock.s_w_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_id, orders.o_w_id, orders.o_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id] } │ ├─StreamExchange { dist: HashShard(orders.o_id, customer.c_d_id, customer.c_w_id, orders.o_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: customer.c_id = orders.o_c_id AND customer.c_w_id = orders.o_w_id AND customer.c_d_id = orders.o_d_id, output: [customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_id, orders.o_d_id, orders.o_w_id, customer.c_id] } │ │ ├─StreamExchange { dist: HashShard(customer.c_d_id, customer.c_w_id, customer.c_id) } @@ -695,7 +695,7 @@ │ └─StreamExchange { dist: HashShard(stock.s_w_id, stock.s_i_id) } │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } └─StreamExchange { dist: HashShard(supplier.s_suppkey, supplier.s_nationkey) } - └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_nationkey, nation.n_name, region.r_regionkey, nation.n_nationkey] } + └─StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_nationkey, nation.n_name, region.r_regionkey] } ├─StreamExchange { dist: HashShard(region.r_regionkey) } │ └─StreamProject { exprs: [region.r_regionkey] } │ └─StreamFilter { predicate: (region.r_name = 'EUROPE':Varchar) } @@ -718,7 +718,7 @@ └── StreamExchange Hash([1]) from 1 Fragment 1 - StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey AND $expr2 = supplier.s_nationkey, output: [order_line.ol_amount, nation.n_name, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, stock.s_w_id, stock.s_i_id, order_line.ol_i_id, $expr1, $expr2, region.r_regionkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey AND $expr2 = supplier.s_nationkey, output: [order_line.ol_amount, nation.n_name, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_id, order_line.ol_number, order_line.ol_i_id, orders.o_w_id, orders.o_d_id, $expr1, $expr2, region.r_regionkey, supplier.s_suppkey, supplier.s_nationkey] } ├── left table: 1 ├── right table: 3 ├── left degree table: 2 @@ -727,8 +727,8 @@ └── StreamExchange Hash([0, 1]) from 9 Fragment 2 - StreamProject { exprs: [order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, stock.s_w_id, stock.s_i_id, order_line.ol_i_id] } - └── StreamHashJoin { type: Inner, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND orders.o_w_id = stock.s_w_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id] } + StreamProject { exprs: [order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_id, order_line.ol_number, order_line.ol_i_id, orders.o_w_id, orders.o_d_id] } + └── StreamHashJoin { type: Inner, predicate: orders.o_id = order_line.ol_o_id AND orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id AND orders.o_w_id = stock.s_w_id, output: [customer.c_state, order_line.ol_amount, stock.s_i_id, stock.s_w_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_id, orders.o_w_id, orders.o_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id] } ├── left table: 5 ├── right table: 7 ├── left degree table: 6 @@ -770,7 +770,7 @@ └── BatchPlanNode Fragment 9 - StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_nationkey, nation.n_name, region.r_regionkey, nation.n_nationkey] } { left table: 21, right table: 23, left degree table: 22, right degree table: 24 } + StreamHashJoin { type: Inner, predicate: region.r_regionkey = nation.n_regionkey, output: [supplier.s_suppkey, supplier.s_nationkey, nation.n_name, region.r_regionkey] } { left table: 21, right table: 23, left degree table: 22, right degree table: 24 } ├── StreamExchange Hash([0]) from 10 └── StreamExchange Hash([3]) from 11 @@ -798,26 +798,21 @@ Table 0 { columns: [ nation_n_name, sum(order_line_ol_amount), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 - ├── columns: [ order_line_ol_amount, $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, stock_s_w_id, stock_s_i_id, order_line_ol_i_id ] - ├── primary key: [ $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] - ├── distribution key: [ 1, 2 ] - └── read pk prefix len hint: 2 + Table 1 { columns: [ order_line_ol_amount, $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_id, order_line_ol_number, order_line_ol_i_id, orders_o_w_id, orders_o_d_id ], primary key: [ $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1, 2 ], read pk prefix len hint: 2 } - Table 2 { columns: [ $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, stock_s_w_id, stock_s_i_id, order_line_ol_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC ], value indices: [ 15 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 2 { columns: [ $expr1, $expr2, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_id, order_line_ol_number, order_line_ol_i_id, orders_o_w_id, orders_o_d_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, region_r_regionkey, nation_n_nationkey ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, region_r_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 5 { columns: [ customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_id, orders_o_d_id, orders_o_w_id, customer_c_id ], primary key: [ $3 ASC, $5 ASC, $4 ASC, $0 ASC, $1 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 0, 1, 5 ], read pk prefix len hint: 6 } Table 6 { columns: [ orders_o_id, orders_o_w_id, orders_o_d_id, customer_c_d_id, customer_c_w_id, orders_o_w_id_0, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 0, 3, 4, 5 ], read pk prefix len hint: 6 } - Table 7 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, stock_s_i_id, stock_s_w_id, order_line_ol_number, order_line_ol_i_id ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $1 ASC, $2 ASC, $5 ASC, $6 ASC, $4 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 5 ], read pk prefix len hint: 6 } + Table 7 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, stock_s_i_id, stock_s_w_id, order_line_ol_number, order_line_ol_i_id ], primary key: [ $0 ASC, $2 ASC, $1 ASC, $1 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 5 ], read pk prefix len hint: 6 } - Table 8 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_d_id_0, order_line_ol_w_id_0, stock_s_w_id, order_line_ol_number, stock_s_i_id, order_line_ol_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 0, 3, 4, 5 ], read pk prefix len hint: 6 } + Table 8 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_d_id_0, order_line_ol_w_id_0, stock_s_w_id, order_line_ol_number, order_line_ol_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0, 3, 4, 5 ], read pk prefix len hint: 6 } Table 9 { columns: [ customer_c_id, customer_c_d_id, customer_c_w_id, customer_c_state ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1, 2, 0 ], read pk prefix len hint: 3 } @@ -847,9 +842,9 @@ Table 22 { columns: [ region_r_regionkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 23 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, nation_n_regionkey, nation_n_nationkey ], primary key: [ $3 ASC, $0 ASC, $4 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3 ], read pk prefix len hint: 1 } + Table 23 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, nation_n_regionkey, nation_n_nationkey ], primary key: [ $3 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 24 { columns: [ nation_n_regionkey, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 24 { columns: [ nation_n_regionkey, supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 25 { columns: [ vnode, r_regionkey, region_backfill_finished, region_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } @@ -996,15 +991,15 @@ └─StreamProject { exprs: [supplier.s_nationkey, $expr3, $expr4, sum(order_line.ol_amount), supplier.s_nationkey] } └─StreamHashAgg { group_key: [supplier.s_nationkey, $expr3, $expr4], aggs: [sum(order_line.ol_amount), count] } └─StreamExchange { dist: HashShard(supplier.s_nationkey, $expr3, $expr4) } - └─StreamProject { exprs: [supplier.s_nationkey, Substr(customer.c_state, 1:Int32, 1:Int32) as $expr3, Extract('YEAR':Varchar, orders.o_entry_d) as $expr4, order_line.ol_amount, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, nation.n_nationkey, $expr1, supplier.s_suppkey, nation.n_nationkey, $expr2] } + └─StreamProject { exprs: [supplier.s_nationkey, Substr(customer.c_state, 1:Int32, 1:Int32) as $expr3, Extract('YEAR':Varchar, orders.o_entry_d) as $expr4, order_line.ol_amount, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_c_id, $expr1, $expr2] } └─StreamFilter { predicate: (((nation.n_name = 'JAPAN':Varchar) AND (nation.n_name = 'CHINA':Varchar)) OR ((nation.n_name = 'CHINA':Varchar) AND (nation.n_name = 'JAPAN':Varchar))) } └─StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: all } ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, nation.n_nationkey, $expr1] } - │ └─StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, $expr1, nation.n_nationkey] } + │ └─StreamProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_c_id, $expr1] } + │ └─StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_c_id, $expr1, nation.n_nationkey] } │ ├─StreamExchange { dist: HashShard($expr1) } - │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, (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, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id] } - │ │ └─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: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id] } + │ │ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, (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, orders.o_c_id] } + │ │ └─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: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id] } │ │ ├─StreamExchange { dist: HashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_w_id) } │ │ │ └─StreamHashJoin { type: Inner, predicate: stock.s_w_id = order_line.ol_supply_w_id AND stock.s_i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_amount, order_line.ol_number] } │ │ │ ├─StreamExchange { dist: HashShard(stock.s_i_id, stock.s_w_id) } @@ -1036,15 +1031,15 @@ └── StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 - StreamProject { exprs: [supplier.s_nationkey, Substr(customer.c_state, 1:Int32, 1:Int32) as $expr3, Extract('YEAR':Varchar, orders.o_entry_d) as $expr4, order_line.ol_amount, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, nation.n_nationkey, $expr1, supplier.s_suppkey, nation.n_nationkey, $expr2] } + StreamProject { exprs: [supplier.s_nationkey, Substr(customer.c_state, 1:Int32, 1:Int32) as $expr3, Extract('YEAR':Varchar, orders.o_entry_d) as $expr4, order_line.ol_amount, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_c_id, $expr1, $expr2] } └── StreamFilter { predicate: (((nation.n_name = 'JAPAN':Varchar) AND (nation.n_name = 'CHINA':Varchar)) OR ((nation.n_name = 'CHINA':Varchar) AND (nation.n_name = 'JAPAN':Varchar))) } └── StreamHashJoin { type: Inner, predicate: $expr2 = supplier.s_suppkey, output: all } { left table: 1, right table: 3, left degree table: 2, right degree table: 4 } ├── StreamExchange Hash([4]) from 2 └── StreamExchange Hash([0]) from 11 Fragment 2 - StreamProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, nation.n_nationkey, $expr1] } - └── StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, $expr1, nation.n_nationkey] } + StreamProject { exprs: [order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr2, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_c_id, $expr1] } + └── StreamHashJoin { type: Inner, predicate: $expr1 = nation.n_nationkey, output: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, nation.n_name, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_c_id, $expr1, nation.n_nationkey] } ├── left table: 5 ├── right table: 7 ├── left degree table: 6 @@ -1053,8 +1048,8 @@ └── StreamExchange Hash([0]) from 10 Fragment 3 - StreamProject { exprs: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, (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, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id] } - └── 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: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id] } + StreamProject { exprs: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, (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, orders.o_c_id] } + └── 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: [stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_entry_d, customer.c_state, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id] } ├── left table: 9 ├── right table: 11 ├── left degree table: 10 @@ -1116,27 +1111,17 @@ Table 0 { columns: [ supplier_s_nationkey, $expr3, $expr4, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 1 - ├── columns: [ order_line_ol_amount, orders_o_entry_d, customer_c_state, nation_n_name, $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id, nation_n_nationkey, $expr1 ] - ├── primary key: [ $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $18 ASC, $19 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19 ] - ├── distribution key: [ 4 ] - └── read pk prefix len hint: 1 + Table 1 { columns: [ order_line_ol_amount, orders_o_entry_d, customer_c_state, nation_n_name, $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, $expr1 ], primary key: [ $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id, nation_n_nationkey, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC ], value indices: [ 16 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr2, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_name, nation_n_nationkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 - ├── columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_amount, orders_o_entry_d, customer_c_state, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id ] - ├── primary key: [ $5 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16 ] - ├── distribution key: [ 5 ] - └── read pk prefix len hint: 1 + Table 5 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_amount, orders_o_entry_d, customer_c_state, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id ], primary key: [ $5 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 5 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC ], value indices: [ 14 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1146,9 +1131,9 @@ Table 10 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, stock_s_w_id, stock_s_i_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 2, 1, 0, 3, 4 ], read pk prefix len hint: 5 } - Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $8 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 1, 2, 4, 5 ], read pk prefix len hint: 5 } + Table 11 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 1, 2, 4, 5 ], read pk prefix len hint: 5 } - Table 12 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, customer_c_id, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 2, 1, 0, 3, 4 ], read pk prefix len hint: 5 } + Table 12 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 1, 0, 3, 4 ], read pk prefix len hint: 5 } Table 13 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } @@ -1276,14 +1261,14 @@ └─StreamProject { exprs: [$expr3, (sum($expr4) / sum(order_line.ol_amount)) as $expr5] } └─StreamHashAgg { group_key: [$expr3], aggs: [sum($expr4), sum(order_line.ol_amount), count] } └─StreamExchange { dist: HashShard($expr3) } - └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_entry_d) as $expr3, Case((nation.n_name = 'INDIA':Varchar), order_line.ol_amount, 0:Decimal) as $expr4, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1, nation.n_nationkey, region.r_regionkey, nation.n_regionkey, $expr2] } - └─StreamHashJoin { type: Inner, predicate: $expr2 = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1, $expr2, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_entry_d) as $expr3, Case((nation.n_name = 'INDIA':Varchar), order_line.ol_amount, 0:Decimal) as $expr4, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, supplier.s_nationkey, $expr1, nation.n_regionkey, $expr2] } + └─StreamHashJoin { type: Inner, predicate: $expr2 = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, supplier.s_nationkey, $expr1, $expr2, nation.n_nationkey, nation.n_regionkey] } ├─StreamExchange { dist: HashShard($expr2) } - │ └─StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, nation.n_name, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1] } - │ └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [orders.o_entry_d, customer.c_state, order_line.ol_amount, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + │ └─StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, nation.n_name, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, supplier.s_nationkey, $expr1] } + │ └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [orders.o_entry_d, customer.c_state, order_line.ol_amount, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, $expr1, supplier.s_suppkey, supplier.s_nationkey] } │ ├─StreamExchange { dist: HashShard($expr1) } - │ │ └─StreamProject { exprs: [orders.o_entry_d, customer.c_state, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ │ └─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 AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id, output: [orders.o_entry_d, customer.c_state, stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ │ └─StreamProject { exprs: [orders.o_entry_d, customer.c_state, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id] } + │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id, output: [orders.o_entry_d, customer.c_state, stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, customer.c_d_id, customer.c_w_id, item.i_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, customer.c_d_id, customer.c_w_id) } │ │ │ └─StreamHashJoin { type: Inner, predicate: orders.o_c_id = customer.c_id AND orders.o_w_id = customer.c_w_id AND orders.o_d_id = customer.c_d_id, output: [orders.o_id, orders.o_d_id, orders.o_w_id, orders.o_entry_d, customer.c_d_id, customer.c_w_id, customer.c_state, orders.o_c_id, customer.c_id] } │ │ │ ├─StreamExchange { dist: HashShard(orders.o_d_id, orders.o_w_id, orders.o_c_id) } @@ -1330,8 +1315,8 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_entry_d) as $expr3, Case((nation.n_name = 'INDIA':Varchar), order_line.ol_amount, 0:Decimal) as $expr4, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1, nation.n_nationkey, region.r_regionkey, nation.n_regionkey, $expr2] } - └── StreamHashJoin { type: Inner, predicate: $expr2 = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1, $expr2, nation.n_nationkey, region.r_regionkey, nation.n_regionkey] } + StreamProject { exprs: [Extract('YEAR':Varchar, orders.o_entry_d) as $expr3, Case((nation.n_name = 'INDIA':Varchar), order_line.ol_amount, 0:Decimal) as $expr4, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, supplier.s_nationkey, $expr1, nation.n_regionkey, $expr2] } + └── StreamHashJoin { type: Inner, predicate: $expr2 = nation.n_nationkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, supplier.s_nationkey, $expr1, $expr2, nation.n_nationkey, nation.n_regionkey] } ├── left table: 1 ├── right table: 3 ├── left degree table: 2 @@ -1340,14 +1325,18 @@ └── StreamExchange Hash([0]) from 15 Fragment 2 - StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, nation.n_name, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1] } - └── StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [orders.o_entry_d, customer.c_state, order_line.ol_amount, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 5, right table: 7, left degree table: 6, right degree table: 8 } + StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, nation.n_name, (Ascii(Substr(customer.c_state, 1:Int32, 1:Int32)) - 65:Int32)::Int64 as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, supplier.s_nationkey, $expr1] } + └── StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [orders.o_entry_d, customer.c_state, order_line.ol_amount, nation.n_name, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id, $expr1, supplier.s_suppkey, supplier.s_nationkey] } + ├── left table: 5 + ├── right table: 7 + ├── left degree table: 6 + ├── right degree table: 8 ├── StreamExchange Hash([3]) from 3 └── StreamExchange Hash([0]) from 12 Fragment 3 - StreamProject { exprs: [orders.o_entry_d, customer.c_state, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, stock.s_w_id, stock.s_i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - └── 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 AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id, output: [orders.o_entry_d, customer.c_state, stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, customer.c_w_id, customer.c_d_id, customer.c_id, orders.o_c_id, item.i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + StreamProject { exprs: [orders.o_entry_d, customer.c_state, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, item.i_id, stock.s_w_id, order_line.ol_number, stock.s_i_id, customer.c_d_id, customer.c_w_id] } + └── StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id AND customer.c_d_id = order_line.ol_d_id AND customer.c_w_id = order_line.ol_w_id, output: [orders.o_entry_d, customer.c_state, stock.s_i_id, stock.s_w_id, order_line.ol_amount, orders.o_w_id, orders.o_d_id, orders.o_id, orders.o_c_id, customer.c_d_id, customer.c_w_id, item.i_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } ├── left table: 9 ├── right table: 11 ├── left degree table: 10 @@ -1434,43 +1423,33 @@ Table 0 { columns: [ $expr3, sum($expr4), sum(order_line_ol_amount), count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ orders_o_entry_d, order_line_ol_amount, nation_n_name, $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id, item_i_id, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, $expr1 ] - ├── primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $18 ASC, $19 ASC, $20 ASC, $21 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21 ] + ├── columns: [ orders_o_entry_d, order_line_ol_amount, nation_n_name, $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, supplier_s_nationkey, $expr1 ] + ├── primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 3 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id, item_i_id, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, $expr1, _degree ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC, $18 ASC ] - ├── value indices: [ 19 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 + Table 2 { columns: [ $expr2, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, supplier_s_nationkey, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ], value indices: [ 13 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ nation_n_nationkey, nation_n_regionkey, region_r_regionkey ], primary key: [ $0 ASC, $2 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ nation_n_nationkey, nation_n_regionkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ nation_n_nationkey, region_r_regionkey, nation_n_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ nation_n_nationkey, nation_n_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 - ├── columns: [ orders_o_entry_d, customer_c_state, order_line_ol_amount, $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id, item_i_id, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ] - ├── primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC, $17 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ] - ├── distribution key: [ 3 ] - └── read pk prefix len hint: 1 + Table 5 { columns: [ orders_o_entry_d, customer_c_state, order_line_ol_amount, $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id ], primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ], distribution key: [ 3 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_c_id, item_i_id, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC ], value indices: [ 15 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr1, orders_o_w_id, orders_o_d_id, orders_o_id, orders_o_c_id, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, customer_c_d_id, customer_c_w_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 11 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 9 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $8 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 5 } + Table 9 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_entry_d, customer_c_d_id, customer_c_w_id, customer_c_state, orders_o_c_id, customer_c_id ], primary key: [ $2 ASC, $1 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 0, 4, 5 ], read pk prefix len hint: 5 } - Table 10 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, customer_c_id, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } + Table 10 { columns: [ orders_o_w_id, orders_o_d_id, orders_o_id, customer_c_d_id, customer_c_w_id, orders_o_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } - Table 11 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, item_i_id, order_line_ol_number ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $3 ASC, $4 ASC, $6 ASC, $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } + Table 11 { columns: [ stock_s_i_id, stock_s_w_id, order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_amount, item_i_id, order_line_ol_number ], primary key: [ $4 ASC, $3 ASC, $2 ASC, $3 ASC, $4 ASC, $6 ASC, $1 ASC, $7 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } - Table 12 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, item_i_id, stock_s_w_id, stock_s_i_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } + Table 12 { columns: [ order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_d_id_0, order_line_ol_w_id_0, item_i_id, stock_s_w_id, order_line_ol_number, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 2, 3, 4 ], read pk prefix len hint: 5 } Table 13 { columns: [ orders_o_id, orders_o_d_id, orders_o_w_id, orders_o_c_id, orders_o_entry_d ], primary key: [ $3 ASC, $2 ASC, $1 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 1, 2, 3 ], read pk prefix len hint: 3 } @@ -1594,11 +1573,11 @@ └─StreamProject { exprs: [nation.n_name, $expr2, sum(order_line.ol_amount)] } └─StreamHashAgg { group_key: [nation.n_name, $expr2], aggs: [sum(order_line.ol_amount), count] } └─StreamExchange { dist: HashShard(nation.n_name, $expr2) } - └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_entry_d) as $expr2, order_line.ol_amount, item.i_id, stock.s_w_id, stock.s_i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1] } - └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, item.i_id, stock.s_w_id, stock.s_i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └─StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_entry_d) as $expr2, order_line.ol_amount, item.i_id, stock.s_w_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number, stock.s_i_id, supplier.s_nationkey, $expr1] } + └─StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, item.i_id, stock.s_w_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number, stock.s_i_id, $expr1, supplier.s_suppkey, supplier.s_nationkey] } ├─StreamExchange { dist: HashShard($expr1) } - │ └─StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, item.i_id, stock.s_w_id, stock.s_i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, orders.o_entry_d, order_line.ol_amount, item.i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + │ └─StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, item.i_id, stock.s_w_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number, stock.s_i_id] } + │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, orders.o_entry_d, order_line.ol_amount, item.i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } │ ├─StreamExchange { dist: HashShard(item.i_id, stock.s_w_id) } │ │ └─StreamHashJoin { type: Inner, predicate: item.i_id = stock.s_i_id, output: all } │ │ ├─StreamExchange { dist: HashShard(item.i_id) } @@ -1632,8 +1611,8 @@ └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_entry_d) as $expr2, order_line.ol_amount, item.i_id, stock.s_w_id, stock.s_i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1] } - └── StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, item.i_id, stock.s_w_id, stock.s_i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, $expr1, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [nation.n_name, Extract('YEAR':Varchar, orders.o_entry_d) as $expr2, order_line.ol_amount, item.i_id, stock.s_w_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number, stock.s_i_id, supplier.s_nationkey, $expr1] } + └── StreamHashJoin { type: Inner, predicate: $expr1 = supplier.s_suppkey, output: [order_line.ol_amount, orders.o_entry_d, nation.n_name, item.i_id, stock.s_w_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number, stock.s_i_id, $expr1, supplier.s_suppkey, supplier.s_nationkey] } ├── left table: 1 ├── right table: 3 ├── left degree table: 2 @@ -1642,8 +1621,8 @@ └── StreamExchange Hash([0]) from 9 Fragment 2 - StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, item.i_id, stock.s_w_id, stock.s_i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } - └── StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, orders.o_entry_d, order_line.ol_amount, item.i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + StreamProject { exprs: [orders.o_entry_d, order_line.ol_amount, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1, item.i_id, stock.s_w_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number, stock.s_i_id] } + └── StreamHashJoin { type: Inner, predicate: stock.s_i_id = order_line.ol_i_id AND stock.s_w_id = order_line.ol_supply_w_id AND item.i_id = order_line.ol_i_id, output: [stock.s_i_id, stock.s_w_id, orders.o_entry_d, order_line.ol_amount, item.i_id, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } ├── left table: 5 ├── right table: 7 ├── left degree table: 6 @@ -1706,18 +1685,13 @@ Table 0 { columns: [ nation_n_name, $expr2, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 1 - ├── columns: [ orders_o_entry_d, order_line_ol_amount, $expr1, item_i_id, stock_s_w_id, stock_s_i_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ] - ├── primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ] - ├── distribution key: [ 2 ] - └── read pk prefix len hint: 1 + Table 1 { columns: [ orders_o_entry_d, order_line_ol_amount, $expr1, item_i_id, stock_s_w_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, stock_s_i_id ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 2 { columns: [ $expr1, item_i_id, stock_s_w_id, stock_s_i_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 11 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ $expr1, item_i_id, stock_s_w_id, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, nation_n_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 { columns: [ item_i_id, stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 2 ], read pk prefix len hint: 3 } @@ -1725,17 +1699,12 @@ Table 7 ├── columns: [ orders_o_entry_d, order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_amount, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ] - ├── primary key: [ $1 ASC, $2 ASC, $1 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ] + ├── primary key: [ $1 ASC, $2 ASC, $1 ASC, $4 ASC, $5 ASC, $6 ASC, $10 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 1, 2 ] └── read pk prefix len hint: 3 - Table 8 - ├── columns: [ order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_i_id_0, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ] - ├── value indices: [ 10 ] - ├── distribution key: [ 2, 1 ] - └── read pk prefix len hint: 3 + Table 8 { columns: [ order_line_ol_i_id, order_line_ol_supply_w_id, order_line_ol_i_id_0, orders_o_w_id, orders_o_d_id, orders_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 2, 1 ], read pk prefix len hint: 3 } Table 9 { columns: [ item_i_id ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1826,9 +1795,9 @@ └─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, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, $expr1, nation.n_nationkey] } + └─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, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id, orders.o_id] } + │ └─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) } @@ -1855,7 +1824,7 @@ └── StreamExchange Hash([0, 1, 2, 3, 5]) from 1 Fragment 1 - 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, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id, orders.o_id, $expr1, nation.n_nationkey] } + 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] } ├── left table: 1 ├── right table: 3 ├── left degree table: 2 @@ -1864,9 +1833,13 @@ └── StreamExchange Hash([0]) from 7 Fragment 2 - 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, customer.c_w_id, customer.c_d_id, orders.o_w_id, orders.o_d_id, orders.o_id] } + 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 } { left table: 5, right table: 7, left degree table: 6, right degree table: 8 } + └── 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 } + ├── left table: 5 + ├── right table: 7 + ├── left degree table: 6 + ├── right degree table: 8 ├── StreamExchange Hash([1, 2, 0, 1, 2]) from 3 └── StreamExchange Hash([1, 2, 7, 8, 9]) from 4 @@ -1905,18 +1878,13 @@ Table 0 { columns: [ customer_c_id, customer_c_last, customer_c_city, customer_c_phone, nation_n_name, sum(order_line_ol_amount), count ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } Table 1 - ├── columns: [ order_line_ol_amount, customer_c_id, customer_c_last, customer_c_city, customer_c_phone, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, customer_c_w_id, customer_c_d_id, orders_o_w_id, orders_o_d_id, orders_o_id ] - ├── primary key: [ $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $1 ASC, $12 ASC, $13 ASC, $14 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── columns: [ order_line_ol_amount, customer_c_id, customer_c_last, customer_c_city, customer_c_phone, $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number ] + ├── primary key: [ $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] ├── distribution key: [ 5 ] └── read pk prefix len hint: 1 - Table 2 - ├── columns: [ $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, customer_c_w_id, customer_c_d_id, customer_c_id, orders_o_w_id, orders_o_d_id, orders_o_id, _degree ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ] - ├── value indices: [ 11 ] - ├── distribution key: [ 0 ] - └── read pk prefix len hint: 1 + Table 2 { columns: [ $expr1, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, customer_c_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 3 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2031,7 +1999,7 @@ │ └─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, nation.n_nationkey, supplier.s_nationkey, stock.s_w_id] } + │ └─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) } @@ -2049,7 +2017,7 @@ └─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, nation.n_nationkey, supplier.s_nationkey, stock.s_w_id] } + └─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) } @@ -2082,7 +2050,7 @@ └── StreamExchange NoShuffle from 2 Fragment 2 - StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = $expr1, output: [stock.s_i_id, stock.s_order_cnt, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, stock.s_w_id] } + 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] } ├── left table: 3 ├── right table: 5 ├── left degree table: 4 @@ -2091,12 +2059,17 @@ └── StreamExchange Hash([2]) from 6 Fragment 3 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } { left table: 7, right table: 9, left degree table: 8, right degree table: 10 } + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: all } + ├── left table: 7 + ├── right table: 9 + ├── left degree table: 8 + ├── right degree table: 10 ├── StreamExchange Hash([1]) from 4 └── StreamExchange Hash([0]) from 5 Fragment 4 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 11 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_nationkey], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + ├── state table: 11 ├── Upstream └── BatchPlanNode @@ -2123,7 +2096,12 @@ StreamStatelessSimpleAgg { aggs: [sum(stock.s_order_cnt)] } └── StreamExchange NoShuffle from 2 - Table 0 { columns: [ stock_s_i_id, sum(stock_s_order_cnt), $expr2 ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ stock_s_i_id, sum(stock_s_order_cnt), $expr2 ] + ├── primary key: [ $2 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 1 { columns: [ $expr3 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } @@ -2131,15 +2109,15 @@ Table 3 ├── columns: [ supplier_s_suppkey, supplier_s_nationkey, nation_n_nationkey ] - ├── primary key: [ $0 ASC, $2 ASC, $1 ASC ] + ├── primary key: [ $0 ASC, $1 ASC ] ├── value indices: [ 0, 1, 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 Table 4 - ├── columns: [ supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ] - ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] - ├── value indices: [ 3 ] + ├── columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 @@ -2150,11 +2128,26 @@ ├── distribution key: [ 2 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 + ├── columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 3 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 7 { columns: [ supplier_s_suppkey, supplier_s_nationkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 7 + ├── columns: [ supplier_s_suppkey, supplier_s_nationkey ] + ├── primary key: [ $1 ASC, $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [ 1 ] + └── read pk prefix len hint: 1 - Table 8 { columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 + ├── columns: [ supplier_s_nationkey, supplier_s_suppkey, _degree ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 9 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2223,7 +2216,7 @@ └─StreamProject { exprs: [orders.o_ol_cnt, sum($expr1), sum($expr2)] } └─StreamHashAgg { group_key: [orders.o_ol_cnt], aggs: [sum($expr1), sum($expr2), count] } └─StreamExchange { dist: HashShard(orders.o_ol_cnt) } - └─StreamProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + └─StreamProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } └─StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } └─StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } ├─StreamExchange { dist: HashShard(orders.o_id, orders.o_d_id, orders.o_w_id) } @@ -2243,7 +2236,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - StreamProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } + StreamProject { exprs: [orders.o_ol_cnt, Case(((orders.o_carrier_id = 1:Int32) OR (orders.o_carrier_id = 2:Int32)), 1:Int32, 0:Int32) as $expr1, Case(((orders.o_carrier_id <> 1:Int32) AND (orders.o_carrier_id <> 2:Int32)), 1:Int32, 0:Int32) as $expr2, orders.o_w_id, orders.o_d_id, orders.o_id, order_line.ol_number] } └── StreamFilter { predicate: (orders.o_entry_d <= order_line.ol_delivery_d) } └── StreamHashJoin { type: Inner, predicate: orders.o_w_id = order_line.ol_w_id AND orders.o_d_id = order_line.ol_d_id AND orders.o_id = order_line.ol_o_id, output: all } { left table: 1, right table: 3, left degree table: 2, right degree table: 4 } ├── StreamExchange Hash([0, 1, 2]) from 2 @@ -2256,7 +2249,8 @@ Fragment 3 StreamFilter { predicate: (order_line.ol_delivery_d < '2030-01-01 00:00:00':Timestamp) } - └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } { state table: 6 } + └── Chain { table: order_line, columns: [order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_delivery_d, order_line.ol_number], pk: [order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number], dist: UpstreamHashShard(order_line.ol_o_id, order_line.ol_d_id, order_line.ol_w_id, order_line.ol_number) } + ├── state table: 6 ├── Upstream └── BatchPlanNode @@ -2418,7 +2412,7 @@ └─StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum(order_line.ol_amount)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum(order_line.ol_amount)] } - └─StreamProject { exprs: [Case(Like(item.i_data, 'PR%':Varchar), order_line.ol_amount, 0:Decimal) as $expr1, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, item.i_id, order_line.ol_i_id] } + └─StreamProject { exprs: [Case(Like(item.i_data, 'PR%':Varchar), order_line.ol_amount, 0:Decimal) as $expr1, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id] } └─StreamHashJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id, output: [order_line.ol_amount, item.i_data, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id, item.i_id] } ├─StreamExchange { dist: HashShard(order_line.ol_i_id) } │ └─StreamProject { exprs: [order_line.ol_i_id, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number] } @@ -2439,7 +2433,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum($expr1), sum(order_line.ol_amount)] } - └── StreamProject { exprs: [Case(Like(item.i_data, 'PR%':Varchar), order_line.ol_amount, 0:Decimal) as $expr1, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, item.i_id, order_line.ol_i_id] } + └── StreamProject { exprs: [Case(Like(item.i_data, 'PR%':Varchar), order_line.ol_amount, 0:Decimal) as $expr1, order_line.ol_amount, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id] } └── StreamHashJoin { type: Inner, predicate: order_line.ol_i_id = item.i_id, output: [order_line.ol_amount, item.i_data, order_line.ol_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, order_line.ol_i_id, item.i_id] } ├── left table: 1 ├── right table: 3 @@ -2643,10 +2637,10 @@ └─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, item.i_id, $expr1] } - └─StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id, $expr1] } + └─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, item.i_id] } + │ └─StreamProject { exprs: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, ((stock.s_w_id * stock.s_i_id) % 10000:Int32)::Int64 as $expr1] } │ └─StreamHashJoin { type: Inner, predicate: stock.s_i_id = item.i_id, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id] } │ ├─StreamExchange { dist: HashShard(stock.s_i_id) } │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } @@ -2669,17 +2663,13 @@ └── StreamExchange Hash([0, 1, 2]) from 1 Fragment 1 - 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, item.i_id, $expr1] } - └── StreamHashJoin { type: LeftAnti, predicate: $expr1 = supplier.s_suppkey, output: [stock.s_i_id, stock.s_w_id, item.i_name, item.i_price, item.i_data, item.i_id, $expr1] } - ├── left table: 2 - ├── right table: 4 - ├── left degree table: 3 - ├── right degree table: 5 + 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 } { left table: 2, right table: 4, left degree table: 3, right degree table: 5 } ├── StreamExchange Hash([5]) from 2 └── StreamExchange Hash([0]) from 5 Fragment 2 - 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, item.i_id] } + 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] } ├── left table: 6 ├── right table: 8 @@ -2721,13 +2711,13 @@ └── read pk prefix len hint: 4 Table 2 - ├── columns: [ stock_s_i_id, stock_s_w_id, item_i_name, item_i_price, item_i_data, $expr1, item_i_id ] - ├── primary key: [ $5 ASC, $1 ASC, $0 ASC, $6 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── columns: [ stock_s_i_id, stock_s_w_id, item_i_name, item_i_price, item_i_data, $expr1 ] + ├── primary key: [ $5 ASC, $1 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 5 ] └── read pk prefix len hint: 1 - Table 3 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, item_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 { columns: [ supplier_s_suppkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2749,7 +2739,13 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 11 { columns: [ vnode, i_id, item_backfill_finished, item_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 11 + ├── columns: [ vnode, i_id, item_backfill_finished, item_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 Table 12 ├── columns: [ vnode, s_suppkey, supplier_backfill_finished, supplier_row_count ] @@ -3177,8 +3173,8 @@ └─BatchFilter { predicate: (order_line.ol_delivery_d > '2010-05-23 12:00:00':Timestamp) } └─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), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], pk_conflict: NoCheck } - └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + 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 } + └─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 } │ ├─StreamExchange { dist: HashShard(supplier.s_nationkey) } @@ -3206,9 +3202,9 @@ └─StreamTableScan { table: item, columns: [item.i_id, item.i_data], pk: [item.i_id], dist: UpstreamHashShard(item.i_id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], stream_key: [supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], pk_columns: [s_name, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey], pk_conflict: NoCheck } + 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 } ├── materialized table: 4294967294 - └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = $expr1, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0]) from 1 └── StreamExchange Hash([0]) from 4 @@ -3264,9 +3260,9 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 1 { columns: [ supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 2 { columns: [ $expr1, stock_s_i_id, stock_s_w_id, stock_s_quantity ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -3308,7 +3304,7 @@ Table 21 { columns: [ vnode, i_id, item_backfill_finished, item_row_count ], 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, nation.n_nationkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 4 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } - id: ch_q21 before: @@ -3371,12 +3367,12 @@ 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_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1] } + └─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, orders.o_w_id, orders.o_d_id, orders.o_id, $expr1, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + │ └─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, orders.o_w_id, orders.o_d_id, orders.o_id] } - │ │ └─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, orders.o_w_id, orders.o_d_id, orders.o_id] } + │ │ └─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: (stock.s_w_id = stock.s_w_id) } │ │ │ └─StreamTableScan { table: stock, columns: [stock.s_i_id, stock.s_w_id], pk: [stock.s_w_id, stock.s_i_id], dist: UpstreamHashShard(stock.s_i_id, stock.s_w_id) } @@ -3410,7 +3406,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - 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_w_id, order_line.ol_d_id, order_line.ol_o_id, order_line.ol_number, orders.o_w_id, orders.o_d_id, orders.o_id, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, $expr1] } + 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] } ├── left table: 1 ├── right table: 3 ├── left degree table: 2 @@ -3419,13 +3415,13 @@ └── StreamExchange Hash([0, 1, 2]) from 11 Fragment 2 - 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, orders.o_w_id, orders.o_d_id, orders.o_id, $expr1, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 5, right table: 7, left degree table: 6, right degree table: 8 } + 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] } { left table: 5, right table: 7, left degree table: 6, right degree table: 8 } ├── StreamExchange Hash([4]) from 3 └── StreamExchange Hash([0]) from 8 Fragment 3 - 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, orders.o_w_id, orders.o_d_id, orders.o_id] } - └── 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, orders.o_w_id, orders.o_d_id, orders.o_id] } { left table: 9, right table: 11, left degree table: 10, right degree table: 12 } + 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] } { left table: 9, right table: 11, left degree table: 10, right degree table: 12 } ├── StreamExchange Hash([1, 0, 1]) from 4 └── StreamExchange Hash([2, 3, 5]) from 5 @@ -3478,33 +3474,33 @@ Table 0 { columns: [ supplier_s_name, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 1 - ├── columns: [ 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, orders_o_w_id, orders_o_d_id, orders_o_id, $expr1, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey ] - ├── primary key: [ $1 ASC, $3 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $12 ASC, $13 ASC, $14 ASC, $11 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── columns: [ 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 ] + ├── primary key: [ $1 ASC, $3 ASC, $2 ASC, $5 ASC, $6 ASC, $7 ASC, $10 ASC, $8 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 1, 3, 2 ] └── read pk prefix len hint: 3 - Table 2 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, stock_s_w_id, stock_s_i_id, order_line_ol_number, orders_o_w_id, orders_o_d_id, orders_o_id, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC ], value indices: [ 13 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 2 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, stock_s_w_id, stock_s_i_id, order_line_ol_number, supplier_s_nationkey, $expr1, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 3 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_delivery_d, order_line_ol_number ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 4 { columns: [ order_line_ol_o_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 5 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_number, orders_o_w_id, orders_o_d_id, orders_o_id ], primary key: [ $4 ASC, $5 ASC, $6 ASC, $2 ASC, $1 ASC, $0 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 5 { columns: [ order_line_ol_o_id, order_line_ol_d_id, order_line_ol_w_id, order_line_ol_delivery_d, $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_number ], primary key: [ $4 ASC, $5 ASC, $6 ASC, $1 ASC, $0 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_w_id, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ], value indices: [ 10 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ $expr1, stock_s_w_id, stock_s_i_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 7 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 7 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 8 { columns: [ supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 8 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 9 { columns: [ stock_s_i_id, stock_s_w_id ], primary key: [ $1 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 1, 0, 1 ], read pk prefix len hint: 3 } Table 10 { columns: [ stock_s_w_id, stock_s_i_id, stock_s_w_id_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 11 { 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, orders_o_w_id, order_line_ol_number, orders_o_d_id, orders_o_id ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $1 ASC, $0 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2, 3, 5 ], read pk prefix len hint: 3 } + Table 11 { 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, orders_o_w_id, order_line_ol_number, orders_o_d_id, orders_o_id ], primary key: [ $2 ASC, $3 ASC, $5 ASC, $1 ASC, $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2, 3, 5 ], read pk prefix len hint: 3 } - Table 12 { columns: [ order_line_ol_w_id, order_line_ol_i_id, orders_o_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, orders_o_d_id, orders_o_id, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + Table 12 { columns: [ order_line_ol_w_id, order_line_ol_i_id, orders_o_w_id, order_line_ol_d_id, order_line_ol_o_id, order_line_ol_number, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } Table 13 { columns: [ vnode, s_w_id, s_i_id, stock_backfill_finished, stock_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } diff --git a/src/frontend/planner_test/tests/testdata/output/delta_join.yaml b/src/frontend/planner_test/tests/testdata/output/delta_join.yaml index 29ab1cb7a71b9..8c65e70f4bf3a 100644 --- a/src/frontend/planner_test/tests/testdata/output/delta_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/delta_join.yaml @@ -33,8 +33,8 @@ /* should generate delta join plan, and stream index scan */ select * from a join b on a.a1 = b.b1 ; stream_plan: |- - StreamMaterialize { columns: [a1, a2, b1, b2], stream_key: [a1, b1], pk_columns: [a1, b1], pk_conflict: NoCheck } - └─StreamExchange { dist: HashShard(a.a1, b.b1) } + StreamMaterialize { columns: [a1, a2, b1, b2], stream_key: [a1], pk_columns: [a1], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(a.a1) } └─StreamDeltaJoin { type: Inner, predicate: a.a1 = b.b1, output: all } ├─StreamTableScan { table: a, columns: [a.a1, a.a2], pk: [a.a1], dist: UpstreamHashShard(a.a1) } └─StreamTableScan { table: b, columns: [b.b1, b.b2], pk: [b.b1], dist: UpstreamHashShard(b.b1) } diff --git a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml index 0ad51e1e404b1..85d76188f3e76 100644 --- a/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/distribution_derive.yaml @@ -962,7 +962,7 @@ └─BatchExchange { order: [], dist: HashShard(a.k1) } └─BatchScan { table: a, columns: [a.k1], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, a.k1, ak1.k1], pk_columns: [ak1.a._row_id, a.k1, ak1.k1], pk_conflict: NoCheck } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } ├─StreamExchange { dist: HashShard(ak1.k1) } │ └─StreamTableScan { table: ak1, columns: [ak1.k1, ak1.v, ak1.a._row_id], pk: [ak1.a._row_id], dist: UpstreamHashShard(ak1.k1) } @@ -971,9 +971,13 @@ └─StreamTableScan { table: a, columns: [a.k1, a._row_id], pk: [a._row_id], dist: UpstreamHashShard(a._row_id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, a.k1, ak1.k1], pk_columns: [ak1.a._row_id, a.k1, ak1.k1], pk_conflict: NoCheck } + StreamMaterialize { columns: [v, bv, ak1.a._row_id(hidden), ak1.k1(hidden), a.k1(hidden)], stream_key: [ak1.a._row_id, ak1.k1], pk_columns: [ak1.a._row_id, ak1.k1], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } + └── StreamHashJoin { type: Inner, predicate: ak1.k1 = a.k1, output: [ak1.v, count, ak1.a._row_id, ak1.k1, a.k1] } + ├── left table: 0 + ├── right table: 2 + ├── left degree table: 1 + ├── right degree table: 3 ├── StreamExchange Hash([0]) from 1 └── StreamHashAgg { group_key: [a.k1], aggs: [count] } { intermediate state table: 5, state tables: [], distinct tables: [] } └── StreamExchange Hash([0]) from 2 @@ -1014,7 +1018,12 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 4294967294 { columns: [ v, bv, ak1.a._row_id, ak1.k1, a.k1 ], primary key: [ $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 3 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ v, bv, ak1.a._row_id, ak1.k1, a.k1 ] + ├── primary key: [ $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 3 ] + └── read pk prefix len hint: 2 - id: aggk1_join_Ak1_onk1 before: @@ -1146,7 +1155,7 @@ └─BatchExchange { order: [], dist: HashShard(b.k1) } └─BatchScan { table: b, columns: [b.k1], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1, b.k1], pk_columns: [a.k1, b.k1], pk_conflict: NoCheck } + StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } ├─StreamHashAgg { group_key: [a.k1], aggs: [count] } │ └─StreamExchange { dist: HashShard(a.k1) } @@ -1156,7 +1165,7 @@ └─StreamTableScan { table: b, columns: [b.k1, b._row_id], pk: [b._row_id], dist: UpstreamHashShard(b._row_id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1, b.k1], pk_columns: [a.k1, b.k1], pk_conflict: NoCheck } + StreamMaterialize { columns: [num, bv, a.k1(hidden), b.k1(hidden)], stream_key: [a.k1], pk_columns: [a.k1], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: Inner, predicate: a.k1 = b.k1, output: [count, count, a.k1, b.k1] } ├── left table: 0 @@ -1178,15 +1187,40 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 0 + ├── columns: [ a_k1, count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 1 { columns: [ a_k1, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 1 + ├── columns: [ a_k1, _degree ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 2 { columns: [ b_k1, count ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 + ├── columns: [ b_k1, count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 3 { columns: [ b_k1, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 + ├── columns: [ b_k1, _degree ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 - Table 4 { columns: [ a_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 + ├── columns: [ a_k1, count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 5 ├── columns: [ vnode, _row_id, a_backfill_finished, a_row_count ] @@ -1196,7 +1230,12 @@ ├── read pk prefix len hint: 1 └── vnode column idx: 0 - Table 6 { columns: [ b_k1, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 + ├── columns: [ b_k1, count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 Table 7 ├── columns: [ vnode, _row_id, b_backfill_finished, b_row_count ] @@ -1208,10 +1247,10 @@ Table 4294967294 ├── columns: [ num, bv, a.k1, b.k1 ] - ├── primary key: [ $2 ASC, $3 ASC ] + ├── primary key: [ $2 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 2 ] - └── read pk prefix len hint: 2 + └── read pk prefix len hint: 1 - sql: | create table t1 (row_id int, uid int, v int, created_at timestamp); diff --git a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml index 2f72bc6d4f4a2..8e63beb9798c1 100644 --- a/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/lateral_subquery.yaml @@ -41,7 +41,7 @@ └─BatchFilter { predicate: IsNotNull(all_sales.salesperson_id) } └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.amount], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), all_sales._row_id(hidden), salesperson.id(hidden), all_sales.amount(hidden), salesperson.id#1(hidden)], stream_key: [salesperson._row_id, all_sales._row_id, salesperson.id, salesperson.id#1, all_sales.amount], pk_columns: [salesperson._row_id, all_sales._row_id, salesperson.id, salesperson.id#1, all_sales.amount], pk_conflict: NoCheck } + StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), all_sales._row_id(hidden), salesperson.id(hidden), all_sales.amount(hidden), salesperson.id#1(hidden)], stream_key: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_columns: [salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM salesperson.id AND all_sales.amount = max(all_sales.amount), output: [salesperson.name, max(all_sales.amount), all_sales.customer_name, salesperson._row_id, all_sales._row_id, salesperson.id, all_sales.amount, salesperson.id] } ├─StreamHashJoin { type: Inner, predicate: salesperson.id = all_sales.salesperson_id, output: [salesperson.id, salesperson.name, all_sales.customer_name, all_sales.amount, salesperson._row_id, all_sales._row_id] } │ ├─StreamExchange { dist: HashShard(salesperson.id) } @@ -86,7 +86,7 @@ └─BatchFilter { predicate: IsNotNull(all_sales.salesperson_id) } └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, all_sales.salesperson_id, salesperson.id], pk_columns: [salesperson._row_id, all_sales.salesperson_id, salesperson.id], pk_conflict: NoCheck } + StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } ├─StreamExchange { dist: HashShard(salesperson.id) } │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } @@ -123,7 +123,7 @@ └─BatchFilter { predicate: IsNotNull(all_sales.salesperson_id) } └─BatchScan { table: all_sales, columns: [all_sales.salesperson_id, all_sales.customer_name, all_sales.amount], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, all_sales.salesperson_id, salesperson.id], pk_columns: [salesperson._row_id, all_sales.salesperson_id, salesperson.id], pk_conflict: NoCheck } + StreamMaterialize { columns: [name, amount, customer_name, salesperson._row_id(hidden), salesperson.id(hidden), all_sales.salesperson_id(hidden)], stream_key: [salesperson._row_id, salesperson.id], pk_columns: [salesperson._row_id, salesperson.id], pk_conflict: NoCheck } └─StreamHashJoin { type: LeftOuter, predicate: salesperson.id IS NOT DISTINCT FROM all_sales.salesperson_id, output: [salesperson.name, all_sales.amount, all_sales.customer_name, salesperson._row_id, salesperson.id, all_sales.salesperson_id] } ├─StreamExchange { dist: HashShard(salesperson.id) } │ └─StreamTableScan { table: salesperson, columns: [salesperson.id, salesperson.name, salesperson._row_id], pk: [salesperson._row_id], dist: UpstreamHashShard(salesperson._row_id) } @@ -164,7 +164,7 @@ └─BatchExchange { order: [], dist: HashShard(t.arr) } └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, arr, unnest, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, t.arr, projected_row_id, arr], pk_columns: [t._row_id, t.arr, projected_row_id, arr], pk_conflict: NoCheck } + StreamMaterialize { columns: [x, arr, unnest, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] } ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml index e4f1ff1136cdb..b2b0ca006057a 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark.yaml @@ -180,7 +180,7 @@ └─BatchFilter { predicate: (auction.category = 10:Int32) } └─BatchScan { table: auction, columns: [auction.id, auction.seller, auction.category], distribution: UpstreamHashShard(auction.id) } stream_plan: |- - StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, person.id, auction.seller], pk_columns: [id, person.id, auction.seller], pk_conflict: NoCheck } + StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } ├─StreamExchange { dist: HashShard(auction.seller) } │ └─StreamProject { exprs: [auction.id, auction.seller] } @@ -191,7 +191,7 @@ └─StreamTableScan { table: person, columns: [person.id, person.name, person.city, person.state], pk: [person.id], dist: UpstreamHashShard(person.id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, person.id, auction.seller], pk_columns: [id, person.id, auction.seller], pk_conflict: NoCheck } + StreamMaterialize { columns: [name, city, state, id, auction.seller(hidden), person.id(hidden)], stream_key: [id, auction.seller], pk_columns: [id, auction.seller], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: Inner, predicate: auction.seller = person.id, output: [person.name, person.city, person.state, auction.id, auction.seller, person.id] } ├── left table: 0 @@ -222,16 +222,28 @@ Table 3 { columns: [ person_id, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 4 + ├── columns: [ vnode, id, auction_backfill_finished, auction_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 - Table 5 { columns: [ vnode, id, person_backfill_finished, person_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 + ├── columns: [ vnode, id, person_backfill_finished, person_row_count ] + ├── 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: [ name, city, state, id, auction.seller, person.id ] - ├── primary key: [ $3 ASC, $5 ASC, $4 ASC ] + ├── primary key: [ $3 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 4 ] - └── read pk prefix len hint: 3 + └── read pk prefix len hint: 2 - id: nexmark_q4 before: @@ -440,7 +452,7 @@ └─BatchFilter { predicate: IsNotNull(bid.date_time) } └─BatchScan { table: bid, columns: [bid.auction, bid.date_time], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [bid.auction, count, window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } @@ -462,7 +474,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamProject { exprs: [bid.auction, count, window_start, window_start] } └── StreamFilter { predicate: (count >= max(count)) } @@ -501,13 +513,24 @@ Table 4 { columns: [ bid_auction, window_start, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 5 { columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 5 + ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 Table 6 { columns: [ window_start, count, bid_auction ], primary key: [ $0 ASC, $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, num, window_start, window_start#1 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, num, window_start, window_start#1 ] + ├── primary key: [ $0 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 2 - id: nexmark_q6_group_top1 before: @@ -823,7 +846,7 @@ └─BatchProject { exprs: [auction.date_time, auction.seller, TumbleStart(auction.date_time, '00:00:10':Interval) as $expr3] } └─BatchScan { table: auction, columns: [auction.date_time, auction.seller], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2, auction.seller, $expr3, $expr4], pk_columns: [id, starttime, $expr2, auction.seller, $expr3, $expr4], pk_conflict: NoCheck } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] } ├─StreamExchange { dist: HashShard(person.id, $expr1, $expr2) } │ └─StreamProject { exprs: [person.id, $expr1, $expr2, first_value(person.name order_by(person.name ASC))] } @@ -839,7 +862,7 @@ └─StreamTableScan { table: auction, columns: [auction.date_time, auction.seller, auction.id], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2, auction.seller, $expr3, $expr4], pk_columns: [id, starttime, $expr2, auction.seller, $expr3, $expr4], pk_conflict: NoCheck } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), auction.seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, starttime, $expr2], pk_columns: [id, starttime, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: Inner, predicate: person.id = auction.seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [person.id, first_value(person.name order_by(person.name ASC)), $expr1, $expr2, auction.seller, $expr3, $expr4] } ├── left table: 0 @@ -885,7 +908,7 @@ Table 8 { columns: [ vnode, id, auction_backfill_finished, auction_row_count ], 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: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 6 } + Table 4294967294 { columns: [ id, name, starttime, $expr2, auction.seller, $expr3, $expr4 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } - id: nexmark_q9 before: @@ -1106,7 +1129,7 @@ └─StreamExchange [no_shuffle] { dist: UpstreamHashShard(side_input.key) } └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } stream_plan: |- - StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, side_input.key, $expr1], pk_columns: [bid._row_id, side_input.key, $expr1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck } └─StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } ├─StreamExchange { dist: HashShard($expr1) } │ └─StreamProject { exprs: [bid.auction, bid.bidder, bid.price, bid.date_time, (bid.auction % 10000:Int32) as $expr1, bid._row_id] } @@ -1115,7 +1138,7 @@ └─StreamTableScan { table: side_input, columns: [side_input.key, side_input.value], pk: [side_input.key], dist: UpstreamHashShard(side_input.key) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, side_input.key, $expr1], pk_columns: [bid._row_id, side_input.key, $expr1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, bidder, price, date_time, value, bid._row_id(hidden), $expr1(hidden), side_input.key(hidden)], stream_key: [bid._row_id, $expr1], pk_columns: [bid._row_id, $expr1], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamTemporalJoin { type: Inner, predicate: $expr1 = side_input.key, output: [bid.auction, bid.bidder, bid.price, bid.date_time, side_input.value, bid._row_id, $expr1, side_input.key] } ├── StreamExchange Hash([4]) from 1 @@ -1136,7 +1159,12 @@ Table 1 { columns: [ vnode, key, side_input_backfill_finished, side_input_row_count ], 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: [ auction, bidder, price, date_time, value, bid._row_id, $expr1, side_input.key ], primary key: [ $5 ASC, $7 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 6 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, value, bid._row_id, $expr1, side_input.key ] + ├── primary key: [ $5 ASC, $6 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 6 ] + └── read pk prefix len hint: 2 - id: nexmark_q14 before: @@ -1763,7 +1791,7 @@ └─BatchExchange { order: [], dist: UpstreamHashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction.id, auction], pk_columns: [bid._row_id, auction.id, auction], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } ├─StreamExchange { dist: HashShard(bid.auction) } │ └─StreamTableScan { table: bid, columns: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } @@ -1772,7 +1800,7 @@ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category], pk: [auction.id], dist: UpstreamHashShard(auction.id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction.id, auction], pk_columns: [bid._row_id, auction.id, auction], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id(hidden), auction.id(hidden)], stream_key: [bid._row_id, auction], pk_columns: [bid._row_id, auction], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: Inner, predicate: bid.auction = auction.id, output: [bid.auction, bid.bidder, bid.price, bid.channel, bid.url, bid.date_time, auction.item_name, auction.description, auction.initial_bid, auction.reserve, auction.date_time, auction.expires, auction.seller, auction.category, bid._row_id, auction.id] } ├── left table: 0 @@ -1807,10 +1835,10 @@ Table 4294967294 ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, bid._row_id, auction.id ] - ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] + ├── primary key: [ $14 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] ├── distribution key: [ 0 ] - └── read pk prefix len hint: 3 + └── read pk prefix len hint: 2 - id: nexmark_q21 before: @@ -1914,7 +1942,7 @@ └─BatchExchange { order: [], dist: HashShard(bid.auction) } └─BatchScan { table: bid, columns: [bid.auction, bid.price], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id, bid.auction], pk_columns: [auction_id, bid.auction], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } └─StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } ├─StreamExchange { dist: HashShard(auction.id) } │ └─StreamTableScan { table: auction, columns: [auction.id, auction.item_name], pk: [auction.id], dist: UpstreamHashShard(auction.id) } @@ -1924,7 +1952,7 @@ └─StreamTableScan { table: bid, columns: [bid.auction, bid.price, bid._row_id], pk: [bid._row_id], dist: UpstreamHashShard(bid._row_id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id, bid.auction], pk_columns: [auction_id, bid.auction], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, bid.auction(hidden)], stream_key: [auction_id], pk_columns: [auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: LeftOuter, predicate: auction.id = bid.auction, output: [auction.id, auction.item_name, max(bid.price), bid.auction] } ├── left table: 0 @@ -1964,14 +1992,20 @@ Table 5 { columns: [ bid_auction, max(bid_price), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 6 { columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 6 + ├── columns: [ vnode, _row_id, bid_backfill_finished, bid_row_count ] + ├── 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: [ auction_id, auction_item_name, current_highest_bid, bid.auction ] - ├── primary key: [ $0 ASC, $3 ASC ] + ├── primary key: [ $0 ASC ] ├── value indices: [ 0, 1, 2, 3 ] ├── distribution key: [ 0 ] - └── read pk prefix len hint: 2 + └── read pk prefix len hint: 1 - id: nexmark_q102 before: 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 7b1e28d485686..4c85c0fec237a 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source.yaml @@ -408,7 +408,7 @@ └─BatchFilter { predicate: IsNotNull(date_time) } └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, count, window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } @@ -434,7 +434,7 @@ └─StreamSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamProject { exprs: [auction, count, window_start, window_start] } └── StreamFilter { predicate: (count >= max(count)) } @@ -479,7 +479,12 @@ Table 7 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, num, window_start, window_start#1 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, num, window_start, window_start#1 ] + ├── primary key: [ $0 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 2 - id: nexmark_q6 before: @@ -731,7 +736,7 @@ └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } └─BatchSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2, seller, $expr3, $expr4], pk_columns: [id, name, starttime, $expr2, seller, $expr3, $expr4], pk_conflict: NoCheck } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } └─StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } ├─StreamExchange { dist: HashShard(id, $expr1, $expr2) } │ └─StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } @@ -748,7 +753,7 @@ └─StreamSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2, seller, $expr3, $expr4], pk_columns: [id, name, starttime, $expr2, seller, $expr3, $expr4], pk_conflict: NoCheck } + StreamMaterialize { columns: [id, name, starttime, $expr2(hidden), seller(hidden), $expr3(hidden), $expr4(hidden)], stream_key: [id, name, starttime, $expr2], pk_columns: [id, name, starttime, $expr2], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0, 2, 3]) from 1 @@ -787,7 +792,12 @@ Table 7 { columns: [ partition_id, offset_info ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 7 } + Table 4294967294 + ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4 ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 2, 3 ] + └── read pk prefix len hint: 4 - id: nexmark_q9 before: @@ -1764,7 +1774,7 @@ └─BatchExchange { order: [], dist: HashShard(auction) } └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction, auction_id], pk_columns: [_row_id, auction, auction_id], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } ├─StreamExchange { dist: HashShard(id) } │ └─StreamRowIdGen { row_id_index: 10 } @@ -1776,7 +1786,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, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction, auction_id], pk_columns: [_row_id, auction, auction_id], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0]) from 1 @@ -1813,10 +1823,10 @@ Table 4294967294 ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction ] - ├── primary key: [ $3 ASC, $4 ASC, $0 ASC ] + ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0 ] - └── read pk prefix len hint: 3 + └── read pk prefix len hint: 2 - id: nexmark_q102 before: 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 802133d356833..879910f1b27c4 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 @@ -323,7 +323,7 @@ ) AS MaxBids ON AuctionBids.starttime = MaxBids.starttime_c AND AuctionBids.num >= MaxBids.maxn; stream_plan: |- - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [$expr3, count, window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } @@ -359,7 +359,7 @@ └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamProject { exprs: [$expr3, count, window_start, window_start] } └── StreamFilter { predicate: (count >= max(count)) } @@ -380,7 +380,9 @@ Fragment 3 StreamHopWindow { time_col: $expr4, slide: 00:00:02, size: 00:00:10, output: [$expr3, window_start, _row_id] } └── StreamProject { exprs: [Field(bid, 0:Int32) as $expr3, Field(bid, 5:Int32) as $expr4, _row_id] } - └── StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, person, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } { left table: 5, right table: 6 } + └── StreamDynamicFilter { predicate: ($expr1 > $expr2), output_watermarks: [$expr1], output: [event_type, person, auction, bid, $expr1, _row_id], cleaned_by_watermark: true } + ├── left table: 5 + ├── right table: 6 ├── StreamFilter { predicate: IsNotNull(Field(bid, 5:Int32)) AND (event_type = 2:Int32) } │ └── StreamRowIdGen { row_id_index: 5 } │ └── StreamProject { exprs: [event_type, person, auction, bid, Proctime as $expr1, _row_id], output_watermarks: [$expr1] } @@ -405,7 +407,12 @@ Table 4 { columns: [ $expr3, window_start, count ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 5 { columns: [ event_type, person, auction, bid, $expr1, _row_id ], primary key: [ $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + Table 5 + ├── columns: [ event_type, person, auction, bid, $expr1, _row_id ] + ├── primary key: [ $4 ASC, $5 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5 ] + ├── distribution key: [ 5 ] + └── read pk prefix len hint: 1 Table 6 { columns: [ $expr2 ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } @@ -417,7 +424,12 @@ Table 10 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, num, window_start, window_start#1 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ auction, num, window_start, window_start#1 ] + ├── primary key: [ $0 ASC, $2 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 2 - id: nexmark_q6 before: @@ -704,7 +716,7 @@ AND P.starttime = A.starttime AND P.endtime = A.endtime; stream_plan: |- - StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6, $expr8, $expr9, $expr10], pk_columns: [id, name, starttime, $expr6, $expr8, $expr9, $expr10], pk_conflict: NoCheck } + StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck } └─StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } ├─StreamExchange { dist: HashShard($expr2, $expr5, $expr6) } │ └─StreamAppendOnlyDedup { dedup_cols: [$expr2, $expr3, $expr5, $expr6] } @@ -733,7 +745,7 @@ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6, $expr8, $expr9, $expr10], pk_columns: [id, name, starttime, $expr6, $expr8, $expr9, $expr10], pk_conflict: NoCheck } + StreamMaterialize { columns: [id, name, starttime, $expr6(hidden), $expr8(hidden), $expr9(hidden), $expr10(hidden)], stream_key: [id, name, starttime, $expr6], pk_columns: [id, name, starttime, $expr6], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin [append_only] { type: Inner, predicate: $expr2 = $expr8 AND $expr5 = $expr9 AND $expr6 = $expr10, output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0, 2, 3]) from 1 @@ -779,7 +791,12 @@ Table 6 { columns: [ $expr8, $expr9, $expr10 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4294967294 { columns: [ id, name, starttime, $expr6, $expr8, $expr9, $expr10 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 7 } + Table 4294967294 + ├── columns: [ id, name, starttime, $expr6, $expr8, $expr9, $expr10 ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 2, 3 ] + └── read pk prefix len hint: 4 - id: nexmark_q9 before: @@ -1357,7 +1374,7 @@ GROUP BY b1.auction ) b ON a.id = b.auction; stream_plan: |- - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, auction_id], pk_columns: [_row_id, $expr5, auction_id], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } ├─StreamExchange { dist: HashShard($expr2) } │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } @@ -1385,7 +1402,7 @@ └─StreamNow { output: [now] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, $expr5, auction_id], pk_columns: [_row_id, $expr5, auction_id], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr5(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr5, output: [$expr2, $expr3, max($expr6), _row_id, $expr5] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0]) from 1 @@ -1438,10 +1455,10 @@ Table 4294967294 ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr5 ] - ├── primary key: [ $3 ASC, $4 ASC, $0 ASC ] + ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0 ] - └── read pk prefix len hint: 3 + └── read pk prefix len hint: 2 - id: nexmark_q102 before: 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 bc0ae6bd25441..73e5cff7a0ed1 100644 --- a/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_watermark.yaml @@ -396,7 +396,7 @@ └─BatchProject { 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 $expr3, _row_id] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck, watermark_columns: [window_start(hidden), window_start#1(hidden)] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck, watermark_columns: [window_start(hidden), window_start#1(hidden)] } └─StreamProject { exprs: [$expr2, count, window_start, window_start], output_watermarks: [window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin [window] { type: Inner, predicate: window_start = window_start, output_watermarks: [window_start, window_start], output: all } @@ -426,7 +426,7 @@ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck, watermark_columns: [window_start(hidden), window_start#1(hidden)] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck, watermark_columns: [window_start(hidden), window_start#1(hidden)] } ├── materialized table: 4294967294 └── StreamProject { exprs: [$expr2, count, window_start, window_start], output_watermarks: [window_start, window_start] } └── StreamFilter { predicate: (count >= max(count)) } @@ -475,10 +475,10 @@ Table 8 { columns: [ window_start, max(count), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4294967294 { columns: [ auction, num, window_start, window_start#1 ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 3 } + Table 4294967294 { columns: [ auction, num, window_start, window_start#1 ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 2 ], read pk prefix len hint: 2 } eowc_stream_plan: |- - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck, watermark_columns: [window_start(hidden)] } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck, watermark_columns: [window_start(hidden)] } └─StreamEowcSort { sort_column: window_start } └─StreamProject { exprs: [$expr2, count, window_start, window_start], output_watermarks: [window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } @@ -844,7 +844,7 @@ └─BatchProject { 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 $expr6, _row_id] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5, $expr7, $expr6, $expr8], pk_columns: [id, name, starttime, $expr5, $expr7, $expr6, $expr8], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } + StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) } │ └─StreamAppendOnlyDedup { dedup_cols: [$expr3, $expr4, $expr2, $expr5] } @@ -873,7 +873,7 @@ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5, $expr7, $expr6, $expr8], pk_columns: [id, name, starttime, $expr5, $expr7, $expr6, $expr8], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } + StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime, $expr5(hidden), $expr6(hidden), $expr8(hidden)] } ├── materialized table: 4294967294 └── StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0, 2, 3]) from 1 @@ -920,10 +920,10 @@ Table 7 { columns: [ $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } - Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 7 } + Table 4294967294 { columns: [ id, name, starttime, $expr5, $expr7, $expr6, $expr8 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 4 } eowc_stream_plan: |- - StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5, $expr7, $expr6, $expr8], pk_columns: [id, name, starttime, $expr5, $expr7, $expr6, $expr8], pk_conflict: NoCheck, watermark_columns: [starttime] } + StreamMaterialize { columns: [id, name, starttime, $expr5(hidden), $expr7(hidden), $expr6(hidden), $expr8(hidden)], stream_key: [id, name, starttime, $expr5], pk_columns: [id, name, starttime, $expr5], pk_conflict: NoCheck, watermark_columns: [starttime] } └─StreamEowcSort { sort_column: $expr2 } └─StreamHashJoin [window, append_only] { type: Inner, predicate: $expr2 = $expr6 AND $expr5 = $expr8 AND $expr3 = $expr7, output_watermarks: [$expr2, $expr5, $expr6, $expr8], output: all } ├─StreamExchange { dist: HashShard($expr3, $expr2, $expr5) } @@ -1908,7 +1908,7 @@ └─BatchProject { 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 $expr4, _row_id] } └─BatchSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, $expr4, auction_id], pk_columns: [_row_id, $expr4, auction_id], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } └─StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } ├─StreamExchange { dist: HashShard($expr2) } │ └─StreamProject { exprs: [Field(auction, 0:Int32) as $expr2, Field(auction, 1:Int32) as $expr3, _row_id] } @@ -1934,7 +1934,7 @@ └─StreamSource { source: nexmark, columns: [event_type, person, auction, bid, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, $expr4, auction_id], pk_columns: [_row_id, $expr4, auction_id], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), $expr4(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: LeftOuter, predicate: $expr2 = $expr4, output: [$expr2, $expr3, max($expr5), _row_id, $expr4] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0]) from 1 @@ -1976,10 +1976,10 @@ Table 4294967294 ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, $expr4 ] - ├── primary key: [ $3 ASC, $4 ASC, $0 ASC ] + ├── primary key: [ $3 ASC, $0 ASC ] ├── value indices: [ 0, 1, 2, 3, 4 ] ├── distribution key: [ 0 ] - └── read pk prefix len hint: 3 + └── read pk prefix len hint: 2 eowc_stream_error: |- Not supported: The query cannot be executed in Emit-On-Window-Close mode. diff --git a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml index 796a6521f731c..8180c1bed023b 100644 --- a/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml +++ b/src/frontend/planner_test/tests/testdata/output/over_window_function.yaml @@ -184,7 +184,7 @@ └─BatchExchange { order: [], dist: HashShard(t.y) } └─BatchScan { table: t, columns: [t.x, t.y, t.w], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, y, sum, max, min, t._row_id(hidden), t.y(hidden)], stream_key: [t._row_id, t.y, y], pk_columns: [t._row_id, t.y, y], pk_conflict: NoCheck } + StreamMaterialize { columns: [x, y, sum, max, min, t._row_id(hidden), t.y(hidden)], stream_key: [t._row_id, y], pk_columns: [t._row_id, y], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: t.y = t.y, output: [t.x, t.y, sum(t.x), max(t.x), min(t.w), t._row_id, t.y] } ├─StreamExchange { dist: HashShard(t.y) } │ └─StreamShare { id: 1 } diff --git a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml index ca81bfc0f1d3d..55131ed1614cd 100644 --- a/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml +++ b/src/frontend/planner_test/tests/testdata/output/pk_derive.yaml @@ -20,7 +20,7 @@ ON Tone.id = Ttwo.id; stream_plan: |- - StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], stream_key: [t1.id, t2.id], pk_columns: [t1.id, t2.id], pk_conflict: NoCheck } + StreamMaterialize { columns: [max_v1, max_v2, t1.id(hidden), t2.id(hidden)], stream_key: [t1.id], pk_columns: [t1.id], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: t1.id = t2.id, output: [max(t1.v1), max(t2.v2), t1.id, t2.id] } ├─StreamProject { exprs: [t1.id, max(t1.v1)] } │ └─StreamHashAgg { group_key: [t1.id], aggs: [max(t1.v1), count] } @@ -50,7 +50,7 @@ ON Tone.id = Ttwo.id; stream_plan: |- - StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], stream_key: [t.id, t.id#1], pk_columns: [t.id, t.id#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [max_v, min_v, t.id(hidden), t.id#1(hidden)], stream_key: [t.id], pk_columns: [t.id], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: t.id = t.id, output: [max(t.v), min(t.v), t.id, t.id] } ├─StreamProject { exprs: [t.id, max(t.v)] } │ └─StreamHashAgg { group_key: [t.id], aggs: [max(t.v), count] } diff --git a/src/frontend/planner_test/tests/testdata/output/share.yaml b/src/frontend/planner_test/tests/testdata/output/share.yaml index 78aeb4084f38c..2815b00784b1d 100644 --- a/src/frontend/planner_test/tests/testdata/output/share.yaml +++ b/src/frontend/planner_test/tests/testdata/output/share.yaml @@ -121,7 +121,7 @@ └─BatchFilter { predicate: IsNotNull(date_time) } └─BatchSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start, window_start#1], pk_columns: [auction, window_start, window_start#1], pk_conflict: NoCheck } + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } └─StreamProject { exprs: [auction, count, window_start, window_start] } └─StreamFilter { predicate: (count >= max(count)) } └─StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } @@ -150,7 +150,7 @@ create table t(a int, b int); with cte as (select count(*) from t) select * from cte union all select * from cte; stream_plan: |- - StreamMaterialize { columns: [count, 0:Int32(hidden)], stream_key: [0:Int32], pk_columns: [0:Int32], pk_conflict: NoCheck } + StreamMaterialize { columns: [count, $src(hidden)], stream_key: [$src], pk_columns: [$src], pk_conflict: NoCheck } └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } │ └─StreamProject { exprs: [sum0(count), 0:Int32] } @@ -173,7 +173,7 @@ create table t(a int, b int); with cte as (select count(*) from t) select * from cte union all select * from cte; stream_plan: |- - StreamMaterialize { columns: [count, 0:Int32(hidden)], stream_key: [0:Int32], pk_columns: [0:Int32], pk_conflict: NoCheck } + StreamMaterialize { columns: [count, $src(hidden)], stream_key: [$src], pk_columns: [$src], pk_conflict: NoCheck } └─StreamUnion { all: true } ├─StreamExchange { dist: HashShard(0:Int32) } │ └─StreamProject { exprs: [sum0(count), 0:Int32] } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 64116534a99b6..1ad1d9f92c418 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -534,7 +534,7 @@ └─BatchExchange { order: [], dist: HashShard(t.x) } └─BatchScan { table: t, columns: [t.x], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, y, k, sum_x, t.x(hidden)], stream_key: [k, t.x, x], pk_columns: [k, t.x, x], pk_conflict: NoCheck } + StreamMaterialize { columns: [x, y, k, sum_x, t.x(hidden)], stream_key: [k, x], pk_columns: [k, x], pk_conflict: NoCheck } └─StreamHashJoin { type: LeftOuter, predicate: t.x IS NOT DISTINCT FROM t.x, output: [t.x, t.y, t.k, sum(Unnest($0)), t.x] } ├─StreamExchange { dist: HashShard(t.x) } │ └─StreamTableScan { table: t, columns: [t.x, t.y, t.k], pk: [t.k], dist: UpstreamHashShard(t.k) } @@ -631,7 +631,7 @@ └─BatchFilter { predicate: IsNotNull(integers.i) } └─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, integers.i, i], pk_columns: [i, integers._row_id, integers.i], pk_conflict: NoCheck } + 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 } └─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], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } @@ -689,7 +689,7 @@ └─BatchFilter { predicate: IsNotNull(integers.i) } └─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, integers.i, i], pk_columns: [i, integers._row_id, integers.i], pk_conflict: NoCheck } + 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 } └─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], pk: [integers._row_id], dist: UpstreamHashShard(integers._row_id) } @@ -847,7 +847,7 @@ └─BatchExchange { order: [], dist: HashShard(1:Int32) } └─BatchValues { rows: [[1:Int32], [2:Int32]] } stream_plan: |- - StreamMaterialize { columns: [col, k, ts, window_start, window_end], stream_key: [col, k, window_start, window_end], pk_columns: [col, k, window_start, window_end], pk_conflict: NoCheck } + StreamMaterialize { columns: [col, k, ts, window_start, window_end], stream_key: [col, window_start, window_end], pk_columns: [col, window_start, window_end], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: 1:Int32 = t1.k, output: all } ├─StreamAppendOnlyDedup { dedup_cols: [1:Int32] } │ └─StreamExchange { dist: HashShard(1:Int32) } diff --git a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml index 88edfa45ad938..a1020b8d16ee5 100644 --- a/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/temporal_join.yaml @@ -5,7 +5,7 @@ create table version(id2 int, a2 int, b2 int, primary key (id2)); select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on id1= id2 stream_plan: |- - StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id2, id1], pk_columns: [stream._row_id, id2, id1], pk_conflict: NoCheck } + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2, output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } @@ -20,7 +20,7 @@ create table version(id2 int, a2 int, b2 int, primary key (id2)); select id1, a1, id2, a2 from stream join version FOR SYSTEM_TIME AS OF PROCTIME() on id1 = id2 where a2 < 10; stream_plan: |- - StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id2, id1], pk_columns: [stream._row_id, id2, id1], pk_conflict: NoCheck } + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } @@ -32,7 +32,7 @@ create table version(id2 int, a2 int, b2 int, primary key (id2)); select id1, a1, id2, a2 from stream, version FOR SYSTEM_TIME AS OF PROCTIME() where id1 = id2 AND a2 < 10; stream_plan: |- - StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id2, id1], pk_columns: [stream._row_id, id2, id1], pk_conflict: NoCheck } + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1], pk_columns: [stream._row_id, id1], pk_conflict: NoCheck } └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND (version.a2 < 10:Int32), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } @@ -44,7 +44,7 @@ create table version(id2 int, a2 int, b2 int, primary key (id2, a2)); select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and id1 = id2 where b2 != a2; stream_plan: |- - StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id2, a2, id1, a1], pk_columns: [stream._row_id, id2, a2, id1, a1], pk_conflict: NoCheck } + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden)], stream_key: [stream._row_id, id1, a1], pk_columns: [stream._row_id, id1, a1], pk_conflict: NoCheck } └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version.id2 AND stream.a1 = version.a2 AND (version.b2 <> version.a2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id] } ├─StreamExchange { dist: HashShard(stream.id1, stream.a1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } @@ -100,8 +100,8 @@ join version1 FOR SYSTEM_TIME AS OF PROCTIME() on stream.k = version1.k join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.k = version2.k where a1 < 10; stream_plan: |- - StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version1.k(hidden), version2.k(hidden)], stream_key: [stream._row_id, version1.k, k, version2.k], pk_columns: [stream._row_id, version1.k, k, version2.k], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version1.k, version2.k] } + StreamMaterialize { columns: [k, x1, x2, a1, b1, stream._row_id(hidden), version2.k(hidden)], stream_key: [stream._row_id, k], pk_columns: [stream._row_id, k], pk_conflict: NoCheck } + └─StreamTemporalJoin { type: Inner, predicate: stream.k = version2.k, output: [stream.k, version1.x1, version2.x2, stream.a1, stream.b1, stream._row_id, version2.k] } ├─StreamTemporalJoin { type: Inner, predicate: stream.k = version1.k, output: [stream.k, stream.a1, stream.b1, version1.x1, stream._row_id, version1.k] } │ ├─StreamExchange { dist: HashShard(stream.k) } │ │ └─StreamFilter { predicate: (stream.a1 < 10:Int32) } @@ -120,8 +120,8 @@ join version1 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id1 = version1.id1 join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10; stream_plan: |- - StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version1.id1(hidden), version2.id2(hidden)], stream_key: [stream._row_id, version1.id1, id1, version2.id2, id2], pk_columns: [stream._row_id, version1.id1, id1, version2.id2, id2], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version1.id1, version2.id2] } + StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } ├─StreamExchange { dist: HashShard(stream.id2) } │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } │ ├─StreamExchange { dist: HashShard(stream.id1) } @@ -141,8 +141,8 @@ join version1 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id1 = version1.id1 join version2 FOR SYSTEM_TIME AS OF PROCTIME() on stream.id2 = version2.id2 where a1 < 10; stream_plan: |- - StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version1.id1(hidden), version2.id2(hidden)], stream_key: [stream._row_id, version1.id1, id1, version2.id2, id2], pk_columns: [stream._row_id, version1.id1, id1, version2.id2, id2], pk_conflict: NoCheck } - └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version1.id1, version2.id2] } + StreamMaterialize { columns: [id1, x1, id2, x2, a1, b1, stream._row_id(hidden), version2.id2(hidden)], stream_key: [stream._row_id, id1, id2], pk_columns: [stream._row_id, id1, id2], pk_conflict: NoCheck } + └─StreamTemporalJoin { type: Inner, predicate: stream.id2 = version2.id2, output: [stream.id1, version1.x1, stream.id2, version2.x2, stream.a1, stream.b1, stream._row_id, version2.id2] } ├─StreamExchange { dist: HashShard(stream.id2) } │ └─StreamTemporalJoin { type: Inner, predicate: stream.id1 = version1.id1, output: [stream.id1, stream.id2, stream.a1, stream.b1, version1.x1, stream._row_id, version1.id1] } │ ├─StreamExchange { dist: HashShard(stream.id1) } @@ -240,7 +240,7 @@ create index idx3 on version (c2); select id1, a1, id2, a2 from stream left join version FOR SYSTEM_TIME AS OF PROCTIME() on a1 = a2 and b1 = b2 and c1 = c2 and id1 = id2; stream_plan: |- - StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden), stream.c1(hidden)], stream_key: [stream._row_id, id2, id1, a1, stream.b1, stream.c1], pk_columns: [stream._row_id, id2, id1, a1, stream.b1, stream.c1], pk_conflict: NoCheck } + StreamMaterialize { columns: [id1, a1, id2, a2, stream._row_id(hidden), stream.b1(hidden), stream.c1(hidden)], stream_key: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_columns: [stream._row_id, id1, a1, stream.b1, stream.c1], pk_conflict: NoCheck } └─StreamTemporalJoin { type: LeftOuter, predicate: stream.id1 = version.id2 AND (stream.a1 = version.a2) AND (stream.b1 = version.b2) AND (stream.c1 = version.c2), output: [stream.id1, stream.a1, version.id2, version.a2, stream._row_id, stream.b1, stream.c1] } ├─StreamExchange { dist: HashShard(stream.id1) } │ └─StreamTableScan { table: stream, columns: [stream.id1, stream.a1, stream.b1, stream.c1, stream._row_id], pk: [stream._row_id], dist: UpstreamHashShard(stream._row_id) } diff --git a/src/frontend/planner_test/tests/testdata/output/tpch.yaml b/src/frontend/planner_test/tests/testdata/output/tpch.yaml index 1fdcdfee27b4d..e4bc067ba2988 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch.yaml @@ -321,13 +321,13 @@ └─BatchFilter { predicate: IsNotNull(partsupp.ps_partkey) } └─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, region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), part.p_partkey(hidden), partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), min(partsupp.ps_supplycost)(hidden)], stream_key: [region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)], 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), part.p_partkey(hidden), partsupp.ps_suppkey(hidden), min(partsupp.ps_supplycost)(hidden)], stream_key: [region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } └─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: [$expr1] } - └─StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost), Vnode(supplier.s_suppkey) as $expr1] } - └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } + └─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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost), Vnode(supplier.s_suppkey) as $expr1] } + └─StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } ├─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) } @@ -341,10 +341,10 @@ │ └─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], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost), partsupp.ps_partkey] } + └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost)] } ├─StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } │ └─StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } - │ └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } + │ └─StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, region.r_regionkey, supplier.s_nationkey] } │ ├─StreamExchange { dist: HashShard(part.p_partkey) } │ │ └─StreamProject { exprs: [part.p_partkey] } │ │ └─StreamHashAgg { group_key: [part.p_partkey], aggs: [count] } @@ -352,7 +352,7 @@ │ │ └─StreamFilter { predicate: (part.p_size = 4:Int32) AND Like(part.p_type, '%TIN':Varchar) } │ │ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_type, part.p_size], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } - │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, 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) } @@ -377,16 +377,16 @@ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.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, region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), part.p_partkey(hidden), partsupp.ps_partkey(hidden), partsupp.ps_suppkey(hidden), min(partsupp.ps_supplycost)(hidden)], stream_key: [region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)], pk_conflict: NoCheck } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, region.r_regionkey(hidden), nation.n_nationkey(hidden), supplier.s_suppkey(hidden), part.p_partkey(hidden), partsupp.ps_suppkey(hidden), min(partsupp.ps_supplycost)(hidden)], stream_key: [region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], pk_columns: [s_acctbal, n_name, s_name, p_partkey, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, min(partsupp.ps_supplycost)], pk_conflict: NoCheck } ├── materialized table: 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } + └── 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } └── StreamTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0 } { state table: 0 } └── StreamExchange Single from 1 Fragment 1 StreamGroupTopN { order: [supplier.s_acctbal DESC, nation.n_name ASC, supplier.s_name ASC, part.p_partkey ASC], limit: 100, offset: 0, group_key: [$expr1] } { state table: 1 } - └── StreamProject { exprs: [supplier.s_acctbal, supplier.s_name, nation.n_name, part.p_partkey, part.p_mfgr, supplier.s_address, supplier.s_phone, supplier.s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost), Vnode(supplier.s_suppkey) as $expr1] } - └── StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } { left table: 2, right table: 4, left degree table: 3, right degree table: 5 } + └── 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost), Vnode(supplier.s_suppkey) as $expr1] } + └── StreamHashJoin { type: Inner, predicate: supplier.s_suppkey = partsupp.ps_suppkey, 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost)] } { left table: 2, right table: 4, left degree table: 3, right degree table: 5 } ├── StreamExchange Hash([1]) from 2 └── StreamExchange Hash([2]) from 7 @@ -418,10 +418,10 @@ └── BatchPlanNode Fragment 7 - StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost), partsupp.ps_partkey] } { left table: 17, right table: 19, left degree table: 18, right degree table: 20 } + StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey AND min(partsupp.ps_supplycost) = partsupp.ps_supplycost, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, part.p_partkey, min(partsupp.ps_supplycost)] } { left table: 17, right table: 19, left degree table: 18, right degree table: 20 } ├── StreamProject { exprs: [part.p_partkey, min(partsupp.ps_supplycost)] } │ └── StreamHashAgg { group_key: [part.p_partkey], aggs: [min(partsupp.ps_supplycost), count] } { intermediate state table: 22, state tables: [ 21 ], distinct tables: [] } - │ └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, region.r_regionkey, nation.n_nationkey, supplier.s_nationkey] } { left table: 23, right table: 25, left degree table: 24, right degree table: 26 } + │ └── StreamHashJoin { type: LeftOuter, predicate: part.p_partkey IS NOT DISTINCT FROM partsupp.ps_partkey, output: [part.p_partkey, partsupp.ps_supplycost, partsupp.ps_partkey, partsupp.ps_suppkey, region.r_regionkey, supplier.s_nationkey] } { left table: 23, right table: 25, left degree table: 24, right degree table: 26 } │ ├── StreamExchange Hash([0]) from 8 │ └── StreamExchange Hash([0]) from 9 └── StreamHashJoin { type: Inner, predicate: part.p_partkey = partsupp.ps_partkey, output: [part.p_partkey, part.p_mfgr, partsupp.ps_suppkey, partsupp.ps_supplycost, partsupp.ps_partkey] } { left table: 45, right table: 47, left degree table: 46, right degree table: 48 } @@ -438,7 +438,7 @@ └── BatchPlanNode Fragment 9 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } { left table: 29, right table: 31, left degree table: 30, right degree table: 32 } + StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_nationkey, region.r_regionkey, nation.n_nationkey] } { left table: 29, right table: 31, left degree table: 30, right degree table: 32 } ├── StreamExchange Hash([2]) from 10 └── StreamExchange Hash([0]) from 13 @@ -487,17 +487,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey_0, partsupp_ps_partkey, partsupp_ps_suppkey, min(partsupp_ps_supplycost), $expr1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey_0, partsupp_ps_suppkey, min(partsupp_ps_supplycost), $expr1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC, $14 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey_0, partsupp_ps_partkey, partsupp_ps_suppkey, min(partsupp_ps_supplycost), $expr1 ], primary key: [ $15 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ], distribution key: [ 10 ], read pk prefix len hint: 1, vnode column idx: 15 } + Table 1 { columns: [ supplier_s_acctbal, supplier_s_name, nation_n_name, part_p_partkey, part_p_mfgr, supplier_s_address, supplier_s_phone, supplier_s_comment, region_r_regionkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey_0, partsupp_ps_suppkey, min(partsupp_ps_supplycost), $expr1 ], primary key: [ $14 ASC, $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 10 ], read pk prefix len hint: 1, vnode column idx: 14 } Table 2 { columns: [ 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 ], primary key: [ $1 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 1 ], read pk prefix len hint: 1 } Table 3 { columns: [ supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, part_p_partkey_0, min(partsupp_ps_supplycost), partsupp_ps_partkey ], primary key: [ $2 ASC, $3 ASC, $0 ASC, $5 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 4 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, part_p_partkey_0, min(partsupp_ps_supplycost) ], primary key: [ $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 5 { columns: [ partsupp_ps_suppkey, part_p_partkey, part_p_partkey_0, partsupp_ps_partkey, min(partsupp_ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ partsupp_ps_suppkey, part_p_partkey, min(partsupp_ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 { columns: [ nation_n_nationkey, nation_n_name, region_r_regionkey ], primary key: [ $0 ASC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -525,11 +525,11 @@ Table 18 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 19 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, partsupp_ps_supplycost, partsupp_ps_partkey ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 19 { columns: [ part_p_partkey, part_p_mfgr, partsupp_ps_suppkey, partsupp_ps_supplycost, partsupp_ps_partkey ], primary key: [ $0 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 20 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + Table 20 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } - Table 21 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 21 { columns: [ part_p_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, region_r_regionkey, supplier_s_nationkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 22 { columns: [ part_p_partkey, min(partsupp_ps_supplycost), count ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -537,17 +537,17 @@ Table 24 { columns: [ part_p_partkey, _degree ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 25 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, supplier_s_suppkey, supplier_s_nationkey, region_r_regionkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $5 ASC, $6 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 25 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, partsupp_ps_suppkey, supplier_s_nationkey, region_r_regionkey, nation_n_nationkey ], primary key: [ $0 ASC, $2 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 26 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, region_r_regionkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 26 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, region_r_regionkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 27 { columns: [ part_p_partkey, count ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 28 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 29 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $2 ASC, $0 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 29 { columns: [ partsupp_ps_partkey, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ], primary key: [ $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 30 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 30 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 31 { columns: [ nation_n_nationkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -589,7 +589,7 @@ Table 50 { columns: [ vnode, ps_partkey, ps_suppkey, partsupp_backfill_finished, partsupp_row_count ], 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, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost) ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [], read pk prefix len hint: 11 } + Table 4294967294 { columns: [ s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, region.r_regionkey, nation.n_nationkey, supplier.s_suppkey, part.p_partkey, partsupp.ps_suppkey, min(partsupp.ps_supplycost) ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $13 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 } - id: tpch_q3 before: @@ -1005,8 +1005,8 @@ └─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, customer.c_custkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, customer.c_nationkey] } - └─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, customer.c_custkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, customer.c_nationkey] } + └─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) } @@ -1017,7 +1017,7 @@ │ └─StreamFilter { predicate: (nation.n_nationkey = nation.n_nationkey) } │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name, nation.n_regionkey], pk: [nation.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, customer.c_custkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey] } + └─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) } @@ -1050,8 +1050,8 @@ └── StreamExchange Hash([0]) from 2 Fragment 2 - StreamProject { exprs: [nation.n_name, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, region.r_regionkey, nation.n_nationkey, orders.o_orderkey, customer.c_custkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, customer.c_nationkey] } - └── 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, customer.c_custkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey, customer.c_nationkey] } + 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] } ├── left table: 3 ├── right table: 5 ├── left degree table: 4 @@ -1078,7 +1078,7 @@ └── BatchPlanNode Fragment 6 - 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, customer.c_custkey, orders.o_custkey, lineitem.l_orderkey, lineitem.l_linenumber, supplier.s_suppkey, lineitem.l_suppkey] } + 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] } ├── left table: 13 ├── right table: 15 ├── left degree table: 14 @@ -1129,13 +1129,13 @@ Table 4 { columns: [ nation_n_nationkey, nation_n_nationkey_0, region_r_regionkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } Table 5 - ├── columns: [ customer_c_nationkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey ] - ├── primary key: [ $3 ASC, $0 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── columns: [ 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 ] + ├── primary key: [ $3 ASC, $0 ASC, $4 ASC, $5 ASC, $7 ASC, $8 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] ├── distribution key: [ 0, 3 ] └── read pk prefix len hint: 2 - Table 6 { columns: [ supplier_s_nationkey, customer_c_nationkey, orders_o_orderkey, customer_c_custkey, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 9 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } + Table 6 { columns: [ supplier_s_nationkey, customer_c_nationkey, orders_o_orderkey, orders_o_custkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 1, 0 ], read pk prefix len hint: 2 } Table 7 { columns: [ region_r_regionkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1149,13 +1149,13 @@ Table 12 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 13 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, customer_c_custkey ], primary key: [ $0 ASC, $1 ASC, $3 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 13 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, customer_c_custkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 14 { columns: [ orders_o_orderkey, customer_c_nationkey, customer_c_custkey, orders_o_custkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 14 { columns: [ orders_o_orderkey, customer_c_nationkey, orders_o_custkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } - Table 15 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, supplier_s_suppkey ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $6 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 3 ], read pk prefix len hint: 2 } + Table 15 { columns: [ lineitem_l_orderkey, lineitem_l_extendedprice, lineitem_l_discount, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, supplier_s_suppkey ], primary key: [ $0 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 0, 3 ], read pk prefix len hint: 2 } - Table 16 { columns: [ lineitem_l_orderkey, supplier_s_nationkey, lineitem_l_linenumber, supplier_s_suppkey, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + Table 16 { columns: [ lineitem_l_orderkey, supplier_s_nationkey, lineitem_l_linenumber, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } Table 17 { columns: [ orders_o_orderkey, orders_o_custkey ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 1 ], read pk prefix len hint: 1 } @@ -1355,7 +1355,7 @@ └─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, orders.o_orderkey] } + └─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) } @@ -1397,7 +1397,7 @@ └── StreamExchange Hash([0, 1, 2]) from 2 Fragment 2 - 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, orders.o_orderkey] } + 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 } { left table: 3, right table: 5, left degree table: 4, right degree table: 6 } ├── StreamExchange Hash([1]) from 3 @@ -1429,7 +1429,8 @@ Fragment 7 StreamFilter { predicate: (lineitem.l_shipdate >= '1983-01-01':Date) AND (lineitem.l_shipdate <= '2000-12-31':Date) } - └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } { state table: 17 } + └── Chain { table: lineitem, columns: [lineitem.l_orderkey, lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_shipdate, lineitem.l_linenumber], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } + ├── state table: 17 ├── Upstream └── BatchPlanNode @@ -1645,9 +1646,9 @@ └─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, orders.o_orderkey] } - └─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, orders.o_orderkey] } - └─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, orders.o_orderkey] } + └─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) } @@ -1699,9 +1700,9 @@ └── StreamExchange Hash([0]) from 2 Fragment 2 - 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, orders.o_orderkey] } - └── 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, orders.o_orderkey] } - └── 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, orders.o_orderkey] } + 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] } ├── left table: 3 ├── right table: 5 ├── left degree table: 4 @@ -1800,12 +1801,12 @@ Table 5 ├── columns: [ 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 ] - ├── primary key: [ $3 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC, $10 ASC ] + ├── primary key: [ $3 ASC, $5 ASC, $6 ASC, $7 ASC, $8 ASC, $9 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] ├── distribution key: [ 3 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ orders_o_custkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ orders_o_custkey, nation_n_nationkey, supplier_s_suppkey, part_p_partkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 6 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 { columns: [ nation_n_nationkey, region_r_regionkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -1974,8 +1975,8 @@ └─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_partkey, partsupp.ps_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └─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_partkey, partsupp.ps_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + └─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) } @@ -1985,7 +1986,7 @@ │ └─StreamExchange { dist: HashShard(partsupp.ps_partkey) } │ └─StreamFilter { predicate: (partsupp.ps_suppkey = partsupp.ps_suppkey) } │ └─StreamTableScan { table: partsupp, columns: [partsupp.ps_partkey, partsupp.ps_suppkey, partsupp.ps_supplycost], pk: [partsupp.ps_partkey, partsupp.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_orderkey, lineitem.l_linenumber] } + └─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) } @@ -2017,14 +2018,14 @@ └── StreamExchange Hash([0, 1]) from 2 Fragment 2 - 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_partkey, partsupp.ps_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } - └── 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_partkey, partsupp.ps_suppkey, nation.n_nationkey, supplier.s_suppkey, orders.o_orderkey, lineitem.l_orderkey, lineitem.l_linenumber] } + 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] } ├── left table: 3 ├── right table: 5 ├── left degree table: 4 ├── right degree table: 6 ├── StreamExchange Hash([2]) from 3 - └── 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_orderkey, lineitem.l_linenumber] } { left table: 13, right table: 15, left degree table: 14, right degree table: 16 } + └── 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] } { left table: 13, right table: 15, left degree table: 14, right degree table: 16 } ├── StreamExchange Hash([1]) from 6 └── StreamExchange Hash([2]) from 9 @@ -2087,9 +2088,9 @@ Table 4 { columns: [ part_p_partkey, partsupp_ps_suppkey, partsupp_ps_partkey, partsupp_ps_suppkey_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 3 ], read pk prefix len hint: 4 } - Table 5 { columns: [ 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_orderkey, lineitem_l_linenumber ], primary key: [ $3 ASC, $4 ASC, $3 ASC, $1 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ], distribution key: [ 1 ], read pk prefix len hint: 4 } + Table 5 { columns: [ 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 ], primary key: [ $3 ASC, $4 ASC, $3 ASC, $1 ASC, $8 ASC, $9 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 1 ], read pk prefix len hint: 4 } - Table 6 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_partkey_0, supplier_s_suppkey, nation_n_nationkey, orders_o_orderkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC, $7 ASC ], value indices: [ 8 ], distribution key: [ 3 ], read pk prefix len hint: 4 } + Table 6 { columns: [ lineitem_l_partkey, lineitem_l_suppkey, lineitem_l_partkey_0, supplier_s_suppkey, nation_n_nationkey, orders_o_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 7 ], distribution key: [ 3 ], read pk prefix len hint: 4 } Table 7 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2107,9 +2108,9 @@ Table 14 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 15 { columns: [ 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 ], primary key: [ $2 ASC, $6 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 15 { columns: [ 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 ], primary key: [ $2 ASC, $6 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 16 { columns: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 16 { columns: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 17 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2229,8 +2230,8 @@ └─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, orders.o_orderkey] } - └─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, orders.o_orderkey] } + └─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) } @@ -2259,8 +2260,8 @@ StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } { state table: 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] } { intermediate state table: 2, state tables: [], distinct tables: [] } - └── 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, orders.o_orderkey] } - └── 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, orders.o_orderkey] } + └── 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] } ├── left table: 3 ├── right table: 5 ├── left degree table: 4 @@ -2302,7 +2303,12 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2 ], primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC, $8 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 + ├── columns: [ customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2 ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC, $3 ASC, $6 ASC, $4 ASC, $5 ASC, $7 ASC, $8 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 Table 1 ├── columns: [ customer_c_custkey, customer_c_name, sum($expr1), customer_c_acctbal, nation_n_name, customer_c_address, customer_c_phone, customer_c_comment, $expr2 ] @@ -2318,9 +2324,9 @@ Table 4 { columns: [ customer_c_custkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ lineitem_l_extendedprice, lineitem_l_discount, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, orders_o_orderkey ], primary key: [ $2 ASC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } + Table 5 { columns: [ lineitem_l_extendedprice, lineitem_l_discount, orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, orders_o_orderkey ], primary key: [ $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 2 ], read pk prefix len hint: 1 } - Table 6 { columns: [ orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, orders_o_orderkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ orders_o_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 { columns: [ nation_n_nationkey, nation_n_name ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2454,9 +2460,9 @@ ├─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_suppkey, nation.n_nationkey, supplier.s_nationkey] } + │ └─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_suppkey, supplier.s_nationkey, nation.n_nationkey] } + │ └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_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) } @@ -2472,9 +2478,9 @@ └─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_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └─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_suppkey, supplier.s_nationkey, nation.n_nationkey] } + └─StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_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) } @@ -2508,11 +2514,11 @@ └── StreamExchange Broadcast from 8 Fragment 2 - StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + StreamProject { exprs: [partsupp.ps_partkey, (partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr1, partsupp.ps_suppkey, supplier.s_nationkey] } └── StreamExchange NoShuffle from 3 Fragment 3 - StreamHashJoin { type: Inner, predicate: supplier.s_nationkey = nation.n_nationkey, output: [partsupp.ps_partkey, partsupp.ps_availqty, partsupp.ps_supplycost, partsupp.ps_suppkey, supplier.s_suppkey, supplier.s_nationkey, nation.n_nationkey] } + 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] } ├── left table: 5 ├── right table: 7 ├── left degree table: 6 @@ -2554,7 +2560,7 @@ Fragment 9 StreamStatelessSimpleAgg { aggs: [sum($expr2)] } - └── StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └── StreamProject { exprs: [(partsupp.ps_supplycost * partsupp.ps_availqty::Decimal) as $expr2, partsupp.ps_partkey, partsupp.ps_suppkey, supplier.s_nationkey] } └── StreamExchange NoShuffle from 3 Table 0 { columns: [ partsupp_ps_partkey, sum($expr1), $expr4 ], primary key: [ $1 DESC, $2 ASC ], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } @@ -2569,12 +2575,12 @@ Table 5 ├── columns: [ partsupp_ps_partkey, partsupp_ps_availqty, partsupp_ps_supplycost, supplier_s_nationkey, partsupp_ps_suppkey, supplier_s_suppkey ] - ├── primary key: [ $3 ASC, $0 ASC, $4 ASC, $5 ASC ] + ├── primary key: [ $3 ASC, $0 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 3 ] └── read pk prefix len hint: 1 - Table 6 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 6 { columns: [ supplier_s_nationkey, partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 7 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -2670,7 +2676,7 @@ └─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_orderkey, lineitem.l_linenumber] } + └─StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $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], pk: [orders.o_orderkey], dist: UpstreamHashShard(orders.o_orderkey) } @@ -2696,7 +2702,7 @@ └── StreamExchange Hash([0]) from 2 Fragment 2 - 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_orderkey, lineitem.l_linenumber] } + StreamProject { exprs: [lineitem.l_shipmode, Case(((orders.o_orderpriority = '1-URGENT':Varchar) OR (orders.o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $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] } { left table: 3, right table: 5, left degree table: 4, right degree table: 6 } ├── StreamExchange Hash([0]) from 3 └── StreamExchange Hash([0]) from 4 @@ -2963,7 +2969,7 @@ └─StreamSimpleAgg { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum($expr2)] } - └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } │ └─StreamProject { exprs: [lineitem.l_partkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber] } @@ -2984,7 +2990,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum($expr1), sum($expr2)] } - └── StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └── StreamProject { exprs: [Case(Like(part.p_type, 'PROMO%':Varchar), (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)), 0:Decimal) as $expr1, (lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr2, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } └── StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: [lineitem.l_extendedprice, lineitem.l_discount, part.p_type, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey] } ├── left table: 1 ├── right table: 3 @@ -3093,13 +3099,13 @@ └─LogicalProject { exprs: [lineitem.l_suppkey, (lineitem.l_extendedprice * (1:Int32::Decimal - lineitem.l_discount)) as $expr2] } └─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, lineitem.l_suppkey(hidden)], 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), lineitem.l_suppkey] } + 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: [$expr3] } - └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey, Vnode(sum($expr1)) as $expr3] } - └─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), lineitem.l_suppkey] } + └─StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as $expr3] } + └─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) } @@ -3126,16 +3132,16 @@ └─StreamTableScan { table: lineitem, columns: [lineitem.l_suppkey, lineitem.l_extendedprice, lineitem.l_discount, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_shipdate], pk: [lineitem.l_orderkey, lineitem.l_linenumber], dist: UpstreamHashShard(lineitem.l_orderkey, lineitem.l_linenumber) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, lineitem.l_suppkey(hidden)], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_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 } { state table: 0 } └── StreamExchange Single from 1 Fragment 1 StreamGroupTopN { order: [supplier.s_suppkey ASC], limit: 1, offset: 0, group_key: [$expr3] } { state table: 1 } - └── StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), lineitem.l_suppkey, Vnode(sum($expr1)) as $expr3] } - └── 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), lineitem.l_suppkey] } + └── StreamProject { exprs: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone, sum($expr1), Vnode(sum($expr1)) as $expr3] } + └── 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)] } ├── left table: 2 ├── right table: 4 ├── left degree table: 3 @@ -3153,7 +3159,8 @@ └── StreamExchange NoShuffle from 4 Fragment 3 - Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } { state table: 10 } + Chain { table: supplier, columns: [supplier.s_suppkey, supplier.s_name, supplier.s_address, supplier.s_phone], pk: [supplier.s_suppkey], dist: UpstreamHashShard(supplier.s_suppkey) } + ├── state table: 10 ├── Upstream └── BatchPlanNode @@ -3180,19 +3187,13 @@ └── StreamProject { exprs: [lineitem.l_suppkey, sum($expr1), Vnode(lineitem.l_suppkey) as $expr2] } └── StreamExchange NoShuffle from 4 - Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), lineitem_l_suppkey, $expr3 ], primary key: [ $0 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), $expr3 ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 - ├── columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), lineitem_l_suppkey, $expr3 ] - ├── primary key: [ $6 ASC, $0 ASC, $5 ASC, $4 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] - ├── distribution key: [ 4 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 6 + Table 1 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), $expr3 ], primary key: [ $5 ASC, $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 1, vnode column idx: 5 } - Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), lineitem_l_suppkey ], primary key: [ $4 ASC, $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 1 } + Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_phone, sum($expr1), lineitem_l_suppkey ], primary key: [ $4 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 4 ], read pk prefix len hint: 1 } - Table 3 { columns: [ sum($expr1), supplier_s_suppkey, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ sum($expr1), supplier_s_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 { columns: [ max(max(sum($expr1))) ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -3220,7 +3221,7 @@ Table 16 { columns: [ $expr2, max(sum($expr1)), count ], 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, lineitem.l_suppkey ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_suppkey, s_name, s_address, s_phone, total_revenue ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q16 before: @@ -3300,7 +3301,7 @@ └─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: all } + └─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) } @@ -3332,7 +3333,7 @@ └── StreamExchange Hash([1, 2, 3]) from 2 Fragment 2 - StreamHashJoin { type: LeftAnti, predicate: partsupp.ps_suppkey = supplier.s_suppkey, output: all } + 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] } ├── left table: 4 ├── right table: 6 ├── left degree table: 5 @@ -3399,12 +3400,12 @@ Table 4 ├── columns: [ partsupp_ps_suppkey, part_p_brand, part_p_type, part_p_size, partsupp_ps_partkey, part_p_partkey ] - ├── primary key: [ $0 ASC, $4 ASC, $5 ASC ] + ├── primary key: [ $0 ASC, $4 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5 ] ├── distribution key: [ 0 ] └── read pk prefix len hint: 1 - Table 5 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, part_p_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 { columns: [ supplier_s_suppkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -3513,7 +3514,7 @@ └─StreamSimpleAgg { aggs: [sum(sum(lineitem.l_extendedprice)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } + └─StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, part.p_partkey] } └─StreamFilter { predicate: (lineitem.l_quantity < $expr1) } └─StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } ├─StreamExchange { dist: HashShard(part.p_partkey) } @@ -3549,7 +3550,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum(lineitem.l_extendedprice)] } - └── StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey, part.p_partkey] } + └── StreamProject { exprs: [lineitem.l_extendedprice, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey, part.p_partkey, part.p_partkey] } └── StreamFilter { predicate: (lineitem.l_quantity < $expr1) } └── StreamHashJoin { type: Inner, predicate: part.p_partkey IS NOT DISTINCT FROM part.p_partkey, output: all } { left table: 1, right table: 3, left degree table: 2, right degree table: 4 } ├── StreamExchange Hash([2]) from 2 @@ -3741,7 +3742,7 @@ └─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: all } + └─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] } @@ -3769,7 +3770,11 @@ StreamGroupTopN { order: [orders.o_totalprice DESC, orders.o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } { state table: 1 } └── StreamProject { exprs: [customer.c_name, customer.c_custkey, orders.o_orderkey, orders.o_orderdate, orders.o_totalprice, sum(lineitem.l_quantity), Vnode(orders.o_orderkey) as $expr1] } └── StreamHashAgg { group_key: [customer.c_custkey, customer.c_name, orders.o_orderkey, orders.o_totalprice, orders.o_orderdate], aggs: [sum(lineitem.l_quantity), count] } { intermediate state table: 2, state tables: [], distinct tables: [] } - └── StreamHashJoin { type: LeftSemi, predicate: orders.o_orderkey = lineitem.l_orderkey, output: all } { left table: 3, right table: 5, left degree table: 4, right degree table: 6 } + └── 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] } + ├── left table: 3 + ├── right table: 5 + ├── left degree table: 4 + ├── right degree table: 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] } │ ├── left table: 7 │ ├── right table: 9 @@ -3836,12 +3841,12 @@ Table 3 ├── columns: [ customer_c_custkey, customer_c_name, orders_o_orderkey, orders_o_totalprice, orders_o_orderdate, lineitem_l_quantity, lineitem_l_orderkey, lineitem_l_linenumber ] - ├── primary key: [ $2 ASC, $0 ASC, $6 ASC, $7 ASC ] + ├── primary key: [ $2 ASC, $0 ASC, $7 ASC ] ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] ├── distribution key: [ 2 ] └── read pk prefix len hint: 1 - Table 4 { columns: [ orders_o_orderkey, customer_c_custkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ orders_o_orderkey, customer_c_custkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 5 { columns: [ lineitem_l_orderkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -3944,7 +3949,7 @@ └─StreamSimpleAgg { aggs: [sum(sum($expr1)), count] } └─StreamExchange { dist: Single } └─StreamStatelessSimpleAgg { aggs: [sum($expr1)] } - └─StreamProject { exprs: [(lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └─StreamProject { exprs: [(lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } └─StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Decimal)) AND (lineitem.l_quantity <= 11:Decimal)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Decimal)) AND (lineitem.l_quantity <= 40:Decimal)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Decimal)) AND (lineitem.l_quantity <= 20:Decimal)) AND (part.p_size <= 15:Int32))) } └─StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } ├─StreamExchange { dist: HashShard(lineitem.l_partkey) } @@ -3967,7 +3972,7 @@ Fragment 1 StreamStatelessSimpleAgg { aggs: [sum($expr1)] } - └── StreamProject { exprs: [(lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber, part.p_partkey, lineitem.l_partkey] } + └── StreamProject { exprs: [(lineitem.l_extendedprice * (1:Decimal - lineitem.l_discount)) as $expr1, lineitem.l_orderkey, lineitem.l_linenumber, lineitem.l_partkey] } └── StreamFilter { predicate: (((((((part.p_brand = 'Brand#52':Varchar) AND In(part.p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND (lineitem.l_quantity >= 1:Decimal)) AND (lineitem.l_quantity <= 11:Decimal)) AND (part.p_size <= 5:Int32)) OR (((((part.p_brand = 'Brand#24':Varchar) AND In(part.p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND (lineitem.l_quantity >= 30:Decimal)) AND (lineitem.l_quantity <= 40:Decimal)) AND (part.p_size <= 10:Int32))) OR (((((part.p_brand = 'Brand#32':Varchar) AND In(part.p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND (lineitem.l_quantity >= 10:Decimal)) AND (lineitem.l_quantity <= 20:Decimal)) AND (part.p_size <= 15:Int32))) } └── StreamHashJoin { type: Inner, predicate: lineitem.l_partkey = part.p_partkey, output: all } { left table: 1, right table: 3, left degree table: 2, right degree table: 4 } ├── StreamExchange Hash([0]) from 2 @@ -4064,13 +4069,13 @@ └─BatchFilter { predicate: (lineitem.l_shipdate >= '1994-01-01':Date) AND (lineitem.l_shipdate < '1995-01-01 00:00:00':Timestamp) } └─BatchScan { table: lineitem, columns: [lineitem.l_partkey, lineitem.l_suppkey, lineitem.l_quantity, lineitem.l_shipdate], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } - └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + 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: [$expr3] } - └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as $expr3] } - └─StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └─StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as $expr3] } + └─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) } @@ -4080,7 +4085,7 @@ │ └─StreamFilter { predicate: (nation.n_name = 'KENYA':Varchar) } │ └─StreamTableScan { table: nation, columns: [nation.n_nationkey, nation.n_name], pk: [nation.n_nationkey], dist: UpstreamHashShard(nation.n_nationkey) } └─StreamExchange { dist: HashShard(partsupp.ps_suppkey) } - └─StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey, lineitem.l_partkey, lineitem.l_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) } @@ -4100,16 +4105,16 @@ └─StreamTableScan { table: part, columns: [part.p_partkey, part.p_name], pk: [part.p_partkey], dist: UpstreamHashShard(part.p_partkey) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), nation.n_nationkey(hidden), supplier.s_nationkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } + StreamMaterialize { columns: [s_name, s_address, supplier.s_suppkey(hidden), supplier.s_nationkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } ├── materialized table: 4294967294 - └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } └── StreamTopN { order: [supplier.s_name ASC], limit: 1, offset: 0 } { state table: 0 } └── StreamExchange Single from 1 Fragment 1 StreamGroupTopN { order: [supplier.s_name ASC], limit: 1, offset: 0, group_key: [$expr3] } { state table: 1 } - └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as $expr3] } - └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, nation.n_nationkey, supplier.s_nationkey] } + └── StreamProject { exprs: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey, Vnode(supplier.s_suppkey) as $expr3] } + └── StreamHashJoin { type: LeftSemi, predicate: supplier.s_suppkey = partsupp.ps_suppkey, output: [supplier.s_name, supplier.s_address, supplier.s_suppkey, supplier.s_nationkey] } ├── left table: 2 ├── right table: 4 ├── left degree table: 3 @@ -4136,7 +4141,7 @@ └── BatchPlanNode Fragment 5 - StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey, lineitem.l_partkey, lineitem.l_suppkey] } + StreamHashJoin { type: LeftSemi, predicate: partsupp.ps_partkey = part.p_partkey, output: [partsupp.ps_suppkey, partsupp.ps_partkey] } ├── left table: 12 ├── right table: 14 ├── left degree table: 13 @@ -4154,7 +4159,10 @@ ├── right degree table: 19 ├── StreamExchange Hash([0, 1]) from 7 └── 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] } { intermediate state table: 21, state tables: [], distinct tables: [] } + └── StreamHashAgg { group_key: [lineitem.l_partkey, lineitem.l_suppkey], aggs: [sum(lineitem.l_quantity), count] } + ├── intermediate state table: 21 + ├── state tables: [] + ├── distinct tables: [] └── StreamExchange Hash([0, 1]) from 8 Fragment 7 @@ -4179,23 +4187,17 @@ ├── Upstream └── BatchPlanNode - Table 0 { columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, $expr3 ], primary key: [ $0 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [], read pk prefix len hint: 0 } + Table 0 { columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, supplier_s_nationkey, $expr3 ], primary key: [ $0 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 0 } - Table 1 - ├── columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, $expr3 ] - ├── primary key: [ $5 ASC, $0 ASC, $2 ASC, $3 ASC, $4 ASC ] - ├── value indices: [ 0, 1, 2, 3, 4, 5 ] - ├── distribution key: [ 2 ] - ├── read pk prefix len hint: 1 - └── vnode column idx: 5 + Table 1 { columns: [ supplier_s_name, supplier_s_address, supplier_s_suppkey, supplier_s_nationkey, $expr3 ], primary key: [ $4 ASC, $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 2 ], read pk prefix len hint: 1, vnode column idx: 4 } - Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $4 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey, nation_n_nationkey ], primary key: [ $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ supplier_s_suppkey, nation_n_nationkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ supplier_s_suppkey, supplier_s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 4 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, lineitem_l_partkey, lineitem_l_suppkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 4 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 5 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, lineitem_l_partkey, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 5 { columns: [ partsupp_ps_suppkey, partsupp_ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 6 { columns: [ supplier_s_suppkey, supplier_s_name, supplier_s_address, supplier_s_nationkey ], primary key: [ $3 ASC, $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 3 ], read pk prefix len hint: 1 } @@ -4209,9 +4211,9 @@ Table 11 { columns: [ vnode, n_nationkey, nation_backfill_finished, nation_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } - Table 12 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, lineitem_l_partkey, lineitem_l_suppkey ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 12 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, lineitem_l_partkey, lineitem_l_suppkey ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, lineitem_l_partkey, lineitem_l_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 13 { columns: [ partsupp_ps_partkey, partsupp_ps_suppkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 14 { columns: [ part_p_partkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } @@ -4233,7 +4235,7 @@ Table 23 { columns: [ vnode, p_partkey, part_backfill_finished, part_row_count ], 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, nation.n_nationkey, supplier.s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [], read pk prefix len hint: 1 } + Table 4294967294 { columns: [ s_name, s_address, supplier.s_suppkey, supplier.s_nationkey ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [], read pk prefix len hint: 1 } - id: tpch_q21 before: @@ -4349,7 +4351,7 @@ └─StreamProject { exprs: [supplier.s_name, count, Vnode(supplier.s_name) as $expr1] } └─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_orderkey, lineitem.l_linenumber] } + └─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] } @@ -4396,7 +4398,7 @@ └── StreamExchange Hash([0]) from 2 Fragment 2 - 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_orderkey, lineitem.l_linenumber] } + 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] } ├── left table: 3 ├── right table: 5 ├── left degree table: 4 @@ -4501,9 +4503,9 @@ Table 12 { columns: [ supplier_s_suppkey, nation_n_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 13 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $2 ASC, $0 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + Table 13 { columns: [ lineitem_l_orderkey, lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber ], primary key: [ $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 1 ], read pk prefix len hint: 1 } - Table 14 { columns: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 14 { columns: [ lineitem_l_suppkey, orders_o_orderkey, lineitem_l_linenumber, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 15 { columns: [ nation_n_nationkey ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } 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 e476722be5a18..eaaa1f8e5c8d5 100644 --- a/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml +++ b/src/frontend/planner_test/tests/testdata/output/tpch_variant.yaml @@ -242,7 +242,7 @@ └─BatchExchange { order: [], dist: HashShard(r_regionkey) } └─BatchSource { source: region, columns: [r_regionkey, r_name, r_comment, _row_id], filter: (None, None) } stream_plan: |- - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, p_partkey#1, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, p_partkey#1, ps_supplycost], pk_conflict: NoCheck } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } └─StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } ├─StreamExchange { dist: HashShard(p_partkey) } │ └─StreamShare { id: 26 } @@ -342,7 +342,7 @@ └─StreamSource { source: nation, columns: [n_nationkey, n_name, n_regionkey, n_comment, _row_id] } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, p_partkey#1, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, p_partkey#1, ps_supplycost], pk_conflict: NoCheck } + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), _row_id#1(hidden), r_regionkey(hidden), _row_id#2(hidden), _row_id#3(hidden), _row_id#4(hidden), ps_suppkey(hidden), n_nationkey(hidden), ps_supplycost(hidden), p_partkey#1(hidden)], stream_key: [_row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, p_partkey, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamHashJoin { type: Inner, predicate: p_partkey IS NOT DISTINCT FROM p_partkey AND ps_supplycost = min(ps_supplycost), output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, _row_id, r_regionkey, _row_id, _row_id, _row_id, ps_suppkey, n_nationkey, ps_supplycost, p_partkey] } { left table: 0, right table: 2, left degree table: 1, right degree table: 3 } ├── StreamExchange Hash([0]) from 1 @@ -540,7 +540,7 @@ Table 43 { columns: [ n_regionkey, _row_id, _degree ], 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, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $17 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3 ], 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, _row_id#1, r_regionkey, _row_id#2, _row_id#3, _row_id#4, ps_suppkey, n_nationkey, ps_supplycost, p_partkey#1 ], primary key: [ $0 DESC, $2 ASC, $1 ASC, $3 ASC, $8 ASC, $9 ASC, $10 ASC, $11 ASC, $12 ASC, $13 ASC, $14 ASC, $15 ASC, $16 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17 ], distribution key: [ 3 ], read pk prefix len hint: 13 } - id: tpch_q5 before: @@ -629,7 +629,7 @@ └─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, c_nationkey, n_nationkey] } + └─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] } @@ -670,7 +670,7 @@ └── StreamExchange Hash([0]) from 1 Fragment 1 - 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, c_nationkey, n_nationkey] } + 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] } ├── left table: 1 ├── right table: 3 @@ -1524,7 +1524,7 @@ └─StreamProject { exprs: [n_name, $expr1, RoundDigit(sum($expr2), 2:Int32) 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, s_suppkey, ps_suppkey, ps_partkey] } + └─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] } @@ -1565,7 +1565,7 @@ └── StreamExchange Hash([0, 1]) from 1 Fragment 1 - 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, s_suppkey, ps_suppkey, ps_partkey] } + 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] } ├── left table: 1 ├── right table: 3 @@ -1911,9 +1911,9 @@ Table 1 { columns: [ s_suppkey, _row_id, _row_id_0, s_nationkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 2 { columns: [ ps_suppkey, _row_id, ps_partkey, ps_partkey_0, ps_suppkey_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 2 { columns: [ ps_suppkey, _row_id, ps_partkey, ps_partkey_0, ps_suppkey_0 ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 0, 1, 2, 3, 4 ], distribution key: [ 0 ], read pk prefix len hint: 1 } - Table 3 { columns: [ ps_suppkey, _row_id, ps_partkey, ps_partkey_0, ps_suppkey_0, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + Table 3 { columns: [ ps_suppkey, _row_id, ps_partkey, _degree ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } Table 4 { columns: [ s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id ], primary key: [ $3 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3 ], read pk prefix len hint: 1 } diff --git a/src/frontend/planner_test/tests/testdata/output/union.yaml b/src/frontend/planner_test/tests/testdata/output/union.yaml index bc4c1fd4f1d50..14e7b7e65cb70 100644 --- a/src/frontend/planner_test/tests/testdata/output/union.yaml +++ b/src/frontend/planner_test/tests/testdata/output/union.yaml @@ -10,39 +10,56 @@ └─BatchExchange { order: [], dist: Single } └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), null:Serial(hidden), 0:Int32(hidden)], stream_key: [t1._row_id, null:Serial, 0:Int32], pk_columns: [t1._row_id, null:Serial, 0:Int32], pk_conflict: NoCheck } + StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), $src(hidden)], stream_key: [t1._row_id, $src], pk_columns: [t1._row_id, $src], pk_conflict: NoCheck } └─StreamUnion { all: true } - ├─StreamExchange { dist: HashShard(t1._row_id, null:Serial, 0:Int32) } - │ └─StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, null:Serial, 0:Int32] } + ├─StreamExchange { dist: HashShard(t1._row_id, 0:Int32) } + │ └─StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(null:Serial, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.a, t2.b, t2.c, null:Serial, t2._row_id, 1:Int32] } + └─StreamExchange { dist: HashShard(t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } stream_dist_plan: |+ Fragment 0 - StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), null:Serial(hidden), 0:Int32(hidden)], stream_key: [t1._row_id, null:Serial, 0:Int32], pk_columns: [t1._row_id, null:Serial, 0:Int32], pk_conflict: NoCheck } + StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), $src(hidden)], stream_key: [t1._row_id, $src], pk_columns: [t1._row_id, $src], pk_conflict: NoCheck } ├── materialized table: 4294967294 └── StreamUnion { all: true } - ├── StreamExchange Hash([3, 4, 5]) from 1 - └── StreamExchange Hash([3, 4, 5]) from 2 + ├── StreamExchange Hash([3, 4]) from 1 + └── StreamExchange Hash([3, 4]) from 2 Fragment 1 - StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, null:Serial, 0:Int32] } + StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } ├── Upstream └── BatchPlanNode Fragment 2 - StreamProject { exprs: [t2.a, t2.b, t2.c, null:Serial, t2._row_id, 1:Int32] } + StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 1 } ├── Upstream └── BatchPlanNode - Table 0 { columns: [ vnode, _row_id, t1_backfill_finished, t1_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 0 + ├── columns: [ vnode, _row_id, t1_backfill_finished, t1_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 - Table 1 { columns: [ vnode, _row_id, t2_backfill_finished, t2_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + Table 1 + ├── columns: [ vnode, _row_id, t2_backfill_finished, t2_row_count ] + ├── 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: [ a, b, c, t1._row_id, null:Serial, 0:Int32 ], primary key: [ $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 3, 4, 5 ], read pk prefix len hint: 3 } + Table 4294967294 + ├── columns: [ a, b, c, t1._row_id, $src ] + ├── primary key: [ $3 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 3, 4 ] + └── read pk prefix len hint: 2 - sql: | create table t1 (a int, b numeric, c bigint); @@ -68,11 +85,11 @@ └─StreamHashAgg { group_key: [t1.a, t1.b, t1.c], aggs: [count] } └─StreamExchange { dist: HashShard(t1.a, t1.b, t1.c) } └─StreamUnion { all: true } - ├─StreamExchange { dist: HashShard(t1._row_id, null:Serial, 0:Int32) } - │ └─StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, null:Serial, 0:Int32] } + ├─StreamExchange { dist: HashShard(t1._row_id, 0:Int32) } + │ └─StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(null:Serial, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.a, t2.b, t2.c, null:Serial, t2._row_id, 1:Int32] } + └─StreamExchange { dist: HashShard(t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } stream_dist_plan: |+ Fragment 0 @@ -87,18 +104,18 @@ Fragment 1 StreamUnion { all: true } - ├── StreamExchange Hash([3, 4, 5]) from 2 - └── StreamExchange Hash([3, 4, 5]) from 3 + ├── StreamExchange Hash([3, 4]) from 2 + └── StreamExchange Hash([3, 4]) from 3 Fragment 2 - StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, null:Serial, 0:Int32] } + StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } ├── state table: 1 ├── Upstream └── BatchPlanNode Fragment 3 - StreamProject { exprs: [t2.a, t2.b, t2.c, null:Serial, t2._row_id, 1:Int32] } + StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } ├── state table: 2 ├── Upstream @@ -292,3 +309,347 @@ └─BatchHashAgg { group_key: [1:Int32], aggs: [] } └─BatchExchange { order: [], dist: HashShard(1:Int32) } └─BatchValues { rows: [[1:Int32], [2:Int32], [3:Int32], [4:Int32], [5:Int32], [5:Int32]] } +- name: test merged union stream key (2 columns, row_id + src_col) + sql: | + create table t1 (a int, b numeric, c bigint); + create table t2 (a int, b numeric, c bigint); + create table t3 (a int, b numeric, c bigint); + create table t4 (a int, b numeric, c bigint); + create table t5 (a int, b numeric, c bigint); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + batch_plan: |- + BatchUnion { all: true } + ├─BatchExchange { order: [], dist: Single } + │ └─BatchScan { table: t1, columns: [t1.a, t1.b, t1.c], distribution: SomeShard } + ├─BatchExchange { order: [], dist: Single } + │ └─BatchScan { table: t2, columns: [t2.a, t2.b, t2.c], distribution: SomeShard } + ├─BatchExchange { order: [], dist: Single } + │ └─BatchScan { table: t3, columns: [t3.a, t3.b, t3.c], distribution: SomeShard } + ├─BatchExchange { order: [], dist: Single } + │ └─BatchScan { table: t4, columns: [t4.a, t4.b, t4.c], distribution: SomeShard } + └─BatchExchange { order: [], dist: Single } + └─BatchScan { table: t5, columns: [t5.a, t5.b, t5.c], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), $src(hidden)], stream_key: [t1._row_id, $src], pk_columns: [t1._row_id, $src], pk_conflict: NoCheck } + └─StreamUnion { all: true } + ├─StreamExchange { dist: HashShard(t1._row_id, 0:Int32) } + │ └─StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } + │ └─StreamTableScan { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } + ├─StreamExchange { dist: HashShard(t2._row_id, 1:Int32) } + │ └─StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } + │ └─StreamTableScan { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } + ├─StreamExchange { dist: HashShard(t3._row_id, 2:Int32) } + │ └─StreamProject { exprs: [t3.a, t3.b, t3.c, t3._row_id, 2:Int32] } + │ └─StreamTableScan { table: t3, columns: [t3.a, t3.b, t3.c, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } + ├─StreamExchange { dist: HashShard(t4._row_id, 3:Int32) } + │ └─StreamProject { exprs: [t4.a, t4.b, t4.c, t4._row_id, 3:Int32] } + │ └─StreamTableScan { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } + └─StreamExchange { dist: HashShard(t5._row_id, 4:Int32) } + └─StreamProject { exprs: [t5.a, t5.b, t5.c, t5._row_id, 4:Int32] } + └─StreamTableScan { table: t5, columns: [t5.a, t5.b, t5.c, t5._row_id], pk: [t5._row_id], dist: UpstreamHashShard(t5._row_id) } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [a, b, c, t1._row_id(hidden), $src(hidden)], stream_key: [t1._row_id, $src], pk_columns: [t1._row_id, $src], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamUnion { all: true } + ├── StreamExchange Hash([3, 4]) from 1 + ├── StreamExchange Hash([3, 4]) from 2 + ├── StreamExchange Hash([3, 4]) from 3 + ├── StreamExchange Hash([3, 4]) from 4 + └── StreamExchange Hash([3, 4]) from 5 + + Fragment 1 + StreamProject { exprs: [t1.a, t1.b, t1.c, t1._row_id, 0:Int32] } + └── Chain { table: t1, columns: [t1.a, t1.b, t1.c, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } { state table: 0 } + ├── Upstream + └── BatchPlanNode + + Fragment 2 + StreamProject { exprs: [t2.a, t2.b, t2.c, t2._row_id, 1:Int32] } + └── Chain { table: t2, columns: [t2.a, t2.b, t2.c, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } { state table: 1 } + ├── Upstream + └── BatchPlanNode + + Fragment 3 + StreamProject { exprs: [t3.a, t3.b, t3.c, t3._row_id, 2:Int32] } + └── Chain { table: t3, columns: [t3.a, t3.b, t3.c, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } { state table: 2 } + ├── Upstream + └── BatchPlanNode + + Fragment 4 + StreamProject { exprs: [t4.a, t4.b, t4.c, t4._row_id, 3:Int32] } + └── Chain { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } + ├── Upstream + └── BatchPlanNode + + Fragment 5 + StreamProject { exprs: [t5.a, t5.b, t5.c, t5._row_id, 4:Int32] } + └── Chain { table: t5, columns: [t5.a, t5.b, t5.c, t5._row_id], pk: [t5._row_id], dist: UpstreamHashShard(t5._row_id) } { state table: 4 } + ├── Upstream + └── BatchPlanNode + + Table 0 + ├── columns: [ vnode, _row_id, t1_backfill_finished, t1_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 1 + ├── columns: [ vnode, _row_id, t2_backfill_finished, t2_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 2 + ├── columns: [ vnode, _row_id, t3_backfill_finished, t3_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 3 + ├── columns: [ vnode, _row_id, t4_backfill_finished, t4_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 4 + ├── columns: [ vnode, _row_id, t5_backfill_finished, t5_row_count ] + ├── 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: [ a, b, c, t1._row_id, $src ] + ├── primary key: [ $3 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 3, 4 ] + └── read pk prefix len hint: 2 + +- name: test merged union stream key (5 columns, row_id + src_col + a + b + c) + sql: | + create table t1 (a int, b numeric, c bigint, primary key (a)); + create table t2 (a int, b numeric, c bigint, primary key (b)); + create table t3 (a int, b numeric, c bigint, primary key (c)); + create table t4 (a int, b numeric, c bigint); + create table t5 (a int, b numeric, c bigint, primary key (a, b)); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [a, b, c, t1.a(hidden), null:Int64(hidden), null:Decimal(hidden), null:Serial(hidden), $src(hidden)], stream_key: [t1.a, null:Decimal, null:Int64, null:Serial, $src], pk_columns: [t1.a, null:Decimal, null:Int64, null:Serial, $src], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamUnion { all: true } + ├── StreamExchange Hash([3, 5, 4, 6, 7]) from 1 + ├── StreamExchange Hash([3, 5, 4, 6, 7]) from 2 + ├── StreamExchange Hash([3, 5, 4, 6, 7]) from 3 + ├── StreamExchange Hash([3, 5, 4, 6, 7]) from 4 + └── StreamExchange Hash([3, 5, 4, 6, 7]) from 5 + + Fragment 1 + StreamProject { exprs: [t1.a, t1.b, t1.c, t1.a, null:Int64, null:Decimal, null:Serial, 0:Int32] } + └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } + ├── Upstream + └── BatchPlanNode + + Fragment 2 + StreamProject { exprs: [t2.a, t2.b, t2.c, null:Int32, null:Int64, t2.b, null:Serial, 1:Int32] } + └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } + ├── Upstream + └── BatchPlanNode + + Fragment 3 + StreamProject { exprs: [t3.a, t3.b, t3.c, null:Int32, t3.c, null:Decimal, null:Serial, 2:Int32] } + └── Chain { table: t3, columns: [t3.a, t3.b, t3.c], pk: [t3.c], dist: UpstreamHashShard(t3.c) } { state table: 2 } + ├── Upstream + └── BatchPlanNode + + Fragment 4 + StreamProject { exprs: [t4.a, t4.b, t4.c, null:Int32, null:Int64, null:Decimal, t4._row_id, 3:Int32] } + └── Chain { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } + ├── Upstream + └── BatchPlanNode + + Fragment 5 + StreamProject { exprs: [t5.a, t5.b, t5.c, t5.a, null:Int64, t5.b, null:Serial, 4:Int32] } + └── Chain { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } + ├── Upstream + └── BatchPlanNode + + Table 0 { columns: [ vnode, a, t1_backfill_finished, t1_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 1 { columns: [ vnode, b, t2_backfill_finished, t2_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 2 { columns: [ vnode, c, t3_backfill_finished, t3_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 3 { columns: [ vnode, _row_id, t4_backfill_finished, t4_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 4 { columns: [ vnode, a, b, t5_backfill_finished, t5_row_count ], 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: [ a, b, c, t1.a, null:Int64, null:Decimal, null:Serial, $src ], primary key: [ $3 ASC, $5 ASC, $4 ASC, $6 ASC, $7 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ], distribution key: [ 3, 5, 4, 6, 7 ], read pk prefix len hint: 5 } + +- name: test merged union stream key (4 columns, row_id + src_col + a + b) + sql: | + create table t1 (a int, b numeric, c bigint, primary key (a)); + create table t2 (a int, b numeric, c bigint, primary key (b)); + create table t3 (a int, b numeric, c bigint); + create table t4 (a int, b numeric, c bigint); + create table t5 (a int, b numeric, c bigint, primary key (a, b)); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [a, b, c, t1.a(hidden), null:Decimal(hidden), null:Serial(hidden), $src(hidden)], stream_key: [t1.a, null:Decimal, null:Serial, $src], pk_columns: [t1.a, null:Decimal, null:Serial, $src], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamUnion { all: true } + ├── StreamExchange Hash([3, 4, 5, 6]) from 1 + ├── StreamExchange Hash([3, 4, 5, 6]) from 2 + ├── StreamExchange Hash([3, 4, 5, 6]) from 3 + ├── StreamExchange Hash([3, 4, 5, 6]) from 4 + └── StreamExchange Hash([3, 4, 5, 6]) from 5 + + Fragment 1 + StreamProject { exprs: [t1.a, t1.b, t1.c, t1.a, null:Decimal, null:Serial, 0:Int32] } + └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } + ├── Upstream + └── BatchPlanNode + + Fragment 2 + StreamProject { exprs: [t2.a, t2.b, t2.c, null:Int32, t2.b, null:Serial, 1:Int32] } + └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } + ├── Upstream + └── BatchPlanNode + + Fragment 3 + StreamProject { exprs: [t3.a, t3.b, t3.c, null:Int32, null:Decimal, t3._row_id, 2:Int32] } + └── Chain { table: t3, columns: [t3.a, t3.b, t3.c, t3._row_id], pk: [t3._row_id], dist: UpstreamHashShard(t3._row_id) } { state table: 2 } + ├── Upstream + └── BatchPlanNode + + Fragment 4 + StreamProject { exprs: [t4.a, t4.b, t4.c, null:Int32, null:Decimal, t4._row_id, 3:Int32] } + └── Chain { table: t4, columns: [t4.a, t4.b, t4.c, t4._row_id], pk: [t4._row_id], dist: UpstreamHashShard(t4._row_id) } { state table: 3 } + ├── Upstream + └── BatchPlanNode + + Fragment 5 + StreamProject { exprs: [t5.a, t5.b, t5.c, t5.a, t5.b, null:Serial, 4:Int32] } + └── Chain { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } + ├── Upstream + └── BatchPlanNode + + Table 0 { columns: [ vnode, a, t1_backfill_finished, t1_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 1 { columns: [ vnode, b, t2_backfill_finished, t2_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 2 { columns: [ vnode, _row_id, t3_backfill_finished, t3_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 3 { columns: [ vnode, _row_id, t4_backfill_finished, t4_row_count ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 4 { columns: [ vnode, a, b, t5_backfill_finished, t5_row_count ], 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: [ a, b, c, t1.a, null:Decimal, null:Serial, $src ], primary key: [ $3 ASC, $4 ASC, $5 ASC, $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 3, 4, 5, 6 ], read pk prefix len hint: 4 } + +- name: test merged union stream key (3 columns, src_col + a + b) + sql: | + create table t1 (a int, b numeric, c bigint, primary key (a)); + create table t2 (a int, b numeric, c bigint, primary key (b)); + create table t3 (a int, b numeric, c bigint, primary key (b)); + create table t4 (a int, b numeric, c bigint, primary key (b, a)); + create table t5 (a int, b numeric, c bigint, primary key (a, b)); + select * from t1 union all select * from t2 union all select * from t3 union all select * from t4 union all select * from t5; + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [a, b, c, $src(hidden)], stream_key: [a, b, $src], pk_columns: [a, b, $src], pk_conflict: NoCheck } + ├── materialized table: 4294967294 + └── StreamUnion { all: true } + ├── StreamExchange Hash([0, 1, 3]) from 1 + ├── StreamExchange Hash([0, 1, 3]) from 2 + ├── StreamExchange Hash([0, 1, 3]) from 3 + ├── StreamExchange Hash([0, 1, 3]) from 4 + └── StreamExchange Hash([0, 1, 3]) from 5 + + Fragment 1 + StreamProject { exprs: [t1.a, t1.b, t1.c, 0:Int32] } + └── Chain { table: t1, columns: [t1.a, t1.b, t1.c], pk: [t1.a], dist: UpstreamHashShard(t1.a) } { state table: 0 } + ├── Upstream + └── BatchPlanNode + + Fragment 2 + StreamProject { exprs: [t2.a, t2.b, t2.c, 1:Int32] } + └── Chain { table: t2, columns: [t2.a, t2.b, t2.c], pk: [t2.b], dist: UpstreamHashShard(t2.b) } { state table: 1 } + ├── Upstream + └── BatchPlanNode + + Fragment 3 + StreamProject { exprs: [t3.a, t3.b, t3.c, 2:Int32] } + └── Chain { table: t3, columns: [t3.a, t3.b, t3.c], pk: [t3.b], dist: UpstreamHashShard(t3.b) } { state table: 2 } + ├── Upstream + └── BatchPlanNode + + Fragment 4 + StreamProject { exprs: [t4.a, t4.b, t4.c, 3:Int32] } + └── Chain { table: t4, columns: [t4.a, t4.b, t4.c], pk: [t4.b, t4.a], dist: UpstreamHashShard(t4.a, t4.b) } { state table: 3 } + ├── Upstream + └── BatchPlanNode + + Fragment 5 + StreamProject { exprs: [t5.a, t5.b, t5.c, 4:Int32] } + └── Chain { table: t5, columns: [t5.a, t5.b, t5.c], pk: [t5.a, t5.b], dist: UpstreamHashShard(t5.a, t5.b) } { state table: 4 } + ├── Upstream + └── BatchPlanNode + + Table 0 + ├── columns: [ vnode, a, t1_backfill_finished, t1_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 1 + ├── columns: [ vnode, b, t2_backfill_finished, t2_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 2 + ├── columns: [ vnode, b, t3_backfill_finished, t3_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 3 + ├── columns: [ vnode, b, a, t4_backfill_finished, t4_row_count ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 4 + ├── columns: [ vnode, a, b, t5_backfill_finished, t5_row_count ] + ├── 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: [ a, b, c, $src ] + ├── primary key: [ $0 ASC, $1 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 0, 1, 3 ] + └── read pk prefix len hint: 3 + diff --git a/src/frontend/planner_test/tests/testdata/output/watermark.yaml b/src/frontend/planner_test/tests/testdata/output/watermark.yaml index d1916a33192c6..e4ef42b121528 100644 --- a/src/frontend/planner_test/tests/testdata/output/watermark.yaml +++ b/src/frontend/planner_test/tests/testdata/output/watermark.yaml @@ -140,13 +140,13 @@ create table t2 (ts timestamp with time zone, v1 int, v2 int, watermark for ts as ts - INTERVAL '1' SECOND) append only; select * from t1 Union all select * from t2; stream_plan: |- - StreamMaterialize { columns: [ts, v1, v2, t1._row_id(hidden), null:Serial(hidden), 0:Int32(hidden)], stream_key: [t1._row_id, null:Serial, 0:Int32], pk_columns: [t1._row_id, null:Serial, 0:Int32], pk_conflict: NoCheck, watermark_columns: [ts] } + StreamMaterialize { columns: [ts, v1, v2, t1._row_id(hidden), $src(hidden)], stream_key: [t1._row_id, $src], pk_columns: [t1._row_id, $src], pk_conflict: NoCheck, watermark_columns: [ts] } └─StreamUnion { all: true, output_watermarks: [t1.ts] } - ├─StreamExchange { dist: HashShard(t1._row_id, null:Serial, 0:Int32) } - │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Serial, 0:Int32], output_watermarks: [t1.ts] } + ├─StreamExchange { dist: HashShard(t1._row_id, 0:Int32) } + │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, 0:Int32], output_watermarks: [t1.ts] } │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(null:Serial, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Serial, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } + └─StreamExchange { dist: HashShard(t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: union sql: | @@ -159,11 +159,11 @@ └─StreamExchange { dist: HashShard(t1.ts, t1.v1, t1.v2) } └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2], output_watermarks: [t1.ts] } └─StreamUnion { all: true, output_watermarks: [t1.ts] } - ├─StreamExchange { dist: HashShard(t1._row_id, null:Serial, 0:Int32) } - │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, null:Serial, 0:Int32], output_watermarks: [t1.ts] } + ├─StreamExchange { dist: HashShard(t1._row_id, 0:Int32) } + │ └─StreamProject { exprs: [t1.ts, t1.v1, t1.v2, t1._row_id, 0:Int32], output_watermarks: [t1.ts] } │ └─StreamTableScan { table: t1, columns: [t1.ts, t1.v1, t1.v2, t1._row_id], pk: [t1._row_id], dist: UpstreamHashShard(t1._row_id) } - └─StreamExchange { dist: HashShard(null:Serial, t2._row_id, 1:Int32) } - └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, null:Serial, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } + └─StreamExchange { dist: HashShard(t2._row_id, 1:Int32) } + └─StreamProject { exprs: [t2.ts, t2.v1, t2.v2, t2._row_id, 1:Int32], output_watermarks: [t2.ts] } └─StreamTableScan { table: t2, columns: [t2.ts, t2.v1, t2.v2, t2._row_id], pk: [t2._row_id], dist: UpstreamHashShard(t2._row_id) } - name: tumble sql: | diff --git a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml index 208eec58ecdba..867855fe7b44d 100644 --- a/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml +++ b/src/frontend/planner_test/tests/testdata/output/with_ordinality.yaml @@ -24,8 +24,8 @@ └─BatchExchange { order: [], dist: HashShard(t.arr) } └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, arr, unnest, ordinality, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, t.arr, projected_row_id, arr], pk_columns: [t._row_id, t.arr, projected_row_id, arr], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, t.arr, projected_row_id] } + StreamMaterialize { columns: [x, arr, unnest, ordinality, t._row_id(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -48,8 +48,8 @@ └─BatchExchange { order: [], dist: HashShard(t.arr) } └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, arr, foo, ordinality, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, t.arr, projected_row_id, arr], pk_columns: [t._row_id, t.arr, projected_row_id, arr], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, t.arr, projected_row_id] } + StreamMaterialize { columns: [x, arr, foo, ordinality, t._row_id(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -72,8 +72,8 @@ └─BatchExchange { order: [], dist: HashShard(t.arr) } └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, arr, a, ordinality, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, t.arr, projected_row_id, arr], pk_columns: [t._row_id, t.arr, projected_row_id, arr], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, t.arr, projected_row_id] } + StreamMaterialize { columns: [x, arr, a, ordinality, t._row_id(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -96,8 +96,8 @@ └─BatchExchange { order: [], dist: HashShard(t.arr) } └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, arr, a, ord, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, t.arr, projected_row_id, arr], pk_columns: [t._row_id, t.arr, projected_row_id, arr], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, t.arr, projected_row_id] } + StreamMaterialize { columns: [x, arr, a, ord, t._row_id(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -121,8 +121,8 @@ └─BatchExchange { order: [], dist: HashShard(t.arr) } └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, arr, a, ord, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden)], stream_key: [t._row_id, t.arr, projected_row_id, arr], pk_columns: [t._row_id, t.arr, projected_row_id, arr], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, t.arr, projected_row_id] } + StreamMaterialize { columns: [x, arr, a, ord, t._row_id(hidden), projected_row_id(hidden)], stream_key: [t._row_id, projected_row_id, arr], pk_columns: [t._row_id, projected_row_id, arr], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -159,11 +159,11 @@ └─BatchExchange { order: [], dist: HashShard(t.arr) } └─BatchScan { table: t, columns: [t.arr], distribution: SomeShard } stream_plan: |- - StreamMaterialize { columns: [x, arr, unnest, ordinality, arr_2, ordinality_2, t._row_id(hidden), t.arr(hidden), projected_row_id(hidden), t.arr#1(hidden), projected_row_id#1(hidden)], stream_key: [t._row_id, t.arr, projected_row_id, arr, t.arr#1, projected_row_id#1], pk_columns: [t._row_id, t.arr, projected_row_id, arr, t.arr#1, projected_row_id#1], pk_conflict: NoCheck } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), $expr1, Unnest($0), (projected_row_id + 1:Int64) as $expr2, t._row_id, t.arr, projected_row_id, t.arr, projected_row_id] } - └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), $expr1, projected_row_id, t.arr, Unnest($0), t._row_id, t.arr, projected_row_id] } + StreamMaterialize { columns: [x, arr, unnest, ordinality, arr_2, ordinality_2, t._row_id(hidden), projected_row_id(hidden), projected_row_id#1(hidden)], stream_key: [t._row_id, projected_row_id, arr, projected_row_id#1], pk_columns: [t._row_id, projected_row_id, arr, projected_row_id#1], pk_conflict: NoCheck } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), $expr1, Unnest($0), (projected_row_id + 1:Int64) as $expr2, t._row_id, projected_row_id, projected_row_id] } + └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, Unnest($0), $expr1, projected_row_id, t.arr, Unnest($0), t._row_id, projected_row_id] } ├─StreamShare { id: 8 } - │ └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, t.arr, projected_row_id] } + │ └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } │ └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } │ ├─StreamExchange { dist: HashShard(t.arr) } │ │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } @@ -176,7 +176,7 @@ └─StreamProject { exprs: [t.arr] } └─StreamHashAgg { group_key: [t.arr], aggs: [count] } └─StreamShare { id: 8 } - └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, t.arr, projected_row_id] } + └─StreamProject { exprs: [t.x, t.arr, Unnest($0), (projected_row_id + 1:Int64) as $expr1, t._row_id, projected_row_id] } └─StreamHashJoin { type: Inner, predicate: t.arr IS NOT DISTINCT FROM t.arr, output: [t.x, t.arr, projected_row_id, t.arr, Unnest($0), t._row_id] } ├─StreamExchange { dist: HashShard(t.arr) } │ └─StreamTableScan { table: t, columns: [t.x, t.arr, t._row_id], pk: [t._row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 7479348c4b80f..d585caf70a920 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -34,12 +34,12 @@ use risingwave_connector::source::cdc::{ CITUS_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, }; use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; -use risingwave_connector::source::filesystem::S3_CONNECTOR; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; use risingwave_connector::source::test_source::TEST_CONNECTOR; use risingwave_connector::source::{ SourceEncode, SourceFormat, SourceStruct, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, - KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, PULSAR_CONNECTOR, + KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, + S3_V2_CONNECTOR, }; use risingwave_pb::catalog::{ PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc, @@ -893,6 +893,9 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock hashmap!( Format::Plain => vec![Encode::Csv, Encode::Json], ), + S3_V2_CONNECTOR => hashmap!( + Format::Plain => vec![Encode::Csv, Encode::Json], + ), MYSQL_CDC_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Bytes], Format::Debezium => vec![Encode::Json], @@ -925,8 +928,9 @@ pub fn validate_compatibility( .get(&connector) .ok_or_else(|| { RwError::from(ProtocolError(format!( - "connector {} is not supported", - connector + "connector {:?} is not supported, accept {:?}", + connector, + CONNECTORS_COMPATIBLE_FORMATS.keys() ))) })?; if connector != KAFKA_CONNECTOR { diff --git a/src/frontend/src/optimizer/plan_node/generic/agg.rs b/src/frontend/src/optimizer/plan_node/generic/agg.rs index 107eec5e51b01..2fb251ca89aa6 100644 --- a/src/frontend/src/optimizer/plan_node/generic/agg.rs +++ b/src/frontend/src/optimizer/plan_node/generic/agg.rs @@ -412,7 +412,9 @@ impl Agg { | AggKind::FirstValue | AggKind::LastValue | AggKind::StringAgg - | AggKind::ArrayAgg => { + | AggKind::ArrayAgg + | AggKind::JsonbAgg + | AggKind::JsonbObjectAgg => { // columns with order requirement in state table let sort_keys = { match agg_call.agg_kind { @@ -425,7 +427,8 @@ impl Agg { AggKind::FirstValue | AggKind::LastValue | AggKind::StringAgg - | AggKind::ArrayAgg => { + | AggKind::ArrayAgg + | AggKind::JsonbAgg => { if agg_call.order_by.is_empty() { me.ctx().warn_to_user(format!( "{} without ORDER BY may produce non-deterministic result", @@ -447,6 +450,11 @@ impl Agg { }) .collect() } + AggKind::JsonbObjectAgg => agg_call + .order_by + .iter() + .map(|o| (o.order_type, o.column_index)) + .collect(), _ => unreachable!(), } }; @@ -455,7 +463,11 @@ impl Agg { AggKind::FirstValue | AggKind::LastValue | AggKind::StringAgg - | AggKind::ArrayAgg => agg_call.inputs.iter().map(|i| i.index).collect(), + | AggKind::ArrayAgg + | AggKind::JsonbAgg + | AggKind::JsonbObjectAgg => { + agg_call.inputs.iter().map(|i| i.index).collect() + } _ => vec![], }; let state = gen_materialized_input_state(sort_keys, include_keys); diff --git a/src/frontend/src/optimizer/plan_node/generic/join.rs b/src/frontend/src/optimizer/plan_node/generic/join.rs index 9429e29b968ea..95bee8413f56f 100644 --- a/src/frontend/src/optimizer/plan_node/generic/join.rs +++ b/src/frontend/src/optimizer/plan_node/generic/join.rs @@ -91,8 +91,10 @@ impl GenericPlanNode for Join { } fn stream_key(&self) -> Option> { - let _left_len = self.left.schema().len(); - let _right_len = self.right.schema().len(); + let left_len = self.left.schema().len(); + let right_len = self.right.schema().len(); + let eq_predicate = EqJoinPredicate::create(left_len, right_len, self.on.clone()); + let left_pk = self.left.stream_key()?; let right_pk = self.right.stream_key()?; let l2i = self.l2i_col_mapping(); @@ -100,65 +102,77 @@ impl GenericPlanNode for Join { let full_out_col_num = self.internal_column_num(); let i2o = ColIndexMapping::with_remaining_columns(&self.output_indices, full_out_col_num); - let pk_indices = left_pk + let mut pk_indices = left_pk .iter() .map(|index| l2i.try_map(*index)) .chain(right_pk.iter().map(|index| r2i.try_map(*index))) .flatten() .map(|index| i2o.try_map(index)) - .collect::>>(); + .collect::>>()?; // NOTE(st1page): add join keys in the pk_indices a work around before we really have stream // key. - pk_indices.and_then(|mut pk_indices: Vec| { - let left_len = self.left.schema().len(); - let right_len = self.right.schema().len(); - let eq_predicate = EqJoinPredicate::create(left_len, right_len, self.on.clone()); - - let l2i = self.l2i_col_mapping(); - let r2i = self.r2i_col_mapping(); - let full_out_col_num = self.internal_column_num(); - let i2o = - ColIndexMapping::with_remaining_columns(&self.output_indices, full_out_col_num); - - let either_or_both = self.add_which_join_key_to_pk(); - - for (lk, rk) in eq_predicate.eq_indexes() { - match either_or_both { - EitherOrBoth::Left(_) => { - if let Some(lk) = l2i.try_map(lk) { - let out_k = i2o.try_map(lk)?; - if !pk_indices.contains(&out_k) { - pk_indices.push(out_k); - } + let l2i = self.l2i_col_mapping(); + let r2i = self.r2i_col_mapping(); + let full_out_col_num = self.internal_column_num(); + let i2o = ColIndexMapping::with_remaining_columns(&self.output_indices, full_out_col_num); + + let either_or_both = self.add_which_join_key_to_pk(); + + for (lk, rk) in eq_predicate.eq_indexes() { + match either_or_both { + EitherOrBoth::Left(_) => { + // Remove right-side join-key column it from pk_indices. + // This may happen when right-side join-key is included in right-side PK. + // e.g. select a, b where a.bid = b.id + // Here the pk_indices should be [a.id, a.bid] instead of [a.id, b.id, a.bid], + // because b.id = a.bid, so either of them would be enough. + if let Some(rk) = r2i.try_map(rk) { + if let Some(out_k) = i2o.try_map(rk) { + pk_indices.retain(|&x| x != out_k); + } + } + // Add left-side join-key column in pk_indices + if let Some(lk) = l2i.try_map(lk) { + let out_k = i2o.try_map(lk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); } } - EitherOrBoth::Right(_) => { - if let Some(rk) = r2i.try_map(rk) { - let out_k = i2o.try_map(rk)?; - if !pk_indices.contains(&out_k) { - pk_indices.push(out_k); - } + } + EitherOrBoth::Right(_) => { + // Remove left-side join-key column it from pk_indices + // See the example above + if let Some(lk) = l2i.try_map(lk) { + if let Some(out_k) = i2o.try_map(lk) { + pk_indices.retain(|&x| x != out_k); } } - EitherOrBoth::Both(_, _) => { - if let Some(lk) = l2i.try_map(lk) { - let out_k = i2o.try_map(lk)?; - if !pk_indices.contains(&out_k) { - pk_indices.push(out_k); - } + // Add right-side join-key column in pk_indices + if let Some(rk) = r2i.try_map(rk) { + let out_k = i2o.try_map(rk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); } - if let Some(rk) = r2i.try_map(rk) { - let out_k = i2o.try_map(rk)?; - if !pk_indices.contains(&out_k) { - pk_indices.push(out_k); - } + } + } + EitherOrBoth::Both(_, _) => { + if let Some(lk) = l2i.try_map(lk) { + let out_k = i2o.try_map(lk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); } } - }; - } - Some(pk_indices) - }) + if let Some(rk) = r2i.try_map(rk) { + let out_k = i2o.try_map(rk)?; + if !pk_indices.contains(&out_k) { + pk_indices.push(out_k); + } + } + } + }; + } + Some(pk_indices) } fn ctx(&self) -> OptimizerContextRef { diff --git a/src/frontend/src/optimizer/plan_node/generic/union.rs b/src/frontend/src/optimizer/plan_node/generic/union.rs index bc736eed4e153..3e6a5b9b9bab6 100644 --- a/src/frontend/src/optimizer/plan_node/generic/union.rs +++ b/src/frontend/src/optimizer/plan_node/generic/union.rs @@ -33,7 +33,13 @@ pub struct Union { impl GenericPlanNode for Union { fn schema(&self) -> Schema { - self.inputs[0].schema().clone() + let mut schema = self.inputs[0].schema().clone(); + if let Some(source_col) = self.source_col { + schema.fields[source_col].name = "$src".to_string(); + schema + } else { + schema + } } fn stream_key(&self) -> Option> { diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index d2dbcf4787eb0..caf04021755f8 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -17,11 +17,14 @@ use std::ops::Bound; use std::ops::Bound::{Excluded, Included, Unbounded}; use std::rc::Rc; +use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{ColumnCatalog, Schema, KAFKA_TIMESTAMP_COLUMN_NAME}; -use risingwave_common::error::Result; -use risingwave_connector::source::DataType; +use risingwave_common::catalog::{ + ColumnCatalog, ColumnDesc, Field, Schema, KAFKA_TIMESTAMP_COLUMN_NAME, +}; +use risingwave_common::error::{ErrorCode, Result, RwError, TrackingIssue}; +use risingwave_connector::source::{ConnectorProperties, DataType}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::GeneratedColumnDesc; @@ -35,10 +38,13 @@ use super::{ use crate::catalog::source_catalog::SourceCatalog; use crate::expr::{Expr, ExprImpl, ExprRewriter, ExprType, InputRef}; use crate::optimizer::optimizer_context::OptimizerContextRef; +use crate::optimizer::plan_node::stream_fs_fetch::StreamFsFetch; use crate::optimizer::plan_node::utils::column_names_pretty; use crate::optimizer::plan_node::{ - ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, ToStreamContext, + ColumnPruningContext, PredicatePushdownContext, RewriteStreamContext, StreamDedup, + ToStreamContext, }; +use crate::optimizer::property::{Distribution, Order, RequiredDist}; use crate::utils::{ColIndexMapping, Condition, IndexRewriter}; /// `LogicalSource` returns contents of a table or other equivalent object @@ -152,6 +158,73 @@ impl LogicalSource { Ok(Some(exprs)) } + fn rewrite_new_s3_plan(&self) -> Result { + let logical_source = generic::Source { + catalog: self.core.catalog.clone(), + column_catalog: vec![ + ColumnCatalog { + column_desc: ColumnDesc::from_field_with_column_id( + &Field { + name: "filename".to_string(), + data_type: DataType::Varchar, + sub_fields: vec![], + type_name: "".to_string(), + }, + 0, + ), + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc::from_field_with_column_id( + &Field { + name: "last_edit_time".to_string(), + data_type: DataType::Timestamp, + sub_fields: vec![], + type_name: "".to_string(), + }, + 1, + ), + is_hidden: false, + }, + ColumnCatalog { + column_desc: ColumnDesc::from_field_with_column_id( + &Field { + name: "file_size".to_string(), + data_type: DataType::Int64, + sub_fields: vec![], + type_name: "".to_string(), + }, + 0, + ), + is_hidden: false, + }, + ], + row_id_index: None, + gen_row_id: false, + ..self.core.clone() + }; + let mut new_s3_plan: PlanRef = StreamSource { + base: PlanBase::new_stream_with_logical( + &logical_source, + Distribution::Single, + true, // `list` will keep listing all objects, it must be append-only + false, + FixedBitSet::with_capacity(logical_source.column_catalog.len()), + ), + logical: logical_source, + } + .into(); + new_s3_plan = RequiredDist::shard_by_key(3, &[0]) + .enforce_if_not_satisfies(new_s3_plan, &Order::any())?; + new_s3_plan = StreamDedup::new(generic::Dedup { + input: new_s3_plan, + dedup_cols: vec![0], + }) + .into(); + + Ok(new_s3_plan) + } + /// `row_id_index` in source node should rule out generated column #[must_use] fn rewrite_row_id_idx(columns: &[ColumnCatalog], row_id_index: Option) -> Option { @@ -200,14 +273,18 @@ impl LogicalSource { } } - fn wrap_with_optional_generated_columns_stream_proj(&self) -> Result { + fn wrap_with_optional_generated_columns_stream_proj( + &self, + input: Option, + ) -> Result { if let Some(exprs) = &self.output_exprs { - let source = StreamSource::new(self.rewrite_to_stream_batch_source()); - let logical_project = generic::Project::new(exprs.to_vec(), source.into()); + let source: PlanRef = + dispatch_new_s3_plan(self.rewrite_to_stream_batch_source(), input); + let logical_project = generic::Project::new(exprs.to_vec(), source); Ok(StreamProject::new(logical_project).into()) } else { - let source = StreamSource::new(self.core.clone()); - Ok(source.into()) + let source = dispatch_new_s3_plan(self.core.clone(), input); + Ok(source) } } @@ -452,6 +529,16 @@ impl PredicatePushdown for LogicalSource { impl ToBatch for LogicalSource { fn to_batch(&self) -> Result { + if self.core.catalog.is_some() + && ConnectorProperties::is_new_fs_connector_b_tree_map( + &self.core.catalog.as_ref().unwrap().properties, + ) + { + return Err(RwError::from(ErrorCode::NotImplemented( + "New S3 connector for batch".to_string(), + TrackingIssue::from(None), + ))); + } let source = self.wrap_with_optional_generated_columns_batch_proj()?; Ok(source) } @@ -459,11 +546,20 @@ impl ToBatch for LogicalSource { impl ToStream for LogicalSource { fn to_stream(&self, _ctx: &mut ToStreamContext) -> Result { - let mut plan = if self.core.for_table { - StreamSource::new(self.rewrite_to_stream_batch_source()).into() + let mut plan_prefix: Option = None; + let mut plan: PlanRef; + if self.core.catalog.is_some() + && ConnectorProperties::is_new_fs_connector_b_tree_map( + &self.core.catalog.as_ref().unwrap().properties, + ) + { + plan_prefix = Some(self.rewrite_new_s3_plan()?); + } + plan = if self.core.for_table { + dispatch_new_s3_plan(self.rewrite_to_stream_batch_source(), plan_prefix) } else { // Create MV on source. - self.wrap_with_optional_generated_columns_stream_proj()? + self.wrap_with_optional_generated_columns_stream_proj(plan_prefix)? }; if let Some(catalog) = self.source_catalog() @@ -490,3 +586,12 @@ impl ToStream for LogicalSource { )) } } + +#[inline] +fn dispatch_new_s3_plan(source: generic::Source, input: Option) -> PlanRef { + if let Some(input) = input { + StreamFsFetch::new(input, source).into() + } else { + StreamSource::new(source).into() + } +} diff --git a/src/frontend/src/optimizer/plan_node/logical_union.rs b/src/frontend/src/optimizer/plan_node/logical_union.rs index 10371fda3c2b0..51e4e620cf4ca 100644 --- a/src/frontend/src/optimizer/plan_node/logical_union.rs +++ b/src/frontend/src/optimizer/plan_node/logical_union.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::max; +use std::collections::BTreeMap; + use itertools::Itertools; use risingwave_common::catalog::Schema; use risingwave_common::error::Result; @@ -174,7 +177,7 @@ impl ToStream for LogicalUnion { rewrites.push(input.logical_rewrite_for_stream(ctx)?); } - let original_schema_contain_all_input_pks = + let original_schema_contain_all_input_stream_keys = rewrites.iter().all(|(new_input, col_index_mapping)| { let original_schema_new_pos = (0..original_schema_len) .map(|x| col_index_mapping.map(x)) @@ -185,7 +188,7 @@ impl ToStream for LogicalUnion { .all(|x| original_schema_new_pos.contains(x)) }); - if original_schema_contain_all_input_pks { + if original_schema_contain_all_input_stream_keys { // Add one more column at the end of the original schema to identify the record came // from which input. [original_schema + source_col] let new_inputs = rewrites @@ -223,29 +226,45 @@ impl ToStream for LogicalUnion { Ok((new_union.into(), out_col_change)) } else { // In order to ensure all inputs have the same schema for new union, we construct new - // schema like that: [original_schema + input1_pk + input2_pk + ... + - // source_col] - let input_pk_types = rewrites - .iter() - .flat_map(|(new_input, _)| { - new_input - .expect_stream_key() - .iter() - .map(|x| new_input.schema().fields[*x].data_type()) - }) - .collect_vec(); - let input_pk_nulls = input_pk_types + // schema like that: [original_schema + merged_stream_key + source_col] + // where merged_stream_key is merged by the types of each input stream key. + // If all inputs have the same stream key column types, we have a small merged_stream_key. Otherwise, we will have a large merged_stream_key. + + let (merged_stream_key_types, types_offset) = { + let mut max_types_counter = BTreeMap::default(); + for (new_input, _) in &rewrites { + let mut types_counter = BTreeMap::default(); + for x in new_input.expect_stream_key() { + types_counter + .entry(new_input.schema().fields[*x].data_type()) + .and_modify(|x| *x += 1) + .or_insert(1); + } + for (key, val) in types_counter { + max_types_counter + .entry(key) + .and_modify(|x| *x = max(*x, val)) + .or_insert(val); + } + } + + let mut merged_stream_key_types = vec![]; + let mut types_offset = BTreeMap::default(); + let mut offset = 0; + for (key, val) in max_types_counter { + let _ = types_offset.insert(key.clone(), offset); + offset += val; + merged_stream_key_types.extend(std::iter::repeat(key.clone()).take(val)); + } + + (merged_stream_key_types, types_offset) + }; + + let input_stream_key_nulls = merged_stream_key_types .iter() .map(|t| ExprImpl::Literal(Literal::new(None, t.clone()).into())) .collect_vec(); - let input_pk_lens = rewrites - .iter() - .map(|(new_input, _)| new_input.expect_stream_key().len()) - .collect_vec(); - let mut input_pk_offsets = vec![0]; - for (i, len) in input_pk_lens.into_iter().enumerate() { - input_pk_offsets.push(input_pk_offsets[i] + len) - } + let new_inputs = rewrites .into_iter() .enumerate() @@ -262,18 +281,22 @@ impl ToStream for LogicalUnion { ) }) .collect_vec(); - // input1_pk + input2_pk + ... - let mut input_pks = input_pk_nulls.clone(); - for (j, pk_idx) in new_input.expect_stream_key().iter().enumerate() { - input_pks[input_pk_offsets[i] + j] = ExprImpl::InputRef( - InputRef::new( - *pk_idx, - new_input.schema().fields[*pk_idx].data_type.clone(), - ) - .into(), - ); + // merged_stream_key + let mut input_stream_keys = input_stream_key_nulls.clone(); + let mut types_counter = BTreeMap::default(); + for stream_key_idx in new_input.expect_stream_key() { + let data_type = + new_input.schema().fields[*stream_key_idx].data_type.clone(); + let count = *types_counter + .entry(data_type.clone()) + .and_modify(|x| *x += 1) + .or_insert(1); + let type_start_offset = *types_offset.get(&data_type).unwrap(); + + input_stream_keys[type_start_offset + count - 1] = + ExprImpl::InputRef(InputRef::new(*stream_key_idx, data_type).into()); } - exprs.extend(input_pks); + exprs.extend(input_stream_keys); // source_col exprs.push(ExprImpl::Literal( Literal::new(Some((i as i32).to_scalar_value()), DataType::Int32).into(), @@ -285,7 +308,7 @@ impl ToStream for LogicalUnion { let new_union = LogicalUnion::new_with_source_col( self.all(), new_inputs, - Some(original_schema_len + input_pk_types.len()), + Some(original_schema_len + merged_stream_key_types.len()), ); // We have already used project to map rewrite input to the origin schema, so we can use // identity with the new schema len. diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index a4ec6191dedf3..fed912cc480b4 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -703,6 +703,7 @@ mod stream_eowc_over_window; mod stream_exchange; mod stream_expand; mod stream_filter; +mod stream_fs_fetch; mod stream_group_topn; mod stream_hash_agg; mod stream_hash_join; @@ -787,6 +788,7 @@ pub use stream_eowc_over_window::StreamEowcOverWindow; pub use stream_exchange::StreamExchange; pub use stream_expand::StreamExpand; pub use stream_filter::StreamFilter; +pub use stream_fs_fetch::StreamFsFetch; pub use stream_group_topn::StreamGroupTopN; pub use stream_hash_agg::StreamHashAgg; pub use stream_hash_join::StreamHashJoin; @@ -912,6 +914,7 @@ macro_rules! for_all_plan_nodes { , { Stream, EowcOverWindow } , { Stream, EowcSort } , { Stream, OverWindow } + , { Stream, FsFetch } } }; } @@ -1019,6 +1022,7 @@ macro_rules! for_stream_plan_nodes { , { Stream, EowcOverWindow } , { Stream, EowcSort } , { Stream, OverWindow } + , { Stream, FsFetch } } }; } diff --git a/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs new file mode 100644 index 0000000000000..4a1e449a21ab6 --- /dev/null +++ b/src/frontend/src/optimizer/plan_node/stream_fs_fetch.rs @@ -0,0 +1,125 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::rc::Rc; + +use fixedbitset::FixedBitSet; +use itertools::Itertools; +use pretty_xmlish::{Pretty, XmlNode}; +use risingwave_pb::stream_plan::stream_node::NodeBody; +use risingwave_pb::stream_plan::{PbStreamFsFetch, StreamFsFetchNode}; + +use super::{PlanBase, PlanRef, PlanTreeNodeUnary}; +use crate::catalog::source_catalog::SourceCatalog; +use crate::optimizer::plan_node::generic::GenericPlanRef; +use crate::optimizer::plan_node::utils::{childless_record, Distill}; +use crate::optimizer::plan_node::{generic, ExprRewritable, StreamNode}; +use crate::optimizer::property::Distribution; +use crate::stream_fragmenter::BuildFragmentGraphState; + +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +pub struct StreamFsFetch { + pub base: PlanBase, + input: PlanRef, + logical: generic::Source, +} + +impl PlanTreeNodeUnary for StreamFsFetch { + fn input(&self) -> PlanRef { + self.input.clone() + } + + fn clone_with_input(&self, input: PlanRef) -> Self { + Self::new(input, self.logical.clone()) + } +} +impl_plan_tree_node_for_unary! { StreamFsFetch } + +impl StreamFsFetch { + pub fn new(input: PlanRef, source: generic::Source) -> Self { + let base = PlanBase::new_stream_with_logical( + &source, + Distribution::SomeShard, + source.catalog.as_ref().map_or(true, |s| s.append_only), + false, + FixedBitSet::with_capacity(source.column_catalog.len()), + ); + + Self { + base, + input, + logical: source, + } + } + + fn get_columns(&self) -> Vec<&str> { + self.logical + .column_catalog + .iter() + .map(|column| column.name()) + .collect() + } + + pub fn source_catalog(&self) -> Option> { + self.logical.catalog.clone() + } +} + +impl Distill for StreamFsFetch { + fn distill<'a>(&self) -> XmlNode<'a> { + let columns = self + .get_columns() + .iter() + .map(|ele| Pretty::from(ele.to_string())) + .collect(); + let col = Pretty::Array(columns); + childless_record("StreamFsFetch", vec![("columns", col)]) + } +} + +impl ExprRewritable for StreamFsFetch {} + +impl StreamNode for StreamFsFetch { + fn to_stream_prost_body(&self, state: &mut BuildFragmentGraphState) -> NodeBody { + // `StreamFsFetch` is same as source in proto def, so the following code is the same as `StreamSource` + let source_catalog = self.source_catalog(); + let source_inner = source_catalog.map(|source_catalog| PbStreamFsFetch { + source_id: source_catalog.id, + source_name: source_catalog.name.clone(), + state_table: Some( + generic::Source::infer_internal_table_catalog() + .with_id(state.gen_table_id_wrapped()) + .to_internal_table_prost(), + ), + info: Some(source_catalog.info.clone()), + row_id_index: self.logical.row_id_index.map(|index| index as _), + columns: self + .logical + .column_catalog + .iter() + .map(|c| c.to_protobuf()) + .collect_vec(), + properties: source_catalog.properties.clone().into_iter().collect(), + rate_limit: self + .base + .ctx() + .session_ctx() + .config() + .get_streaming_rate_limit(), + }); + NodeBody::StreamFsFetch(StreamFsFetchNode { + node_inner: source_inner, + }) + } +} diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index f188889189464..a870be1de5840 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -32,7 +32,7 @@ use crate::stream_fragmenter::BuildFragmentGraphState; #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub struct StreamSource { pub base: PlanBase, - logical: generic::Source, + pub(crate) logical: generic::Source, } impl StreamSource { diff --git a/src/frontend/src/utils/stream_graph_formatter.rs b/src/frontend/src/utils/stream_graph_formatter.rs index 93acc73bd7957..2e9e6d1bb01ec 100644 --- a/src/frontend/src/utils/stream_graph_formatter.rs +++ b/src/frontend/src/utils/stream_graph_formatter.rs @@ -163,6 +163,12 @@ impl StreamGraphFormatter { self.pretty_add_table(source.get_state_table().unwrap()), )); } + stream_node::NodeBody::StreamFsFetch(node) if let Some(fetch) = &node.node_inner => { + fields.push(( + "fs fetch state table", + self.pretty_add_table(fetch.get_state_table().unwrap()), + )) + } stream_node::NodeBody::Materialize(node) => fields.push(( "materialized table", self.pretty_add_table(node.get_table().unwrap()), @@ -304,6 +310,7 @@ impl StreamGraphFormatter { stream_node::NodeBody::BarrierRecv(_) | stream_node::NodeBody::Values(_) | stream_node::NodeBody::Source(_) | + stream_node::NodeBody::StreamFsFetch(_) | stream_node::NodeBody::NoOp(_) => {} }; diff --git a/src/jni_core/src/jvm_runtime.rs b/src/jni_core/src/jvm_runtime.rs index 62630528961dc..5d93a122ab4a1 100644 --- a/src/jni_core/src/jvm_runtime.rs +++ b/src/jni_core/src/jvm_runtime.rs @@ -21,7 +21,7 @@ use std::sync::LazyLock; use jni::strings::JNIString; use jni::{InitArgsBuilder, JNIVersion, JavaVM, NativeMethod}; use risingwave_common::error::{ErrorCode, RwError}; -use risingwave_common::util::resource_util::memory::total_memory_available_bytes; +use risingwave_common::util::resource_util::memory::system_memory_available_bytes; pub static JVM: LazyLock> = LazyLock::new(|| { let libs_path = if let Ok(libs_path) = std::env::var("CONNECTOR_LIBS_PATH") { @@ -65,7 +65,8 @@ pub static JVM: LazyLock> = LazyLock::new(|| { heap_size } else { // Use 10% of total memory by default - format!("{}", total_memory_available_bytes() / 10) + // TODO: should use compute-node's total_memory_bytes + format!("{}", system_memory_available_bytes() / 10) }; // Build the VM properties diff --git a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs index c744c79c89995..980c3030a98fb 100644 --- a/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/intra_compaction_picker.rs @@ -623,53 +623,4 @@ pub mod tests { assert!(is_l0_trivial_move(&ret)); assert_eq!(ret.input_levels[0].table_infos.len(), 1); } - - #[test] - fn test_issue_11154() { - let mut local_stats = LocalPickerStatistic::default(); - let mut l0 = generate_l0_overlapping_sublevels(vec![ - vec![ - generate_table(4, 1, 1, 200, 1), - generate_table(5, 1, 400, 600, 1), - ], - vec![ - generate_table(6, 1, 1, 200, 1), - generate_table(7, 1, 400, 600, 1), - ], - vec![ - generate_table(8, 1, 1, 200, 1), - generate_table(9, 1, 400, 600, 1), - ], - vec![generate_table(10, 1, 1, 600, 1)], - ]); - // We can set level_type only because the input above is valid. - for s in &mut l0.sub_levels { - s.level_type = LevelType::Nonoverlapping as i32; - } - let levels = Levels { - l0: Some(l0), - levels: vec![generate_level(1, vec![generate_table(3, 1, 0, 100000, 1)])], - member_table_ids: vec![1], - ..Default::default() - }; - let levels_handler = vec![LevelHandler::new(0), LevelHandler::new(1)]; - - // Pick with large max_compaction_bytes results all sub levels included in input. - let config = Arc::new( - CompactionConfigBuilder::new() - .max_compaction_bytes(800) - .sub_level_max_compaction_bytes(50000) - .max_bytes_for_level_base(500000) - .level0_sub_level_compact_level_count(1) - .build(), - ); - // Only include sub-level 0 results will violate MAX_WRITE_AMPLIFICATION. - // So all sub-levels are included to make write amplification < MAX_WRITE_AMPLIFICATION. - let mut picker = IntraCompactionPicker::new(config); - let ret = picker - .pick_compaction(&levels, &levels_handler, &mut local_stats) - .unwrap(); - // avoid add sst_10 and cause a big task - assert_eq!(3, ret.input_levels.len()); - } } diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index c705bd3b43aae..c17fa305be0e4 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -209,6 +209,13 @@ impl NonOverlapSubLevelPicker { break; } + // more than 1 sub_level + if ret.total_file_count > 1 && ret.total_file_size >= self.max_compaction_bytes + || ret.total_file_count >= self.max_file_count as usize + { + break; + } + let mut overlap_files_range = overlap_info.check_multiple_include(&target_level.table_infos); if overlap_files_range.is_empty() { @@ -288,15 +295,6 @@ impl NonOverlapSubLevelPicker { .map(|(_, files)| files.len()) .sum::(); - // more than 1 sub_level - if ret.total_file_count > 1 - && (ret.total_file_size + (add_files_size + current_level_size) - >= self.max_compaction_bytes - || ret.total_file_count + add_files_count >= self.max_file_count as usize) - { - break; - } - if ret .sstable_infos .iter() diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index 3fc8d2eec303a..5fa90a77d48cb 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -239,7 +239,7 @@ pub fn start(opts: MetaNodeOpts) -> Pin + Send>> { validate_config(&config); - let total_memory_bytes = resource_util::memory::total_memory_available_bytes(); + let total_memory_bytes = resource_util::memory::system_memory_available_bytes(); let heap_profiler = HeapProfiler::new(total_memory_bytes, config.server.heap_profiling.clone()); // Run a background heap profiler diff --git a/src/source/src/connector_source.rs b/src/source/src/connector_source.rs index 445bf0f6dbb90..733ba6a8c4a83 100644 --- a/src/source/src/connector_source.rs +++ b/src/source/src/connector_source.rs @@ -18,17 +18,22 @@ use std::sync::Arc; use futures::future::try_join_all; use futures::stream::pending; use futures::StreamExt; +use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::catalog::ColumnId; use risingwave_common::error::ErrorCode::ConnectorError; -use risingwave_common::error::{internal_error, Result}; +use risingwave_common::error::{internal_error, Result, RwError}; use risingwave_common::util::select_all; use risingwave_connector::dispatch_source_prop; use risingwave_connector::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; +use risingwave_connector::source::filesystem::{FsPage, FsPageItem, S3SplitEnumerator}; use risingwave_connector::source::{ - create_split_reader, BoxSourceWithStateStream, Column, ConnectorProperties, ConnectorState, - SourceColumnDesc, SourceContext, SplitReader, + create_split_reader, BoxSourceWithStateStream, BoxTryStream, Column, ConnectorProperties, + ConnectorState, FsFilterCtrlCtx, FsListInner, SourceColumnDesc, SourceContext, + SourceEnumeratorContext, SplitEnumerator, SplitReader, }; +use tokio::time; +use tokio::time::{Duration, MissedTickBehavior}; #[derive(Clone, Debug)] pub struct ConnectorSource { @@ -38,6 +43,15 @@ pub struct ConnectorSource { pub connector_message_buffer_size: usize, } +#[derive(Clone, Debug)] +pub struct FsListCtrlContext { + pub interval: Duration, + pub last_tick: Option, + + pub filter_ctx: FsFilterCtrlCtx, +} +pub type FsListCtrlContextRef = Arc; + impl ConnectorSource { pub fn new( properties: HashMap, @@ -74,6 +88,25 @@ impl ConnectorSource { .collect::>>() } + pub async fn get_source_list(&self) -> Result> { + let config = self.config.clone(); + let lister = match config { + ConnectorProperties::S3(prop) => { + S3SplitEnumerator::new(*prop, Arc::new(SourceEnumeratorContext::default())).await? + } + other => return Err(internal_error(format!("Unsupported source: {:?}", other))), + }; + + Ok(build_fs_list_stream( + FsListCtrlContext { + interval: Duration::from_secs(60), + last_tick: None, + filter_ctx: FsFilterCtrlCtx, + }, + lister, + )) + } + pub async fn stream_reader( &self, state: ConnectorState, @@ -147,3 +180,35 @@ impl ConnectorSource { }) } } + +#[try_stream(boxed, ok = FsPage, error = RwError)] +async fn build_fs_list_stream( + mut ctrl_ctx: FsListCtrlContext, + mut list_op: impl FsListInner + Send + 'static, +) { + let mut interval = time::interval(ctrl_ctx.interval); + interval.set_missed_tick_behavior(MissedTickBehavior::Skip); + + // controlling whether request for next page + fn page_ctrl_logic(_ctx: &FsListCtrlContext, has_finished: bool, _page_num: usize) -> bool { + !has_finished + } + + loop { + let mut page_num = 0; + ctrl_ctx.last_tick = Some(time::Instant::now()); + 'inner: loop { + let (fs_page, has_finished) = list_op.get_next_page::().await?; + let matched_items = fs_page + .into_iter() + .filter(|item| list_op.filter_policy(&ctrl_ctx.filter_ctx, page_num, item)) + .collect_vec(); + yield matched_items; + page_num += 1; + if !page_ctrl_logic(&ctrl_ctx, has_finished, page_num) { + break 'inner; + } + } + interval.tick().await; + } +} diff --git a/src/source/src/fs_connector_source.rs b/src/source/src/fs_connector_source.rs index 974f0561e0f2d..671f5b99c5bae 100644 --- a/src/source/src/fs_connector_source.rs +++ b/src/source/src/fs_connector_source.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +// *** NOTICE: TO BE DEPRECATED *** // + use std::collections::HashMap; use std::sync::Arc; diff --git a/src/source/src/lib.rs b/src/source/src/lib.rs index 30c7d90cfe771..1a4f5d5f47280 100644 --- a/src/source/src/lib.rs +++ b/src/source/src/lib.rs @@ -21,6 +21,7 @@ #![feature(hash_extract_if)] #![feature(type_alias_impl_trait)] #![feature(box_patterns)] +#![feature(stmt_expr_attributes)] pub use table::*; diff --git a/src/source/src/source_desc.rs b/src/source/src/source_desc.rs index 4d4b9f9cb5b80..bf28c00e42f1f 100644 --- a/src/source/src/source_desc.rs +++ b/src/source/src/source_desc.rs @@ -21,7 +21,8 @@ use risingwave_common::error::{Result, RwError}; use risingwave_connector::parser::SpecificParserConfig; use risingwave_connector::source::monitor::SourceMetrics; use risingwave_connector::source::{ - SourceColumnDesc, SourceColumnType, SourceEncode, SourceFormat, SourceStruct, + ConnectorProperties, SourceColumnDesc, SourceColumnType, SourceEncode, SourceFormat, + SourceStruct, }; use risingwave_connector::ConnectorParams; use risingwave_pb::catalog::PbStreamSourceInfo; @@ -33,12 +34,14 @@ use crate::fs_connector_source::FsConnectorSource; pub const DEFAULT_CONNECTOR_MESSAGE_BUFFER_SIZE: usize = 16; /// `SourceDesc` describes a stream source. -#[derive(Debug)] +#[derive(Debug, Clone)] pub struct SourceDesc { pub source: ConnectorSource, pub source_struct: SourceStruct, pub columns: Vec, pub metrics: Arc, + + pub is_new_fs_source: bool, } /// `FsSourceDesc` describes a stream source. @@ -101,13 +104,19 @@ impl SourceDescBuilder { columns } - pub fn build(self) -> Result { + pub fn build(mut self) -> Result { let columns = self.column_catalogs_to_source_column_descs(); let source_struct = extract_source_struct(&self.source_info)?; let psrser_config = SpecificParserConfig::new(source_struct, &self.source_info, &self.properties)?; + let is_new_fs_source = ConnectorProperties::is_new_fs_connector_hash_map(&self.properties); + if is_new_fs_source { + // new fs source requires `connector='s3_v2' but we simply reuse S3 connector` + ConnectorProperties::rewrite_upstream_source_key_hash_map(&mut self.properties); + } + let source = ConnectorSource::new( self.properties, columns.clone(), @@ -120,6 +129,7 @@ impl SourceDescBuilder { source_struct, columns, metrics: self.metrics, + is_new_fs_source, }) } diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index 49c1977c0f034..9e0fac5826d6a 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -71,7 +71,7 @@ tracing = "0.1" tracing-futures = { version = "0.2", features = ["futures-03"] } xorf = "0.8.1" xxhash-rust = { version = "0.8.7", features = ["xxh32", "xxh64"] } -zstd = { version = "0.12", default-features = false } +zstd = { version = "0.13", default-features = false } [target.'cfg(target_os = "linux")'.dependencies] procfs = { version = "0.15", default-features = false } diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index 50bfa6cdb0634..627cd3491d78f 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -88,7 +88,7 @@ pub async fn prepare_start_parameters( &storage_memory_config, ))); let total_memory_available_bytes = - (resource_util::memory::total_memory_available_bytes() as f64 + (resource_util::memory::system_memory_available_bytes() as f64 * config.storage.compactor_memory_available_proportion) as usize; let meta_cache_capacity_bytes = storage_opts.meta_cache_capacity_mb * (1 << 20); let compactor_memory_limit_bytes = match config.storage.compactor_memory_limit_mb { diff --git a/src/storage/src/hummock/compactor/compaction_utils.rs b/src/storage/src/hummock/compactor/compaction_utils.rs index 5f28a076888ef..3c1332d09317c 100644 --- a/src/storage/src/hummock/compactor/compaction_utils.rs +++ b/src/storage/src/hummock/compactor/compaction_utils.rs @@ -158,6 +158,63 @@ pub fn build_multi_compaction_filter(compact_task: &CompactTask) -> MultiCompact multi_filter } +const MAX_FILE_COUNT: usize = 32; + +fn generate_splits_fast( + sstable_infos: &Vec, + compaction_size: u64, + context: CompactorContext, +) -> HummockResult> { + let worker_num = context.compaction_executor.worker_num(); + let parallel_compact_size = (context.storage_opts.parallel_compact_size_mb as u64) << 20; + + let parallelism = (compaction_size + parallel_compact_size - 1) / parallel_compact_size; + + let parallelism = std::cmp::min( + worker_num, + std::cmp::min( + parallelism as usize, + context.storage_opts.max_sub_compaction as usize, + ), + ); + let mut indexes = vec![]; + for sst in sstable_infos { + let key_range = sst.key_range.as_ref().unwrap(); + indexes.push( + FullKey { + user_key: FullKey::decode(&key_range.left).user_key, + epoch: HummockEpoch::MAX, + } + .encode(), + ); + indexes.push( + FullKey { + user_key: FullKey::decode(&key_range.right).user_key, + epoch: HummockEpoch::MAX, + } + .encode(), + ); + } + indexes.sort_by(|a, b| KeyComparator::compare_encoded_full_key(a.as_ref(), b.as_ref())); + indexes.dedup(); + if indexes.len() <= parallelism { + return Ok(vec![]); + } + let mut splits = vec![]; + splits.push(KeyRange_vec::new(vec![], vec![])); + let parallel_key_count = indexes.len() / parallelism; + let mut last_split_key_count = 0; + for key in indexes { + if last_split_key_count >= parallel_key_count { + splits.last_mut().unwrap().right = key.clone(); + splits.push(KeyRange_vec::new(key.clone(), vec![])); + last_split_key_count = 0; + } + last_split_key_count += 1; + } + Ok(splits) +} + pub async fn generate_splits( sstable_infos: &Vec, compaction_size: u64, @@ -165,6 +222,9 @@ pub async fn generate_splits( ) -> HummockResult> { let parallel_compact_size = (context.storage_opts.parallel_compact_size_mb as u64) << 20; if compaction_size > parallel_compact_size { + if sstable_infos.len() > MAX_FILE_COUNT { + return generate_splits_fast(sstable_infos, compaction_size, context); + } let mut indexes = vec![]; // preload the meta and get the smallest key to split sub_compaction for sstable_info in sstable_infos { diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 3d079ce4a188e..583bab3d10b3c 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -152,17 +152,20 @@ impl CompactorRunner { let mut local_stats = StoreLocalStatistic::default(); for table_info in sstable_infos { - let table = sstable_store.sstable(table_info, &mut local_stats).await?; - let mut range_tombstone_list = table.value().meta.monotonic_tombstone_events.clone(); - range_tombstone_list.iter_mut().for_each(|tombstone| { - if filter.should_delete(FullKey::from_user_key( - tombstone.event_key.left_user_key.as_ref(), - tombstone.new_epoch, - )) { - tombstone.new_epoch = HummockEpoch::MAX; - } - }); - builder.add_delete_events(range_tombstone_list); + if table_info.range_tombstone_count > 0 { + let table = sstable_store.sstable(table_info, &mut local_stats).await?; + let mut range_tombstone_list = + table.value().meta.monotonic_tombstone_events.clone(); + range_tombstone_list.iter_mut().for_each(|tombstone| { + if filter.should_delete(FullKey::from_user_key( + tombstone.event_key.left_user_key.as_ref(), + tombstone.new_epoch, + )) { + tombstone.new_epoch = HummockEpoch::MAX; + } + }); + builder.add_delete_events(range_tombstone_list); + } } let aggregator = builder.build_for_compaction(); @@ -891,10 +894,8 @@ mod tests { use super::*; use crate::hummock::compactor::StateCleanUpCompactionFilter; use crate::hummock::iterator::test_utils::mock_sstable_store; - use crate::hummock::test_utils::{ - default_builder_opt_for_test, gen_test_sstable_with_range_tombstone, - }; - use crate::hummock::{create_monotonic_events, DeleteRangeTombstone}; + use crate::hummock::test_utils::{default_builder_opt_for_test, gen_test_sstable_impl}; + use crate::hummock::{create_monotonic_events, DeleteRangeTombstone, Xor16FilterBuilder}; #[tokio::test] async fn test_delete_range_aggregator_with_filter() { @@ -914,26 +915,26 @@ mod tests { 1, ), ]; - let mut sstable_info_1 = gen_test_sstable_with_range_tombstone( + let mut sstable_info_1 = gen_test_sstable_impl::( default_builder_opt_for_test(), 1, kv_pairs.clone().into_iter(), range_tombstones.clone(), sstable_store.clone(), + CachePolicy::NotFill, ) - .await - .get_sstable_info(); + .await; sstable_info_1.table_ids = vec![1]; - let mut sstable_info_2 = gen_test_sstable_with_range_tombstone( + let mut sstable_info_2 = gen_test_sstable_impl::( default_builder_opt_for_test(), 2, kv_pairs.into_iter(), range_tombstones.clone(), sstable_store.clone(), + CachePolicy::NotFill, ) - .await - .get_sstable_info(); + .await; sstable_info_2.table_ids = vec![2]; let compact_task = CompactTask { diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 78c7d484385e8..1329f08eb6d99 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -139,7 +139,10 @@ impl MaterializedInputState { agg_call.args.arg_types(), )) } - AggKind::StringAgg | AggKind::ArrayAgg => Box::new(GenericAggStateCache::new( + AggKind::StringAgg + | AggKind::ArrayAgg + | AggKind::JsonbAgg + | AggKind::JsonbObjectAgg => Box::new(GenericAggStateCache::new( OrderedStateCache::new(), agg_call.args.arg_types(), )), diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs new file mode 100644 index 0000000000000..3e7ea84bcfcce --- /dev/null +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -0,0 +1,350 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::{Debug, Formatter}; +use std::ops::Bound; +use std::sync::Arc; + +use either::Either; +use futures::pin_mut; +use futures::stream::{self, StreamExt}; +use futures_async_stream::try_stream; +use risingwave_common::catalog::{ColumnId, Schema, TableId}; +use risingwave_common::hash::VnodeBitmapExt; +use risingwave_common::row::{OwnedRow, Row}; +use risingwave_common::types::{ScalarRef, ScalarRefImpl}; +use risingwave_connector::source::filesystem::FsSplit; +use risingwave_connector::source::{ + BoxSourceWithStateStream, SourceContext, SourceCtrlOpts, SplitImpl, SplitMetaData, + StreamChunkWithState, +}; +use risingwave_connector::ConnectorParams; +use risingwave_source::source_desc::SourceDesc; +use risingwave_storage::store::PrefetchOptions; +use risingwave_storage::StateStore; + +use crate::executor::stream_reader::StreamReaderWithPause; +use crate::executor::{ + expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, Message, + Mutation, PkIndices, PkIndicesRef, SourceStateTableHandler, StreamExecutorError, + StreamExecutorResult, StreamSourceCore, +}; + +const SPLIT_BATCH_SIZE: usize = 1000; + +type SplitBatch = Option>; + +pub struct FsFetchExecutor { + actor_ctx: ActorContextRef, + + identity: String, + + schema: Schema, + + pk_indices: PkIndices, + + /// Streaming source for external + stream_source_core: Option>, + + /// Upstream list executor. + upstream: Option, + + // control options for connector level + source_ctrl_opts: SourceCtrlOpts, + + // config for the connector node + connector_params: ConnectorParams, +} + +impl FsFetchExecutor { + #[allow(clippy::too_many_arguments)] + pub fn new( + actor_ctx: ActorContextRef, + schema: Schema, + pk_indices: PkIndices, + stream_source_core: StreamSourceCore, + executor_id: u64, + upstream: BoxedExecutor, + source_ctrl_opts: SourceCtrlOpts, + connector_params: ConnectorParams, + ) -> Self { + Self { + actor_ctx, + identity: format!("FsFetchExecutor {:X}", executor_id), + schema, + pk_indices, + stream_source_core: Some(stream_source_core), + upstream: Some(upstream), + source_ctrl_opts, + connector_params, + } + } + + async fn replace_with_new_batch_reader( + splits_on_fetch: &mut usize, + state_store_handler: &SourceStateTableHandler, + column_ids: Vec, + source_ctx: SourceContext, + source_desc: &SourceDesc, + stream: &mut StreamReaderWithPause, + ) -> StreamExecutorResult<()> { + let mut batch = Vec::with_capacity(SPLIT_BATCH_SIZE); + 'vnodes: for vnode in state_store_handler.state_store.vnodes().iter_vnodes() { + let table_iter = state_store_handler + .state_store + .iter_with_vnode( + vnode, + &(Bound::::Unbounded, Bound::::Unbounded), + PrefetchOptions::new_for_exhaust_iter(), + ) + .await?; + pin_mut!(table_iter); + + while let Some(item) = table_iter.next().await { + let row = item?; + let split = match row.datum_at(1) { + Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { + SplitImpl::from(FsSplit::restore_from_json(jsonb_ref.to_owned_scalar())?) + } + _ => unreachable!(), + }; + batch.push(split); + + if batch.len() >= SPLIT_BATCH_SIZE { + break 'vnodes; + } + } + } + + if batch.is_empty() { + stream.replace_data_stream(stream::pending().boxed()); + } else { + *splits_on_fetch += batch.len(); + let batch_reader = + Self::build_batched_stream_reader(column_ids, source_ctx, source_desc, Some(batch)) + .await?; + stream.replace_data_stream(batch_reader); + } + + Ok(()) + } + + async fn build_batched_stream_reader( + column_ids: Vec, + source_ctx: SourceContext, + source_desc: &SourceDesc, + batch: SplitBatch, + ) -> StreamExecutorResult { + source_desc + .source + .stream_reader(batch, column_ids, Arc::new(source_ctx)) + .await + .map_err(StreamExecutorError::connector_error) + } + + fn build_source_ctx(&self, source_desc: &SourceDesc, source_id: TableId) -> SourceContext { + SourceContext::new_with_suppressor( + self.actor_ctx.id, + source_id, + self.actor_ctx.fragment_id, + source_desc.metrics.clone(), + self.source_ctrl_opts.clone(), + self.connector_params.connector_client.clone(), + self.actor_ctx.error_suppressor.clone(), + ) + } + + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn into_stream(mut self) { + let mut upstream = self.upstream.take().unwrap().execute(); + let barrier = expect_first_barrier(&mut upstream).await?; + + let mut core = self.stream_source_core.take().unwrap(); + let mut state_store_handler = core.split_state_store; + + // Build source description from the builder. + let source_desc_builder = core.source_desc_builder.take().unwrap(); + + let source_desc = source_desc_builder + .build() + .map_err(StreamExecutorError::connector_error)?; + + // Initialize state table. + state_store_handler.init_epoch(barrier.epoch); + + let mut splits_on_fetch: usize = 0; + let mut stream = StreamReaderWithPause::::new( + upstream, + stream::pending().boxed(), + ); + + if barrier.is_pause_on_startup() { + stream.pause_stream(); + } + + // If it is a recovery startup, + // there can be file assignments in the state table. + // Hence we try building a reader first. + Self::replace_with_new_batch_reader( + &mut splits_on_fetch, + &state_store_handler, + core.column_ids.clone(), + self.build_source_ctx(&source_desc, core.source_id), + &source_desc, + &mut stream, + ) + .await?; + + yield Message::Barrier(barrier); + + while let Some(msg) = stream.next().await { + match msg { + Err(e) => { + tracing::error!("Fetch Error: {:?}", e); + splits_on_fetch = 0; + } + Ok(msg) => { + match msg { + // This branch will be preferred. + Either::Left(msg) => { + match &msg { + Message::Barrier(barrier) => { + if let Some(mutation) = barrier.mutation.as_deref() { + match mutation { + Mutation::Pause => stream.pause_stream(), + Mutation::Resume => stream.resume_stream(), + _ => (), + } + } + + state_store_handler + .state_store + .commit(barrier.epoch) + .await?; + + if let Some(vnode_bitmap) = + barrier.as_update_vnode_bitmap(self.actor_ctx.id) + { + // if _cache_may_stale, we must rebuild the stream to adjust vnode mappings + let (_prev_vnode_bitmap, cache_may_stale) = + state_store_handler + .state_store + .update_vnode_bitmap(vnode_bitmap); + + if cache_may_stale { + splits_on_fetch = 0; + } + } + + if splits_on_fetch == 0 { + Self::replace_with_new_batch_reader( + &mut splits_on_fetch, + &state_store_handler, + core.column_ids.clone(), + self.build_source_ctx(&source_desc, core.source_id), + &source_desc, + &mut stream, + ) + .await?; + } + + // Propagate the barrier. + yield msg; + } + // Receiving file assignments from upstream list executor, + // store into state table and try building a new reader. + Message::Chunk(chunk) => { + let file_assignment = chunk + .data_chunk() + .rows() + .map(|row| { + let filename = row.datum_at(0).unwrap().into_utf8(); + let size = row.datum_at(2).unwrap().into_int64(); + FsSplit::new(filename.to_owned(), 0, size as usize) + }) + .collect(); + state_store_handler.take_snapshot(file_assignment).await?; + } + _ => unreachable!(), + } + } + // StreamChunk from FsSourceReader, and the reader reads only one file. + // If the file read out, replace with a new file reader. + Either::Right(StreamChunkWithState { + chunk, + split_offset_mapping, + }) => { + let mapping = split_offset_mapping.unwrap(); + for (split_id, offset) in mapping { + let row = state_store_handler + .get(split_id.clone()) + .await? + .expect("The fs_split should be in the state table."); + let fs_split = match row.datum_at(1) { + Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { + FsSplit::restore_from_json(jsonb_ref.to_owned_scalar())? + } + _ => unreachable!(), + }; + + if offset.parse::().unwrap() >= fs_split.size { + splits_on_fetch -= 1; + state_store_handler.delete(split_id).await?; + } else { + state_store_handler + .set(split_id, fs_split.encode_to_json()) + .await?; + } + } + + yield Message::Chunk(chunk); + } + } + } + } + } + } +} + +impl Executor for FsFetchExecutor { + fn execute(self: Box) -> BoxedMessageStream { + self.into_stream().boxed() + } + + fn schema(&self) -> &Schema { + &self.schema + } + + fn pk_indices(&self) -> PkIndicesRef<'_> { + &self.pk_indices + } + + fn identity(&self) -> &str { + self.identity.as_str() + } +} + +impl Debug for FsFetchExecutor { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + if let Some(core) = &self.stream_source_core { + f.debug_struct("FsFetchExecutor") + .field("source_id", &core.source_id) + .field("column_ids", &core.column_ids) + .field("pk_indices", &self.pk_indices) + .finish() + } else { + f.debug_struct("FsFetchExecutor").finish() + } + } +} diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index ae77adb427e23..bba0e30eb5712 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +// *** NOTICE: TO BE DEPRECATED *** // + use std::fmt::{Debug, Formatter}; use std::sync::Arc; diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs new file mode 100644 index 0000000000000..53e8854594ce4 --- /dev/null +++ b/src/stream/src/executor/source/list_executor.rs @@ -0,0 +1,231 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::fmt::Formatter; +use std::sync::Arc; + +use anyhow::anyhow; +use either::Either; +use futures::StreamExt; +use futures_async_stream::try_stream; +use risingwave_common::array::Op; +use risingwave_common::catalog::Schema; +use risingwave_common::system_param::local_manager::SystemParamsReaderRef; +use risingwave_connector::source::filesystem::FsPage; +use risingwave_connector::source::{BoxTryStream, SourceCtrlOpts}; +use risingwave_connector::ConnectorParams; +use risingwave_source::source_desc::{SourceDesc, SourceDescBuilder}; +use risingwave_storage::StateStore; +use tokio::sync::mpsc::UnboundedReceiver; + +use crate::executor::error::StreamExecutorError; +use crate::executor::monitor::StreamingMetrics; +use crate::executor::stream_reader::StreamReaderWithPause; +use crate::executor::*; + +#[allow(dead_code)] +pub struct FsListExecutor { + actor_ctx: ActorContextRef, + + identity: String, + + schema: Schema, + + pk_indices: PkIndices, + + /// Streaming source for external + stream_source_core: Option>, + + /// Metrics for monitor. + metrics: Arc, + + /// Receiver of barrier channel. + barrier_receiver: Option>, + + /// System parameter reader to read barrier interval + system_params: SystemParamsReaderRef, + + // control options for connector level + source_ctrl_opts: SourceCtrlOpts, + + // config for the connector node + connector_params: ConnectorParams, +} + +impl FsListExecutor { + #[allow(clippy::too_many_arguments)] + pub fn new( + actor_ctx: ActorContextRef, + schema: Schema, + pk_indices: PkIndices, + stream_source_core: Option>, + metrics: Arc, + barrier_receiver: UnboundedReceiver, + system_params: SystemParamsReaderRef, + executor_id: u64, + source_ctrl_opts: SourceCtrlOpts, + connector_params: ConnectorParams, + ) -> Self { + Self { + actor_ctx, + identity: format!("FsListExecutor {:X}", executor_id), + schema, + pk_indices, + stream_source_core, + metrics, + barrier_receiver: Some(barrier_receiver), + system_params, + source_ctrl_opts, + connector_params, + } + } + + async fn build_chunked_paginate_stream( + &self, + source_desc: &SourceDesc, + ) -> StreamExecutorResult> { + let stream = source_desc + .source + .get_source_list() + .await + .map_err(StreamExecutorError::connector_error)?; + + Ok(stream + .map(|item| item.map(Self::map_fs_page_to_chunk)) + .boxed()) + } + + fn map_fs_page_to_chunk(page: FsPage) -> StreamChunk { + let rows = page + .into_iter() + .map(|split| { + ( + Op::Insert, + OwnedRow::new(vec![ + Some(ScalarImpl::Utf8(split.name.into_boxed_str())), + Some(ScalarImpl::Timestamp(split.timestamp)), + Some(ScalarImpl::Int64(split.size)), + ]), + ) + }) + .collect::>(); + StreamChunk::from_rows( + &rows, + &[DataType::Varchar, DataType::Timestamp, DataType::Int64], + ) + } + + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn into_stream(mut self) { + let mut barrier_receiver = self.barrier_receiver.take().unwrap(); + let barrier = barrier_receiver + .recv() + .instrument_await("source_recv_first_barrier") + .await + .ok_or_else(|| { + anyhow!( + "failed to receive the first barrier, actor_id: {:?}, source_id: {:?}", + self.actor_ctx.id, + self.stream_source_core.as_ref().unwrap().source_id + ) + })?; + + let mut core = self.stream_source_core.unwrap(); + + // Build source description from the builder. + let source_desc_builder: SourceDescBuilder = core.source_desc_builder.take().unwrap(); + let source_desc = source_desc_builder + .build() + .map_err(StreamExecutorError::connector_error)?; + + // Return the ownership of `stream_source_core` to the source executor. + self.stream_source_core = Some(core); + + let chunked_paginate_stream = self.build_chunked_paginate_stream(&source_desc).await?; + + let barrier_stream = barrier_to_message_stream(barrier_receiver).boxed(); + let mut stream = + StreamReaderWithPause::::new(barrier_stream, chunked_paginate_stream); + + if barrier.is_pause_on_startup() { + stream.pause_stream(); + } + + yield Message::Barrier(barrier); + + while let Some(msg) = stream.next().await { + match msg { + Err(e) => { + tracing::warn!("encountered an error, recovering. {:?}", e); + // todo: rebuild stream here + } + Ok(msg) => match msg { + // Barrier arrives. + Either::Left(msg) => match &msg { + Message::Barrier(barrier) => { + if let Some(mutation) = barrier.mutation.as_deref() { + match mutation { + Mutation::Pause => stream.pause_stream(), + Mutation::Resume => stream.resume_stream(), + _ => (), + } + } + + // Propagate the barrier. + yield msg; + } + // Only barrier can be received. + _ => unreachable!(), + }, + // Chunked FsPage arrives. + Either::Right(chunk) => { + yield Message::Chunk(chunk); + } + }, + } + } + } +} + +impl Executor for FsListExecutor { + fn execute(self: Box) -> BoxedMessageStream { + self.into_stream().boxed() + } + + fn schema(&self) -> &Schema { + &self.schema + } + + fn pk_indices(&self) -> PkIndicesRef<'_> { + &self.pk_indices + } + + fn identity(&self) -> &str { + self.identity.as_str() + } +} + +impl Debug for FsListExecutor { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + if let Some(core) = &self.stream_source_core { + f.debug_struct("FsListExecutor") + .field("source_id", &core.source_id) + .field("column_ids", &core.column_ids) + .field("pk_indices", &self.pk_indices) + .finish() + } else { + f.debug_struct("FsListExecutor").finish() + } + } +} diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 1b06120561f51..18f7346777d6b 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -19,12 +19,15 @@ mod fs_source_executor; pub use fs_source_executor::*; use risingwave_common::bail; pub use state_table_handler::*; +pub mod fetch_executor; +pub use fetch_executor::*; pub mod source_executor; +pub mod list_executor; pub mod state_table_handler; - use futures_async_stream::try_stream; +pub use list_executor::*; use tokio::sync::mpsc::UnboundedReceiver; use crate::executor::error::StreamExecutorError; diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index 85c6d2d780d60..d2226eb93867b 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -14,8 +14,10 @@ use std::collections::HashSet; use std::ops::{Bound, Deref}; +use std::sync::Arc; use futures::{pin_mut, StreamExt}; +use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::{DatabaseId, SchemaId}; use risingwave_common::constants::hummock::PROPERTIES_RETENTION_SECOND_KEY; use risingwave_common::hash::VirtualNode; @@ -56,6 +58,21 @@ impl SourceStateTableHandler { } } + pub async fn from_table_catalog_with_vnodes( + table_catalog: &PbTable, + store: S, + vnodes: Option>, + ) -> Self { + // The state of source should not be cleaned up by retention_seconds + assert!(!table_catalog + .properties + .contains_key(&String::from(PROPERTIES_RETENTION_SECOND_KEY))); + + Self { + state_store: StateTable::from_table_catalog(table_catalog, store, vnodes).await, + } + } + pub fn init_epoch(&mut self, epoch: EpochPair) { self.state_store.init_epoch(epoch); } @@ -159,7 +176,7 @@ impl SourceStateTableHandler { Ok(()) } - async fn delete(&mut self, key: SplitId) -> StreamExecutorResult<()> { + pub async fn delete(&mut self, key: SplitId) -> StreamExecutorResult<()> { if let Some(prev_row) = self.get(key).await? { self.state_store.delete(prev_row); } diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index bdb19f022ec37..2ec9476d0e904 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -166,5 +166,6 @@ pub async fn create_executor( NodeBody::NoOp => NoOpExecutorBuilder, NodeBody::EowcOverWindow => EowcOverWindowExecutorBuilder, NodeBody::OverWindow => OverWindowExecutorBuilder, + NodeBody::StreamFsFetch => FsFetchExecutorBuilder, } } diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs new file mode 100644 index 0000000000000..ecf242f044b04 --- /dev/null +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -0,0 +1,114 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::sync::Arc; + +use risingwave_common::catalog::{ColumnId, Field, Schema, TableId}; +use risingwave_common::types::DataType; +use risingwave_connector::source::SourceCtrlOpts; +use risingwave_pb::stream_plan::StreamFsFetchNode; +use risingwave_source::source_desc::SourceDescBuilder; +use risingwave_storage::StateStore; + +use crate::error::StreamResult; +use crate::executor::{ + BoxedExecutor, Executor, FsFetchExecutor, SourceStateTableHandler, StreamSourceCore, +}; +use crate::from_proto::ExecutorBuilder; +use crate::task::{ExecutorParams, LocalStreamManagerCore}; + +pub struct FsFetchExecutorBuilder; + +#[async_trait::async_trait] +impl ExecutorBuilder for FsFetchExecutorBuilder { + type Node = StreamFsFetchNode; + + async fn new_boxed_executor( + params: ExecutorParams, + node: &Self::Node, + store: impl StateStore, + _stream: &mut LocalStreamManagerCore, + ) -> StreamResult { + let [upstream]: [_; 1] = params.input.try_into().unwrap(); + + let source = node.node_inner.as_ref().unwrap(); + + let source_id = TableId::new(source.source_id); + let source_name = source.source_name.clone(); + let source_info = source.get_info()?; + + let source_desc_builder = SourceDescBuilder::new( + source.columns.clone(), + params.env.source_metrics(), + source.row_id_index.map(|x| x as _), + source.properties.clone(), + source_info.clone(), + params.env.connector_params(), + params.env.config().developer.connector_message_buffer_size, + params.pk_indices.clone(), + ); + + let source_ctrl_opts = SourceCtrlOpts { + chunk_size: params.env.config().developer.chunk_size, + }; + + let column_ids: Vec<_> = source + .columns + .iter() + .map(|column| ColumnId::from(column.get_column_desc().unwrap().column_id)) + .collect(); + let fields = source + .columns + .iter() + .map(|prost| { + let column_desc = prost.column_desc.as_ref().unwrap(); + let data_type = DataType::from(column_desc.column_type.as_ref().unwrap()); + let name = column_desc.name.clone(); + Field::with_name(data_type, name) + }) + .collect(); + let schema = Schema::new(fields); + + let vnodes = Some(Arc::new( + params + .vnode_bitmap + .expect("vnodes not set for fetch executor"), + )); + let state_table_handler = SourceStateTableHandler::from_table_catalog_with_vnodes( + source.state_table.as_ref().unwrap(), + store.clone(), + vnodes, + ) + .await; + let stream_source_core = StreamSourceCore::new( + source_id, + source_name, + column_ids, + source_desc_builder, + state_table_handler, + ); + + Ok(FsFetchExecutor::new( + params.actor_context, + schema, + params.pk_indices, + stream_source_core, + params.executor_id, + upstream, + source_ctrl_opts, + params.env.connector_params(), + ) + .boxed()) + } +} diff --git a/src/stream/src/from_proto/source/mod.rs b/src/stream/src/from_proto/source/mod.rs new file mode 100644 index 0000000000000..cb83889465a73 --- /dev/null +++ b/src/stream/src/from_proto/source/mod.rs @@ -0,0 +1,20 @@ +// Copyright 2023 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +mod trad_source; +pub use trad_source::SourceExecutorBuilder; +mod fs_fetch; +pub use fs_fetch::FsFetchExecutorBuilder; + +use super::*; diff --git a/src/stream/src/from_proto/source.rs b/src/stream/src/from_proto/source/trad_source.rs similarity index 91% rename from src/stream/src/from_proto/source.rs rename to src/stream/src/from_proto/source/trad_source.rs index 77bbcc53e69c5..24b77af41e0f6 100644 --- a/src/stream/src/from_proto/source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -16,7 +16,7 @@ use risingwave_common::catalog::{ColumnId, Field, Schema, TableId}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::OrderType; use risingwave_connector::source::external::{ExternalTableType, SchemaTableName}; -use risingwave_connector::source::SourceCtrlOpts; +use risingwave_connector::source::{ConnectorProperties, SourceCtrlOpts}; use risingwave_pb::stream_plan::SourceNode; use risingwave_source::source_desc::SourceDescBuilder; use risingwave_storage::panic_store::PanicStateStore; @@ -24,7 +24,7 @@ use tokio::sync::mpsc::unbounded_channel; use super::*; use crate::executor::external::ExternalStorageTable; -use crate::executor::source::StreamSourceCore; +use crate::executor::source::{FsListExecutor, StreamSourceCore}; use crate::executor::source_executor::SourceExecutor; use crate::executor::state_table_handler::SourceStateTableHandler; use crate::executor::{CdcBackfillExecutor, FlowControlExecutor, FsSourceExecutor}; @@ -115,6 +115,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { .map(|c| c.to_ascii_lowercase()) .unwrap_or_default(); let is_fs_connector = FS_CONNECTORS.contains(&connector.as_str()); + let is_fs_v2_connector = + ConnectorProperties::is_new_fs_connector_hash_map(&source.properties); if is_fs_connector { FsSourceExecutor::new( @@ -129,6 +131,20 @@ impl ExecutorBuilder for SourceExecutorBuilder { source_ctrl_opts, )? .boxed() + } else if is_fs_v2_connector { + FsListExecutor::new( + params.actor_context.clone(), + schema.clone(), + params.pk_indices.clone(), + Some(stream_source_core), + params.executor_stats.clone(), + barrier_receiver, + system_params, + params.executor_id, + source_ctrl_opts.clone(), + params.env.connector_params(), + ) + .boxed() } else { let source_exec = SourceExecutor::new( params.actor_context.clone(), diff --git a/src/tests/sqlsmith/src/sql_gen/functions.rs b/src/tests/sqlsmith/src/sql_gen/functions.rs index 3583b820f1204..01cbb0604d262 100644 --- a/src/tests/sqlsmith/src/sql_gen/functions.rs +++ b/src/tests/sqlsmith/src/sql_gen/functions.rs @@ -49,6 +49,7 @@ impl<'a, R: Rng> SqlGenerator<'a, R> { 4 => self.gen_overlay(context), _ => unreachable!(), }, + T::Bytea => self.gen_decode(context), _ => match self.rng.gen_bool(0.5) { true => self.gen_case(ret, context), false => self.gen_coalesce(ret, context), @@ -121,6 +122,16 @@ impl<'a, R: Rng> SqlGenerator<'a, R> { .collect() } + fn gen_decode(&mut self, context: SqlGeneratorContext) -> Expr { + let input_string = self.gen_expr(&DataType::Varchar, context); + let encoding = &["base64", "hex", "escape"].choose(&mut self.rng).unwrap(); + let args = vec![ + input_string, + Expr::Value(Value::SingleQuotedString(encoding.to_string())), + ]; + Expr::Function(make_simple_func("decode", &args)) + } + fn gen_fixed_func(&mut self, ret: &DataType, context: SqlGeneratorContext) -> Expr { let funcs = match FUNC_TABLE.get(ret) { None => return self.gen_simple_scalar(ret), diff --git a/src/tests/sqlsmith/src/sql_gen/types.rs b/src/tests/sqlsmith/src/sql_gen/types.rs index ea3c00e45e1da..06d170e604ace 100644 --- a/src/tests/sqlsmith/src/sql_gen/types.rs +++ b/src/tests/sqlsmith/src/sql_gen/types.rs @@ -109,7 +109,10 @@ impl TryFrom for CastSig { /// effectiveness, e.g. cause it to crash. static FUNC_BAN_LIST: LazyLock> = LazyLock::new(|| { [ - ExprType::Repeat, // FIXME: https://github.com/risingwavelabs/risingwave/issues/8003 + // FIXME: https://github.com/risingwavelabs/risingwave/issues/8003 + ExprType::Repeat, + // The format argument needs to be handled specially. It is still generated in `gen_special_func`. + ExprType::Decode, ] .into_iter() .collect() @@ -117,6 +120,9 @@ static FUNC_BAN_LIST: LazyLock> = LazyLock::new(|| { /// Table which maps functions' return types to possible function signatures. // ENABLE: https://github.com/risingwavelabs/risingwave/issues/5826 +// TODO: Create a `SPECIAL_FUNC` table. +// Otherwise when we dump the function table, we won't include those functions in +// gen_special_func. pub(crate) static FUNC_TABLE: LazyLock>> = LazyLock::new(|| { let mut funcs = HashMap::>::new(); diff --git a/src/workspace-hack/Cargo.toml b/src/workspace-hack/Cargo.toml index c5fe625a0c65f..15d296b56b5b3 100644 --- a/src/workspace-hack/Cargo.toml +++ b/src/workspace-hack/Cargo.toml @@ -68,7 +68,7 @@ lexical-write-integer = { version = "0.8", default-features = false, features = libc = { version = "0.2", features = ["extra_traits"] } lock_api = { version = "0.4", features = ["arc_lock"] } log = { version = "0.4", default-features = false, features = ["kv_unstable", "std"] } -madsim-rdkafka = { git = "https://github.com/madsim-rs/madsim.git", rev = "fedb1e3", features = ["cmake-build", "gssapi", "ssl-vendored", "zstd"] } +madsim-rdkafka = { version = "0.3", features = ["cmake-build", "gssapi", "ssl-vendored", "zstd"] } madsim-tokio = { version = "0.2", default-features = false, features = ["fs", "io-util", "macros", "net", "process", "rt", "rt-multi-thread", "signal", "sync", "time", "tracing"] } md-5 = { version = "0.10" } mio = { version = "0.8", features = ["net", "os-ext"] }