From 2064b5b145cda3c0b04061a3e819bf68b32d0333 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Fri, 22 Nov 2024 17:28:10 +0800 Subject: [PATCH 001/163] fix: memory allocation of standalone mode (#19477) --- docker/README.md | 100 +++++++++--------- docker/docker-compose-with-azblob.yml | 13 ++- docker/docker-compose-with-gcs.yml | 13 ++- docker/docker-compose-with-local-fs.yml | 12 ++- docker/docker-compose-with-obs.yml | 13 ++- docker/docker-compose-with-oss.yml | 13 ++- docker/docker-compose-with-s3.yml | 13 ++- docker/docker-compose-with-sqlite.yml | 13 ++- docker/docker-compose.yml | 13 ++- src/cmd_all/src/standalone.rs | 135 +++++++----------------- src/compute/src/lib.rs | 7 ++ src/compute/src/memory/controller.rs | 6 +- src/compute/src/memory/manager.rs | 4 +- src/compute/src/server.rs | 18 ++-- src/frontend/src/lib.rs | 2 +- src/frontend/src/session.rs | 12 ++- 16 files changed, 181 insertions(+), 206 deletions(-) diff --git a/docker/README.md b/docker/README.md index 865f7c64c5183..13782a0447bb8 100644 --- a/docker/README.md +++ b/docker/README.md @@ -1,43 +1,21 @@ -# Docker Images +# RisingWave Docker Images -## Published images +## Run RisingWave with Docker Compose -- `latest` on GHCR (latest nightly build): `ghcr.io/risingwavelabs/risingwave:latest` -- `latest` on Docker Hub (latest release): `risingwavelabs/risingwave:latest` -- Other tags available on both GHCR and Docker Hub: - - `nightly-yyyyMMdd`, e.g., `nightly-20230108` - - `vX.Y.Z`, e.g., `v0.1.15` +Docker Compose allows you to easily launch a RisingWave instance on a single node. If you are using more than one server, please refer to [Deploy RisingWave on Kubernetes](https://docs.risingwave.com/deploy/risingwave-kubernetes). -## Build the images -The docker images for x86_64 are built with AVX2 SIMD extensions, while the images for aarch64 are built with NEON SIMD extensions. These must be available on your machine. If your machine does not support these extensions, you must build the docker image with the build-arg `simd_disabled=true`. - -To build the images, simply run: - -``` -docker build . -f docker/Dockerfile -``` - -from the project root. - -To build the images without SIMD vector extensions, run - -``` -docker build . -f docker/Dockerfile --build-arg simd_disabled=true -``` - -from the project root and run any subsequent docker commands on the resultant image. - -## Use the images - -To ensure you are using the latest version of RisingWave image, +Ensure you are using the latest version of RisingWave image: ``` # Ensure risingwave image is of latest version docker pull ghcr.io/risingwavelabs/risingwave:latest ``` -### playground +### Playground + +Playground mode does not persist any data. **Never** use it for production purpose. + To start a RisingWave playground, run ``` @@ -45,25 +23,27 @@ To start a RisingWave playground, run docker run -it --pull=always -p 4566:4566 -p 5691:5691 ghcr.io/risingwavelabs/risingwave:latest playground ``` -### standalone minio -To start a RisingWave standalone mode with minio backend, run +### Standalone (MinIO backend) + +To start a RisingWave standalone instance with MinIO backend, run ``` # Start all components docker-compose up ``` -### distributed cluster minio -To start a RisingWave cluster with minio backend, run +**⚠️ Important Notes: Memory is crucial for RisingWave!** Inappropriate memory configuration may lead to OOM (out-of-memory) errors or poor performance. Before deploying Docker Compose, ensure that the correct memory settings are configured in the `docker-compose.yaml` file. Here are examples of some typical settings. + +| Memory for RW container (`resource.limits.memory`) | 8 GiB | 14 GiB | 28 GiB | 58 GiB | +|----------------------------------------------------|---|---|---|---| +| `compute-opts.total-memory-bytes` | 6 GiB | 10 GiB | 20 GiB | 46 GiB | +| `frontend-opts.frontend-total-memory-bytes` | 1 GiB | 2 GiB | 4 GiB | 6 GiB | +| `compactor-opts.compactor-total-memory-bytes` | 1 GiB | 2 GiB | 4 GiB | 6 GiB | +| `compute-opts.memory-manager-target-bytes` | 5.6 GiB | 9.8 GiB | 20.8 GiB | 44.8 GiB | -``` -# Start all components -docker-compose -f docker-compose-distributed.yml up -``` -It will start a minio, a meta node, a compute node, a frontend, a compactor, a prometheus and a redpanda instance. +### Standalone (S3 backend) -### s3 and other s3-compatible storage backend To start a RisingWave cluster with s3 backend, configure the aws credit in [aws.env](https://github.com/risingwavelabs/risingwave/blob/main/docker/aws.env). If you want to use some s3 compatible storage like Tencent Cloud COS, just configure one more [endpoint](https://github.com/risingwavelabs/risingwave/blob/a2684461e379ce73f8d730982147439e2379de16/docker/aws.env#L7). After configuring the environment and fill in your [bucket name](https://github.com/risingwavelabs/risingwave/blob/a2684461e379ce73f8d730982147439e2379de16/docker/docker-compose-with-s3.yml#L196), run @@ -75,7 +55,8 @@ docker-compose -f docker-compose-with-s3.yml up It will run with s3 (compatible) object storage with a meta node, a compute node, a frontend, a compactor, a prometheus and a redpanda instance. -### Start with other storage products of public cloud vendors +### Standalone (other backend) + To start a RisingWave cluster with other storage backend, like Google Cloud Storage, Alicloud OSS or Azure Blob Storage, configure the authentication information in [multiple_object_storage.env](https://github.com/risingwavelabs/risingwave/blob/main/docker/multiple_object_storage.env), fill in your [bucket name](https://github.com/risingwavelabs/risingwave/blob/a2684461e379ce73f8d730982147439e2379de16/docker/docker-compose-with-gcs.yml#L196). and run @@ -86,27 +67,39 @@ docker-compose -f docker-compose-with-xxx.yml up It will run RisingWave with corresponding (object) storage products. -### Start with HDFS backend -To start a RisingWave cluster with HDFS, mount your `HADDOP_HOME` in [compactor node volumes](https://github.com/risingwavelabs/risingwave/blob/a2684461e379ce73f8d730982147439e2379de16/docker/docker-compose-with-hdfs.yml#L28), [compute node volumes](https://github.com/risingwavelabs/risingwave/blob/a2684461e379ce73f8d730982147439e2379de16/docker/docker-compose-with-hdfs.yml#L112) [compute node volumes](https://github.com/risingwavelabs/risingwave/blob/a2684461e379ce73f8d730982147439e2379de16/docker/docker-compose-with-hdfs.yml#L218), fill in the [cluster_name/namenode](https://github.com/risingwavelabs/risingwave/blob/a2684461e379ce73f8d730982147439e2379de16/docker/docker-compose-with-hdfs.yml#L202), -and run +> [!NOTE] +> +> For RisingWave kernel hackers, we always recommend using [risedev](../src/risedevtool/README.md) to start the full cluster, instead of using docker images. +> See [CONTRIBUTING](../CONTRIBUTING.md) for more information. + + +## Published images + +- `latest` on GHCR (latest nightly build): `ghcr.io/risingwavelabs/risingwave:latest` +- `latest` on Docker Hub (latest release): `risingwavelabs/risingwave:latest` +- Other tags available on both GHCR and Docker Hub: + - `nightly-yyyyMMdd`, e.g., `nightly-20230108` + - `vX.Y.Z`, e.g., `v0.1.15` + +## Build the images + +The docker images for x86_64 are built with AVX2 SIMD extensions, while the images for aarch64 are built with NEON SIMD extensions. These must be available on your machine. If your machine does not support these extensions, you must build the docker image with the build-arg `simd_disabled=true`. + +To build the images, simply run: ``` -# Start all components -docker-compose -f docker-compose-with-hdfs.yml up +docker build . -f docker/Dockerfile ``` -It will run RisingWave with HDFS. +from the project root. -To clean all data, run: +To build the images without SIMD vector extensions, run ``` -docker-compose down -v +docker build . -f docker/Dockerfile --build-arg simd_disabled=true ``` -> [!NOTE] -> -> For RisingWave kernel hackers, we always recommend using [risedev](../src/risedevtool/README.md) to start the full cluster, instead of using docker images. -> See [CONTRIBUTING](../CONTRIBUTING.md) for more information. +from the project root and run any subsequent docker commands on the resultant image. ## Generate docker-compose.yml @@ -122,4 +115,5 @@ Error { code: "XMinioStorageFull", message: "Storage backend has reached its min ``` Solution: + This usually happens on MacOS with Docker Desktop. The Docker Deskup runs in the macOS Hypervisor. All the data, including logs, images, volumes, and so on, is stored in this hypervisor and the hypervisor has a default disk capacity limit. So when this message emerges, simply cleaning up the unused container or image can help mitigate. You can also increase capacity limit by clicking the Docker Desktop icon in the menu bar, then clicking Preferences > Resources > `Increase Disk image size`. diff --git a/docker/docker-compose-with-azblob.yml b/docker/docker-compose-with-azblob.yml index 99889d846dcb1..d479f9464b5fe 100644 --- a/docker/docker-compose-with-azblob.yml +++ b/docker/docker-compose-with-azblob.yml @@ -21,22 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - #--parallelism 4 \ - #--total-memory-bytes 8589934592 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/docker/docker-compose-with-gcs.yml b/docker/docker-compose-with-gcs.yml index 80466c7cccaba..6380b91a01a95 100644 --- a/docker/docker-compose-with-gcs.yml +++ b/docker/docker-compose-with-gcs.yml @@ -21,22 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - #--parallelism 4 \ - #--total-memory-bytes 8589934592 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/docker/docker-compose-with-local-fs.yml b/docker/docker-compose-with-local-fs.yml index 68483796ac800..abfea87b5a68e 100644 --- a/docker/docker-compose-with-local-fs.yml +++ b/docker/docker-compose-with-local-fs.yml @@ -21,21 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - # --parallelism 4 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/docker/docker-compose-with-obs.yml b/docker/docker-compose-with-obs.yml index f4bf8dc0e74c0..690da92d8b966 100644 --- a/docker/docker-compose-with-obs.yml +++ b/docker/docker-compose-with-obs.yml @@ -21,22 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - #--parallelism 4 \ - #--total-memory-bytes 8589934592 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/docker/docker-compose-with-oss.yml b/docker/docker-compose-with-oss.yml index 7d9563473182a..dab8af1592993 100644 --- a/docker/docker-compose-with-oss.yml +++ b/docker/docker-compose-with-oss.yml @@ -21,22 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - #--parallelism 4 \ - #--total-memory-bytes 8589934592 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/docker/docker-compose-with-s3.yml b/docker/docker-compose-with-s3.yml index d7dc75aa556a6..c9d839220c943 100644 --- a/docker/docker-compose-with-s3.yml +++ b/docker/docker-compose-with-s3.yml @@ -21,22 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - #--parallelism 4 \ - #--total-memory-bytes 8589934592 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/docker/docker-compose-with-sqlite.yml b/docker/docker-compose-with-sqlite.yml index d4081b592c2ab..98d88a415d496 100644 --- a/docker/docker-compose-with-sqlite.yml +++ b/docker/docker-compose-with-sqlite.yml @@ -21,22 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - #--parallelism 4 \ - #--total-memory-bytes 8589934592 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/docker/docker-compose.yml b/docker/docker-compose.yml index e315878c98b77..b7b29313547b7 100644 --- a/docker/docker-compose.yml +++ b/docker/docker-compose.yml @@ -21,22 +21,25 @@ services: --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:5688 \ --async-stack-trace verbose \ - #--parallelism 4 \ - #--total-memory-bytes 8589934592 \ + --parallelism 8 \ + --total-memory-bytes 21474836480 \ --role both \ - --meta-address http://0.0.0.0:5690\" \ + --meta-address http://0.0.0.0:5690 \ + --memory-manager-target-bytes 22333829939 \" \ --frontend-opts=\" \ --config-path /risingwave.toml \ --listen-addr 0.0.0.0:4566 \ --advertise-addr 0.0.0.0:4566 \ --prometheus-listener-addr 0.0.0.0:1250 \ --health-check-listener-addr 0.0.0.0:6786 \ - --meta-addr http://0.0.0.0:5690\" \ + --meta-addr http://0.0.0.0:5690 \ + --frontend-total-memory-bytes=4294967296\" \ --compactor-opts=\" \ --listen-addr 0.0.0.0:6660 \ --prometheus-listener-addr 0.0.0.0:1250 \ --advertise-addr 0.0.0.0:6660 \ - --meta-address http://0.0.0.0:5690\"" + --meta-address http://0.0.0.0:5690 \ + --compactor-total-memory-bytes=4294967296\"" expose: - "6660" - "4566" diff --git a/src/cmd_all/src/standalone.rs b/src/cmd_all/src/standalone.rs index 6eb62492999a0..b5730d8d7845a 100644 --- a/src/cmd_all/src/standalone.rs +++ b/src/cmd_all/src/standalone.rs @@ -391,17 +391,9 @@ It SHOULD NEVER be used in benchmarks and production environment!!!" #[cfg(test)] mod test { - use std::fmt::Debug; - - use expect_test::{expect, Expect}; - use super::*; - fn check(actual: impl Debug, expect: Expect) { - let actual = format!("{:#?}", actual); - expect.assert_eq(&actual); - } - + #[allow(clippy::assertions_on_constants)] #[test] fn test_parse_opt_args() { // Test parsing into standalone-level opts. @@ -426,93 +418,42 @@ mod test { // Test parsing into node-level opts. let actual = parse_standalone_opt_args(&opts); - check( - actual, - expect![[r#" - ParsedStandaloneOpts { - meta_opts: Some( - MetaNodeOpts { - listen_addr: "127.0.0.1:8001", - advertise_addr: "127.0.0.1:9999", - dashboard_host: None, - prometheus_listener_addr: Some( - "127.0.0.1:1234", - ), - sql_endpoint: None, - sql_username: "", - sql_password: [REDACTED alloc::string::String], - sql_database: "", - prometheus_endpoint: None, - prometheus_selector: None, - privatelink_endpoint_default_tags: None, - vpc_id: None, - security_group_id: None, - config_path: "src/config/test.toml", - backend: None, - barrier_interval_ms: None, - sstable_size_mb: None, - block_size_kb: None, - bloom_false_positive: None, - state_store: None, - data_directory: Some( - "some path with spaces", - ), - do_not_config_object_storage_lifecycle: None, - backup_storage_url: None, - backup_storage_directory: None, - heap_profiling_dir: None, - dangerous_max_idle_secs: None, - connector_rpc_endpoint: None, - license_key: None, - license_key_path: None, - temp_secret_file_dir: "./meta/secrets/", - }, - ), - compute_opts: Some( - ComputeNodeOpts { - listen_addr: "127.0.0.1:8000", - advertise_addr: None, - prometheus_listener_addr: "127.0.0.1:1234", - meta_address: List( - [ - http://127.0.0.1:5690/, - ], - ), - config_path: "src/config/test.toml", - total_memory_bytes: 34359738368, - reserved_memory_bytes: None, - parallelism: 10, - role: Both, - metrics_level: None, - data_file_cache_dir: None, - meta_file_cache_dir: None, - async_stack_trace: None, - heap_profiling_dir: None, - connector_rpc_endpoint: None, - temp_secret_file_dir: "./compute/secrets/", - }, - ), - frontend_opts: Some( - FrontendOpts { - listen_addr: "0.0.0.0:4566", - tcp_keepalive_idle_secs: 300, - advertise_addr: None, - meta_addr: List( - [ - http://127.0.0.1:5690/, - ], - ), - prometheus_listener_addr: "127.0.0.1:1234", - frontend_rpc_listener_addr: "127.0.0.1:6786", - config_path: "src/config/test.toml", - metrics_level: None, - enable_barrier_read: None, - temp_secret_file_dir: "./frontend/secrets/", - frontend_total_memory_bytes: 34359738368, - }, - ), - compactor_opts: None, - }"#]], - ); + if let Some(compute_opts) = &actual.compute_opts { + assert_eq!(compute_opts.listen_addr, "127.0.0.1:8000"); + assert_eq!(compute_opts.total_memory_bytes, 34359738368); + assert_eq!(compute_opts.parallelism, 10); + assert_eq!(compute_opts.temp_secret_file_dir, "./compute/secrets/"); + assert_eq!(compute_opts.prometheus_listener_addr, "127.0.0.1:1234"); + assert_eq!(compute_opts.config_path, "src/config/test.toml"); + } else { + assert!(false); + } + if let Some(meta_opts) = &actual.meta_opts { + assert_eq!(meta_opts.listen_addr, "127.0.0.1:8001"); + assert_eq!(meta_opts.advertise_addr, "127.0.0.1:9999"); + assert_eq!( + meta_opts.data_directory, + Some("some path with spaces".to_string()) + ); + assert_eq!(meta_opts.temp_secret_file_dir, "./meta/secrets/"); + assert_eq!( + meta_opts.prometheus_listener_addr, + Some("127.0.0.1:1234".to_string()) + ); + assert_eq!(meta_opts.config_path, "src/config/test.toml"); + } else { + assert!(false); + } + + if let Some(frontend_opts) = &actual.frontend_opts { + assert_eq!(frontend_opts.config_path, "src/config/test.toml"); + assert_eq!(frontend_opts.temp_secret_file_dir, "./frontend/secrets/"); + assert_eq!(frontend_opts.frontend_total_memory_bytes, 34359738368); + assert_eq!(frontend_opts.prometheus_listener_addr, "127.0.0.1:1234"); + } else { + assert!(false); + } + + assert!(actual.compactor_opts.is_none()); } } diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index 1336a84980cea..ef4b5c5e32d3b 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -92,6 +92,13 @@ pub struct ComputeNodeOpts { #[clap(long, env = "RW_RESERVED_MEMORY_BYTES")] pub reserved_memory_bytes: Option, + /// Target memory usage for Memory Manager. + /// If not set, the default value is `total_memory_bytes` - `reserved_memory_bytes` + /// + /// It's strongly recommended to set it for standalone deployment. + #[clap(long, env = "RW_MEMORY_MANAGER_TARGET_BYTES")] + pub memory_manager_target_bytes: Option, + /// The parallelism that the compute node will register to the scheduler of the meta service. #[clap(long, env = "RW_PARALLELISM", default_value_t = default_parallelism())] #[override_opts(if_absent, path = streaming.actor_runtime_worker_threads_num)] diff --git a/src/compute/src/memory/controller.rs b/src/compute/src/memory/controller.rs index 2857b0e6b0d66..878471a7bedb2 100644 --- a/src/compute/src/memory/controller.rs +++ b/src/compute/src/memory/controller.rs @@ -85,10 +85,10 @@ pub struct LruWatermarkController { impl LruWatermarkController { pub fn new(config: &MemoryManagerConfig) -> Self { - let threshold_stable = (config.total_memory as f64 * config.threshold_stable) as usize; - let threshold_graceful = (config.total_memory as f64 * config.threshold_graceful) as usize; + let threshold_stable = (config.target_memory as f64 * config.threshold_stable) as usize; + let threshold_graceful = (config.target_memory as f64 * config.threshold_graceful) as usize; let threshold_aggressive = - (config.total_memory as f64 * config.threshold_aggressive) as usize; + (config.target_memory as f64 * config.threshold_aggressive) as usize; Self { metrics: config.metrics.clone(), diff --git a/src/compute/src/memory/manager.rs b/src/compute/src/memory/manager.rs index 235ab5802fbfd..5384838eec378 100644 --- a/src/compute/src/memory/manager.rs +++ b/src/compute/src/memory/manager.rs @@ -22,7 +22,9 @@ use risingwave_stream::executor::monitor::StreamingMetrics; use super::controller::LruWatermarkController; pub struct MemoryManagerConfig { - pub total_memory: usize, + /// [`MemoryManager`] will try to control the jemalloc-reported memory usage + /// to be lower than this + pub target_memory: usize, pub threshold_aggressive: f64, pub threshold_graceful: f64, diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index eda0ee9c159da..d86a516771802 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -288,15 +288,14 @@ pub async fn compute_node_serve( batch_mem_limit(compute_memory_bytes, opts.role.for_serving()), )); - // NOTE: Due to some limits, we use `compute_memory_bytes + storage_memory_bytes` as - // `total_compute_memory_bytes` for memory control. This is just a workaround for some - // memory control issues and should be modified as soon as we figure out a better solution. - // - // Related issues: - // - https://github.com/risingwavelabs/risingwave/issues/8696 - // - https://github.com/risingwavelabs/risingwave/issues/8822 + let target_memory = if let Some(v) = opts.memory_manager_target_bytes { + v + } else { + compute_memory_bytes + storage_memory_bytes + }; + let memory_mgr = MemoryManager::new(MemoryManagerConfig { - total_memory: compute_memory_bytes + storage_memory_bytes, + target_memory, threshold_aggressive: config .streaming .developer @@ -546,8 +545,7 @@ fn print_memory_config( reserved_memory_bytes: usize, ) { let memory_config = format!( - "\n\ - Memory outline:\n\ + "Memory outline:\n\ > total_memory: {}\n\ > storage_memory: {}\n\ > block_cache_capacity: {}\n\ diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index bf03edb6a87dd..6424da42a1510 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -162,7 +162,7 @@ pub struct FrontendOpts { )] pub temp_secret_file_dir: String, - /// Total available memory for the frontend node in bytes. Used by both computing and storage. + /// Total available memory for the frontend node in bytes. Used for batch computing. #[clap(long, env = "RW_FRONTEND_TOTAL_MEMORY_BYTES", default_value_t = default_frontend_total_memory_bytes())] pub frontend_total_memory_bytes: usize, } diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index fa4836e73e958..a920dc32653f7 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -62,6 +62,7 @@ use risingwave_common::util::addr::HostAddr; use risingwave_common::util::cluster_limit; use risingwave_common::util::cluster_limit::ActorCountPerParallelism; use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::pretty_bytes::convert; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; @@ -450,9 +451,16 @@ impl FrontendEnv { // Run a background heap profiler heap_profiler.start(); - let mem_context = risingwave_common::memory::MemoryContext::root( + let batch_memory_limit = total_memory_bytes as f64 * FRONTEND_BATCH_MEMORY_PROPORTION; + let mem_context = MemoryContext::root( frontend_metrics.batch_total_mem.clone(), - (total_memory_bytes as f64 * FRONTEND_BATCH_MEMORY_PROPORTION) as u64, + batch_memory_limit as u64, + ); + + info!( + "Frontend total_memory: {} batch_memory: {}", + convert(total_memory_bytes as _), + convert(batch_memory_limit as _), ); Ok(( From fd5d0912fdebd031873651041866df6a2b8651b6 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Fri, 22 Nov 2024 17:37:42 +0800 Subject: [PATCH 002/163] fix: when target is table, the sink downstream pk is not set (#19515) --- e2e_test/sink/sink_into_table/parallelism.slt | 6 +- src/frontend/planner_test/src/lib.rs | 1 + .../tests/testdata/input/sink_into_table.yaml | 12 ++ .../testdata/output/emit_on_window_close.yaml | 2 +- .../tests/testdata/output/sink.yaml | 18 +- .../testdata/output/sink_into_table.yaml | 17 ++ src/frontend/src/handler/create_sink.rs | 23 +- src/frontend/src/optimizer/mod.rs | 47 +++- .../src/optimizer/plan_node/stream_sink.rs | 204 +++++++++--------- 9 files changed, 193 insertions(+), 137 deletions(-) diff --git a/e2e_test/sink/sink_into_table/parallelism.slt b/e2e_test/sink/sink_into_table/parallelism.slt index 8359d2731d196..d5bbecf10f01a 100644 --- a/e2e_test/sink/sink_into_table/parallelism.slt +++ b/e2e_test/sink/sink_into_table/parallelism.slt @@ -8,16 +8,16 @@ statement ok SET STREAMING_PARALLELISM TO 2; statement ok -create table t_simple (v1 int, v2 int); +create table t_simple (v1 int, v2 int) append only; statement ok -create table m_simple (v1 int primary key, v2 int); +create table m_simple (v1 int, v2 int) append only; statement ok SET STREAMING_PARALLELISM TO 3; statement ok -create sink s_simple_1 into m_simple as select v1, v2 from t_simple; +create sink s_simple_1 into m_simple as select v1, v2 from t_simple with (type = 'append-only'); query I select distinct parallelism from rw_fragment_parallelism where name in ('t_simple', 'm_simple', 's_simple_1'); diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 528fa88ef3506..0efa5f66865e8 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -850,6 +850,7 @@ impl TestCase { false, None, None, + false, ) { Ok(sink_plan) => { ret.sink_plan = Some(explain_plan(&sink_plan.into())); diff --git a/src/frontend/planner_test/tests/testdata/input/sink_into_table.yaml b/src/frontend/planner_test/tests/testdata/input/sink_into_table.yaml index 1191cd6a68966..24b3df8902faf 100644 --- a/src/frontend/planner_test/tests/testdata/input/sink_into_table.yaml +++ b/src/frontend/planner_test/tests/testdata/input/sink_into_table.yaml @@ -9,3 +9,15 @@ explain create sink ss into t from s with (type = 'append-only'); expected_outputs: - explain_output +- sql: | + create table t1 (a int primary key, b int); + create table t2 (a int, b int primary key); + explain create sink s into t1 from t2; + expected_outputs: + - explain_output +- sql: | + create table t1 (a int primary key, b int); + create table t2 (a int, b int primary key); + explain create sink s into t1 as select b from t2; + expected_outputs: + - explain_output diff --git a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml index faaae7b4c9895..22d4fbf9134ad 100644 --- a/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml +++ b/src/frontend/planner_test/tests/testdata/output/emit_on_window_close.yaml @@ -206,7 +206,7 @@ emit on window close WITH (connector = 'blackhole'); explain_output: | - StreamSink { type: upsert, columns: [tm, foo, bar, t._row_id(hidden), lag, max, sum], pk: [t._row_id, t.bar] } + StreamSink { type: upsert, columns: [tm, foo, bar, t._row_id(hidden), lag, max, sum], downstream_pk: [] } └─StreamEowcOverWindow { window_functions: [first_value(t.foo) OVER(PARTITION BY t.bar ORDER BY t.tm ASC ROWS BETWEEN 2 PRECEDING AND 2 PRECEDING), max(t.foo) OVER(PARTITION BY t.bar ORDER BY t.tm ASC ROWS BETWEEN 1 PRECEDING AND 1 FOLLOWING), sum(t.foo) OVER(PARTITION BY t.bar ORDER BY t.tm ASC ROWS BETWEEN 2 PRECEDING AND CURRENT ROW EXCLUDE CURRENT ROW)] } └─StreamEowcSort { sort_column: t.tm } └─StreamExchange { dist: HashShard(t.bar) } diff --git a/src/frontend/planner_test/tests/testdata/output/sink.yaml b/src/frontend/planner_test/tests/testdata/output/sink.yaml index 12ba038043062..db5d533223cb4 100644 --- a/src/frontend/planner_test/tests/testdata/output/sink.yaml +++ b/src/frontend/planner_test/tests/testdata/output/sink.yaml @@ -9,7 +9,7 @@ table.name='t1sink', type='upsert'); explain_output: | - StreamSink { type: upsert, columns: [v1, v2, v3, v5, t1.v4(hidden)], pk: [t1.v3, t1.v4] } + StreamSink { type: upsert, columns: [v1, v2, v3, v5, t1.v4(hidden)], downstream_pk: [t1.v1, t1.v2] } └─StreamExchange { dist: HashShard(t1.v1, t1.v2) } └─StreamTableScan { table: t1, columns: [v1, v2, v3, v5, v4] } - id: create_upsert_jdbc_sink_with_downstream_pk2 @@ -22,7 +22,7 @@ table.name='t1sink', type='upsert'); explain_output: | - StreamSink { type: upsert, columns: [v1, v2, v3, v5], pk: [t1.v1, t1.v2] } + StreamSink { type: upsert, columns: [v1, v2, v3, v5], downstream_pk: [t1.v3, t1.v5] } └─StreamExchange { dist: HashShard(t1.v3, t1.v5) } └─StreamTableScan { table: t1, columns: [v1, v2, v3, v5] } - id: create_upsert_jdbc_sink_with_downstream_pk1 @@ -36,7 +36,7 @@ type='upsert'); explain_output: |+ Fragment 0 - StreamSink { type: upsert, columns: [v1, v2, v3, v5, t1.v4(hidden)], pk: [t1.v3, t1.v4] } + StreamSink { type: upsert, columns: [v1, v2, v3, v5, t1.v4(hidden)], downstream_pk: [t1.v1, t1.v2] } ├── tables: [ Sink: 0 ] ├── output: [ t1.v1, t1.v2, t1.v3, t1.v5, t1.v4 ] ├── stream key: [ t1.v3, t1.v4 ] @@ -89,7 +89,7 @@ type='upsert'); explain_output: |+ Fragment 0 - StreamSink { type: upsert, columns: [v1, v2, v3, v5], pk: [t1.v1, t1.v2] } + StreamSink { type: upsert, columns: [v1, v2, v3, v5], downstream_pk: [t1.v3, t1.v5] } ├── tables: [ Sink: 0 ] ├── output: [ t1.v1, t1.v2, t1.v3, t1.v5 ] ├── stream key: [ t1.v1, t1.v2 ] @@ -152,7 +152,7 @@ primary_key='v1,v2' ); explain_output: | - StreamSink { type: upsert, columns: [v1, v2, v3, v5, t1.v4(hidden)], pk: [t1.v3, t1.v4] } + StreamSink { type: upsert, columns: [v1, v2, v3, v5, t1.v4(hidden)], downstream_pk: [t1.v1, t1.v2] } └─StreamExchange { dist: HashShard(t1.v1, t1.v2) } └─StreamTableScan { table: t1, columns: [v1, v2, v3, v5, v4] } - id: downstream_pk_same_with_upstream @@ -165,7 +165,7 @@ primary_key='v2,v1' ); explain_output: | - StreamSink { type: upsert, columns: [v2, v1, count], pk: [t1.v1, t1.v2] } + StreamSink { type: upsert, columns: [v2, v1, count], downstream_pk: [t1.v2, t1.v1] } └─StreamProject { exprs: [t1.v2, t1.v1, count] } └─StreamHashAgg { group_key: [t1.v1, t1.v2], aggs: [count] } └─StreamExchange { dist: HashShard(t1.v1, t1.v2) } @@ -175,7 +175,7 @@ create table t2 (a int, b int, watermark for b as b - 4) append only; explain create sink sk1 from t2 emit on window close with (connector='blackhole'); explain_output: | - StreamSink { type: upsert, columns: [a, b, t2._row_id(hidden)], pk: [t2._row_id] } + StreamSink { type: upsert, columns: [a, b, t2._row_id(hidden)], downstream_pk: [] } └─StreamEowcSort { sort_column: t2.b } └─StreamTableScan { table: t2, columns: [a, b, _row_id] } - id: create_mock_iceberg_sink_append_only_with_sparse_partition @@ -238,7 +238,7 @@ primary_key = 'v1' ); explain_output: | - StreamSink { type: upsert, columns: [v1, v2, v3, v4, t1._row_id(hidden)], pk: [t1._row_id] } + StreamSink { type: upsert, columns: [v1, v2, v3, v4, t1._row_id(hidden)], downstream_pk: [t1.v1] } └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [t1.v1, t1.v2, t1.v3, t1.v4, t1._row_id, Row(t1.v1, IcebergTransform('bucket[1]':Varchar, t1.v2), IcebergTransform('truncate[1]':Varchar, t1.v3), null:Int32) as $expr1] } └─StreamTableScan { table: t1, columns: [v1, v2, v3, v4, _row_id] } @@ -260,5 +260,5 @@ primary_key = 'v1' ); explain_output: | - StreamSink { type: upsert, columns: [v1, v2, v3, v4, t1._row_id(hidden)], pk: [t1._row_id] } + StreamSink { type: upsert, columns: [v1, v2, v3, v4, t1._row_id(hidden)], downstream_pk: [t1.v1] } └─StreamTableScan { table: t1, columns: [v1, v2, v3, v4, _row_id] } diff --git a/src/frontend/planner_test/tests/testdata/output/sink_into_table.yaml b/src/frontend/planner_test/tests/testdata/output/sink_into_table.yaml index 1fc6df6613a98..ac9485bc61abb 100644 --- a/src/frontend/planner_test/tests/testdata/output/sink_into_table.yaml +++ b/src/frontend/planner_test/tests/testdata/output/sink_into_table.yaml @@ -12,3 +12,20 @@ StreamProject { exprs: [s.x, Proctime as $expr1, (Proctime - '00:01:00':Interval) as $expr2, null:Serial], output_watermarks: [$expr1, $expr2] } └─StreamSink { type: append-only, columns: [x, s._row_id(hidden)] } └─StreamTableScan { table: s, columns: [x, _row_id] } +- sql: | + create table t1 (a int primary key, b int); + create table t2 (a int, b int primary key); + explain create sink s into t1 from t2; + explain_output: | + StreamProject { exprs: [t2.a, t2.b] } + └─StreamSink { type: upsert, columns: [a, b], downstream_pk: [t2.a] } + └─StreamExchange { dist: HashShard(t2.a) } + └─StreamTableScan { table: t2, columns: [a, b] } +- sql: | + create table t1 (a int primary key, b int); + create table t2 (a int, b int primary key); + explain create sink s into t1 as select b from t2; + explain_output: | + StreamProject { exprs: [t2.b, null:Int32] } + └─StreamSink { type: upsert, columns: [b], downstream_pk: [t2.b] } + └─StreamTableScan { table: t2, columns: [b] } diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index 7172e2ba7220b..cfcddc8b8bb4a 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -26,7 +26,7 @@ use risingwave_common::bail; use risingwave_common::catalog::{ColumnCatalog, DatabaseId, ObjectId, Schema, SchemaId, UserId}; use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::DataType; -use risingwave_connector::sink::catalog::{SinkCatalog, SinkFormatDesc, SinkType}; +use risingwave_connector::sink::catalog::{SinkCatalog, SinkFormatDesc}; use risingwave_connector::sink::iceberg::{IcebergConfig, ICEBERG_SINK}; use risingwave_connector::sink::kafka::KAFKA_SINK; use risingwave_connector::sink::{ @@ -229,8 +229,6 @@ pub async fn gen_sink_plan( } } - let target_table = target_table_catalog.as_ref().map(|catalog| catalog.id()); - let sink_plan = plan_root.gen_sink_plan( sink_table_name, definition, @@ -240,8 +238,9 @@ pub async fn gen_sink_plan( sink_from_table_name, format_desc, without_backfill, - target_table, + target_table_catalog.clone(), partition_info, + user_specified_columns, )?; let sink_desc = sink_plan.sink_desc().clone(); @@ -282,22 +281,6 @@ pub async fn gen_sink_plan( } } - let user_defined_primary_key_table = table_catalog.row_id_index.is_none(); - let sink_is_append_only = sink_catalog.sink_type == SinkType::AppendOnly - || sink_catalog.sink_type == SinkType::ForceAppendOnly; - - if !user_defined_primary_key_table && !sink_is_append_only { - return Err(RwError::from(ErrorCode::BindError( - "Only append-only sinks can sink to a table without primary keys. please try to add type = 'append-only' in the with option. e.g. create sink s into t as select * from t1 with (type = 'append-only')".to_string(), - ))); - } - - if table_catalog.append_only && !sink_is_append_only { - return Err(RwError::from(ErrorCode::BindError( - "Only append-only sinks can sink to a append only table. please try to add type = 'append-only' in the with option. e.g. create sink s into t as select * from t1 with (type = 'append-only')".to_string(), - ))); - } - let table_columns_without_rw_timestamp = table_catalog.columns_without_rw_timestamp(); let exprs = derive_default_column_project_for_sink( &sink_catalog, diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 323e38f990321..9572a46d90959 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::num::NonZeroU32; use std::ops::DerefMut; +use std::sync::Arc; pub mod plan_node; @@ -41,7 +42,7 @@ mod plan_expr_visitor; mod rule; use std::assert_matches::assert_matches; -use std::collections::HashMap; +use std::collections::{BTreeMap, HashMap}; use fixedbitset::FixedBitSet; use itertools::Itertools as _; @@ -51,7 +52,7 @@ use plan_expr_rewriter::ConstEvalRewriter; use property::Order; use risingwave_common::bail; use risingwave_common::catalog::{ - ColumnCatalog, ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, TableId, + ColumnCatalog, ColumnDesc, ColumnId, ConflictBehavior, Field, Schema, }; use risingwave_common::types::DataType; use risingwave_common::util::column_index_mapping::ColIndexMapping; @@ -83,6 +84,7 @@ use crate::optimizer::plan_node::{ use crate::optimizer::plan_visitor::{RwTimestampValidator, TemporalJoinValidator}; use crate::optimizer::property::Distribution; use crate::utils::{ColIndexMappingRewriteExt, WithOptionsSecResolved}; +use crate::TableCatalog; /// `PlanRoot` is used to describe a plan. planner will construct a `PlanRoot` with `LogicalNode`. /// and required distribution and order. And `PlanRoot` can generate corresponding streaming or @@ -958,8 +960,9 @@ impl PlanRoot { sink_from_table_name: String, format_desc: Option, without_backfill: bool, - target_table: Option, + target_table: Option>, partition_info: Option, + user_specified_columns: bool, ) -> Result { let stream_scan_type = if without_backfill { StreamScanType::UpstreamOnly @@ -977,12 +980,17 @@ impl PlanRoot { self.gen_optimized_stream_plan_inner(emit_on_window_close, stream_scan_type)?; assert_eq!(self.phase, PlanPhase::Stream); assert_eq!(stream_plan.convention(), Convention::Stream); + let target_columns_to_plan_mapping = target_table.as_ref().map(|t| { + let columns = t.columns_without_rw_timestamp(); + self.target_columns_to_plan_mapping(&columns, user_specified_columns) + }); StreamSink::create( stream_plan, sink_name, db_name, sink_from_table_name, target_table, + target_columns_to_plan_mapping, self.required_dist.clone(), self.required_order.clone(), self.out_fields.clone(), @@ -1012,6 +1020,39 @@ impl PlanRoot { .config() .streaming_use_snapshot_backfill() } + + /// used when the plan has a target relation such as DML and sink into table, return the mapping from table's columns to the plan's schema + pub fn target_columns_to_plan_mapping( + &self, + tar_cols: &[ColumnCatalog], + user_specified_columns: bool, + ) -> Vec> { + #[allow(clippy::disallowed_methods)] + let visible_cols: Vec<(usize, String)> = self + .out_fields + .ones() + .zip_eq(self.out_names.iter().cloned()) + .collect_vec(); + + let visible_col_idxes = visible_cols.iter().map(|(i, _)| *i).collect_vec(); + let visible_col_idxes_by_name = visible_cols + .iter() + .map(|(i, name)| (name.as_ref(), *i)) + .collect::>(); + + tar_cols + .iter() + .enumerate() + .filter(|(_, tar_col)| tar_col.can_dml()) + .map(|(tar_i, tar_col)| { + if user_specified_columns { + visible_col_idxes_by_name.get(tar_col.name()).cloned() + } else { + (tar_i < visible_col_idxes.len()).then(|| visible_cols[tar_i].0) + } + }) + .collect() + } } fn find_version_column_index( diff --git a/src/frontend/src/optimizer/plan_node/stream_sink.rs b/src/frontend/src/optimizer/plan_node/stream_sink.rs index 48dc4dad85c5c..74ce8587ef6d4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_sink.rs +++ b/src/frontend/src/optimizer/plan_node/stream_sink.rs @@ -14,13 +14,14 @@ use std::assert_matches::assert_matches; use std::io::{Error, ErrorKind}; +use std::sync::Arc; use anyhow::anyhow; use fixedbitset::FixedBitSet; use icelake::types::Transform; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{ColumnCatalog, CreateType, TableId}; +use risingwave_common::catalog::{ColumnCatalog, CreateType}; use risingwave_common::types::{DataType, StructType}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::match_sink_name_str; @@ -43,7 +44,7 @@ use super::utils::{ childless_record, infer_kv_log_store_table_catalog_inner, Distill, IndicesDisplay, }; use super::{generic, ExprRewritable, PlanBase, PlanRef, StreamNode, StreamProject}; -use crate::error::{ErrorCode, Result}; +use crate::error::{ErrorCode, Result, RwError}; use crate::expr::{ExprImpl, FunctionCall, InputRef}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::plan_can_use_background_ddl; @@ -195,85 +196,6 @@ impl StreamSink { &self.sink_desc } - #[allow(clippy::too_many_arguments)] - pub fn create( - input: PlanRef, - name: String, - db_name: String, - sink_from_table_name: String, - target_table: Option, - user_distributed_by: RequiredDist, - user_order_by: Order, - user_cols: FixedBitSet, - out_names: Vec, - definition: String, - properties: WithOptionsSecResolved, - format_desc: Option, - partition_info: Option, - ) -> Result { - let columns = derive_columns(input.schema(), out_names, &user_cols)?; - let (input, mut sink) = Self::derive_sink_desc( - input, - user_distributed_by, - name, - db_name, - sink_from_table_name, - target_table, - user_order_by, - columns, - definition, - properties, - format_desc, - partition_info, - )?; - - let unsupported_sink = - |sink: &str| Err(SinkError::Config(anyhow!("unsupported sink type {}", sink))); - - // check and ensure that the sink connector is specified and supported - let sink_decouple = match sink.properties.get(CONNECTOR_TYPE_KEY) { - Some(connector) => { - match_sink_name_str!( - connector.to_lowercase().as_str(), - SinkType, - { - // the table sink is created by with properties - if connector == TABLE_SINK && sink.target_table.is_none() { - unsupported_sink(TABLE_SINK) - } else { - SinkType::set_default_commit_checkpoint_interval( - &mut sink, - &input.ctx().session_ctx().config().sink_decouple(), - )?; - SinkType::is_sink_decouple( - &input.ctx().session_ctx().config().sink_decouple(), - ) - } - }, - |other: &str| unsupported_sink(other) - )? - } - None => { - return Err( - SinkError::Config(anyhow!("connector not specified when create sink")).into(), - ); - } - }; - // For file sink, it must have sink_decouple turned on. - if !sink_decouple && sink.is_file_sink() { - return Err( - SinkError::Config(anyhow!("File sink can only be created with sink_decouple enabled. Please run `set sink_decouple = true` first.")).into(), - ); - } - let log_store_type = if sink_decouple { - SinkLogStoreType::KvLogStore - } else { - SinkLogStoreType::InMemoryLogStore - }; - - Ok(Self::new(input, sink, log_store_type)) - } - fn derive_iceberg_sink_distribution( input: PlanRef, partition_info: Option, @@ -308,27 +230,67 @@ impl StreamSink { } #[allow(clippy::too_many_arguments)] - fn derive_sink_desc( + pub fn create( mut input: PlanRef, - user_distributed_by: RequiredDist, name: String, db_name: String, - sink_from_name: String, - target_table: Option, + sink_from_table_name: String, + target_table: Option>, + target_table_mapping: Option>>, + user_distributed_by: RequiredDist, user_order_by: Order, - columns: Vec, + user_cols: FixedBitSet, + out_names: Vec, definition: String, properties: WithOptionsSecResolved, format_desc: Option, partition_info: Option, - ) -> Result<(PlanRef, SinkDesc)> { + ) -> Result { let sink_type = Self::derive_sink_type(input.append_only(), &properties, format_desc.as_ref())?; + + let columns = derive_columns(input.schema(), out_names, &user_cols)?; let (pk, _) = derive_pk(input.clone(), user_order_by, &columns); - let mut downstream_pk = - Self::parse_downstream_pk(&columns, properties.get(DOWNSTREAM_PK_KEY))?; + let mut downstream_pk = { + let from_properties = + Self::parse_downstream_pk(&columns, properties.get(DOWNSTREAM_PK_KEY))?; + if let Some(t) = &target_table { + let user_defined_primary_key_table = t.row_id_index.is_none(); + let sink_is_append_only = + sink_type == SinkType::AppendOnly || sink_type == SinkType::ForceAppendOnly; + + if !user_defined_primary_key_table && !sink_is_append_only { + return Err(RwError::from(ErrorCode::BindError( + "Only append-only sinks can sink to a table without primary keys. please try to add type = 'append-only' in the with option. e.g. create sink s into t as select * from t1 with (type = 'append-only')".to_string(), + ))); + } + + if t.append_only && !sink_is_append_only { + return Err(RwError::from(ErrorCode::BindError( + "Only append-only sinks can sink to a append only table. please try to add type = 'append-only' in the with option. e.g. create sink s into t as select * from t1 with (type = 'append-only')".to_string(), + ))); + } + + if sink_type != SinkType::Upsert { + vec![] + } else { + let target_table_mapping = target_table_mapping.unwrap(); + t.pk() + .iter() + .map(|c| { + target_table_mapping[c.column_index].ok_or( + ErrorCode::SinkError(Box::new(Error::new(ErrorKind::InvalidInput, + "When using non append only sink into table, the primary key of the table must be included in the sink result.".to_string() + ))).into())}) + .try_collect::<_, _, RwError>()? + } + } else { + from_properties + } + }; let mut extra_partition_col_idx = None; + let required_dist = match input.distribution() { Distribution::Single => RequiredDist::single(), _ => { @@ -392,11 +354,11 @@ impl StreamSink { CreateType::Foreground }; let (properties, secret_refs) = properties.into_parts(); - let sink_desc = SinkDesc { + let mut sink_desc = SinkDesc { id: SinkId::placeholder(), name, db_name, - sink_from_name, + sink_from_name: sink_from_table_name, definition, columns, plan_pk: pk, @@ -406,11 +368,56 @@ impl StreamSink { secret_refs, sink_type, format_desc, - target_table, + target_table: target_table.as_ref().map(|catalog| catalog.id()), extra_partition_col_idx, create_type, }; - Ok((input, sink_desc)) + + let unsupported_sink = + |sink: &str| Err(SinkError::Config(anyhow!("unsupported sink type {}", sink))); + + // check and ensure that the sink connector is specified and supported + let sink_decouple = match sink_desc.properties.get(CONNECTOR_TYPE_KEY) { + Some(connector) => { + match_sink_name_str!( + connector.to_lowercase().as_str(), + SinkType, + { + // the table sink is created by with properties + if connector == TABLE_SINK && sink_desc.target_table.is_none() { + unsupported_sink(TABLE_SINK) + } else { + SinkType::set_default_commit_checkpoint_interval( + &mut sink_desc, + &input.ctx().session_ctx().config().sink_decouple(), + )?; + SinkType::is_sink_decouple( + &input.ctx().session_ctx().config().sink_decouple(), + ) + } + }, + |other: &str| unsupported_sink(other) + )? + } + None => { + return Err( + SinkError::Config(anyhow!("connector not specified when create sink")).into(), + ); + } + }; + // For file sink, it must have sink_decouple turned on. + if !sink_decouple && sink_desc.is_file_sink() { + return Err( + SinkError::Config(anyhow!("File sink can only be created with sink_decouple enabled. Please run `set sink_decouple = true` first.")).into(), + ); + } + let log_store_type = if sink_decouple { + SinkLogStoreType::KvLogStore + } else { + SinkLogStoreType::InMemoryLogStore + }; + + Ok(Self::new(input, sink_desc, log_store_type)) } fn is_user_defined_append_only(properties: &WithOptionsSecResolved) -> Result { @@ -572,16 +579,11 @@ impl Distill for StreamSink { vec.push(("type", Pretty::from(sink_type))); vec.push(("columns", column_names)); if self.sink_desc.sink_type.is_upsert() { - let pk = IndicesDisplay { - indices: &self - .sink_desc - .plan_pk - .iter() - .map(|k| k.column_index) - .collect_vec(), + let sink_pk = IndicesDisplay { + indices: &self.sink_desc.downstream_pk.clone(), schema: self.base.schema(), }; - vec.push(("pk", pk.distill())); + vec.push(("downstream_pk", sink_pk.distill())); } childless_record("StreamSink", vec) } From bc06ffdb782ea5a9f11d19825f0f61c1723991c1 Mon Sep 17 00:00:00 2001 From: Li0k Date: Fri, 22 Nov 2024 18:44:33 +0800 Subject: [PATCH 003/163] feat(compaction): Limit the size of the new overlapping level (#19277) --- proto/hummock.proto | 5 + src/common/src/config.rs | 4 + .../src/cmd_impl/hummock/compaction_group.rs | 4 + src/ctl/src/lib.rs | 5 + .../hummock/compaction/compaction_config.rs | 1 + src/meta/src/hummock/manager/commit_epoch.rs | 107 +++++++++++++-- .../compaction/compaction_group_manager.rs | 3 + src/meta/src/hummock/manager/transaction.rs | 54 ++++---- .../hummock_test/src/hummock_storage_tests.rs | 127 ++++++++++++++++++ .../src/monitor/monitored_storage_metrics.rs | 4 +- 10 files changed, 273 insertions(+), 41 deletions(-) diff --git a/proto/hummock.proto b/proto/hummock.proto index 869c5af867d43..15f3d61a7cf2b 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -660,6 +660,7 @@ message RiseCtlUpdateCompactionConfigRequest { uint64 sst_allowed_trivial_move_min_size = 19; uint32 split_weight_by_vnode = 20; bool disable_auto_group_scheduling = 21; + uint64 max_overlapping_level_size = 22; } } repeated uint64 compaction_group_ids = 1; @@ -858,6 +859,10 @@ message CompactionConfig { // The limitation of auto group scheduling optional bool disable_auto_group_scheduling = 23; + + // The limitation of the max size of the overlapping-level for the compaction + // hummock will reorg the commit-sstables to the multi overlapping-level if the size of the commit-sstables is larger than `max_overlapping_level_size` + optional uint64 max_overlapping_level_size = 24; } message TableStats { diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 1f67057801c4f..393a3a05acb4d 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -2237,6 +2237,10 @@ pub mod default { pub fn disable_auto_group_scheduling() -> bool { false } + + pub fn max_overlapping_level_size() -> u64 { + 256 * MB + } } pub mod object_store_config { diff --git a/src/ctl/src/cmd_impl/hummock/compaction_group.rs b/src/ctl/src/cmd_impl/hummock/compaction_group.rs index d109d6eabda67..e164c0b060eb0 100644 --- a/src/ctl/src/cmd_impl/hummock/compaction_group.rs +++ b/src/ctl/src/cmd_impl/hummock/compaction_group.rs @@ -68,6 +68,7 @@ pub fn build_compaction_config_vec( max_l0_compact_level: Option, sst_allowed_trivial_move_min_size: Option, disable_auto_group_scheduling: Option, + max_overlapping_level_size: Option, ) -> Vec { let mut configs = vec![]; if let Some(c) = max_bytes_for_level_base { @@ -127,6 +128,9 @@ pub fn build_compaction_config_vec( if let Some(c) = disable_auto_group_scheduling { configs.push(MutableConfig::DisableAutoGroupScheduling(c)) } + if let Some(c) = max_overlapping_level_size { + configs.push(MutableConfig::MaxOverlappingLevelSize(c)) + } configs } diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index 99de4cd9b17b9..c13e83cb62b00 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -85,6 +85,7 @@ enum ComputeCommands { ShowConfig { host: String }, } +#[allow(clippy::large_enum_variant)] #[derive(Subcommand)] enum HummockCommands { /// list latest Hummock version on meta node @@ -191,6 +192,8 @@ enum HummockCommands { sst_allowed_trivial_move_min_size: Option, #[clap(long)] disable_auto_group_scheduling: Option, + #[clap(long)] + max_overlapping_level_size: Option, }, /// Split given compaction group into two. Moves the given tables to the new group. SplitCompactionGroup { @@ -578,6 +581,7 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { max_l0_compact_level, sst_allowed_trivial_move_min_size, disable_auto_group_scheduling, + max_overlapping_level_size, }) => { cmd_impl::hummock::update_compaction_config( context, @@ -610,6 +614,7 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { max_l0_compact_level, sst_allowed_trivial_move_min_size, disable_auto_group_scheduling, + max_overlapping_level_size, ), ) .await? diff --git a/src/meta/src/hummock/compaction/compaction_config.rs b/src/meta/src/hummock/compaction/compaction_config.rs index d7be9b6e6cbaa..c808c2f548023 100644 --- a/src/meta/src/hummock/compaction/compaction_config.rs +++ b/src/meta/src/hummock/compaction/compaction_config.rs @@ -71,6 +71,7 @@ impl CompactionConfigBuilder { disable_auto_group_scheduling: Some( compaction_config::disable_auto_group_scheduling(), ), + max_overlapping_level_size: Some(compaction_config::max_overlapping_level_size()), }, } } diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index c51c77a5d36a0..67152cba14236 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -15,7 +15,9 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use std::sync::Arc; +use itertools::Itertools; use risingwave_common::catalog::TableId; +use risingwave_common::config::default::compaction_config; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_hummock_sdk::change_log::ChangeLogDelta; use risingwave_hummock_sdk::compaction_group::group_split::split_sst_with_table_ids; @@ -112,7 +114,7 @@ impl HummockManager { let state_table_info = &version.latest_version().state_table_info; let mut table_compaction_group_mapping = state_table_info.build_table_compaction_group_id(); let mut new_table_ids = HashMap::new(); - let mut new_compaction_groups = HashMap::new(); + let mut new_compaction_groups = Vec::new(); let mut compaction_group_manager_txn = None; let mut compaction_group_config: Option> = None; @@ -143,14 +145,13 @@ impl HummockManager { ) }; let new_compaction_group_id = next_compaction_group_id(&self.env).await?; - new_compaction_groups.insert(new_compaction_group_id, compaction_group_config.clone()); - compaction_group_manager.insert( - new_compaction_group_id, - CompactionGroup { - group_id: new_compaction_group_id, - compaction_config: compaction_group_config, - }, - ); + let new_compaction_group = CompactionGroup { + group_id: new_compaction_group_id, + compaction_config: compaction_group_config.clone(), + }; + + new_compaction_groups.push(new_compaction_group.clone()); + compaction_group_manager.insert(new_compaction_group_id, new_compaction_group); on_handle_add_new_table( state_table_info, @@ -165,12 +166,35 @@ impl HummockManager { .correct_commit_ssts(sstables, &table_compaction_group_mapping) .await?; - let modified_compaction_groups: Vec<_> = commit_sstables.keys().cloned().collect(); + let modified_compaction_groups = commit_sstables.keys().cloned().collect_vec(); + // fill compaction_groups + let mut group_id_to_config = HashMap::new(); + if let Some(compaction_group_manager) = compaction_group_manager_txn.as_ref() { + for cg_id in &modified_compaction_groups { + let compaction_group = compaction_group_manager + .get(cg_id) + .unwrap_or_else(|| panic!("compaction group {} should be created", cg_id)) + .compaction_config(); + group_id_to_config.insert(*cg_id, compaction_group); + } + } else { + let compaction_group_manager = self.compaction_group_manager.read().await; + for cg_id in &modified_compaction_groups { + let compaction_group = compaction_group_manager + .try_get_compaction_group_config(*cg_id) + .unwrap_or_else(|| panic!("compaction group {} should be created", cg_id)) + .compaction_config(); + group_id_to_config.insert(*cg_id, compaction_group); + } + } + + let group_id_to_sub_levels = + rewrite_commit_sstables_to_sub_level(commit_sstables, &group_id_to_config); let time_travel_delta = version.pre_commit_epoch( &tables_to_commit, new_compaction_groups, - commit_sstables, + group_id_to_sub_levels, &new_table_ids, new_table_watermarks, change_log_delta, @@ -327,6 +351,7 @@ impl HummockManager { ) -> Result>> { let mut new_sst_id_number = 0; let mut sst_to_cg_vec = Vec::with_capacity(sstables.len()); + let commit_object_id_vec = sstables.iter().map(|s| s.sst_info.object_id).collect_vec(); for commit_sst in sstables { let mut group_table_ids: BTreeMap> = BTreeMap::new(); for table_id in &commit_sst.sst_info.table_ids { @@ -395,6 +420,12 @@ impl HummockManager { } } + // order check + for ssts in commit_sstables.values() { + let object_ids = ssts.iter().map(|s| s.object_id).collect_vec(); + assert!(is_ordered_subset(&commit_object_id_vec, &object_ids)); + } + Ok(commit_sstables) } } @@ -419,3 +450,57 @@ fn on_handle_add_new_table( Ok(()) } + +/// Rewrite the commit sstables to sub-levels based on the compaction group config. +/// The type of `compaction_group_manager_txn` is too complex to be used in the function signature. So we use `HashMap` instead. +fn rewrite_commit_sstables_to_sub_level( + commit_sstables: BTreeMap>, + group_id_to_config: &HashMap>, +) -> BTreeMap>> { + let mut overlapping_sstables: BTreeMap>> = BTreeMap::new(); + for (group_id, inserted_table_infos) in commit_sstables { + let config = group_id_to_config + .get(&group_id) + .expect("compaction group should exist"); + + let mut accumulated_size = 0; + let mut ssts = vec![]; + let sub_level_size_limit = config + .max_overlapping_level_size + .unwrap_or(compaction_config::max_overlapping_level_size()); + + let level = overlapping_sstables.entry(group_id).or_default(); + + for sst in inserted_table_infos { + accumulated_size += sst.sst_size; + ssts.push(sst); + if accumulated_size > sub_level_size_limit { + level.push(ssts); + + // reset the accumulated size and ssts + accumulated_size = 0; + ssts = vec![]; + } + } + + if !ssts.is_empty() { + level.push(ssts); + } + + // The uploader organizes the ssts in decreasing epoch order, so the level needs to be reversed to ensure that the latest epoch is at the top. + level.reverse(); + } + + overlapping_sstables +} + +fn is_ordered_subset(vec_1: &Vec, vec_2: &Vec) -> bool { + let mut vec_2_iter = vec_2.iter().peekable(); + for item in vec_1 { + if vec_2_iter.peek() == Some(&item) { + vec_2_iter.next(); + } + } + + vec_2_iter.peek().is_none() +} diff --git a/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs index 18bb8dfaf86b3..3a6c179c03147 100644 --- a/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs @@ -588,6 +588,9 @@ fn update_compaction_config(target: &mut CompactionConfig, items: &[MutableConfi MutableConfig::DisableAutoGroupScheduling(c) => { target.disable_auto_group_scheduling = Some(*c); } + MutableConfig::MaxOverlappingLevelSize(c) => { + target.max_overlapping_level_size = Some(*c); + } } } } diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 054ae657d594d..8a4276492365d 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -14,7 +14,6 @@ use std::collections::{BTreeMap, HashMap}; use std::ops::{Deref, DerefMut}; -use std::sync::Arc; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::change_log::ChangeLogDelta; @@ -24,11 +23,12 @@ use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::{GroupDelta, HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{CompactionGroupId, FrontendHummockVersionDelta, HummockVersionId}; use risingwave_pb::hummock::{ - CompactionConfig, CompatibilityVersion, GroupConstruct, HummockVersionDeltas, - HummockVersionStats, StateTableInfoDelta, + CompatibilityVersion, GroupConstruct, HummockVersionDeltas, HummockVersionStats, + StateTableInfoDelta, }; use risingwave_pb::meta::subscribe_response::{Info, Operation}; +use crate::hummock::model::CompactionGroup; use crate::manager::NotificationManager; use crate::model::{ InMemValTransaction, MetadataModelResult, Transactional, ValTransaction, VarTransaction, @@ -111,8 +111,8 @@ impl<'a> HummockVersionTransaction<'a> { pub(super) fn pre_commit_epoch( &mut self, tables_to_commit: &HashMap, - new_compaction_groups: HashMap>, - commit_sstables: BTreeMap>, + new_compaction_groups: Vec, + group_id_to_sub_levels: BTreeMap>>, new_table_ids: &HashMap, new_table_watermarks: HashMap, change_log_delta: HashMap, @@ -121,38 +121,36 @@ impl<'a> HummockVersionTransaction<'a> { new_version_delta.new_table_watermarks = new_table_watermarks; new_version_delta.change_log_delta = change_log_delta; - for (compaction_group_id, compaction_group_config) in new_compaction_groups { - { - let group_deltas = &mut new_version_delta - .group_deltas - .entry(compaction_group_id) - .or_default() - .group_deltas; - - #[expect(deprecated)] - group_deltas.push(GroupDelta::GroupConstruct(GroupConstruct { - group_config: Some((*compaction_group_config).clone()), - group_id: compaction_group_id, - parent_group_id: StaticCompactionGroupId::NewCompactionGroup - as CompactionGroupId, - new_sst_start_id: 0, // No need to set it when `NewCompactionGroup` - table_ids: vec![], - version: CompatibilityVersion::SplitGroupByTableId as i32, - split_key: None, - })); - } + for compaction_group in &new_compaction_groups { + let group_deltas = &mut new_version_delta + .group_deltas + .entry(compaction_group.group_id()) + .or_default() + .group_deltas; + + #[expect(deprecated)] + group_deltas.push(GroupDelta::GroupConstruct(GroupConstruct { + group_config: Some(compaction_group.compaction_config().as_ref().clone()), + group_id: compaction_group.group_id(), + parent_group_id: StaticCompactionGroupId::NewCompactionGroup as CompactionGroupId, + new_sst_start_id: 0, // No need to set it when `NewCompactionGroup` + table_ids: vec![], + version: CompatibilityVersion::SplitGroupByTableId as i32, + split_key: None, + })); } // Append SSTs to a new version. - for (compaction_group_id, inserted_table_infos) in commit_sstables { + for (compaction_group_id, sub_levels) in group_id_to_sub_levels { let group_deltas = &mut new_version_delta .group_deltas .entry(compaction_group_id) .or_default() .group_deltas; - let group_delta = GroupDelta::NewL0SubLevel(inserted_table_infos); - group_deltas.push(group_delta); + for sub_level in sub_levels { + group_deltas.push(GroupDelta::NewL0SubLevel(sub_level)); + } } // update state table info diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 7e847fc089aa2..0b216e84c4960 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -2866,3 +2866,130 @@ async fn test_commit_multi_epoch() { assert_eq!(info.committed_epoch, epoch3); } } + +#[tokio::test] +async fn test_commit_with_large_size() { + let test_env = prepare_hummock_test_env().await; + let context_id = test_env.meta_client.context_id(); + let existing_table_id = TableId::new(1); + let initial_epoch = INVALID_EPOCH; + + let commit_epoch = + |epoch, ssts: Vec, new_table_fragment_infos, tables_to_commit: &[TableId]| { + let manager = &test_env.manager; + let tables_to_commit = tables_to_commit + .iter() + .map(|table_id| (*table_id, epoch)) + .collect(); + let sst_to_context = ssts.iter().map(|sst| (sst.object_id, context_id)).collect(); + + let sstables = ssts + .into_iter() + .map(|sst| LocalSstableInfo { + table_stats: sst + .table_ids + .iter() + .map(|&table_id| { + ( + table_id, + TableStats { + total_compressed_size: 10, + ..Default::default() + }, + ) + }) + .collect(), + sst_info: sst, + created_at: u64::MAX, + }) + .collect_vec(); + + async move { + manager + .commit_epoch(CommitEpochInfo { + new_table_watermarks: Default::default(), + sst_to_context, + sstables, + new_table_fragment_infos, + change_log_delta: Default::default(), + tables_to_commit, + }) + .await + .unwrap(); + } + }; + + let epoch1 = initial_epoch.next_epoch(); + let sst1_epoch1 = SstableInfo { + sst_id: 11, + object_id: 1, + table_ids: vec![existing_table_id.table_id], + file_size: 512 << 20, + sst_size: 512 << 20, + ..Default::default() + }; + + let sst1_epoch2 = SstableInfo { + sst_id: 12, + object_id: 2, + table_ids: vec![existing_table_id.table_id], + file_size: 512 << 20, + sst_size: 512 << 20, + ..Default::default() + }; + + let sst1_epoch3 = SstableInfo { + sst_id: 13, + object_id: 3, + table_ids: vec![existing_table_id.table_id], + file_size: 512 << 20, + sst_size: 512 << 20, + ..Default::default() + }; + + commit_epoch( + epoch1, + vec![ + sst1_epoch3.clone(), + sst1_epoch2.clone(), + sst1_epoch1.clone(), + ], + vec![NewTableFragmentInfo { + table_ids: HashSet::from_iter([existing_table_id]), + }], + &[existing_table_id], + ) + .await; + + let cg_id = + get_compaction_group_id_by_table_id(test_env.manager.clone(), existing_table_id.table_id()) + .await; + + let l0_sub_levels = test_env + .manager + .get_current_version() + .await + .levels + .get(&cg_id) + .unwrap() + .l0 + .clone(); + + println!("l0_sub_levels {:?}", l0_sub_levels.sub_levels); + assert_eq!(3, l0_sub_levels.sub_levels.len()); + assert_eq!(1, l0_sub_levels.sub_levels[0].table_infos.len()); + assert_eq!( + sst1_epoch1.object_id, + l0_sub_levels.sub_levels[0].table_infos[0].object_id + ); + assert_eq!(1, l0_sub_levels.sub_levels[1].table_infos.len()); + assert_eq!( + sst1_epoch2.object_id, + l0_sub_levels.sub_levels[1].table_infos[0].object_id + ); + assert_eq!(1, l0_sub_levels.sub_levels[2].table_infos.len()); + assert_eq!( + sst1_epoch3.object_id, + l0_sub_levels.sub_levels[2].table_infos[0].object_id + ); +} diff --git a/src/storage/src/monitor/monitored_storage_metrics.rs b/src/storage/src/monitor/monitored_storage_metrics.rs index 8bd7ef64b6b83..f8e6ee1e24418 100644 --- a/src/storage/src/monitor/monitored_storage_metrics.rs +++ b/src/storage/src/monitor/monitored_storage_metrics.rs @@ -70,8 +70,8 @@ pub fn global_storage_metrics(metric_level: MetricLevel) -> MonitoredStorageMetr impl MonitoredStorageMetrics { pub fn new(registry: &Registry, metric_level: MetricLevel) -> Self { - // 256B ~ max 4GB - let size_buckets = exponential_buckets(256.0, 16.0, 7).unwrap(); + // 256B ~ max 64GB + let size_buckets = exponential_buckets(256.0, 16.0, 8).unwrap(); // 10ms ~ max 2.7h let time_buckets = exponential_buckets(0.01, 10.0, 7).unwrap(); // ----- get ----- From 236e7617aebd55bd61b9ceb43cb2e1fd9b6d59e8 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Fri, 22 Nov 2024 23:06:25 +0800 Subject: [PATCH 004/163] fix: upgrade setuptools because of vulnerablility (#19540) --- e2e_test/iceberg/pyproject.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/e2e_test/iceberg/pyproject.toml b/e2e_test/iceberg/pyproject.toml index eba4bcd12f137..e0daecfdcb579 100644 --- a/e2e_test/iceberg/pyproject.toml +++ b/e2e_test/iceberg/pyproject.toml @@ -8,7 +8,7 @@ authors = ["risingwavelabs"] python = "^3.10" pyspark = { version = "3.4.1", extras = ["sql", "connect"] } tomli = "2.0" -setuptools = "69" +setuptools = "70" [build-system] requires = ["poetry-core"] From b4bca5724e7acb9c39919aaa188ed699a10eb1f4 Mon Sep 17 00:00:00 2001 From: xxchan Date: Sat, 23 Nov 2024 16:16:36 +0800 Subject: [PATCH 005/163] doc(dev-guide): note good first issue (#19543) Signed-off-by: xxchan --- CONTRIBUTING.md | 6 +++--- docs/dev/src/contributing.md | 11 ++++++++++- 2 files changed, 13 insertions(+), 4 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 7403b83afb698..5f305d85d07fb 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,7 +1,7 @@ # Contributing to RisingWave -Contributing documentation has moved to the **[RisingWave Developer Guide](https://risingwavelabs.github.io/risingwave/)**. - +Read the **[RisingWave Developer Guide](https://risingwavelabs.github.io/risingwave/)** for +how to develop and contribute to RisingWave. ## Before hacking on RisingWave @@ -13,4 +13,4 @@ Please read [the process] of how to submit your change to RisingWave. [Community Slack]: https://risingwave.com/slack [file an issue]: https://github.com/risingwavelabs/risingwave/issues/new/choose -[the process]: https://risingwavelabs.github.io/risingwave/contribution +[the process]: https://risingwavelabs.github.io/risingwave/contributing.html diff --git a/docs/dev/src/contributing.md b/docs/dev/src/contributing.md index 38d8b8cc77c33..926138795d617 100644 --- a/docs/dev/src/contributing.md +++ b/docs/dev/src/contributing.md @@ -11,12 +11,21 @@ To report bugs, create a [GitHub issue](https://github.com/risingwavelabs/rising +## Find Something to Work On + +Issues labeled with [ `good first issue` ](https://github.com/risingwavelabs/risingwave/contribute) are suitable for new RisingWave hackers. +They are relatively easy to begin with and can guide you getting familiar with one module of RisingWave. + ## Tests and miscellaneous checks Before submitting your code changes, ensure you fully test them and perform necessary checks. The testing instructions and necessary checks are detailed in other sections of the book. ## Submit a PR +### Ask for Review + +To get your PR reviewed and merged sooner, you can find and `@` mention developers who recently worked on the same files. If you're not sure who to ask, feel free to reach out to any active developers to help find relevant reviewers. Don't hesitate to follow up politely if you haven't received a response, or ask for help in the RisingWave Community Slack channel. We welcome you to be proactive in finding reviewers for your PR! + ### Pull Request title As described in [here](https://github.com/commitizen/conventional-commit-types/blob/master/index.json), a valid PR title should begin with one of the following prefixes: @@ -28,7 +37,7 @@ As described in [here](https://github.com/commitizen/conventional-commit-types/b - `style`: A refactoring that improves code style - `perf`: A code change that improves performance - `test`: Adding missing tests or correcting existing tests -- `build`: Changes that affect the build system or external dependencies (example scopes: `.config`, `.cargo`, `Cargo.toml`) +- `build`: Changes that affect the build system or external dependencies (example scopes: `.config`, `.cargo`, `Cargo.toml`) - `ci`: Changes to RisingWave CI configuration files and scripts (example scopes: `.github`, `ci` (Buildkite)) - `chore`: Other changes that don't modify src or test files - `revert`: Reverts a previous commit From 9abd62a7a63446c345310c1aeeba994c86cf20e8 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Mon, 25 Nov 2024 11:14:40 +0800 Subject: [PATCH 006/163] refactor(optimizer): record error contexts when casting composite types (#19449) Signed-off-by: Bugen Zhao --- Cargo.lock | 8 +- Cargo.toml | 2 +- e2e_test/batch/basic/dml_update.slt.part | 11 +- .../tests/testdata/input/cast.yaml | 17 ++ .../tests/testdata/output/array.yaml | 2 +- .../tests/testdata/output/cast.yaml | 33 ++++ .../tests/testdata/output/expr.yaml | 2 +- .../tests/testdata/output/struct_query.yaml | 2 +- .../tests/testdata/output/update.yaml | 2 +- .../system_catalog/pg_catalog/pg_cast.rs | 10 +- src/frontend/src/error.rs | 8 + src/frontend/src/expr/function_call.rs | 60 +++---- src/frontend/src/expr/mod.rs | 8 +- src/frontend/src/expr/type_inference/cast.rs | 159 +++++++++++++----- src/frontend/src/expr/type_inference/mod.rs | 3 +- 15 files changed, 229 insertions(+), 98 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c8be3e154ddd8..c08e776947cca 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -14299,9 +14299,9 @@ dependencies = [ [[package]] name = "thiserror-ext" -version = "0.1.2" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "a7c19760dc47062bca5c1b3699b032111c93802d51ac47660db11b08afc6bad2" +checksum = "ef4323942237f7cc071061f2c5f0db919e6053c2cdf58c6bc974883073429737" dependencies = [ "thiserror 1.0.63", "thiserror-ext-derive", @@ -14309,9 +14309,9 @@ dependencies = [ [[package]] name = "thiserror-ext-derive" -version = "0.1.2" +version = "0.2.1" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "667c8c48f68021098038115926c64d9950b0582062ae63f7d30638b1168daf03" +checksum = "96541747c50e6c73e094737938f4f5dfaf50c48a31adff4197a3e2a481371674" dependencies = [ "either", "proc-macro2", diff --git a/Cargo.toml b/Cargo.toml index c260bf8c52932..6f9b85aa5c1ae 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -163,7 +163,7 @@ parquet = { version = "53.2", features = ["async"] } mysql_async = { version = "0.34", default-features = false, features = [ "default", ] } -thiserror-ext = "0.1.2" +thiserror-ext = { version = "0.2.1", features = ["backtrace"] } tikv-jemalloc-ctl = { git = "https://github.com/risingwavelabs/jemallocator.git", rev = "64a2d9" } tikv-jemallocator = { git = "https://github.com/risingwavelabs/jemallocator.git", features = [ "profiling", diff --git a/e2e_test/batch/basic/dml_update.slt.part b/e2e_test/batch/basic/dml_update.slt.part index fcc3bbdfce9a2..fc2647cea147b 100644 --- a/e2e_test/batch/basic/dml_update.slt.part +++ b/e2e_test/batch/basic/dml_update.slt.part @@ -93,10 +93,15 @@ select * from t; 889 999 # Multiple assignments, to subquery with cast failure. -# TODO: this currently shows `cannot cast type "record" to "record"` because we wrap the subquery result -# into a struct, which is not quite clear. -statement error cannot cast type +statement error update t set (v1, v2) = (select '888.88', 999); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: cannot cast type "record" to "record" + 2: cannot cast type "character varying" to "integer" in Assign context + # Multiple assignments, to subquery with mismatched column count. statement error number of columns does not match number of values diff --git a/src/frontend/planner_test/tests/testdata/input/cast.yaml b/src/frontend/planner_test/tests/testdata/input/cast.yaml index a21b9e3cb409a..f2344b3dd00a3 100644 --- a/src/frontend/planner_test/tests/testdata/input/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/input/cast.yaml @@ -64,3 +64,20 @@ select count(*) FILTER(WHERE 'y') from t; expected_outputs: - batch_plan +- name: composite type cast error message (array) + sql: | + select '[]'::int[]::bytea[]; + expected_outputs: + - binder_error +- name: composite type cast error message (struct) + sql: | + create table t (v struct, c bool>); + select v::struct, f bool> from t; + expected_outputs: + - binder_error +- name: composite type cast error message (map) + sql: | + create table t (v map(int, int)); + select v::map(int, bytea) from t; + expected_outputs: + - binder_error diff --git a/src/frontend/planner_test/tests/testdata/output/array.yaml b/src/frontend/planner_test/tests/testdata/output/array.yaml index a2b9486fdb33d..259a727d23df6 100644 --- a/src/frontend/planner_test/tests/testdata/output/array.yaml +++ b/src/frontend/planner_test/tests/testdata/output/array.yaml @@ -228,7 +228,7 @@ sql: | create table t (v1 varchar[]); insert into t values ('{c,' || 'd}'); - binder_error: 'Bind error: cannot cast type "character varying" to "character varying[]" in Assign context' + binder_error: cannot cast type "character varying" to "character varying[]" in Assign context - name: string to varchar[] in explicit context sql: | select ('{c,' || 'd}')::varchar[]; diff --git a/src/frontend/planner_test/tests/testdata/output/cast.yaml b/src/frontend/planner_test/tests/testdata/output/cast.yaml index 636f25a9b07de..28cc002eae562 100644 --- a/src/frontend/planner_test/tests/testdata/output/cast.yaml +++ b/src/frontend/planner_test/tests/testdata/output/cast.yaml @@ -80,3 +80,36 @@ └─BatchExchange { order: [], dist: Single } └─BatchSimpleAgg { aggs: [count] } └─BatchScan { table: t, columns: [], distribution: SomeShard } +- name: composite type cast error message (array) + sql: | + select '[]'::int[]::bytea[]; + binder_error: | + Failed to bind expression: CAST(CAST('[]' AS INT[]) AS BYTEA[]) + + Caused by these errors (recent errors listed first): + 1: cannot cast type "integer[]" to "bytea[]" + 2: cannot cast type "integer" to "bytea" in Explicit context +- name: composite type cast error message (struct) + sql: | + create table t (v struct, c bool>); + select v::struct, f bool> from t; + binder_error: | + Failed to bind expression: CAST(v AS STRUCT, f BOOLEAN>) + + Caused by these errors (recent errors listed first): + 1: cannot cast type "struct, c boolean>" to "struct, f boolean>" + 2: cannot cast struct field "a" to struct field "d" + 3: cannot cast type "struct" to "struct" + 4: cannot cast struct field "b" to struct field "e" + 5: cannot cast type "integer" to "bytea" in Explicit context +- name: composite type cast error message (map) + sql: | + create table t (v map(int, int)); + select v::map(int, bytea) from t; + binder_error: | + Failed to bind expression: CAST(v AS MAP(INT,BYTEA)) + + Caused by these errors (recent errors listed first): + 1: cannot cast type "map(integer,integer)" to "map(integer,bytea)" + 2: cannot cast map value + 3: cannot cast type "integer" to "bytea" in Explicit context diff --git a/src/frontend/planner_test/tests/testdata/output/expr.yaml b/src/frontend/planner_test/tests/testdata/output/expr.yaml index eacab421069a9..ce2f86ed2e689 100644 --- a/src/frontend/planner_test/tests/testdata/output/expr.yaml +++ b/src/frontend/planner_test/tests/testdata/output/expr.yaml @@ -304,7 +304,7 @@ Failed to bind expression: concat_ws(v1, 1.2) Caused by: - Bind error: cannot cast type "integer" to "character varying" in Implicit context + cannot cast type "integer" to "character varying" in Implicit context - sql: | create table t (v1 int); select concat_ws() from t; diff --git a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml index 907aa209c6d2c..3f6b4579e44c9 100644 --- a/src/frontend/planner_test/tests/testdata/output/struct_query.yaml +++ b/src/frontend/planner_test/tests/testdata/output/struct_query.yaml @@ -421,7 +421,7 @@ - sql: | CREATE TABLE a (c STRUCT, j INTEGER>); INSERT INTO a VALUES (1); - binder_error: 'Bind error: cannot cast type "integer" to "struct, j integer>" in Assign context' + binder_error: cannot cast type "integer" to "struct, j integer>" in Assign context - name: test struct type alignment in CASE expression sql: | select CASE WHEN false THEN ROW(0, INTERVAL '1') WHEN true THEN ROW(1.1, INTERVAL '1') ELSE ROW(1, INTERVAL '1') END; diff --git a/src/frontend/planner_test/tests/testdata/output/update.yaml b/src/frontend/planner_test/tests/testdata/output/update.yaml index 4a12b492660ad..26c6d52dc5e05 100644 --- a/src/frontend/planner_test/tests/testdata/output/update.yaml +++ b/src/frontend/planner_test/tests/testdata/output/update.yaml @@ -11,7 +11,7 @@ - sql: | create table t (v1 int, v2 int); update t set v1 = true; - binder_error: 'Bind error: cannot cast type "boolean" to "integer" in Assign context' + binder_error: cannot cast type "boolean" to "integer" in Assign context - sql: | create table t (v1 int, v2 int); update t set v1 = v2 + 1; diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs index d5b1332c25b3e..291743ea4ba26 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_cast.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use itertools::Itertools; use risingwave_common::types::{DataType, Fields}; use risingwave_frontend_macro::system_catalog; use crate::catalog::system_catalog::SysCatalogReaderImpl; -use crate::expr::cast_map_array; +use crate::expr::CAST_TABLE; /// The catalog `pg_cast` stores data type conversion paths. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-cast.html`] @@ -31,12 +32,11 @@ struct PgCast { #[system_catalog(table, "pg_catalog.pg_cast")] fn read_pg_cast(_: &SysCatalogReaderImpl) -> Vec { - let mut cast_array = cast_map_array(); - cast_array.sort(); - cast_array + CAST_TABLE .iter() + .sorted() .enumerate() - .map(|(idx, (src, target, ctx))| PgCast { + .map(|(idx, ((src, target), ctx))| PgCast { oid: idx as i32, castsource: DataType::try_from(*src).unwrap().to_oid(), casttarget: DataType::try_from(*target).unwrap().to_oid(), diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index f0cf35e859664..d0615b9f0e13d 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -26,6 +26,8 @@ use risingwave_rpc_client::error::{RpcError, TonicStatusWrapper}; use thiserror::Error; use tokio::task::JoinError; +use crate::expr::CastError; + /// The error type for the frontend crate, acting as the top-level error type for the /// entire RisingWave project. // TODO(error-handling): this is migrated from the `common` crate, and there could @@ -114,6 +116,12 @@ pub enum ErrorCode { #[backtrace] error: BoxedError, }, + #[error(transparent)] + CastError( + #[from] + #[backtrace] + CastError, + ), #[error("Catalog error: {0}")] CatalogError( #[source] diff --git a/src/frontend/src/expr/function_call.rs b/src/frontend/src/expr/function_call.rs index af1f84b321eb5..c5ae4ca178bf8 100644 --- a/src/frontend/src/expr/function_call.rs +++ b/src/frontend/src/expr/function_call.rs @@ -16,12 +16,11 @@ use itertools::Itertools; use risingwave_common::catalog::Schema; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; -use thiserror::Error; -use thiserror_ext::AsReport; -use super::{cast_ok, infer_some_all, infer_type, CastContext, Expr, ExprImpl, Literal}; -use crate::error::{ErrorCode, Result as RwResult}; -use crate::expr::{ExprDisplay, ExprType, ExprVisitor, ImpureAnalyzer}; +use super::type_inference::cast; +use super::{infer_some_all, infer_type, CastContext, CastError, Expr, ExprImpl, Literal}; +use crate::error::Result as RwResult; +use crate::expr::{bail_cast_error, ExprDisplay, ExprType, ExprVisitor, ImpureAnalyzer}; #[derive(Clone, Eq, PartialEq, Hash)] pub struct FunctionCall { @@ -144,22 +143,23 @@ impl FunctionCall { // else when eager parsing fails, just proceed as normal. // Some callers are not ready to handle `'a'::int` error here. } + let source = child.return_type(); if source == target { - Ok(()) - // Casting from unknown is allowed in all context. And PostgreSQL actually does the parsing - // in frontend. - } else if child.is_untyped() || cast_ok(&source, &target, allows) { - // Always Ok below. Safe to mutate `child`. - let owned = std::mem::replace(child, ExprImpl::literal_bool(false)); - *child = Self::new_unchecked(ExprType::Cast, vec![owned], target).into(); - Ok(()) + return Ok(()); + } + + if child.is_untyped() { + // Casting from unknown is allowed in all context. And PostgreSQL actually does the parsing + // in frontend. } else { - Err(CastError(format!( - "cannot cast type \"{}\" to \"{}\" in {:?} context", - source, target, allows - ))) + cast(&source, &target, allows)?; } + + // Always Ok below. Safe to mutate `child`. + let owned = std::mem::replace(child, ExprImpl::literal_bool(false)); + *child = Self::new_unchecked(ExprType::Cast, vec![owned], target).into(); + Ok(()) } /// Cast a `ROW` expression to the target type. We intentionally disallow casting arbitrary @@ -170,13 +170,13 @@ impl FunctionCall { target_type: DataType, allows: CastContext, ) -> Result<(), CastError> { + // Can only cast to a struct type. let DataType::Struct(t) = &target_type else { - return Err(CastError(format!( - "cannot cast type \"{}\" to \"{}\" in {:?} context", - func.return_type(), + bail_cast_error!( + "cannot cast type \"{}\" to \"{}\"", + func.return_type(), // typically "record" target_type, - allows - ))); + ); }; match t.len().cmp(&func.inputs.len()) { std::cmp::Ordering::Equal => { @@ -189,10 +189,8 @@ impl FunctionCall { func.return_type = target_type; Ok(()) } - std::cmp::Ordering::Less => Err(CastError("Input has too few columns.".to_string())), - std::cmp::Ordering::Greater => { - Err(CastError("Input has too many columns.".to_string())) - } + std::cmp::Ordering::Less => bail_cast_error!("input has too few columns"), + std::cmp::Ordering::Greater => bail_cast_error!("input has too many columns"), } } @@ -422,13 +420,3 @@ pub fn is_row_function(expr: &ExprImpl) -> bool { } false } - -#[derive(Debug, Error)] -#[error("{0}")] -pub struct CastError(pub(super) String); - -impl From for ErrorCode { - fn from(value: CastError) -> Self { - ErrorCode::BindError(value.to_report_string()) - } -} diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index c7acdfa5c4a3c..4cc3a18319398 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -66,10 +66,7 @@ pub use risingwave_pb::expr::expr_node::Type as ExprType; pub use session_timezone::{SessionTimezone, TimestamptzExprFinder}; pub use subquery::{Subquery, SubqueryKind}; pub use table_function::{TableFunction, TableFunctionType}; -pub use type_inference::{ - align_types, cast_map_array, cast_ok, cast_sigs, infer_some_all, infer_type, infer_type_name, - infer_type_with_sigmap, CastContext, CastSig, FuncSign, -}; +pub use type_inference::*; pub use user_defined_function::UserDefinedFunction; pub use utils::*; pub use window_function::WindowFunction; @@ -300,7 +297,7 @@ impl ExprImpl { ))), DataType::Int32 => Ok(self), dt if dt.is_int() => Ok(self.cast_explicit(DataType::Int32)?), - _ => Err(CastError("Unsupported input type".to_string())), + _ => bail_cast_error!("unsupported input type"), } } @@ -1171,7 +1168,6 @@ use risingwave_common::bail; use risingwave_common::catalog::Schema; use risingwave_common::row::OwnedRow; -use self::function_call::CastError; use crate::binder::BoundSetExpr; use crate::utils::Condition; diff --git a/src/frontend/src/expr/type_inference/cast.rs b/src/frontend/src/expr/type_inference/cast.rs index 51441c3f70c5b..c9b09fe18ecac 100644 --- a/src/frontend/src/expr/type_inference/cast.rs +++ b/src/frontend/src/expr/type_inference/cast.rs @@ -13,12 +13,15 @@ // limitations under the License. use std::collections::BTreeMap; +use std::error::Error; use std::sync::LazyLock; use itertools::Itertools as _; use parse_display::Display; use risingwave_common::types::{DataType, DataTypeName}; -use risingwave_common::util::iter_util::ZipEqFast; +use risingwave_common::util::iter_util::{ZipEqDebug, ZipEqFast}; +use thiserror::Error; +use thiserror_ext::{Box, Macro}; use crate::error::ErrorCode; use crate::expr::{Expr as _, ExprImpl, InputRef, Literal}; @@ -114,76 +117,156 @@ pub fn align_array_and_element( Ok(array_type) } +/// A stack of error messages for the cast operation. +#[derive(Error, Debug, Box, Macro)] +#[thiserror_ext(newtype(name = CastError), macro(path = "crate::expr"))] +#[error("{message}")] +pub struct CastErrorInner { + pub source: Option, + pub message: Box, +} + +pub type CastResult = Result; + +/// Returns `Ok` if `ok` is true, otherwise returns a placeholder [`CastError`] to be further +/// wrapped with a more informative context in [`cast`]. +fn canbo(ok: bool) -> CastResult { + if ok { + Ok(()) + } else { + bail_cast_error!() + } +} +/// Equivalent to `canbo(false)`. +fn cannot() -> CastResult { + canbo(false) +} + +/// Checks whether casting from `source` to `target` is ok in `allows` context. +/// Returns an error if the cast is not possible. +pub fn cast(source: &DataType, target: &DataType, allows: CastContext) -> Result<(), CastError> { + macro_rules! any { + ($f:ident) => { + source.$f() || target.$f() + }; + } + + if any!(is_struct) { + cast_struct(source, target, allows) + } else if any!(is_array) { + cast_array(source, target, allows) + } else if any!(is_map) { + cast_map(source, target, allows) + } else { + canbo(cast_ok_base(source, target, allows)) + } + .map_err(|inner| { + // Only show "in .. context" once in the error source chain. + let in_context = if inner.source().is_none() { + &format!(" in {:?} context", allows) + } else { + "" + }; + cast_error!( + source = inner, + "cannot cast type \"{}\" to \"{}\"{}", + source, + target, + in_context, + ) + }) +} + /// Checks whether casting from `source` to `target` is ok in `allows` context. +/// +/// Equivalent to `cast(..).is_ok()`, but [`cast`] may be preferred for its error messages. pub fn cast_ok(source: &DataType, target: &DataType, allows: CastContext) -> bool { - cast_ok_struct(source, target, allows) - || cast_ok_array(source, target, allows) - || cast_ok_map(source, target, allows) - || cast_ok_base(source, target, allows) + cast(source, target, allows).is_ok() } /// Checks whether casting from `source` to `target` is ok in `allows` context. /// Both `source` and `target` must be base types, i.e. not struct or array. pub fn cast_ok_base(source: &DataType, target: &DataType, allows: CastContext) -> bool { - matches!(CAST_MAP.get(&(source.into(), target.into())), Some(context) if *context <= allows) + matches!(CAST_TABLE.get(&(source.into(), target.into())), Some(context) if *context <= allows) } -fn cast_ok_struct(source: &DataType, target: &DataType, allows: CastContext) -> bool { +fn cast_struct(source: &DataType, target: &DataType, allows: CastContext) -> CastResult { match (source, target) { (DataType::Struct(lty), DataType::Struct(rty)) => { if lty.is_empty() || rty.is_empty() { unreachable!("record type should be already processed at this point"); } if lty.len() != rty.len() { - // only cast structs of the same length - return false; + bail_cast_error!("cannot cast structs of different lengths"); } // ... and all fields are castable - lty.types() - .zip_eq_fast(rty.types()) - .all(|(src, dst)| src == dst || cast_ok(src, dst, allows)) + lty.iter().zip_eq_debug(rty.iter()).try_for_each( + |((src_name, src_ty), (dst_name, dst_ty))| { + if src_ty == dst_ty { + Ok(()) + } else { + cast(src_ty, dst_ty, allows).map_err(|inner| { + if src_name.is_empty() { + inner + } else if dst_name.is_empty() { + cast_error!( + source = inner, + "cannot cast struct field \"{}\"", + src_name + ) + } else { + cast_error!( + source = inner, + "cannot cast struct field \"{}\" to struct field \"{}\"", + src_name, + dst_name + ) + } + }) + } + }, + ) } // The automatic casts to string types are treated as assignment casts, while the automatic // casts from string types are explicit-only. // https://www.postgresql.org/docs/14/sql-createcast.html#id-1.9.3.58.7.4 - (DataType::Varchar, DataType::Struct(_)) => CastContext::Explicit <= allows, - (DataType::Struct(_), DataType::Varchar) => CastContext::Assign <= allows, - _ => false, + (DataType::Varchar, DataType::Struct(_)) => canbo(CastContext::Explicit <= allows), + (DataType::Struct(_), DataType::Varchar) => canbo(CastContext::Assign <= allows), + _ => cannot(), } } -fn cast_ok_array(source: &DataType, target: &DataType, allows: CastContext) -> bool { +fn cast_array(source: &DataType, target: &DataType, allows: CastContext) -> CastResult { match (source, target) { (DataType::List(source_elem), DataType::List(target_elem)) => { - cast_ok(source_elem, target_elem, allows) + cast(source_elem, target_elem, allows) } // The automatic casts to string types are treated as assignment casts, while the automatic // casts from string types are explicit-only. // https://www.postgresql.org/docs/14/sql-createcast.html#id-1.9.3.58.7.4 - (DataType::Varchar, DataType::List(_)) => CastContext::Explicit <= allows, - (DataType::List(_), DataType::Varchar) => CastContext::Assign <= allows, - _ => false, + (DataType::Varchar, DataType::List(_)) => canbo(CastContext::Explicit <= allows), + (DataType::List(_), DataType::Varchar) => canbo(CastContext::Assign <= allows), + _ => cannot(), } } -fn cast_ok_map(source: &DataType, target: &DataType, allows: CastContext) -> bool { +fn cast_map(source: &DataType, target: &DataType, allows: CastContext) -> CastResult { match (source, target) { - (DataType::Map(source_elem), DataType::Map(target_elem)) => cast_ok( - &source_elem.clone().into_list(), - &target_elem.clone().into_list(), - allows, - ), - _ => false, + (DataType::Map(source_elem), DataType::Map(target_elem)) => { + if source_elem.key() != target_elem.key() { + cast(source_elem.key(), target_elem.key(), allows) + .map_err(|inner| cast_error!(source = inner, "cannot cast map key"))?; + } + if source_elem.value() != target_elem.value() { + cast(source_elem.value(), target_elem.value(), allows) + .map_err(|inner| cast_error!(source = inner, "cannot cast map value"))?; + } + Ok(()) + } + _ => cannot(), } } -pub fn cast_map_array() -> Vec<(DataTypeName, DataTypeName, CastContext)> { - CAST_MAP - .iter() - .map(|((src, target), ctx)| (*src, *target, *ctx)) - .collect_vec() -} - #[derive(Clone, Debug)] pub struct CastSig { pub from_type: DataTypeName, @@ -204,10 +287,10 @@ pub enum CastContext { Explicit, } -pub type CastMap = BTreeMap<(DataTypeName, DataTypeName), CastContext>; +pub type CastTable = BTreeMap<(DataTypeName, DataTypeName), CastContext>; pub fn cast_sigs() -> impl Iterator { - CAST_MAP + CAST_TABLE .iter() .map(|((from_type, to_type), context)| CastSig { from_type: *from_type, @@ -216,7 +299,7 @@ pub fn cast_sigs() -> impl Iterator { }) } -pub static CAST_MAP: LazyLock = LazyLock::new(|| { +pub static CAST_TABLE: LazyLock = LazyLock::new(|| { // cast rules: // 1. implicit cast operations in PG are organized in 3 sequences, // with the reverse direction being assign cast operations. @@ -306,7 +389,7 @@ mod tests { fn test_cast_ok() { // With the help of a script we can obtain the 3 expected cast tables from PG. They are // slightly modified on same-type cast and from-string cast for reasons explained above in - // `build_cast_map`. + // `build_cast_table`. let actual = gen_cast_table(CastContext::Implicit); assert_eq!( diff --git a/src/frontend/src/expr/type_inference/mod.rs b/src/frontend/src/expr/type_inference/mod.rs index 2845f05ec0dae..4c507a9586190 100644 --- a/src/frontend/src/expr/type_inference/mod.rs +++ b/src/frontend/src/expr/type_inference/mod.rs @@ -18,6 +18,7 @@ mod cast; mod func; pub use cast::{ - align_types, cast_map_array, cast_ok, cast_ok_base, cast_sigs, CastContext, CastSig, + align_types, bail_cast_error, cast, cast_error, cast_ok, cast_ok_base, cast_sigs, CastContext, + CastError, CastErrorInner, CastSig, CAST_TABLE, }; pub use func::{infer_some_all, infer_type, infer_type_name, infer_type_with_sigmap, FuncSign}; From 420b60b591f4da6a6de1eea4966833667f010cd3 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Mon, 25 Nov 2024 14:48:00 +0800 Subject: [PATCH 007/163] test(pgwire): complete the test cases to cover all data types (#19520) --- e2e_test/python_client/main.py | 133 ++++++++++++++++++++++++++++++--- 1 file changed, 121 insertions(+), 12 deletions(-) diff --git a/e2e_test/python_client/main.py b/e2e_test/python_client/main.py index bb41ba6c38f34..dcf7c52518b8e 100644 --- a/e2e_test/python_client/main.py +++ b/e2e_test/python_client/main.py @@ -1,19 +1,128 @@ import psycopg +from decimal import Decimal +import math +import unittest +import datetime +import zoneinfo -def test_psycopg_extended_mode(): - conn = psycopg.connect(host='localhost', port='4566', dbname='dev', user='root') - with conn.cursor() as cur: - cur.execute("select Array[1::bigint, 2::bigint, 3::bigint]", binary=True) - assert cur.fetchone() == ([1, 2, 3],) +PG_HOST = 'localhost' +PG_PORT = 4566 +PG_DBNAME = 'dev' +PG_USER = 'root' - cur.execute("select Array['foo', null, 'bar']", binary=True) - assert cur.fetchone() == (['foo', None, 'bar'],) +class TestPsycopgExtendedMode(unittest.TestCase): + def test_psycopg_extended_mode(self): + with psycopg.connect(host=PG_HOST, port=PG_PORT, dbname=PG_DBNAME, user=PG_USER) as conn: + with conn.cursor() as cur: + # Boolean + cur.execute("select true, false, null::boolean", binary=True) + self.assertEqual(cur.fetchone(), (True, False, None)) - cur.execute("select ROW('123 Main St', 'New York', '10001')", binary=True) - assert cur.fetchone() == (('123 Main St', 'New York', '10001'),) + # Integer types + cur.execute("select 1::smallint, 2::integer, 3::bigint", binary=True) + self.assertEqual(cur.fetchone(), (1, 2, 3)) - cur.execute("select array[ROW('123 Main St', 'New York', '10001'), ROW('234 Main St', null, '10001')]", binary=True) - assert cur.fetchone() == ([('123 Main St', 'New York', '10001'), ('234 Main St', None, '10001')],) + # Decimal/Numeric types + cur.execute("select 1.23::decimal, 2.5::real, 3.45::double precision", binary=True) + self.assertEqual(cur.fetchone(), (Decimal('1.23'), 2.5, 3.45)) + + # String + cur.execute("select 'hello'::varchar, null::varchar", binary=True) + self.assertEqual(cur.fetchone(), ('hello', None)) + + # Date/Time types + cur.execute("select '2023-01-01'::date, '12:34:56'::time, '2023-01-01 12:34:56'::timestamp, '2023-01-01 12:34:56+00'::timestamptz", binary=True) + self.assertEqual(cur.fetchone(), (datetime.date(2023, 1, 1), datetime.time(12, 34, 56), datetime.datetime(2023, 1, 1, 12, 34, 56), datetime.datetime(2023, 1, 1, 20, 34, 56, tzinfo=zoneinfo.ZoneInfo(key='Asia/Shanghai')))) + + # Interval + cur.execute("select '1 year 2 months 3 days 4 hours 5 minutes 6 seconds'::interval", binary=True) + self.assertEqual(cur.fetchone(), (datetime.timedelta(days=428, seconds=14706),)) + + # Byte array + cur.execute("select '\\xDEADBEEF'::bytea", binary=True) + self.assertEqual(cur.fetchone(), (b'\xDE\xAD\xBE\xEF',)) + + cur.execute("select '\\x'::bytea", binary=True) + self.assertEqual(cur.fetchone(), (b'',)) + + # Array + cur.execute("select ARRAY[true, false, null]::boolean[]", binary=True) + self.assertEqual(cur.fetchone(), ([True, False, None],)) + + cur.execute("select ARRAY[1, 2, 3]::smallint[]", binary=True) + self.assertEqual(cur.fetchone(), ([1, 2, 3],)) + + cur.execute("select ARRAY[1, 2, 3]::integer[]", binary=True) + self.assertEqual(cur.fetchone(), ([1, 2, 3],)) + + cur.execute("select ARRAY[1, 2, 3]::bigint[]", binary=True) + self.assertEqual(cur.fetchone(), ([1, 2, 3],)) + + cur.execute("select ARRAY[1.1, 2.2, 3.3]::decimal[]", binary=True) + self.assertEqual(cur.fetchone(), ([Decimal('1.1'), Decimal('2.2'), Decimal('3.3')],)) + + cur.execute("select ARRAY[1.1, 2.2, 3.3]::real[]", binary=True) + result = cur.fetchone()[0] # Fetch once and store the result + self.assertAlmostEqual(result[0], 1.1, places=6) + self.assertAlmostEqual(result[1], 2.2, places=6) + self.assertAlmostEqual(result[2], 3.3, places=6) + + cur.execute("select ARRAY[1.1, 2.2, 3.3]::double precision[]", binary=True) + result = cur.fetchone()[0] # Fetch once and store the result + self.assertAlmostEqual(result[0], 1.1, places=6) + self.assertAlmostEqual(result[1], 2.2, places=6) + self.assertAlmostEqual(result[2], 3.3, places=6) + + cur.execute("select ARRAY['foo', 'bar', null]::varchar[]", binary=True) + self.assertEqual(cur.fetchone(), (['foo', 'bar', None],)) + + cur.execute("select ARRAY['\\xDEADBEEF'::bytea, '\\x0102'::bytea]", binary=True) + self.assertEqual(cur.fetchone(), ([b'\xDE\xAD\xBE\xEF', b'\x01\x02'],)) + + cur.execute("select ARRAY['2023-01-01', '2023-01-02']::date[]", binary=True) + self.assertEqual(cur.fetchone(), ([datetime.date(2023, 1, 1), datetime.date(2023, 1, 2)],)) + + cur.execute("select ARRAY['12:34:56', '23:45:01']::time[]", binary=True) + self.assertEqual(cur.fetchone()[0], [datetime.time(12, 34, 56), datetime.time(23, 45, 1)]) + + cur.execute("select ARRAY['2023-01-01 12:34:56', '2023-01-02 23:45:01']::timestamp[]", binary=True) + self.assertEqual(cur.fetchone()[0], [datetime.datetime(2023, 1, 1, 12, 34, 56), datetime.datetime(2023, 1, 2, 23, 45, 1)]) + + cur.execute("select ARRAY['2023-01-01 12:34:56+00', '2023-01-02 23:45:01+00']::timestamptz[]", binary=True) + self.assertEqual(cur.fetchone()[0], [datetime.datetime(2023, 1, 1, 12, 34, 56, tzinfo=datetime.timezone.utc), datetime.datetime(2023, 1, 2, 23, 45, 1, tzinfo=datetime.timezone.utc)]) + + cur.execute("select ARRAY['{\"a\": 1}'::jsonb, '{\"b\": 2}'::jsonb]", binary=True) + self.assertEqual(cur.fetchone(), ([{'a': 1}, {'b': 2}],)) + + # Struct + cur.execute("select ROW('123 Main St'::varchar, 'New York'::varchar, 10001)", binary=True) + self.assertEqual(cur.fetchone(), (('123 Main St', 'New York', 10001),)) + + cur.execute("select array[ROW('123 Main St'::varchar, 'New York'::varchar, 10001), ROW('234 Main St'::varchar, null, 10002)]", binary=True) + self.assertEqual(cur.fetchone(), ([('123 Main St', 'New York', 10001), ('234 Main St', None, 10002)],)) + + # Numeric + cur.execute("select 'NaN'::numeric, 'NaN'::real, 'NaN'::double precision", binary=True) + result = cur.fetchone() + self.assertTrue(result[0].is_nan()) + self.assertTrue(math.isnan(result[1])) + self.assertTrue(math.isnan(result[2])) + + cur.execute("select 'Infinity'::numeric, 'Infinity'::real, 'Infinity'::double precision", binary=True) + self.assertEqual(cur.fetchone(), (float('inf'), float('inf'), float('inf'))) + + cur.execute("select '-Infinity'::numeric, '-Infinity'::real, '-Infinity'::double precision", binary=True) + self.assertEqual(cur.fetchone(), (float('-inf'), float('-inf'), float('-inf'))) + + # JSONB + cur.execute("select '{\"name\": \"John\", \"age\": 30, \"city\": null}'::jsonb", binary=True) + self.assertEqual(cur.fetchone(), ({'name': 'John', 'age': 30, 'city': None},)) + + cur.execute("select '{\"scores\": [85.5, 90, null], \"passed\": true}'::jsonb", binary=True) + self.assertEqual(cur.fetchone(), ({'scores': [85.5, 90, None], 'passed': True},)) + + cur.execute("select '[{\"id\": 1, \"value\": null}, {\"id\": 2, \"value\": \"test\"}]'::jsonb", binary=True) + self.assertEqual(cur.fetchone(), ([{'id': 1, 'value': None}, {'id': 2, 'value': 'test'}],)) if __name__ == '__main__': - test_psycopg_extended_mode() + unittest.main() From 49398f602e8f01c82b8106017647bcdaa2cc5bbc Mon Sep 17 00:00:00 2001 From: Li0k Date: Mon, 25 Nov 2024 16:02:09 +0800 Subject: [PATCH 008/163] feat(storage): reclaim table throughput statistics (#19135) --- src/common/src/config.rs | 1 + src/meta/src/hummock/manager/commit_epoch.rs | 31 ++-- .../compaction/compaction_group_manager.rs | 20 ++- .../compaction/compaction_group_schedule.rs | 155 +++++++++--------- .../src/hummock/manager/compaction/mod.rs | 27 ++- src/meta/src/hummock/manager/mod.rs | 17 +- .../table_write_throughput_statistic.rs | 115 +++++++++++++ src/meta/src/hummock/manager/timer_task.rs | 61 ++++--- .../hummock/local_version/pinned_version.rs | 11 +- 9 files changed, 283 insertions(+), 155 deletions(-) create mode 100644 src/meta/src/hummock/manager/table_write_throughput_statistic.rs diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 393a3a05acb4d..361f524ce775c 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -422,6 +422,7 @@ pub struct MetaConfig { #[serde(default = "default::meta::periodic_scheduling_compaction_group_merge_interval_sec")] pub periodic_scheduling_compaction_group_merge_interval_sec: u64, + #[serde(default)] #[config_doc(nested)] pub meta_store_config: MetaStoreConfig, diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 67152cba14236..37ff58b25a4d4 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -318,29 +318,20 @@ impl HummockManager { async fn collect_table_write_throughput(&self, table_stats: PbTableStatsMap) { let params = self.env.system_params_reader().await; let barrier_interval_ms = params.barrier_interval_ms() as u64; - let checkpoint_secs = { - std::cmp::max( - 1, - params.checkpoint_frequency() * barrier_interval_ms / 1000, - ) - }; - - let mut table_infos = self.history_table_throughput.write(); - let max_table_stat_throuput_window_seconds = std::cmp::max( - self.env.opts.table_stat_throuput_window_seconds_for_split, - self.env.opts.table_stat_throuput_window_seconds_for_merge, + let checkpoint_secs = std::cmp::max( + 1, + params.checkpoint_frequency() * barrier_interval_ms / 1000, ); - let max_sample_size = (max_table_stat_throuput_window_seconds as f64 - / checkpoint_secs as f64) - .ceil() as usize; + let mut table_throughput_statistic_manager = + self.table_write_throughput_statistic_manager.write(); + let timestamp = chrono::Utc::now().timestamp(); + for (table_id, stat) in table_stats { - let throughput = (stat.total_value_size + stat.total_key_size) as u64; - let entry = table_infos.entry(table_id).or_default(); - entry.push_back(throughput); - if entry.len() > max_sample_size { - entry.pop_front(); - } + let throughput = ((stat.total_value_size + stat.total_key_size) as f64 + / checkpoint_secs as f64) as u64; + table_throughput_statistic_manager + .add_table_throughput_with_ts(table_id, throughput, timestamp); } } diff --git a/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs b/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs index 3a6c179c03147..d21913b78edac 100644 --- a/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs +++ b/src/meta/src/hummock/manager/compaction/compaction_group_manager.rs @@ -164,6 +164,7 @@ impl HummockManager { .cloned() .filter(|table_id| !valid_ids.contains(table_id)) .collect_vec(); + // As we have released versioning lock, the version that `to_unregister` is calculated from // may not be the same as the one used in unregister_table_ids. It is OK. self.unregister_table_ids(to_unregister).await @@ -276,10 +277,22 @@ impl HummockManager { &self, table_ids: impl IntoIterator + Send, ) -> Result<()> { - let mut table_ids = table_ids.into_iter().peekable(); - if table_ids.peek().is_none() { + let table_ids = table_ids.into_iter().collect_vec(); + if table_ids.is_empty() { return Ok(()); } + + { + // Remove table write throughput statistics + // The Caller acquires `Send`, so we should safely use `write` lock before the await point. + // The table write throughput statistic accepts data inconsistencies (unregister table ids fail), so we can clean it up in advance. + let mut table_write_throughput_statistic_manager = + self.table_write_throughput_statistic_manager.write(); + for table_id in table_ids.iter().unique() { + table_write_throughput_statistic_manager.remove_table(table_id.table_id); + } + } + let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); let mut version = HummockVersionTransaction::new( @@ -292,7 +305,7 @@ impl HummockManager { let mut modified_groups: HashMap = HashMap::new(); // Remove member tables - for table_id in table_ids.unique() { + for table_id in table_ids.into_iter().unique() { let version = new_version_delta.latest_version(); let Some(info) = version.state_table_info.info().get(&table_id) else { continue; @@ -356,6 +369,7 @@ impl HummockManager { version.latest_version(), ))); commit_multi_var!(self.meta_store_ref(), version, compaction_groups_txn)?; + // No need to handle DeltaType::GroupDestroy during time travel. Ok(()) } diff --git a/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs b/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs index 578a438225d07..8504a75fe9e25 100644 --- a/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs +++ b/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet, VecDeque}; +use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::ops::DerefMut; use std::sync::Arc; @@ -39,6 +39,10 @@ use crate::hummock::manager::transaction::HummockVersionTransaction; use crate::hummock::manager::{commit_multi_var, HummockManager}; use crate::hummock::metrics_utils::remove_compaction_group_in_sst_stat; use crate::hummock::sequence::{next_compaction_group_id, next_sstable_object_id}; +use crate::hummock::table_write_throughput_statistic::{ + TableWriteThroughputStatistic, TableWriteThroughputStatisticManager, +}; +use crate::manager::MetaOpts; impl HummockManager { pub async fn merge_compaction_group( @@ -85,12 +89,26 @@ impl HummockManager { .cloned() .collect_vec(); + if member_table_ids_1.is_empty() { + return Err(Error::CompactionGroup(format!( + "group_1 {} is empty", + group_1 + ))); + } + let mut member_table_ids_2 = state_table_info .compaction_group_member_table_ids(group_2) .iter() .cloned() .collect_vec(); + if member_table_ids_2.is_empty() { + return Err(Error::CompactionGroup(format!( + "group_2 {} is empty", + group_2 + ))); + } + debug_assert!(!member_table_ids_1.is_empty()); debug_assert!(!member_table_ids_2.is_empty()); assert!(member_table_ids_1.is_sorted()); @@ -692,8 +710,7 @@ impl HummockManager { /// Split the compaction group if the group is too large or contains high throughput tables. pub async fn try_split_compaction_group( &self, - table_write_throughput: &HashMap>, - checkpoint_secs: u64, + table_write_throughput_statistic_manager: &TableWriteThroughputStatisticManager, group: CompactionGroupStatistic, ) { if group @@ -707,10 +724,9 @@ impl HummockManager { // split high throughput table to dedicated compaction group for (table_id, table_size) in &group.table_statistic { self.try_move_high_throughput_table_to_dedicated_cg( - table_write_throughput, - table_id, + table_write_throughput_statistic_manager, + *table_id, table_size, - checkpoint_secs, group.group_id, ) .await; @@ -723,28 +739,24 @@ impl HummockManager { /// Try to move the high throughput table to a dedicated compaction group. pub async fn try_move_high_throughput_table_to_dedicated_cg( &self, - table_write_throughput: &HashMap>, - table_id: &u32, + table_write_throughput_statistic_manager: &TableWriteThroughputStatisticManager, + table_id: u32, _table_size: &u64, - checkpoint_secs: u64, parent_group_id: u64, ) { - if !table_write_throughput.contains_key(table_id) { - return; - } + let mut table_throughput = table_write_throughput_statistic_manager + .get_table_throughput_descending( + table_id, + self.env.opts.table_stat_throuput_window_seconds_for_split as i64, + ) + .peekable(); - let table_stat_throughput_sample_size = - ((self.env.opts.table_stat_throuput_window_seconds_for_split as f64) - / (checkpoint_secs as f64)) - .ceil() as usize; - let table_throughput = table_write_throughput.get(table_id).unwrap(); - if table_throughput.len() < table_stat_throughput_sample_size { + if table_throughput.peek().is_none() { return; } let is_high_write_throughput = is_table_high_write_throughput( table_throughput, - table_stat_throughput_sample_size, self.env.opts.table_high_write_throughput_threshold, self.env .opts @@ -759,7 +771,7 @@ impl HummockManager { let ret = self .move_state_tables_to_dedicated_compaction_group( parent_group_id, - &[*table_id], + &[table_id], Some(self.env.opts.partition_vnode_count), ) .await; @@ -830,15 +842,16 @@ impl HummockManager { pub async fn try_merge_compaction_group( &self, - table_write_throughput: &HashMap>, + table_write_throughput_statistic_manager: &TableWriteThroughputStatisticManager, group: &CompactionGroupStatistic, next_group: &CompactionGroupStatistic, - checkpoint_secs: u64, created_tables: &HashSet, ) -> Result<()> { // TODO: remove this check after refactor group id - if group.group_id == StaticCompactionGroupId::StateDefault as u64 - && next_group.group_id == StaticCompactionGroupId::MaterializedView as u64 + if (group.group_id == StaticCompactionGroupId::StateDefault as u64 + && next_group.group_id == StaticCompactionGroupId::MaterializedView as u64) + || (group.group_id == StaticCompactionGroupId::MaterializedView as u64 + && next_group.group_id == StaticCompactionGroupId::StateDefault as u64) { return Err(Error::CompactionGroup(format!( "group-{} and group-{} are both StaticCompactionGroupId", @@ -879,20 +892,10 @@ impl HummockManager { } // do not merge high throughput group - let table_stat_throughput_sample_size = - ((self.env.opts.table_stat_throuput_window_seconds_for_merge as f64) - / (checkpoint_secs as f64)) - .ceil() as usize; - - // merge the group which is low write throughput if !check_is_low_write_throughput_compaction_group( - table_write_throughput, - table_stat_throughput_sample_size, - self.env.opts.table_low_write_throughput_threshold, + table_write_throughput_statistic_manager, group, - self.env - .opts - .table_stat_low_write_throughput_ratio_for_merge, + &self.env.opts, ) { return Err(Error::CompactionGroup(format!( "Not Merge high throughput group {} next_group {}", @@ -918,13 +921,9 @@ impl HummockManager { } if !check_is_low_write_throughput_compaction_group( - table_write_throughput, - table_stat_throughput_sample_size, - self.env.opts.table_low_write_throughput_threshold, + table_write_throughput_statistic_manager, next_group, - self.env - .opts - .table_stat_low_write_throughput_ratio_for_merge, + &self.env.opts, ) { return Err(Error::CompactionGroup(format!( "Not Merge high throughput group {} next group {}", @@ -964,19 +963,15 @@ impl HummockManager { /// Check if the table is high write throughput with the given threshold and ratio. pub fn is_table_high_write_throughput( - table_throughput: &VecDeque, - sample_size: usize, + table_throughput: impl Iterator, threshold: u64, high_write_throughput_ratio: f64, ) -> bool { - assert!(table_throughput.len() >= sample_size); + let mut sample_size = 0; let mut high_write_throughput_count = 0; - for throughput in table_throughput - .iter() - .skip(table_throughput.len().saturating_sub(sample_size)) - { - // only check the latest window_size - if *throughput > threshold { + for statistic in table_throughput { + sample_size += 1; + if statistic.throughput > threshold { high_write_throughput_count += 1; } } @@ -985,19 +980,15 @@ pub fn is_table_high_write_throughput( } pub fn is_table_low_write_throughput( - table_throughput: &VecDeque, - sample_size: usize, + table_throughput: impl Iterator, threshold: u64, low_write_throughput_ratio: f64, ) -> bool { - assert!(table_throughput.len() >= sample_size); - + let mut sample_size = 0; let mut low_write_throughput_count = 0; - for throughput in table_throughput - .iter() - .skip(table_throughput.len().saturating_sub(sample_size)) - { - if *throughput <= threshold { + for statistic in table_throughput { + sample_size += 1; + if statistic.throughput <= threshold { low_write_throughput_count += 1; } } @@ -1006,32 +997,36 @@ pub fn is_table_low_write_throughput( } fn check_is_low_write_throughput_compaction_group( - table_write_throughput: &HashMap>, - sample_size: usize, - threshold: u64, + table_write_throughput_statistic_manager: &TableWriteThroughputStatisticManager, group: &CompactionGroupStatistic, - low_write_throughput_ratio: f64, + opts: &Arc, ) -> bool { - // check table exists - let live_table = group - .table_statistic - .keys() - .filter(|table_id| table_write_throughput.contains_key(table_id)) - .filter(|table_id| table_write_throughput.get(table_id).unwrap().len() >= sample_size) - .cloned() - .collect_vec(); + let mut table_with_statistic = Vec::with_capacity(group.table_statistic.len()); + for table_id in group.table_statistic.keys() { + let mut table_throughput = table_write_throughput_statistic_manager + .get_table_throughput_descending( + *table_id, + opts.table_stat_throuput_window_seconds_for_merge as i64, + ) + .peekable(); + if table_throughput.peek().is_none() { + continue; + } - if live_table.is_empty() { - return false; + table_with_statistic.push(table_throughput); } - live_table.into_iter().all(|table_id| { - let table_write_throughput = table_write_throughput.get(&table_id).unwrap(); + // if all tables in the group do not have enough statistics, return true + if table_with_statistic.is_empty() { + return true; + } + + // check if all tables in the group are low write throughput with enough statistics + table_with_statistic.into_iter().all(|table_throughput| { is_table_low_write_throughput( - table_write_throughput, - sample_size, - threshold, - low_write_throughput_ratio, + table_throughput, + opts.table_low_write_throughput_threshold, + opts.table_stat_low_write_throughput_ratio_for_merge, ) }) } diff --git a/src/meta/src/hummock/manager/compaction/mod.rs b/src/meta/src/hummock/manager/compaction/mod.rs index 6b8df20120528..bfa5e0ec97a56 100644 --- a/src/meta/src/hummock/manager/compaction/mod.rs +++ b/src/meta/src/hummock/manager/compaction/mod.rs @@ -836,8 +836,7 @@ impl HummockManager { self.calculate_vnode_partition( &mut compact_task, group_config.compaction_config.as_ref(), - ) - .await; + ); compact_task.table_watermarks = version .latest_version() .safe_epoch_table_watermarks(&compact_task.existing_table_ids); @@ -1429,7 +1428,7 @@ impl HummockManager { } } - pub(crate) async fn calculate_vnode_partition( + pub(crate) fn calculate_vnode_partition( &self, compact_task: &mut CompactTask, compaction_config: &CompactionConfig, @@ -1473,21 +1472,17 @@ impl HummockManager { .env .opts .compact_task_table_size_partition_threshold_high; - use risingwave_common::system_param::reader::SystemParamsRead; - let params = self.env.system_params_reader().await; - let barrier_interval_ms = params.barrier_interval_ms() as u64; - let checkpoint_secs = std::cmp::max( - 1, - params.checkpoint_frequency() * barrier_interval_ms / 1000, - ); // check latest write throughput - let history_table_throughput = self.history_table_throughput.read(); + let table_write_throughput_statistic_manager = + self.table_write_throughput_statistic_manager.read(); + let timestamp = chrono::Utc::now().timestamp(); for (table_id, compact_table_size) in table_size_info { - let write_throughput = history_table_throughput - .get(&table_id) - .map(|que| que.back().cloned().unwrap_or(0)) - .unwrap_or(0) - / checkpoint_secs; + let write_throughput = table_write_throughput_statistic_manager + .get_table_throughput_descending(table_id, timestamp) + .peekable() + .peek() + .map(|item| item.throughput) + .unwrap_or(0); if compact_table_size > compact_task_table_size_partition_threshold_high && default_partition_count > 0 { diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index 2d16fc31c1049..2e5118bbcd9a7 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap, VecDeque}; +use std::collections::BTreeMap; use std::ops::{Deref, DerefMut}; use std::sync::atomic::AtomicBool; use std::sync::Arc; @@ -34,6 +34,7 @@ use risingwave_pb::hummock::{ HummockVersionStats, PbCompactTaskAssignment, PbCompactionGroupInfo, SubscribeCompactionEventRequest, }; +use table_write_throughput_statistic::TableWriteThroughputStatisticManager; use tokio::sync::mpsc::UnboundedSender; use tokio::sync::{Mutex, Semaphore}; use tonic::Streaming; @@ -58,6 +59,7 @@ pub(crate) mod checkpoint; mod commit_epoch; mod compaction; pub mod sequence; +pub mod table_write_throughput_statistic; pub mod time_travel; mod timer_task; mod transaction; @@ -95,7 +97,8 @@ pub struct HummockManager { version_checkpoint_path: String, version_archive_dir: String, pause_version_checkpoint: AtomicBool, - history_table_throughput: parking_lot::RwLock>>, + table_write_throughput_statistic_manager: + parking_lot::RwLock, // for compactor // `compactor_streams_change_tx` is used to pass the mapping from `context_id` to event_stream @@ -245,6 +248,12 @@ impl HummockManager { state_store_dir, use_new_object_prefix_strategy, ); + + let max_table_statistic_expired_time = std::cmp::max( + env.opts.table_stat_throuput_window_seconds_for_split, + env.opts.table_stat_throuput_window_seconds_for_merge, + ) as i64; + let instance = HummockManager { env, versioning: MonitoredRwLock::new( @@ -276,7 +285,9 @@ impl HummockManager { version_checkpoint_path, version_archive_dir, pause_version_checkpoint: AtomicBool::new(false), - history_table_throughput: parking_lot::RwLock::new(HashMap::default()), + table_write_throughput_statistic_manager: parking_lot::RwLock::new( + TableWriteThroughputStatisticManager::new(max_table_statistic_expired_time), + ), compactor_streams_change_tx, compaction_state: CompactionState::new(), full_gc_state: FullGcState::new().into(), diff --git a/src/meta/src/hummock/manager/table_write_throughput_statistic.rs b/src/meta/src/hummock/manager/table_write_throughput_statistic.rs new file mode 100644 index 0000000000000..ddeb48eafde0b --- /dev/null +++ b/src/meta/src/hummock/manager/table_write_throughput_statistic.rs @@ -0,0 +1,115 @@ +// Copyright 2024 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::collections::{HashMap, VecDeque}; + +#[derive(Debug, Clone)] +pub struct TableWriteThroughputStatistic { + pub throughput: u64, + pub timestamp_secs: i64, +} + +impl AsRef for TableWriteThroughputStatistic { + fn as_ref(&self) -> &TableWriteThroughputStatistic { + self + } +} + +impl TableWriteThroughputStatistic { + pub fn is_expired(&self, max_statistic_expired_secs: i64, timestamp_secs: i64) -> bool { + // max(0) is used to avoid overflow + (timestamp_secs - self.timestamp_secs).max(0) > max_statistic_expired_secs + } +} + +#[derive(Debug, Clone)] +pub struct TableWriteThroughputStatisticManager { + table_throughput: HashMap>, + max_statistic_expired_secs: i64, +} + +impl TableWriteThroughputStatisticManager { + pub fn new(max_statistic_expired_secs: i64) -> Self { + Self { + table_throughput: HashMap::new(), + max_statistic_expired_secs, + } + } + + pub fn add_table_throughput_with_ts( + &mut self, + table_id: u32, + throughput: u64, + timestamp_secs: i64, + ) { + let table_throughput = self.table_throughput.entry(table_id).or_default(); + table_throughput.push_back(TableWriteThroughputStatistic { + throughput, + timestamp_secs, + }); + + // skip expired statistics + while let Some(statistic) = table_throughput.front() { + if statistic.is_expired(self.max_statistic_expired_secs, timestamp_secs) { + table_throughput.pop_front(); + } else { + break; + } + } + + if table_throughput.is_empty() { + self.table_throughput.remove(&table_id); + } + } + + // `get_table_throughput` return the statistics of the table with the given `table_id` within the given `window_secs`. + // The statistics are sorted by timestamp in descending order. + pub fn get_table_throughput_descending( + &self, + table_id: u32, + window_secs: i64, + ) -> impl Iterator { + let timestamp_secs = chrono::Utc::now().timestamp(); + self.table_throughput + .get(&table_id) + .into_iter() + .flatten() + .rev() + .take_while(move |statistic| !statistic.is_expired(window_secs, timestamp_secs)) + } + + pub fn remove_table(&mut self, table_id: u32) { + self.table_throughput.remove(&table_id); + } + + // `avg_write_throughput` returns the average write throughput of the table with the given `table_id` within the given `window_secs`. + pub fn avg_write_throughput(&self, table_id: u32, window_secs: i64) -> f64 { + let mut total_throughput = 0; + let mut total_count = 0; + let mut statistic_iter = self + .get_table_throughput_descending(table_id, window_secs) + .peekable(); + + if statistic_iter.peek().is_none() { + return 0.0; + } + + for statistic in statistic_iter { + total_throughput += statistic.throughput; + total_count += 1; + } + + total_throughput as f64 / total_count as f64 + } +} diff --git a/src/meta/src/hummock/manager/timer_task.rs b/src/meta/src/hummock/manager/timer_task.rs index 7434dd1f40f9b..cd948500aaa6f 100644 --- a/src/meta/src/hummock/manager/timer_task.rs +++ b/src/meta/src/hummock/manager/timer_task.rs @@ -20,7 +20,6 @@ use futures::future::Either; use futures::stream::BoxStream; use futures::{FutureExt, StreamExt}; use itertools::Itertools; -use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_hummock_sdk::compaction_group::hummock_version_ext::get_compaction_group_ids; use risingwave_pb::hummock::compact_task::{self, TaskStatus}; use risingwave_pb::hummock::level_handler::RunningCompactTask; @@ -162,6 +161,7 @@ impl HummockManager { ); scheduling_compaction_group_trigger_interval .set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + scheduling_compaction_group_trigger_interval.reset(); let group_scheduling_split_trigger = IntervalStream::new(scheduling_compaction_group_trigger_interval) .map(|_| HummockTimerEvent::GroupScheduleSplit); @@ -179,6 +179,7 @@ impl HummockManager { ); scheduling_compaction_group_merge_trigger_interval .set_missed_tick_behavior(tokio::time::MissedTickBehavior::Delay); + scheduling_compaction_group_merge_trigger_interval.reset(); let group_scheduling_merge_trigger = IntervalStream::new(scheduling_compaction_group_merge_trigger_interval) .map(|_| HummockTimerEvent::GroupScheduleMerge); @@ -284,8 +285,11 @@ impl HummockManager { .compaction_group_count .set(compaction_group_count as i64); - let tables_throughput = - hummock_manager.history_table_throughput.read().clone(); + let table_write_throughput_statistic_manager = + hummock_manager + .table_write_throughput_statistic_manager + .read() + .clone(); let current_version_levels = &hummock_manager .versioning @@ -304,16 +308,24 @@ impl HummockManager { .set(group_info.group_size as _); // accumulate the throughput of all tables in the group let mut avg_throuput = 0; + let max_statistic_expired_time = std::cmp::max( + hummock_manager + .env + .opts + .table_stat_throuput_window_seconds_for_split, + hummock_manager + .env + .opts + .table_stat_throuput_window_seconds_for_merge, + ); for table_id in group_info.table_statistic.keys() { - if let Some(throuput) = - tables_throughput.get(table_id) - { - let table_avg_throughput = - throuput.iter().sum::() - / throuput.len() as u64; - - avg_throuput += table_avg_throughput; - } + avg_throuput += + table_write_throughput_statistic_manager + .avg_write_throughput( + *table_id, + max_statistic_expired_time as i64, + ) + as u64; } hummock_manager @@ -546,13 +558,7 @@ impl HummockManager { /// 1. `state table throughput`: If the table is in a high throughput state and it belongs to a multi table group, then an attempt will be made to split the table into separate compaction groups to increase its throughput and reduce the impact on write amplification. /// 2. `group size`: If the group size has exceeded the set upper limit, e.g. `max_group_size` * `split_group_size_ratio` async fn on_handle_schedule_group_split(&self) { - let params = self.env.system_params_reader().await; - let barrier_interval_ms = params.barrier_interval_ms() as u64; - let checkpoint_secs = std::cmp::max( - 1, - params.checkpoint_frequency() * barrier_interval_ms / 1000, - ); - let table_write_throughput = self.history_table_throughput.read().clone(); + let table_write_throughput = self.table_write_throughput_statistic_manager.read().clone(); let mut group_infos = self.calculate_compaction_group_statistic().await; group_infos.sort_by_key(|group| group.group_size); group_infos.reverse(); @@ -563,7 +569,7 @@ impl HummockManager { continue; } - self.try_split_compaction_group(&table_write_throughput, checkpoint_secs, group) + self.try_split_compaction_group(&table_write_throughput, group) .await; } } @@ -594,7 +600,8 @@ impl HummockManager { return; } }; - let table_write_throughput = self.history_table_throughput.read().clone(); + let table_write_throughput_statistic_manager = + self.table_write_throughput_statistic_manager.read().clone(); let mut group_infos = self.calculate_compaction_group_statistic().await; // sort by first table id for deterministic merge order group_infos.sort_by_key(|group| { @@ -611,15 +618,6 @@ impl HummockManager { return; } - let params = self.env.system_params_reader().await; - let barrier_interval_ms = params.barrier_interval_ms() as u64; - let checkpoint_secs = { - std::cmp::max( - 1, - params.checkpoint_frequency() * barrier_interval_ms / 1000, - ) - }; - let mut left = 0; let mut right = left + 1; @@ -628,10 +626,9 @@ impl HummockManager { let next_group = &group_infos[right]; match self .try_merge_compaction_group( - &table_write_throughput, + &table_write_throughput_statistic_manager, group, next_group, - checkpoint_secs, &created_tables, ) .await diff --git a/src/storage/src/hummock/local_version/pinned_version.rs b/src/storage/src/hummock/local_version/pinned_version.rs index b8aeace6c8393..bccede6b4add5 100644 --- a/src/storage/src/hummock/local_version/pinned_version.rs +++ b/src/storage/src/hummock/local_version/pinned_version.rs @@ -131,7 +131,16 @@ impl PinnedVersion { } fn levels_by_compaction_groups_id(&self, compaction_group_id: CompactionGroupId) -> &Levels { - self.version.levels.get(&compaction_group_id).unwrap() + self.version + .levels + .get(&compaction_group_id) + .unwrap_or_else(|| { + panic!( + "levels for compaction group {} not found in version {}", + compaction_group_id, + self.id() + ) + }) } pub fn levels(&self, table_id: TableId) -> impl Iterator { From 9bc0b1fb017d0ee0d04249dc01a5f3c5c428c27a Mon Sep 17 00:00:00 2001 From: Shanicky Chen Date: Mon, 25 Nov 2024 17:10:48 +0800 Subject: [PATCH 009/163] feat: introduce node label (#19153) Signed-off-by: Shanicky Chen --- .../com/risingwave/java/utils/MetaClient.java | 3 +- proto/common.proto | 13 +++--- proto/meta.proto | 12 +----- .../src/worker_manager/worker_node_manager.rs | 6 +-- src/common/src/util/worker_util.rs | 2 + .../src/vnode_mapping/vnode_placement.rs | 22 ++++++---- src/compute/src/lib.rs | 9 ++++ src/compute/src/server.rs | 5 ++- src/ctl/src/common/meta_service.rs | 2 +- .../rw_catalog/rw_worker_nodes.rs | 12 +++++- src/frontend/src/handler/show.rs | 2 +- .../src/scheduler/distributed/query.rs | 12 +++--- src/frontend/src/session.rs | 2 +- src/meta/model/migration/src/lib.rs | 2 + .../src/m20241022_072553_node_label.rs | 35 +++++++++++++++ src/meta/model/src/worker_property.rs | 1 + src/meta/src/barrier/context/recovery.rs | 12 +++--- src/meta/src/controller/cluster.rs | 43 ++++++++++++------- src/meta/src/hummock/manager/tests.rs | 10 ++--- src/meta/src/hummock/test_utils.rs | 6 +-- src/meta/src/manager/metadata.rs | 4 +- src/meta/src/stream/scale.rs | 5 ++- src/meta/src/stream/stream_graph/schedule.rs | 2 +- src/meta/src/stream/stream_manager.rs | 5 +-- src/meta/src/stream/test_fragmenter.rs | 11 ++++- src/prost/src/lib.rs | 7 ++- src/rpc_client/src/meta_client.rs | 2 +- src/tests/simulation/src/ctl_ext.rs | 2 +- .../integration_tests/scale/schedulability.rs | 2 +- 29 files changed, 164 insertions(+), 87 deletions(-) create mode 100644 src/meta/model/migration/src/m20241022_072553_node_label.rs diff --git a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java index 28c1383e668f3..75e9324c37e40 100644 --- a/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java +++ b/java/common-utils/src/main/java/com/risingwave/java/utils/MetaClient.java @@ -18,6 +18,7 @@ import com.risingwave.proto.Catalog.Table; import com.risingwave.proto.ClusterServiceGrpc.ClusterServiceBlockingStub; import com.risingwave.proto.Common.HostAddress; +import com.risingwave.proto.Common.WorkerNode.Property; import com.risingwave.proto.Common.WorkerType; import com.risingwave.proto.DdlServiceGrpc.DdlServiceBlockingStub; import com.risingwave.proto.DdlServiceOuterClass.GetTableRequest; @@ -29,7 +30,6 @@ import com.risingwave.proto.Hummock.UnpinVersionBeforeRequest; import com.risingwave.proto.HummockManagerServiceGrpc.HummockManagerServiceBlockingStub; import com.risingwave.proto.Meta.AddWorkerNodeRequest; -import com.risingwave.proto.Meta.AddWorkerNodeRequest.Property; import com.risingwave.proto.Meta.AddWorkerNodeResponse; import com.risingwave.proto.Meta.HeartbeatRequest; import io.grpc.Grpc; @@ -100,7 +100,6 @@ public MetaClient(String metaAddr, ScheduledExecutorService scheduler) { Property.newBuilder() .setIsStreaming(false) .setIsServing(false) - .setWorkerNodeParallelism(0) .build()) .build(); AddWorkerNodeResponse resp = clusterStub.addWorkerNode(req); diff --git a/proto/common.proto b/proto/common.proto index 1030d07d7c343..05301494b5b44 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -56,6 +56,11 @@ message WorkerNode { bool is_unschedulable = 3; // This is used for frontend node to register its rpc address string internal_rpc_host_addr = 4; + // Meta may assign labels to worker nodes to partition workload by label. + // This is used for serverless backfilling of materialized views. + optional string node_label = 5; + + uint32 parallelism = 6; } message Resource { string rw_version = 1; @@ -83,11 +88,9 @@ message WorkerNode { // It's not persistent in meta store. optional uint64 started_at = 9; - uint32 parallelism = 10; - - // Meta may assign labels to worker nodes to partition workload by label. - // This is used for serverless backfilling of materialized views. - string node_label = 11; + // Moved to `Property` message. + reserved 10; + reserved "parallelism"; } message Buffer { diff --git a/proto/meta.proto b/proto/meta.proto index 37527d6a87ac0..e5dda6b83a922 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -342,21 +342,13 @@ service StreamManagerService { } // Below for cluster service. - message AddWorkerNodeRequest { - message Property { - uint64 worker_node_parallelism = 1; - bool is_streaming = 2; - bool is_serving = 3; - bool is_unschedulable = 4; - // This is used for frontend node to register its rpc address - string internal_rpc_host_addr = 5; - } common.WorkerType worker_type = 1; common.HostAddress host = 2; reserved 3; - Property property = 4; + reserved 4; common.WorkerNode.Resource resource = 5; + common.WorkerNode.Property property = 6; } message AddWorkerNodeResponse { diff --git a/src/batch/src/worker_manager/worker_node_manager.rs b/src/batch/src/worker_manager/worker_node_manager.rs index 98923bac62ba5..ed8c30de61141 100644 --- a/src/batch/src/worker_manager/worker_node_manager.rs +++ b/src/batch/src/worker_manager/worker_node_manager.rs @@ -412,12 +412,11 @@ mod tests { r#type: WorkerType::ComputeNode as i32, host: Some(HostAddr::try_from("127.0.0.1:1234").unwrap().to_protobuf()), state: worker_node::State::Running as i32, - parallelism: 0, property: Some(Property { is_unschedulable: false, is_serving: true, is_streaming: true, - internal_rpc_host_addr: "".to_string(), + ..Default::default() }), transactional_id: Some(1), ..Default::default() @@ -427,12 +426,11 @@ mod tests { r#type: WorkerType::ComputeNode as i32, host: Some(HostAddr::try_from("127.0.0.1:1235").unwrap().to_protobuf()), state: worker_node::State::Running as i32, - parallelism: 0, property: Some(Property { is_unschedulable: false, is_serving: true, is_streaming: false, - internal_rpc_host_addr: "".to_string(), + ..Default::default() }), transactional_id: Some(2), ..Default::default() diff --git a/src/common/src/util/worker_util.rs b/src/common/src/util/worker_util.rs index 80ecd3b822536..da11afca6e6b4 100644 --- a/src/common/src/util/worker_util.rs +++ b/src/common/src/util/worker_util.rs @@ -13,3 +13,5 @@ // limitations under the License. pub type WorkerNodeId = u32; + +pub const DEFAULT_COMPUTE_NODE_LABEL: &str = "default"; diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 1f9235bb862ae..33e544693d0ac 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -206,7 +206,7 @@ mod tests { use risingwave_common::hash::WorkerSlotMapping; use risingwave_pb::common::worker_node::Property; - use risingwave_pb::common::WorkerNode; + use risingwave_pb::common::{WorkerNode, WorkerType}; use crate::hash::VirtualNode; @@ -232,7 +232,7 @@ mod tests { is_unschedulable: false, is_serving: true, is_streaming: false, - internal_rpc_host_addr: "".to_string(), + ..Default::default() }; let count_same_vnode_mapping = |wm1: &WorkerSlotMapping, wm2: &WorkerSlotMapping| { @@ -248,10 +248,12 @@ mod tests { count }; + let mut property = serving_property.clone(); + property.parallelism = 1; let worker_1 = WorkerNode { id: 1, - parallelism: 1, - property: Some(serving_property.clone()), + r#type: WorkerType::ComputeNode.into(), + property: Some(property), ..Default::default() }; @@ -263,10 +265,12 @@ mod tests { let re_worker_mapping_2 = place_vnode(None, &[worker_1.clone()], None).unwrap(); assert_eq!(re_worker_mapping_2.iter_unique().count(), 1); + let mut property = serving_property.clone(); + property.parallelism = 50; let worker_2 = WorkerNode { id: 2, - parallelism: 50, - property: Some(serving_property.clone()), + property: Some(property), + r#type: WorkerType::ComputeNode.into(), ..Default::default() }; @@ -282,10 +286,12 @@ mod tests { let score = count_same_vnode_mapping(&re_worker_mapping_2, &re_worker_mapping); assert!(score >= 5); + let mut property = serving_property.clone(); + property.parallelism = 60; let worker_3 = WorkerNode { id: 3, - parallelism: 60, - property: Some(serving_property.clone()), + r#type: WorkerType::ComputeNode.into(), + property: Some(property), ..Default::default() }; let re_pu_mapping_2 = place_vnode( diff --git a/src/compute/src/lib.rs b/src/compute/src/lib.rs index ef4b5c5e32d3b..52e7719afe6f5 100644 --- a/src/compute/src/lib.rs +++ b/src/compute/src/lib.rs @@ -37,6 +37,7 @@ use risingwave_common::util::meta_addr::MetaAddressStrategy; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::util::tokio_util::sync::CancellationToken; +use risingwave_common::util::worker_util::DEFAULT_COMPUTE_NODE_LABEL; use serde::{Deserialize, Serialize}; /// If `total_memory_bytes` is not specified, the default memory limit will be set to @@ -104,6 +105,10 @@ pub struct ComputeNodeOpts { #[override_opts(if_absent, path = streaming.actor_runtime_worker_threads_num)] pub parallelism: usize, + /// The parallelism that the compute node will register to the scheduler of the meta service. + #[clap(long, env = "RW_NODE_LABEL", default_value_t = default_node_label())] + pub node_label: String, + /// Decides whether the compute node can be used for streaming and serving. #[clap(long, env = "RW_COMPUTE_NODE_ROLE", value_enum, default_value_t = default_role())] pub role: Role, @@ -249,6 +254,10 @@ pub fn default_parallelism() -> usize { total_cpu_available().ceil() as usize } +pub fn default_node_label() -> String { + DEFAULT_COMPUTE_NODE_LABEL.to_string() +} + pub fn default_role() -> Role { Role::Both } diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index d86a516771802..aae537271de48 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -42,10 +42,10 @@ use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::{MetricsManager, ObserverManager, TracingExtractLayer}; use risingwave_connector::source::monitor::GLOBAL_SOURCE_METRICS; use risingwave_dml::dml_manager::DmlManager; +use risingwave_pb::common::worker_node::Property; use risingwave_pb::common::WorkerType; use risingwave_pb::compute::config_service_server::ConfigServiceServer; use risingwave_pb::health::health_server::HealthServer; -use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_pb::monitor_service::monitor_service_server::MonitorServiceServer; use risingwave_pb::stream_service::stream_service_server::StreamServiceServer; use risingwave_pb::task_service::exchange_service_server::ExchangeServiceServer; @@ -124,11 +124,12 @@ pub async fn compute_node_serve( WorkerType::ComputeNode, &advertise_addr, Property { - worker_node_parallelism: opts.parallelism as u64, + parallelism: opts.parallelism as u32, is_streaming: opts.role.for_streaming(), is_serving: opts.role.for_serving(), is_unschedulable: false, internal_rpc_host_addr: "".to_string(), + node_label: Some(opts.node_label.clone()), }, &config.meta, ) diff --git a/src/ctl/src/common/meta_service.rs b/src/ctl/src/common/meta_service.rs index 6d70bdf942833..f91f47b6f2951 100644 --- a/src/ctl/src/common/meta_service.rs +++ b/src/ctl/src/common/meta_service.rs @@ -17,8 +17,8 @@ use std::env; use anyhow::{bail, Result}; use risingwave_common::config::MetaConfig; use risingwave_common::util::addr::HostAddr; +use risingwave_pb::common::worker_node::Property; use risingwave_pb::common::WorkerType; -use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_rpc_client::MetaClient; pub struct MetaServiceOpts { diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs index b50c7e4cfd07b..cbe584bee53a4 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs @@ -38,6 +38,7 @@ struct RwWorkerNode { system_total_memory_bytes: Option, system_total_cpu_cores: Option, started_at: Option, + label: Option, } #[system_catalog(table, "rw_catalog.rw_worker_nodes")] @@ -58,7 +59,11 @@ async fn read_rw_worker_nodes_info(reader: &SysCatalogReaderImpl) -> Result Result Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(WorkerProperty::Table) + .add_column(ColumnDef::new(WorkerProperty::Label).string()) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(WorkerProperty::Table) + .drop_column(WorkerProperty::Label) + .to_owned(), + ) + .await + } +} + +#[derive(DeriveIden)] +enum WorkerProperty { + Table, + Label, +} diff --git a/src/meta/model/src/worker_property.rs b/src/meta/model/src/worker_property.rs index ff19cdeb6f65b..09fa662bb2ca7 100644 --- a/src/meta/model/src/worker_property.rs +++ b/src/meta/model/src/worker_property.rs @@ -27,6 +27,7 @@ pub struct Model { pub is_serving: bool, pub is_unschedulable: bool, pub internal_rpc_host_addr: Option, + pub label: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] diff --git a/src/meta/src/barrier/context/recovery.rs b/src/meta/src/barrier/context/recovery.rs index ee25f1ae84117..b352ece5012a1 100644 --- a/src/meta/src/barrier/context/recovery.rs +++ b/src/meta/src/barrier/context/recovery.rs @@ -267,9 +267,7 @@ impl GlobalBarrierWorkerContextImpl { let active_worker_slots: HashSet<_> = active_nodes .current() .values() - .flat_map(|node| { - (0..node.parallelism).map(|idx| WorkerSlotId::new(node.id, idx as usize)) - }) + .flat_map(|node| (0..node.parallelism()).map(|idx| WorkerSlotId::new(node.id, idx))) .collect(); let expired_worker_slots: BTreeSet<_> = all_inuse_worker_slots @@ -298,7 +296,7 @@ impl GlobalBarrierWorkerContextImpl { .current() .values() .flat_map(|worker| { - (0..worker.parallelism).map(move |i| WorkerSlotId::new(worker.id, i as _)) + (0..worker.parallelism()).map(move |i| WorkerSlotId::new(worker.id, i as _)) }) .collect_vec(); @@ -316,7 +314,7 @@ impl GlobalBarrierWorkerContextImpl { .current() .values() .flat_map(|worker| { - (0..worker.parallelism * factor) + (0..worker.parallelism() * factor) .map(move |i| WorkerSlotId::new(worker.id, i as _)) }) .collect_vec(); @@ -372,7 +370,7 @@ impl GlobalBarrierWorkerContextImpl { let current_nodes = active_nodes .current() .values() - .map(|node| (node.id, &node.host, node.parallelism)) + .map(|node| (node.id, &node.host, node.parallelism())) .collect_vec(); warn!( current_nodes = ?current_nodes, @@ -413,7 +411,7 @@ impl GlobalBarrierWorkerContextImpl { let available_parallelism = active_nodes .current() .values() - .map(|worker_node| worker_node.parallelism as usize) + .map(|worker_node| worker_node.parallelism()) .sum(); let table_parallelisms: HashMap<_, _> = { diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 8d58ba8fd6c1b..242cbbafddd41 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -24,14 +24,16 @@ use risingwave_common::hash::WorkerSlotId; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; +use risingwave_common::util::worker_util::DEFAULT_COMPUTE_NODE_LABEL; use risingwave_common::RW_VERSION; use risingwave_license::LicenseManager; use risingwave_meta_model::prelude::{Worker, WorkerProperty}; use risingwave_meta_model::worker::{WorkerStatus, WorkerType}; use risingwave_meta_model::{worker, worker_property, TransactionId, WorkerId}; -use risingwave_pb::common::worker_node::{PbProperty, PbResource, PbState}; +use risingwave_pb::common::worker_node::{ + PbProperty, PbProperty as AddNodeProperty, PbResource, PbState, +}; use risingwave_pb::common::{HostAddress, PbHostAddress, PbWorkerNode, PbWorkerType, WorkerNode}; -use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::update_worker_node_schedulability_request::Schedulability; use sea_orm::prelude::Expr; @@ -76,17 +78,17 @@ impl From for PbWorkerNode { port: info.0.port, }), state: PbState::from(info.0.status) as _, - parallelism: info.1.as_ref().map(|p| p.parallelism).unwrap_or_default() as u32, property: info.1.as_ref().map(|p| PbProperty { is_streaming: p.is_streaming, is_serving: p.is_serving, is_unschedulable: p.is_unschedulable, internal_rpc_host_addr: p.internal_rpc_host_addr.clone().unwrap_or_default(), + node_label: p.label.clone(), + parallelism: info.1.as_ref().map(|p| p.parallelism).unwrap_or_default() as u32, }), transactional_id: info.0.transaction_id.map(|id| id as _), resource: info.2.resource, started_at: info.2.started_at, - node_label: "".to_string(), } } } @@ -394,7 +396,7 @@ impl StreamingClusterInfo { pub fn parallelism(&self) -> usize { self.worker_nodes .values() - .map(|worker| worker.parallelism as usize) + .map(|worker| worker.parallelism()) .sum() } } @@ -443,7 +445,6 @@ fn meta_node_info(host: &str, started_at: Option) -> PbWorkerNode { .map(HostAddr::to_protobuf) .ok(), state: PbState::Running as _, - parallelism: 0, property: None, transactional_id: None, resource: Some(risingwave_pb::common::worker_node::Resource { @@ -452,7 +453,6 @@ fn meta_node_info(host: &str, started_at: Option) -> PbWorkerNode { total_cpu_cores: total_cpu_available() as _, }), started_at, - node_label: "".to_string(), } } @@ -628,7 +628,7 @@ impl ClusterControllerInner { return if worker.worker_type == WorkerType::ComputeNode { let property = property.unwrap(); let mut current_parallelism = property.parallelism as usize; - let new_parallelism = add_property.worker_node_parallelism as usize; + let new_parallelism = add_property.parallelism as usize; match new_parallelism.cmp(¤t_parallelism) { Ordering::Less => { if !self.disable_automatic_parallelism_control { @@ -668,6 +668,13 @@ impl ClusterControllerInner { property.is_streaming = Set(add_property.is_streaming); property.is_serving = Set(add_property.is_serving); property.parallelism = Set(current_parallelism as _); + property.label = Set(Some(add_property.node_label.unwrap_or_else(|| { + tracing::warn!( + "node_label is not set for worker {}, fallback to `default`", + worker.worker_id + ); + DEFAULT_COMPUTE_NODE_LABEL.to_string() + }))); WorkerProperty::update(property).exec(&txn).await?; txn.commit().await?; @@ -678,13 +685,14 @@ impl ClusterControllerInner { let worker_property = worker_property::ActiveModel { worker_id: Set(worker.worker_id), parallelism: Set(add_property - .worker_node_parallelism + .parallelism .try_into() .expect("invalid parallelism")), is_streaming: Set(add_property.is_streaming), is_serving: Set(add_property.is_serving), is_unschedulable: Set(add_property.is_unschedulable), internal_rpc_host_addr: Set(Some(add_property.internal_rpc_host_addr)), + label: Set(None), }; WorkerProperty::insert(worker_property).exec(&txn).await?; txn.commit().await?; @@ -713,13 +721,18 @@ impl ClusterControllerInner { let property = worker_property::ActiveModel { worker_id: Set(worker_id), parallelism: Set(add_property - .worker_node_parallelism + .parallelism .try_into() .expect("invalid parallelism")), is_streaming: Set(add_property.is_streaming), is_serving: Set(add_property.is_serving), is_unschedulable: Set(add_property.is_unschedulable), internal_rpc_host_addr: Set(Some(add_property.internal_rpc_host_addr)), + label: if r#type == PbWorkerType::ComputeNode { + Set(add_property.node_label.clone()) + } else { + Set(None) + }, }; WorkerProperty::insert(property).exec(&txn).await?; } @@ -952,11 +965,11 @@ mod tests { let parallelism_num = 4_usize; let worker_count = 5_usize; let property = AddNodeProperty { - worker_node_parallelism: parallelism_num as _, + parallelism: parallelism_num as _, is_streaming: true, is_serving: true, is_unschedulable: false, - internal_rpc_host_addr: "".to_string(), + ..Default::default() }; let hosts = mock_worker_hosts_for_test(worker_count); let mut worker_ids = vec![]; @@ -999,7 +1012,7 @@ mod tests { // re-register existing worker node with larger parallelism and change its serving mode. let mut new_property = property.clone(); - new_property.worker_node_parallelism = (parallelism_num * 2) as _; + new_property.parallelism = (parallelism_num * 2) as _; new_property.is_serving = false; cluster_ctl .add_worker( @@ -1043,11 +1056,11 @@ mod tests { port: 5001, }; let mut property = AddNodeProperty { - worker_node_parallelism: 4, is_streaming: true, is_serving: true, is_unschedulable: false, - internal_rpc_host_addr: "".to_string(), + parallelism: 4, + ..Default::default() }; let worker_id = cluster_ctl .add_worker( diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 428bf48343f55..c96911aea163d 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -36,10 +36,10 @@ use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, SyncResult, FIRST_VERSION_ID, }; +use risingwave_pb::common::worker_node::Property; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::HummockPinnedVersion; -use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_rpc_client::HummockMetaClient; use thiserror_ext::AsReport; @@ -381,11 +381,11 @@ async fn test_release_context_resource() { WorkerType::ComputeNode, fake_host_address_2, Property { - worker_node_parallelism: fake_parallelism, + parallelism: fake_parallelism, is_streaming: true, is_serving: true, is_unschedulable: false, - internal_rpc_host_addr: "".to_string(), + ..Default::default() }, Default::default(), ) @@ -464,11 +464,11 @@ async fn test_hummock_manager_basic() { WorkerType::ComputeNode, fake_host_address_2, Property { - worker_node_parallelism: fake_parallelism, + parallelism: fake_parallelism, is_streaming: true, is_serving: true, is_unschedulable: false, - internal_rpc_host_addr: "".to_string(), + ..Default::default() }, Default::default(), ) diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index b6631a672e385..2e750a06aa222 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -32,10 +32,10 @@ use risingwave_hummock_sdk::{ CompactionGroupId, HummockEpoch, HummockSstableObjectId, LocalSstableInfo, SyncResult, }; use risingwave_meta_model::WorkerId; +use risingwave_pb::common::worker_node::Property; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::hummock::compact_task::TaskStatus; use risingwave_pb::hummock::CompactionConfig; -use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_rpc_client::HummockMetaClient; use crate::controller::catalog::CatalogController; @@ -347,11 +347,11 @@ pub async fn setup_compute_env_with_metric( WorkerType::ComputeNode, fake_host_address, Property { - worker_node_parallelism: fake_parallelism as _, is_streaming: true, is_serving: true, is_unschedulable: false, - internal_rpc_host_addr: "".to_string(), + parallelism: fake_parallelism as _, + ..Default::default() }, Default::default(), ) diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index a10d405d31dff..f2510a5b75486 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -22,9 +22,8 @@ use futures::future::{select, Either}; use risingwave_common::catalog::{DatabaseId, TableId, TableOption}; use risingwave_meta_model::{ObjectId, SourceId, WorkerId}; use risingwave_pb::catalog::{PbSink, PbSource, PbTable}; -use risingwave_pb::common::worker_node::{PbResource, State}; +use risingwave_pb::common::worker_node::{PbResource, Property as AddNodeProperty, State}; use risingwave_pb::common::{HostAddress, PbWorkerNode, PbWorkerType, WorkerNode, WorkerType}; -use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; use risingwave_pb::meta::list_rate_limits_response::RateLimitInfo; use risingwave_pb::meta::table_fragments::{Fragment, PbFragment}; use risingwave_pb::stream_plan::{PbDispatchStrategy, StreamActor}; @@ -225,7 +224,6 @@ impl ActiveStreamingWorkerNodes { id: node.id, r#type: node.r#type, host: node.host.clone(), - parallelism: node.parallelism, property: node.property.clone(), resource: node.resource.clone(), ..Default::default() diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index d6f1d54b73849..c4a97d25d7392 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -1816,7 +1816,7 @@ impl ScaleController { let schedulable_worker_slots = workers .values() - .map(|worker| (worker.id as WorkerId, worker.parallelism as usize)) + .map(|worker| (worker.id as WorkerId, worker.parallelism())) .collect::>(); // index for no shuffle relation @@ -2576,7 +2576,8 @@ impl GlobalStreamManager { let prev_worker = worker_cache.insert(worker.id, worker.clone()); match prev_worker { - Some(prev_worker) if prev_worker.get_parallelism() != worker.get_parallelism() => { + // todo, add label checking in further changes + Some(prev_worker) if prev_worker.parallelism() != worker.parallelism() => { tracing::info!(worker = worker.id, "worker parallelism changed"); should_trigger = true; } diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index 7a14cb43a342e..0cae1abf24654 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -223,7 +223,7 @@ impl Scheduler { let slots = workers .iter() - .map(|(worker_id, worker)| (*worker_id as WorkerId, worker.parallelism as usize)) + .map(|(worker_id, worker)| (*worker_id as WorkerId, worker.parallelism())) .collect(); let parallelism = default_parallelism.get(); diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index cd76e124a2d1f..b1bbc04fe5635 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -627,10 +627,7 @@ impl GlobalStreamManager { .collect::>(); // Check if the provided parallelism is valid. - let available_parallelism = worker_nodes - .iter() - .map(|w| w.parallelism as usize) - .sum::(); + let available_parallelism = worker_nodes.iter().map(|w| w.parallelism()).sum::(); let max_parallelism = self .metadata_manager .get_job_max_parallelism(table_id) diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index db34e5fd312bd..cfde9187abc66 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -20,7 +20,10 @@ use itertools::Itertools; use risingwave_common::catalog::{DatabaseId, SchemaId, TableId}; use risingwave_common::hash::VirtualNode; use risingwave_pb::catalog::PbTable; -use risingwave_pb::common::{PbColumnOrder, PbDirection, PbNullsAre, PbOrderType, WorkerNode}; +use risingwave_pb::common::worker_node::Property; +use risingwave_pb::common::{ + PbColumnOrder, PbDirection, PbNullsAre, PbOrderType, WorkerNode, WorkerType, +}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; use risingwave_pb::ddl_service::TableJobType; @@ -426,7 +429,11 @@ fn make_cluster_info() -> StreamingClusterInfo { 0, WorkerNode { id: 0, - parallelism: 8, + property: Some(Property { + parallelism: 8, + ..Default::default() + }), + r#type: WorkerType::ComputeNode.into(), ..Default::default() }, )) diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index a4678df091270..15a0d4b4ff1ba 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -23,6 +23,7 @@ pub use prost::Message; use risingwave_error::tonic::ToTonicStatus; use thiserror::Error; +use crate::common::WorkerType; #[rustfmt::skip] #[cfg_attr(madsim, path = "sim/catalog.rs")] @@ -220,7 +221,11 @@ impl stream_plan::MaterializeNode { // Encapsulating the use of parallelism. impl common::WorkerNode { pub fn parallelism(&self) -> usize { - self.parallelism as usize + assert_eq!(self.r#type(), WorkerType::ComputeNode); + self.property + .as_ref() + .expect("property should be exist") + .parallelism as usize } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index d027608e34600..8f4e6779b8e56 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -54,6 +54,7 @@ use risingwave_pb::catalog::{ }; use risingwave_pb::cloud_service::cloud_service_client::CloudServiceClient; use risingwave_pb::cloud_service::*; +use risingwave_pb::common::worker_node::Property; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::connector_service::sink_coordination_service_client::SinkCoordinationServiceClient; use risingwave_pb::ddl_service::alter_owner_request::Object; @@ -68,7 +69,6 @@ use risingwave_pb::hummock::rise_ctl_update_compaction_config_request::mutable_c use risingwave_pb::hummock::subscribe_compaction_event_request::Register; use risingwave_pb::hummock::write_limits::WriteLimit; use risingwave_pb::hummock::*; -use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_pb::meta::cancel_creating_jobs_request::PbJobs; use risingwave_pb::meta::cluster_service_client::ClusterServiceClient; use risingwave_pb::meta::event_log_service_client::EventLogServiceClient; diff --git a/src/tests/simulation/src/ctl_ext.rs b/src/tests/simulation/src/ctl_ext.rs index 3986a826e21e7..2a022165c8525 100644 --- a/src/tests/simulation/src/ctl_ext.rs +++ b/src/tests/simulation/src/ctl_ext.rs @@ -227,7 +227,7 @@ impl Fragment { self.r .worker_nodes .iter() - .map(|w| (w.id, w.parallelism as usize)) + .map(|w| (w.id, w.parallelism())) .collect() } diff --git a/src/tests/simulation/tests/integration_tests/scale/schedulability.rs b/src/tests/simulation/tests/integration_tests/scale/schedulability.rs index a8a6f73eedd3d..77275a39df70a 100644 --- a/src/tests/simulation/tests/integration_tests/scale/schedulability.rs +++ b/src/tests/simulation/tests/integration_tests/scale/schedulability.rs @@ -39,7 +39,7 @@ async fn test_cordon_normal() -> Result<()> { let rest_worker_slots: HashSet<_> = workers .iter() .flat_map(|worker| { - (0..worker.parallelism).map(|idx| WorkerSlotId::new(worker.id, idx as _)) + (0..worker.parallelism()).map(|idx| WorkerSlotId::new(worker.id, idx as _)) }) .collect(); From fcea1bd2200b1bc1de5157fc6b8465e50839a6cb Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Mon, 25 Nov 2024 20:15:04 +0800 Subject: [PATCH 010/163] chore(ci): increase test timeout for deterministic tests (#19532) --- ci/workflows/main-cron.yml | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index f1fffc51dcabb..49bb47eb984b6 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -143,7 +143,7 @@ steps: files: "*-junit.xml" format: "junit" - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 11 + timeout_in_minutes: 13 retry: *auto-retry - label: "end-to-end test (parallel, in-memory) (release)" @@ -347,7 +347,7 @@ steps: - label: "end-to-end test (madsim)" key: "e2e-test-deterministic" - command: "TEST_NUM=32 timeout 120m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=32 timeout 130m ci/scripts/deterministic-e2e-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" @@ -364,12 +364,12 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 120 + timeout_in_minutes: 135 retry: *auto-retry - label: "end-to-end test (madsim, random vnode count)" key: "e2e-test-deterministic-random-vnode-count" - command: "TEST_NUM=32 RW_SIM_RANDOM_VNODE_COUNT=true timeout 120m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=32 RW_SIM_RANDOM_VNODE_COUNT=true timeout 130m ci/scripts/deterministic-e2e-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-e2e-test-deterministic-simulation" @@ -386,12 +386,12 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 120 + timeout_in_minutes: 135 retry: *auto-retry - label: "recovery test (madsim)" key: "recovery-test-deterministic" - command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 timeout 65m ci/scripts/deterministic-recovery-test.sh" + command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 timeout 70m ci/scripts/deterministic-recovery-test.sh" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null || build.pull_request.labels includes "ci/run-recovery-test-deterministic-simulation" @@ -404,7 +404,7 @@ steps: mount-buildkite-agent: true # Only upload zipped files, otherwise the logs is too much. - ./ci/plugins/upload-failure-logs-zipped - timeout_in_minutes: 70 + timeout_in_minutes: 75 retry: *auto-retry # Ddl statements will randomly run with background_ddl. From 6cae4d3fcffb931aa799764494a3f8f96e66c9e8 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 26 Nov 2024 11:12:51 +0800 Subject: [PATCH 011/163] refactor(meta): provide better message for meta store migration failure (#19551) Signed-off-by: Bugen Zhao --- src/meta/src/controller/mod.rs | 38 +++++++++++++++++++++++++++++++--- src/meta/src/manager/env.rs | 34 +++++++----------------------- 2 files changed, 43 insertions(+), 29 deletions(-) diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 876862acb34b9..c01c34bc9927c 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -14,13 +14,14 @@ use std::collections::BTreeMap; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use risingwave_common::hash::VnodeCount; use risingwave_common::util::epoch::Epoch; use risingwave_meta_model::{ connection, database, function, index, object, schema, secret, sink, source, subscription, table, view, }; +use risingwave_meta_model_migration::{MigrationStatus, Migrator, MigratorTrait}; use risingwave_pb::catalog::connection::PbInfo as PbConnectionInfo; use risingwave_pb::catalog::source::PbOptionalAssociatedTableId; use risingwave_pb::catalog::subscription::PbSubscriptionState; @@ -32,7 +33,7 @@ use risingwave_pb::catalog::{ }; use sea_orm::{DatabaseConnection, ModelTrait}; -use crate::MetaError; +use crate::{MetaError, MetaResult}; pub mod catalog; pub mod cluster; @@ -70,11 +71,42 @@ impl SqlMetaStore { #[cfg(any(test, feature = "test"))] pub async fn for_test() -> Self { - use risingwave_meta_model_migration::{Migrator, MigratorTrait}; let conn = sea_orm::Database::connect(IN_MEMORY_STORE).await.unwrap(); Migrator::up(&conn, None).await.unwrap(); Self { conn } } + + /// Check whether the cluster, which uses SQL as the backend, is a new cluster. + /// It determines this by inspecting the applied migrations. If the migration `m20230908_072257_init` has been applied, + /// then it is considered an old cluster. + /// + /// Note: this check should be performed before [`Self::up()`]. + async fn is_first_launch(&self) -> MetaResult { + let migrations = Migrator::get_applied_migrations(&self.conn) + .await + .context("failed to get applied migrations")?; + for migration in migrations { + if migration.name() == "m20230908_072257_init" + && migration.status() == MigrationStatus::Applied + { + return Ok(false); + } + } + Ok(true) + } + + /// Apply all the migrations to the meta store before starting the service. + /// + /// Returns whether the cluster is the first launch. + pub async fn up(&self) -> MetaResult { + let cluster_first_launch = self.is_first_launch().await?; + // Try to upgrade if any new model changes are added. + Migrator::up(&self.conn, None) + .await + .context("failed to upgrade models in meta store")?; + + Ok(cluster_first_launch) + } } pub struct ObjectModel(M, object::Model); diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 2d11556f6ef80..deb4dd104b520 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -16,12 +16,12 @@ use std::ops::Deref; use std::path::PathBuf; use std::sync::Arc; +use anyhow::Context; use risingwave_common::config::{CompactionConfig, DefaultParallelism, ObjectStoreConfig}; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; use risingwave_common::{bail, system_param}; use risingwave_meta_model::prelude::Cluster; -use risingwave_meta_model_migration::{MigrationStatus, Migrator, MigratorTrait}; use risingwave_pb::meta::SystemParams; use risingwave_rpc_client::{ FrontendClientPool, FrontendClientPoolRef, StreamClientPool, StreamClientPoolRef, @@ -329,25 +329,6 @@ impl MetaOpts { } } -/// This function `is_first_launch_for_sql_backend_cluster` is used to check whether the cluster, which uses SQL as the backend, is a new cluster. -/// It determines this by inspecting the applied migrations. If the migration `m20230908_072257_init` has been applied, -/// then it is considered an old cluster. -/// -/// Note: this check should be performed before `Migrator::up()`. -pub async fn is_first_launch_for_sql_backend_cluster( - sql_meta_store: &SqlMetaStore, -) -> MetaResult { - let migrations = Migrator::get_applied_migrations(&sql_meta_store.conn).await?; - for migration in migrations { - if migration.name() == "m20230908_072257_init" - && migration.status() == MigrationStatus::Applied - { - return Ok(false); - } - } - Ok(true) -} - impl MetaSrvEnv { pub async fn new( opts: MetaOpts, @@ -376,12 +357,13 @@ impl MetaSrvEnv { ); } - let cluster_first_launch = - is_first_launch_for_sql_backend_cluster(&meta_store_impl).await?; - // Try to upgrade if any new model changes are added. - Migrator::up(&meta_store_impl.conn, None) - .await - .expect("Failed to upgrade models in meta store"); + let cluster_first_launch = meta_store_impl.up().await.context( + "Failed to initialize the meta store, \ + this may happen if there's existing metadata incompatible with the current version of RisingWave, \ + e.g., downgrading from a newer release or a nightly build to an older one. \ + For a single-node deployment, you may want to reset all data by deleting the data directory, \ + typically located at `~/.risingwave`.", + )?; let notification_manager = Arc::new(NotificationManager::new(meta_store_impl.clone()).await); From b34f4fdf4c711c614a5985ce3aa413a4bd65d2e8 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Tue, 26 Nov 2024 12:44:29 +0800 Subject: [PATCH 012/163] refactor(frontend): better cast error message for `UPDATE` setting to subquery (#19555) Signed-off-by: Bugen Zhao --- e2e_test/batch/basic/dml_update.slt.part | 17 ++++++++++++-- .../tests/testdata/output/update.yaml | 2 +- src/frontend/src/binder/update.rs | 7 +++--- src/frontend/src/expr/subquery.rs | 18 ++++++++++++--- src/frontend/src/expr/type_inference/cast.rs | 23 ++++++------------- 5 files changed, 42 insertions(+), 25 deletions(-) diff --git a/e2e_test/batch/basic/dml_update.slt.part b/e2e_test/batch/basic/dml_update.slt.part index fc2647cea147b..0adc1ce5f27e6 100644 --- a/e2e_test/batch/basic/dml_update.slt.part +++ b/e2e_test/batch/basic/dml_update.slt.part @@ -99,8 +99,21 @@ update t set (v1, v2) = (select '888.88', 999); db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): - 1: cannot cast type "record" to "record" - 2: cannot cast type "character varying" to "integer" in Assign context + 1: cannot cast type "record" to "struct" + 2: cannot cast to struct field "v1" + 3: cannot cast type "character varying" to "integer" in Assign context + + +# Multiple assignments, to subquery (with column name) with cast failure. +statement error +update t set (v1, v2) = (select '888.88' s1, 999 s2); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: cannot cast type "struct" to "struct" + 2: cannot cast struct field "s1" to struct field "v1" + 3: cannot cast type "character varying" to "integer" in Assign context # Multiple assignments, to subquery with mismatched column count. diff --git a/src/frontend/planner_test/tests/testdata/output/update.yaml b/src/frontend/planner_test/tests/testdata/output/update.yaml index 26c6d52dc5e05..42246cabbb0f1 100644 --- a/src/frontend/planner_test/tests/testdata/output/update.yaml +++ b/src/frontend/planner_test/tests/testdata/output/update.yaml @@ -235,7 +235,7 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchUpdate { table: t, exprs: [Field($4, 0:Int32), Field($4, 1:Int32), $2] } - └─BatchProject { exprs: [t.v1, t.v2, t._row_id, t._rw_timestamp, $expr10011::Struct(StructType { field_names: [], field_types: [Int32, Int32] }) as $expr1] } + └─BatchProject { exprs: [t.v1, t.v2, t._row_id, t._rw_timestamp, $expr10011::Struct(StructType { field_names: ["v1", "v2"], field_types: [Int32, Int32] }) as $expr1] } └─BatchNestedLoopJoin { type: LeftOuter, predicate: true, output: all } ├─BatchExchange { order: [], dist: Single } │ └─BatchScan { table: t, columns: [t.v1, t.v2, t._row_id, t._rw_timestamp], distribution: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/binder/update.rs b/src/frontend/src/binder/update.rs index f57ad1d197982..a2038a4d471d9 100644 --- a/src/frontend/src/binder/update.rs +++ b/src/frontend/src/binder/update.rs @@ -161,8 +161,8 @@ impl Binder { for Assignment { id, value } in assignments { let ids: Vec<_> = id - .into_iter() - .map(|id| self.bind_expr(Expr::Identifier(id))) + .iter() + .map(|id| self.bind_expr(Expr::Identifier(id.clone()))) .try_collect()?; match (ids.as_slice(), value) { @@ -206,8 +206,9 @@ impl Binder { bail_bind_error!("number of columns does not match number of values"); } - let target_type = DataType::new_unnamed_struct( + let target_type = DataType::new_struct( ids.iter().map(|id| id.return_type()).collect(), + id.iter().map(|id| id.real_value()).collect(), ); let expr = expr.cast_assign(target_type)?; diff --git a/src/frontend/src/expr/subquery.rs b/src/frontend/src/expr/subquery.rs index 8460f73d5fbba..fc904638790e2 100644 --- a/src/frontend/src/expr/subquery.rs +++ b/src/frontend/src/expr/subquery.rs @@ -14,10 +14,10 @@ use std::hash::Hash; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, StructType}; use super::{Expr, ExprImpl, ExprType}; -use crate::binder::BoundQuery; +use crate::binder::{BoundQuery, UNNAMED_COLUMN}; use crate::expr::{CorrelatedId, Depth}; #[derive(Clone, Debug, PartialEq, Eq)] @@ -91,7 +91,19 @@ impl Expr for Subquery { assert_eq!(types.len(), 1, "Subquery with more than one column"); types[0].clone() } - SubqueryKind::UpdateSet => DataType::new_unnamed_struct(self.query.data_types()), + SubqueryKind::UpdateSet => { + let schema = self.query.schema(); + let struct_type = if schema.fields().iter().any(|f| f.name == UNNAMED_COLUMN) { + StructType::unnamed(self.query.data_types()) + } else { + StructType::new( + (schema.fields().iter().cloned()) + .map(|f| (f.name, f.data_type)) + .collect(), + ) + }; + DataType::Struct(struct_type) + } SubqueryKind::Array => { let types = self.query.data_types(); assert_eq!(types.len(), 1, "Subquery with more than one column"); diff --git a/src/frontend/src/expr/type_inference/cast.rs b/src/frontend/src/expr/type_inference/cast.rs index c9b09fe18ecac..e03ec98883ce9 100644 --- a/src/frontend/src/expr/type_inference/cast.rs +++ b/src/frontend/src/expr/type_inference/cast.rs @@ -206,22 +206,13 @@ fn cast_struct(source: &DataType, target: &DataType, allows: CastContext) -> Cas Ok(()) } else { cast(src_ty, dst_ty, allows).map_err(|inner| { - if src_name.is_empty() { - inner - } else if dst_name.is_empty() { - cast_error!( - source = inner, - "cannot cast struct field \"{}\"", - src_name - ) - } else { - cast_error!( - source = inner, - "cannot cast struct field \"{}\" to struct field \"{}\"", - src_name, - dst_name - ) - } + let cast_from = (!src_name.is_empty()) + .then(|| format!(" struct field \"{}\"", src_name)) + .unwrap_or_default(); + let cast_to = (!dst_name.is_empty()) + .then(|| format!(" to struct field \"{}\"", dst_name)) + .unwrap_or_default(); + cast_error!(source = inner, "cannot cast{}{}", cast_from, cast_to) }) } }, From a58b142d23502e38a902256d3bc7800909e2bca8 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Tue, 26 Nov 2024 13:13:43 +0800 Subject: [PATCH 013/163] chore: bump iceberg version (java) (#19423) Co-authored-by: Dylan --- java/connector-node/risingwave-sink-deltalake/pom.xml | 1 - java/connector-node/risingwave-sink-iceberg/pom.xml | 4 ++-- java/pom.xml | 1 + 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/java/connector-node/risingwave-sink-deltalake/pom.xml b/java/connector-node/risingwave-sink-deltalake/pom.xml index 386ab0ad70fe5..c1918a4855c29 100644 --- a/java/connector-node/risingwave-sink-deltalake/pom.xml +++ b/java/connector-node/risingwave-sink-deltalake/pom.xml @@ -18,7 +18,6 @@ 11 11 - 1.14.4 true diff --git a/java/connector-node/risingwave-sink-iceberg/pom.xml b/java/connector-node/risingwave-sink-iceberg/pom.xml index ac099739abadc..186aceae7e711 100644 --- a/java/connector-node/risingwave-sink-iceberg/pom.xml +++ b/java/connector-node/risingwave-sink-iceberg/pom.xml @@ -16,7 +16,7 @@ risingwave-sink-iceberg - 1.5.2 + 1.7.0 11 11 true @@ -77,7 +77,7 @@ org.apache.parquet parquet-avro - 1.14.3 + ${parquet.version} org.apache.hadoop diff --git a/java/pom.xml b/java/pom.xml index 4f88d5e35f1eb..70ac90e17be71 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -75,6 +75,7 @@ 1.20.3 2.29.2 12.0.14 + 1.14.4 From 96c5431ed3866483217d3ff54d027ee2ed18bd72 Mon Sep 17 00:00:00 2001 From: xxchan Date: Tue, 26 Nov 2024 14:40:15 +0800 Subject: [PATCH 014/163] refactor(meta): rename replace_table to replace_stream_job where appropriate (#19537) Signed-off-by: xxchan --- src/meta/service/src/ddl_service.rs | 6 +- src/meta/src/barrier/checkpoint/control.rs | 2 +- src/meta/src/barrier/command.rs | 24 +-- src/meta/src/barrier/context/context_impl.rs | 6 +- src/meta/src/barrier/mod.rs | 4 +- src/meta/src/barrier/progress.rs | 33 ++-- src/meta/src/controller/streaming_job.rs | 196 ++++++++++--------- src/meta/src/error.rs | 5 +- src/meta/src/manager/streaming_job.rs | 55 +++++- src/meta/src/model/stream.rs | 7 + src/meta/src/rpc/ddl_controller.rs | 105 +++++----- src/meta/src/stream/stream_manager.rs | 29 +-- 12 files changed, 275 insertions(+), 197 deletions(-) diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index ad6f5ba38942e..3556890db04cc 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -40,7 +40,7 @@ use crate::barrier::BarrierManagerRef; use crate::manager::sink_coordination::SinkCoordinatorManager; use crate::manager::{MetaSrvEnv, StreamingJob}; use crate::rpc::ddl_controller::{ - DdlCommand, DdlController, DropMode, ReplaceTableInfo, StreamingJobId, + DdlCommand, DdlController, DropMode, ReplaceStreamJobInfo, StreamingJobId, }; use crate::stream::{GlobalStreamManagerRef, SourceManagerRef}; use crate::MetaError; @@ -91,13 +91,13 @@ impl DdlServiceImpl { source, job_type, }: ReplaceTablePlan, - ) -> ReplaceTableInfo { + ) -> ReplaceStreamJobInfo { let table = table.unwrap(); let col_index_mapping = table_col_index_mapping .as_ref() .map(ColIndexMapping::from_protobuf); - ReplaceTableInfo { + ReplaceStreamJobInfo { streaming_job: StreamingJob::Table( source, table, diff --git a/src/meta/src/barrier/checkpoint/control.rs b/src/meta/src/barrier/checkpoint/control.rs index c91f63196c667..1d7eef6f81b5e 100644 --- a/src/meta/src/barrier/checkpoint/control.rs +++ b/src/meta/src/barrier/checkpoint/control.rs @@ -643,7 +643,7 @@ impl DatabaseCheckpointControl { node.state.resps.extend(resps); finished_jobs.push(TrackingJob::New(TrackingCommand { info, - replace_table_info: None, + replace_stream_job: None, })); }); let task = task.get_or_insert_default(); diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 2a5db2ab9eb5a..4e7a659c5029d 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -84,12 +84,14 @@ pub struct Reschedule { pub newly_created_actors: Vec<(StreamActor, PbActorStatus)>, } -/// Replacing an old table with a new one. All actors in the table job will be rebuilt. -/// Used for `ALTER TABLE` ([`Command::ReplaceTable`]) and sink into table ([`Command::CreateStreamingJob`]). +/// Replacing an old job with a new one. All actors in the job will be rebuilt. +/// Used for `ALTER TABLE` ([`Command::ReplaceStreamJob`]) and sink into table ([`Command::CreateStreamingJob`]). #[derive(Debug, Clone)] -pub struct ReplaceTablePlan { +pub struct ReplaceStreamJobPlan { pub old_fragments: StreamJobFragments, pub new_fragments: StreamJobFragments, + /// Downstream jobs of the replaced job need to update their `Merge` node to + /// connect to the new fragment. pub merge_updates: Vec, pub dispatchers: HashMap>, /// For a table with connector, the `SourceExecutor` actor will also be rebuilt with new actor ids. @@ -104,7 +106,7 @@ pub struct ReplaceTablePlan { pub tmp_id: u32, } -impl ReplaceTablePlan { +impl ReplaceStreamJobPlan { fn fragment_changes(&self) -> HashMap { let mut fragment_changes = HashMap::new(); for fragment in self.new_fragments.fragments.values() { @@ -206,7 +208,7 @@ pub struct SnapshotBackfillInfo { #[derive(Debug, Clone)] pub enum CreateStreamingJobType { Normal, - SinkIntoTable(ReplaceTablePlan), + SinkIntoTable(ReplaceStreamJobPlan), SnapshotBackfill(SnapshotBackfillInfo), } @@ -271,13 +273,13 @@ pub enum Command { fragment_actors: HashMap>, }, - /// `ReplaceTable` command generates a `Update` barrier with the given `merge_updates`. This is + /// `ReplaceStreamJob` command generates a `Update` barrier with the given `merge_updates`. This is /// essentially switching the downstream of the old table fragments to the new ones, and /// dropping the old table fragments. Used for table schema change. /// /// This can be treated as a special case of `RescheduleFragment`, while the upstream fragment /// of the Merge executors are changed additionally. - ReplaceTable(ReplaceTablePlan), + ReplaceStreamJob(ReplaceStreamJobPlan), /// `SourceSplitAssignment` generates a `Splits` barrier for pushing initialized splits or /// changed splits. @@ -384,7 +386,7 @@ impl Command { }) .collect(), ), - Command::ReplaceTable(plan) => Some(plan.fragment_changes()), + Command::ReplaceStreamJob(plan) => Some(plan.fragment_changes()), Command::MergeSnapshotBackfillStreamingJobs(_) => None, Command::SourceSplitAssignment(_) => None, Command::Throttle(_) => None, @@ -688,7 +690,7 @@ impl Command { subscriptions_to_add, })); - if let CreateStreamingJobType::SinkIntoTable(ReplaceTablePlan { + if let CreateStreamingJobType::SinkIntoTable(ReplaceStreamJobPlan { old_fragments, new_fragments: _, merge_updates, @@ -731,7 +733,7 @@ impl Command { })) } - Command::ReplaceTable(ReplaceTablePlan { + Command::ReplaceStreamJob(ReplaceStreamJobPlan { old_fragments, merge_updates, dispatchers, @@ -943,7 +945,7 @@ impl Command { } Some(map) } - Command::ReplaceTable(replace_table) => { + Command::ReplaceStreamJob(replace_table) => { Some(replace_table.new_fragments.actors_to_create()) } _ => None, diff --git a/src/meta/src/barrier/context/context_impl.rs b/src/meta/src/barrier/context/context_impl.rs index a687312f29e16..4d652494ffd36 100644 --- a/src/meta/src/barrier/context/context_impl.rs +++ b/src/meta/src/barrier/context/context_impl.rs @@ -27,7 +27,7 @@ use crate::barrier::context::{GlobalBarrierWorkerContext, GlobalBarrierWorkerCon use crate::barrier::progress::TrackingJob; use crate::barrier::{ BarrierManagerStatus, BarrierWorkerRuntimeInfoSnapshot, Command, CreateStreamingJobCommandInfo, - CreateStreamingJobType, RecoveryReason, ReplaceTablePlan, Scheduled, + CreateStreamingJobType, RecoveryReason, ReplaceStreamJobPlan, Scheduled, }; use crate::hummock::CommitEpochInfo; use crate::{MetaError, MetaResult}; @@ -180,7 +180,7 @@ impl CommandContext { ) .await?; - if let CreateStreamingJobType::SinkIntoTable(ReplaceTablePlan { + if let CreateStreamingJobType::SinkIntoTable(ReplaceStreamJobPlan { new_fragments, dispatchers, init_split_assignment, @@ -223,7 +223,7 @@ impl CommandContext { .await?; } - Command::ReplaceTable(ReplaceTablePlan { + Command::ReplaceStreamJob(ReplaceStreamJobPlan { old_fragments, new_fragments, dispatchers, diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 4143827027693..beb54da013ea2 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -44,8 +44,8 @@ mod utils; mod worker; pub use self::command::{ - BarrierKind, Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, ReplaceTablePlan, - Reschedule, SnapshotBackfillInfo, + BarrierKind, Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, + ReplaceStreamJobPlan, Reschedule, SnapshotBackfillInfo, }; pub use self::info::InflightSubscriptionInfo; pub use self::manager::{BarrierManagerRef, GlobalBarrierManager}; diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index eac060004116a..aa676c01b3bdb 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -27,7 +27,7 @@ use risingwave_pb::stream_service::PbBarrierCompleteResponse; use crate::barrier::info::BarrierInfo; use crate::barrier::{ - Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, ReplaceTablePlan, + Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, ReplaceStreamJobPlan, }; use crate::manager::{DdlType, MetadataManager}; use crate::model::{ActorId, BackfillUpstreamType, StreamJobFragments}; @@ -229,7 +229,7 @@ impl TrackingJob { .catalog_controller .finish_streaming_job( streaming_job.id() as i32, - command.replace_table_info.clone(), + command.replace_stream_job.clone(), ) .await?; Ok(()) @@ -274,7 +274,7 @@ pub struct RecoveredTrackingJob { /// The command tracking by the [`CreateMviewProgressTracker`]. pub(super) struct TrackingCommand { pub info: CreateStreamingJobCommandInfo, - pub replace_table_info: Option, + pub replace_stream_job: Option, } /// Tracking is done as follows: @@ -379,7 +379,10 @@ impl CreateMviewProgressTracker { pub(super) fn update_tracking_jobs<'a>( &mut self, - info: Option<(&CreateStreamingJobCommandInfo, Option<&ReplaceTablePlan>)>, + info: Option<( + &CreateStreamingJobCommandInfo, + Option<&ReplaceStreamJobPlan>, + )>, create_mview_progress: impl IntoIterator, version_stats: &HummockVersionStats, ) { @@ -389,9 +392,9 @@ impl CreateMviewProgressTracker { let finished_commands = { let mut commands = vec![]; // Add the command to tracker. - if let Some((create_job_info, replace_table)) = info + if let Some((create_job_info, replace_stream_job)) = info && let Some(command) = - self.add(create_job_info, replace_table, version_stats) + self.add(create_job_info, replace_stream_job, version_stats) { // Those with no actors to track can be finished immediately. commands.push(command); @@ -429,8 +432,8 @@ impl CreateMviewProgressTracker { if let Some(Command::CreateStreamingJob { info, job_type }) = command { match job_type { CreateStreamingJobType::Normal => Some((info, None)), - CreateStreamingJobType::SinkIntoTable(replace_table) => { - Some((info, Some(replace_table))) + CreateStreamingJobType::SinkIntoTable(replace_stream_job) => { + Some((info, Some(replace_stream_job))) } CreateStreamingJobType::SnapshotBackfill(_) => { // The progress of SnapshotBackfill won't be tracked here @@ -494,24 +497,24 @@ impl CreateMviewProgressTracker { pub fn add( &mut self, info: &CreateStreamingJobCommandInfo, - replace_table: Option<&ReplaceTablePlan>, + replace_stream_job: Option<&ReplaceStreamJobPlan>, version_stats: &HummockVersionStats, ) -> Option { tracing::trace!(?info, "add job to track"); let (info, actors, replace_table_info) = { let CreateStreamingJobCommandInfo { - stream_job_fragments: table_fragments, + stream_job_fragments, .. } = info; - let actors = table_fragments.tracking_progress_actor_ids(); + let actors = stream_job_fragments.tracking_progress_actor_ids(); if actors.is_empty() { // The command can be finished immediately. return Some(TrackingJob::New(TrackingCommand { info: info.clone(), - replace_table_info: replace_table.cloned(), + replace_stream_job: replace_stream_job.cloned(), })); } - (info.clone(), actors, replace_table.cloned()) + (info.clone(), actors, replace_stream_job.cloned()) }; let CreateStreamingJobCommandInfo { @@ -567,7 +570,7 @@ impl CreateMviewProgressTracker { // that the sink job has been created. Some(TrackingJob::New(TrackingCommand { info, - replace_table_info, + replace_stream_job: replace_table_info, })) } else { let old = self.progress_map.insert( @@ -576,7 +579,7 @@ impl CreateMviewProgressTracker { progress, TrackingJob::New(TrackingCommand { info, - replace_table_info, + replace_stream_job: replace_table_info, }), ), ); diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index f993e70beadb6..3c8c793256db9 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -34,10 +34,10 @@ use risingwave_meta_model::{ actor, actor_dispatcher, fragment, index, object, object_dependency, sink, source, streaming_job, table, ActorId, ActorUpstreamActors, ColumnCatalogArray, CreateType, DatabaseId, ExprNodeArray, FragmentId, I32Array, IndexId, JobStatus, ObjectId, SchemaId, SinkId, SourceId, - StreamNode, StreamingParallelism, TableId, TableVersion, UserId, + StreamNode, StreamingParallelism, UserId, }; use risingwave_pb::catalog::source::PbOptionalAssociatedTableId; -use risingwave_pb::catalog::table::{PbOptionalAssociatedSourceId, PbTableVersion}; +use risingwave_pb::catalog::table::PbOptionalAssociatedSourceId; use risingwave_pb::catalog::{PbCreateType, PbTable}; use risingwave_pb::meta::list_rate_limits_response::RateLimitInfo; use risingwave_pb::meta::relation::{PbRelationInfo, RelationInfo}; @@ -62,7 +62,7 @@ use sea_orm::{ RelationTrait, TransactionTrait, }; -use crate::barrier::{ReplaceTablePlan, Reschedule}; +use crate::barrier::{ReplaceStreamJobPlan, Reschedule}; use crate::controller::catalog::CatalogController; use crate::controller::rename::ReplaceTableExprRewriter; use crate::controller::utils::{ @@ -71,7 +71,7 @@ use crate::controller::utils::{ get_internal_tables_by_id, rebuild_fragment_mapping_from_actors, PartialObject, }; use crate::controller::ObjectModel; -use crate::manager::{NotificationVersion, StreamingJob}; +use crate::manager::{NotificationVersion, StreamingJob, StreamingJobType}; use crate::model::{StreamContext, StreamJobFragments, TableParallelism}; use crate::stream::SplitAssignment; use crate::{MetaError, MetaResult}; @@ -678,7 +678,6 @@ impl CatalogController { &self, streaming_job: &StreamingJob, ctx: &StreamContext, - version: &PbTableVersion, specified_parallelism: &Option, max_parallelism: usize, ) -> MetaResult { @@ -687,18 +686,7 @@ impl CatalogController { let txn = inner.db.begin().await?; // 1. check version. - let original_version: Option = Table::find_by_id(id as TableId) - .select_only() - .column(table::Column::Version) - .into_tuple() - .one(&txn) - .await? - .ok_or_else(|| MetaError::catalog_id_not_found(ObjectType::Table.as_str(), id))?; - let original_version = original_version.expect("version for table should exist"); - if version.version != original_version.to_protobuf().version + 1 { - return Err(MetaError::permission_denied("table version is stale")); - } - + streaming_job.verify_version_for_replace(&txn).await?; // 2. check concurrent replace. let referring_cnt = ObjectDependency::find() .join( @@ -716,7 +704,7 @@ impl CatalogController { .await?; if referring_cnt != 0 { return Err(MetaError::permission_denied( - "table is being altered or referenced by some creating jobs", + "job is being altered or referenced by some creating jobs", )); } @@ -727,14 +715,14 @@ impl CatalogController { .into_tuple() .one(&txn) .await? - .ok_or_else(|| MetaError::catalog_id_not_found(ObjectType::Table.as_str(), id))?; + .ok_or_else(|| MetaError::catalog_id_not_found(streaming_job.job_type_str(), id))?; if original_max_parallelism != max_parallelism as i32 { // We already override the max parallelism in `StreamFragmentGraph` before entering this function. // This should not happen in normal cases. bail!( "cannot use a different max parallelism \ - when altering or creating/dropping a sink into an existing table, \ + when replacing streaming job, \ original: {}, new: {}", original_max_parallelism, max_parallelism @@ -747,13 +735,13 @@ impl CatalogController { }; // 4. create streaming object for new replace table. - let obj_id = Self::create_streaming_job_obj( + let new_obj_id = Self::create_streaming_job_obj( &txn, - ObjectType::Table, + streaming_job.object_type(), streaming_job.owner() as _, Some(streaming_job.database_id() as _), Some(streaming_job.schema_id() as _), - PbCreateType::Foreground, + streaming_job.create_type(), ctx, parallelism, max_parallelism, @@ -763,7 +751,7 @@ impl CatalogController { // 5. record dependency for new replace table. ObjectDependency::insert(object_dependency::ActiveModel { oid: Set(id as _), - used_by: Set(obj_id as _), + used_by: Set(new_obj_id as _), ..Default::default() }) .exec(&txn) @@ -771,14 +759,14 @@ impl CatalogController { txn.commit().await?; - Ok(obj_id) + Ok(new_obj_id) } /// `finish_streaming_job` marks job related objects as `Created` and notify frontend. pub async fn finish_streaming_job( &self, job_id: ObjectId, - replace_table_job_info: Option, + replace_stream_job_info: Option, ) -> MetaResult<()> { let mut inner = self.inner.write().await; let txn = inner.db.begin().await?; @@ -913,8 +901,8 @@ impl CatalogController { let fragment_mapping = get_fragment_mappings(&txn, job_id).await?; - let replace_table_mapping_update = match replace_table_job_info { - Some(ReplaceTablePlan { + let replace_table_mapping_update = match replace_stream_job_info { + Some(ReplaceStreamJobPlan { streaming_job, merge_updates, tmp_id, @@ -926,9 +914,11 @@ impl CatalogController { tmp_id as ObjectId, merge_updates, None, - Some(incoming_sink_id as _), - None, - vec![], + SinkIntoTableContext { + creating_sink_id: Some(incoming_sink_id as _), + dropping_sink_id: None, + updated_sink_catalogs: vec![], + }, &txn, streaming_job, ) @@ -976,9 +966,7 @@ impl CatalogController { streaming_job: StreamingJob, merge_updates: Vec, table_col_index_mapping: Option, - creating_sink_id: Option, - dropping_sink_id: Option, - updated_sink_catalogs: Vec, + sink_into_table_context: SinkIntoTableContext, ) -> MetaResult { let inner = self.inner.write().await; let txn = inner.db.begin().await?; @@ -987,9 +975,7 @@ impl CatalogController { tmp_id, merge_updates, table_col_index_mapping, - creating_sink_id, - dropping_sink_id, - updated_sink_catalogs, + sink_into_table_context, &txn, streaming_job, ) @@ -1014,57 +1000,69 @@ impl CatalogController { Ok(version) } + /// TODO: make it general for other streaming jobs. + /// Currently only for replacing table. pub async fn finish_replace_streaming_job_inner( tmp_id: ObjectId, merge_updates: Vec, table_col_index_mapping: Option, - creating_sink_id: Option, - dropping_sink_id: Option, - updated_sink_catalogs: Vec, + SinkIntoTableContext { + creating_sink_id, + dropping_sink_id, + updated_sink_catalogs, + }: SinkIntoTableContext, txn: &DatabaseTransaction, streaming_job: StreamingJob, ) -> MetaResult<(Vec, Vec)> { - // Question: The source catalog should be remain unchanged? - let StreamingJob::Table(_, table, ..) = streaming_job else { - unreachable!("unexpected job: {streaming_job:?}") - }; + let original_job_id = streaming_job.id() as ObjectId; + let job_type = streaming_job.job_type(); - let job_id = table.id as ObjectId; + match streaming_job { + StreamingJob::Table(_source, table, _table_job_type) => { + // The source catalog should remain unchanged - let original_table_catalogs = Table::find_by_id(job_id) - .select_only() - .columns([table::Column::Columns]) - .into_tuple::() - .one(txn) - .await? - .ok_or_else(|| MetaError::catalog_id_not_found("table", job_id))?; + let original_table_catalogs = Table::find_by_id(original_job_id) + .select_only() + .columns([table::Column::Columns]) + .into_tuple::() + .one(txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("table", original_job_id))?; - // For sinks created in earlier versions, we need to set the original_target_columns. - for sink_id in updated_sink_catalogs { - sink::ActiveModel { - sink_id: Set(sink_id as _), - original_target_columns: Set(Some(original_table_catalogs.clone())), - ..Default::default() - } - .update(txn) - .await?; - } + // For sinks created in earlier versions, we need to set the original_target_columns. + for sink_id in updated_sink_catalogs { + sink::ActiveModel { + sink_id: Set(sink_id as _), + original_target_columns: Set(Some(original_table_catalogs.clone())), + ..Default::default() + } + .update(txn) + .await?; + } + // Update the table catalog with the new one. (column catalog is also updated here) + let mut table = table::ActiveModel::from(table); + let mut incoming_sinks = table.incoming_sinks.as_ref().inner_ref().clone(); + if let Some(sink_id) = creating_sink_id { + debug_assert!(!incoming_sinks.contains(&{ sink_id })); + incoming_sinks.push(sink_id as _); + } - let mut table = table::ActiveModel::from(table); - let mut incoming_sinks = table.incoming_sinks.as_ref().inner_ref().clone(); - if let Some(sink_id) = creating_sink_id { - debug_assert!(!incoming_sinks.contains(&{ sink_id })); - incoming_sinks.push(sink_id as _); - } + if let Some(sink_id) = dropping_sink_id { + let drained = incoming_sinks.extract_if(|id| *id == sink_id).collect_vec(); + debug_assert_eq!(drained, vec![sink_id]); + } - if let Some(sink_id) = dropping_sink_id { - let drained = incoming_sinks.extract_if(|id| *id == sink_id).collect_vec(); - debug_assert_eq!(drained, vec![sink_id]); + table.incoming_sinks = Set(incoming_sinks.into()); + table.update(txn).await?; + } + // TODO: support other streaming jobs + _ => unreachable!( + "invalid streaming job type: {:?}", + streaming_job.job_type_str() + ), } - table.incoming_sinks = Set(incoming_sinks.into()); - let table = table.update(txn).await?; - + // 0. update internal tables // Fields including `fragment_id` were placeholder values before. // After table fragments are created, update them for all internal tables. let fragment_info: Vec<(FragmentId, I32Array)> = Fragment::find() @@ -1090,14 +1088,13 @@ impl CatalogController { } } - // let old_fragment_mappings = get_fragment_mappings(&txn, job_id).await?; // 1. replace old fragments/actors with new ones. Fragment::delete_many() - .filter(fragment::Column::JobId.eq(job_id)) + .filter(fragment::Column::JobId.eq(original_job_id)) .exec(txn) .await?; Fragment::update_many() - .col_expr(fragment::Column::JobId, SimpleExpr::from(job_id)) + .col_expr(fragment::Column::JobId, SimpleExpr::from(original_job_id)) .filter(fragment::Column::JobId.eq(tmp_id)) .exec(txn) .await?; @@ -1118,6 +1115,7 @@ impl CatalogController { // TODO: remove cache upstream fragment/actor ids and derive them from `actor_dispatcher` table. let mut to_update_fragment_ids = HashSet::new(); + // 2.1 update downstream actor's upstream_actor_ids for merge_update in merge_updates { assert!(merge_update.removed_upstream_actor_id.is_empty()); assert!(merge_update.new_upstream_fragment_id.is_some()); @@ -1158,6 +1156,7 @@ impl CatalogController { to_update_fragment_ids.insert(fragment_id); } + // 2.2 update downstream fragment's Merge node, and upstream_fragment_id for fragment_id in to_update_fragment_ids { let (fragment_id, mut stream_node, mut upstream_fragment_id) = Fragment::find_by_id(fragment_id) @@ -1201,14 +1200,21 @@ impl CatalogController { // 4. update catalogs and notify. let mut relations = vec![]; - let table_obj = table - .find_related(Object) - .one(txn) - .await? - .ok_or_else(|| MetaError::catalog_id_not_found("object", table.table_id))?; - relations.push(PbRelation { - relation_info: Some(PbRelationInfo::Table(ObjectModel(table, table_obj).into())), - }); + match job_type { + StreamingJobType::Table => { + let (table, table_obj) = Table::find_by_id(original_job_id) + .find_also_related(Object) + .one(txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found("object", original_job_id))?; + relations.push(PbRelation { + relation_info: Some(PbRelationInfo::Table( + ObjectModel(table, table_obj.unwrap()).into(), + )), + }) + } + _ => unreachable!("invalid streaming job type: {:?}", job_type), + } if let Some(table_col_index_mapping) = table_col_index_mapping { let expr_rewriter = ReplaceTableExprRewriter { table_col_index_mapping, @@ -1217,7 +1223,7 @@ impl CatalogController { let index_items: Vec<(IndexId, ExprNodeArray)> = Index::find() .select_only() .columns([index::Column::IndexId, index::Column::IndexItems]) - .filter(index::Column::PrimaryTableId.eq(job_id)) + .filter(index::Column::PrimaryTableId.eq(original_job_id)) .into_tuple() .all(txn) .await?; @@ -1246,15 +1252,15 @@ impl CatalogController { } } - let fragment_mapping: Vec<_> = get_fragment_mappings(txn, job_id as _).await?; + let fragment_mapping: Vec<_> = get_fragment_mappings(txn, original_job_id as _).await?; Ok((relations, fragment_mapping)) } - /// `try_abort_replacing_streaming_job` is used to abort the replacing streaming job, the input `job_id` is the dummy job id. - pub async fn try_abort_replacing_streaming_job(&self, job_id: ObjectId) -> MetaResult<()> { + /// Abort the replacing streaming job by deleting the temporary job object. + pub async fn try_abort_replacing_streaming_job(&self, tmp_job_id: ObjectId) -> MetaResult<()> { let inner = self.inner.write().await; - Object::delete_by_id(job_id).exec(&inner.db).await?; + Object::delete_by_id(tmp_job_id).exec(&inner.db).await?; Ok(()) } @@ -1891,3 +1897,13 @@ fn bitflag_intersects(column: SimpleExpr, value: i32) -> SimpleExpr { fn fragment_type_mask_intersects(value: i32) -> SimpleExpr { bitflag_intersects(fragment::Column::FragmentTypeMask.into_simple_expr(), value) } + +pub struct SinkIntoTableContext { + /// For creating sink into table, this is `Some`, otherwise `None`. + pub creating_sink_id: Option, + /// For dropping sink into table, this is `Some`, otherwise `None`. + pub dropping_sink_id: Option, + /// For alter table (e.g., add column), this is the list of existing sink ids + /// otherwise empty. + pub updated_sink_catalogs: Vec, +} diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index f1c3bb0ffdd8a..cda027c2d634c 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::error::BoxedError; +use risingwave_common::error::{BoxedError, NotImplemented}; use risingwave_common::session_config::SessionConfigError; use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; @@ -132,6 +132,9 @@ pub enum MetaErrorInner { #[error("{0} has been deprecated, please use {1} instead.")] Deprecated(String, String), + + #[error(transparent)] + NotImplemented(#[from] NotImplemented), } impl MetaError { diff --git a/src/meta/src/manager/streaming_job.rs b/src/meta/src/manager/streaming_job.rs index 76d7052589cd9..924cdb0124a9a 100644 --- a/src/meta/src/manager/streaming_job.rs +++ b/src/meta/src/manager/streaming_job.rs @@ -15,19 +15,25 @@ use std::collections::HashSet; use risingwave_common::catalog::TableVersionId; -use risingwave_common::current_cluster_version; use risingwave_common::util::epoch::Epoch; +use risingwave_common::{bail_not_implemented, current_cluster_version}; +use risingwave_meta_model::object::ObjectType; +use risingwave_meta_model::prelude::Table as TableModel; +use risingwave_meta_model::{table, TableId, TableVersion}; use risingwave_pb::catalog::{CreateType, Index, PbSource, Sink, Table}; use risingwave_pb::ddl_service::TableJobType; +use sea_orm::entity::prelude::*; +use sea_orm::{DatabaseTransaction, QuerySelect}; use strum::{EnumDiscriminants, EnumIs}; use super::{get_referred_secret_ids_from_sink, get_referred_secret_ids_from_source}; use crate::model::FragmentId; -use crate::MetaResult; +use crate::{MetaError, MetaResult}; // This enum is used in order to re-use code in `DdlServiceImpl` for creating MaterializedView and // Sink. #[derive(Debug, Clone, EnumDiscriminants, EnumIs)] +#[strum_discriminants(name(StreamingJobType))] pub enum StreamingJob { MaterializedView(Table), Sink(Sink, Option<(Table, Option)>), @@ -255,6 +261,10 @@ impl StreamingJob { } } + pub fn job_type(&self) -> StreamingJobType { + self.into() + } + pub fn job_type_str(&self) -> &'static str { match self { StreamingJob::MaterializedView(_) => "materialized view", @@ -275,6 +285,16 @@ impl StreamingJob { } } + pub fn object_type(&self) -> ObjectType { + match self { + Self::MaterializedView(_) => ObjectType::Table, // Note MV is special. + Self::Sink(_, _) => ObjectType::Sink, + Self::Table(_, _, _) => ObjectType::Table, + Self::Index(_, _) => ObjectType::Index, + Self::Source(_) => ObjectType::Source, + } + } + /// Returns the [`TableVersionId`] if this job is `Table`. pub fn table_version_id(&self) -> Option { if let Self::Table(_, table, ..) = self { @@ -329,4 +349,35 @@ impl StreamingJob { StreamingJob::MaterializedView(_) | StreamingJob::Index(_, _) => Ok(HashSet::new()), } } + + /// Verify the new version is the next version of the original version. + pub async fn verify_version_for_replace(&self, txn: &DatabaseTransaction) -> MetaResult<()> { + let id = self.id(); + + match self { + StreamingJob::Table(_source, table, _table_job_type) => { + let new_version = table.get_version()?.get_version(); + let original_version: Option = TableModel::find_by_id(id as TableId) + .select_only() + .column(table::Column::Version) + .into_tuple() + .one(txn) + .await? + .ok_or_else(|| MetaError::catalog_id_not_found(self.job_type_str(), id))?; + let original_version = original_version + .expect("version for table should exist") + .to_protobuf(); + if new_version != original_version.version + 1 { + return Err(MetaError::permission_denied("table version is stale")); + } + } + StreamingJob::MaterializedView(_) + | StreamingJob::Sink(_, _) + | StreamingJob::Index(_, _) + | StreamingJob::Source(_) => { + bail_not_implemented!("schema change for {}", self.job_type_str()) + } + } + Ok(()) + } } diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index e5490a86365b9..b87d06fdd0898 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -597,6 +597,13 @@ impl StreamJobFragments { }); self } + + /// Panics if the fragment is not found. + pub fn fragment_mut(&mut self, fragment_id: FragmentId) -> &mut Fragment { + self.fragments + .get_mut(&fragment_id) + .unwrap_or_else(|| panic!("fragment {} not found", fragment_id)) + } } #[derive(Debug, Clone, Copy, PartialEq, Eq)] diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index dacf908071533..bb14ecefc1298 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -66,16 +66,17 @@ use tracing::Instrument; use crate::barrier::BarrierManagerRef; use crate::controller::catalog::ReleaseContext; use crate::controller::cluster::StreamingClusterInfo; +use crate::controller::streaming_job::SinkIntoTableContext; use crate::error::{bail_invalid_parameter, bail_unavailable}; use crate::manager::{ DdlType, LocalNotification, MetaSrvEnv, MetadataManager, NotificationVersion, StreamingJob, IGNORED_NOTIFICATION_VERSION, }; -use crate::model::{FragmentId, StreamContext, StreamJobFragments, TableParallelism}; +use crate::model::{StreamContext, StreamJobFragments, TableParallelism}; use crate::stream::{ create_source_worker_handle, validate_sink, ActorGraphBuildResult, ActorGraphBuilder, CompleteStreamFragmentGraph, CreateStreamingJobContext, CreateStreamingJobOption, - GlobalStreamManagerRef, ReplaceTableContext, SourceManagerRef, StreamFragmentGraph, + GlobalStreamManagerRef, ReplaceStreamJobContext, SourceManagerRef, StreamFragmentGraph, }; use crate::{MetaError, MetaResult}; @@ -114,8 +115,9 @@ impl StreamingJobId { } } -// It’s used to describe the information of the table that needs to be replaced and it will be used during replacing table and creating sink into table operations. -pub struct ReplaceTableInfo { +/// It’s used to describe the information of the job that needs to be replaced +/// and it will be used during replacing table and creating sink into table operations. +pub struct ReplaceStreamJobInfo { pub streaming_job: StreamingJob, pub fragment_graph: StreamFragmentGraphProto, pub col_index_mapping: Option, @@ -136,13 +138,13 @@ pub enum DdlCommand { StreamingJob, StreamFragmentGraphProto, CreateType, - Option, + Option, HashSet, ), - DropStreamingJob(StreamingJobId, DropMode, Option), + DropStreamingJob(StreamingJobId, DropMode, Option), AlterName(alter_name_request::Object, String), AlterSwapRename(alter_swap_rename_request::Object), - ReplaceTable(ReplaceTableInfo), + ReplaceTable(ReplaceStreamJobInfo), AlterSourceColumn(Source), AlterObjectOwner(Object, UserId), AlterSetSchema(alter_set_schema_request::Object, SchemaId), @@ -325,7 +327,7 @@ impl DdlController { ctrl.drop_streaming_job(job_id, drop_mode, target_replace_info) .await } - DdlCommand::ReplaceTable(ReplaceTableInfo { + DdlCommand::ReplaceTable(ReplaceStreamJobInfo { streaming_job, fragment_graph, col_index_mapping, @@ -670,7 +672,7 @@ impl DdlController { dropping_sink_id: Option, streaming_job: &StreamingJob, fragment_graph: StreamFragmentGraph, - ) -> MetaResult<(ReplaceTableContext, StreamJobFragments)> { + ) -> MetaResult<(ReplaceStreamJobContext, StreamJobFragments)> { let (mut replace_table_ctx, mut stream_job_fragments) = self .build_replace_table(stream_ctx, streaming_job, fragment_graph, None, tmp_id as _) .await?; @@ -696,7 +698,7 @@ impl DdlController { let target_table = streaming_job.table().unwrap(); - let target_fragment_id = + let union_fragment_id = union_fragment_id.expect("fragment of placeholder merger not found"); if let Some(creating_sink_table_fragments) = creating_sink_table_fragments { @@ -707,8 +709,7 @@ impl DdlController { &sink_fragment, target_table, &mut replace_table_ctx, - &mut stream_job_fragments, - target_fragment_id, + stream_job_fragments.fragment_mut(union_fragment_id), None, ); } @@ -746,8 +747,7 @@ impl DdlController { &sink_fragment, target_table, &mut replace_table_ctx, - &mut stream_job_fragments, - target_fragment_id, + stream_job_fragments.fragment_mut(union_fragment_id), Some(&sink.unique_identity()), ); } @@ -773,9 +773,8 @@ impl DdlController { sink_id: Option, sink_fragment: &PbFragment, table: &Table, - replace_table_ctx: &mut ReplaceTableContext, - stream_job_fragments: &mut StreamJobFragments, - target_fragment_id: FragmentId, + replace_table_ctx: &mut ReplaceStreamJobContext, + union_fragment: &mut PbFragment, unique_identity: Option<&str>, ) { let sink_actor_ids = sink_fragment @@ -784,11 +783,6 @@ impl DdlController { .map(|a| a.actor_id) .collect_vec(); - let union_fragment = stream_job_fragments - .fragments - .get_mut(&target_fragment_id) - .unwrap(); - let downstream_actor_ids = union_fragment .actors .iter() @@ -916,7 +910,7 @@ impl DdlController { &self, mut streaming_job: StreamingJob, fragment_graph: StreamFragmentGraphProto, - affected_table_replace_info: Option, + affected_table_replace_info: Option, dependencies: HashSet, ) -> MetaResult { let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap()); @@ -998,7 +992,7 @@ impl DdlController { ctx: StreamContext, mut streaming_job: StreamingJob, fragment_graph: StreamFragmentGraphProto, - affected_table_replace_info: Option, + affected_table_replace_info: Option, ) -> MetaResult { let mut fragment_graph = StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?; @@ -1019,7 +1013,7 @@ impl DdlController { let affected_table_replace_info = match affected_table_replace_info { Some(replace_table_info) => { - let ReplaceTableInfo { + let ReplaceStreamJobInfo { mut streaming_job, fragment_graph, .. @@ -1116,7 +1110,7 @@ impl DdlController { object_type: ObjectType, object_id: ObjectId, drop_mode: DropMode, - target_replace_info: Option, + target_replace_info: Option, ) -> MetaResult { let (release_ctx, mut version) = match object_type { ObjectType::Database => { @@ -1159,7 +1153,7 @@ impl DdlController { let stream_ctx = StreamContext::from_protobuf(replace_table_info.fragment_graph.get_ctx().unwrap()); - let ReplaceTableInfo { + let ReplaceStreamJobInfo { mut streaming_job, fragment_graph, .. @@ -1187,7 +1181,7 @@ impl DdlController { streaming_job.set_dml_fragment_id(fragment_graph.dml_fragment_id()); let streaming_job = streaming_job; - let table = streaming_job.table().unwrap(); + streaming_job.table().expect("should be table job"); tracing::debug!(id = streaming_job.id(), "replacing table for dropped sink"); let tmp_id = self @@ -1196,7 +1190,6 @@ impl DdlController { .create_job_catalog_for_replace( &streaming_job, &stream_ctx, - table.get_version()?, &fragment_graph.specified_parallelism(), fragment_graph.max_parallelism(), ) @@ -1224,7 +1217,7 @@ impl DdlController { .await?; self.stream_manager - .replace_table(stream_job_fragments, ctx) + .replace_stream_job(stream_job_fragments, ctx) .await?; merge_updates @@ -1240,9 +1233,11 @@ impl DdlController { streaming_job, merge_updates, None, - None, - Some(sink_id), - vec![], + SinkIntoTableContext { + creating_sink_id: None, + dropping_sink_id: Some(sink_id), + updated_sink_catalogs: vec![], + }, ) .await?; Ok(version) @@ -1344,13 +1339,12 @@ impl DdlController { .create_job_catalog_for_replace( &streaming_job, &ctx, - table.get_version()?, &fragment_graph.specified_parallelism(), fragment_graph.max_parallelism(), ) .await?; - tracing::debug!(id = streaming_job.id(), "building replace streaming job"); + tracing::debug!(id = job_id, "building replace streaming job"); let mut updated_sink_catalogs = vec![]; let result: MetaResult> = try { @@ -1383,8 +1377,9 @@ impl DdlController { } } - let target_fragment_id = + let union_fragment_id = union_fragment_id.expect("fragment of placeholder merger not found"); + let union_fragment = stream_job_fragments.fragment_mut(union_fragment_id); let catalogs = self .metadata_manager @@ -1406,8 +1401,7 @@ impl DdlController { &sink_fragment, table, &mut ctx, - &mut stream_job_fragments, - target_fragment_id, + union_fragment, Some(&sink.unique_identity()), ); @@ -1424,7 +1418,7 @@ impl DdlController { .await?; self.stream_manager - .replace_table(stream_job_fragments, ctx) + .replace_stream_job(stream_job_fragments, ctx) .await?; merge_updates }; @@ -1439,9 +1433,11 @@ impl DdlController { streaming_job, merge_updates, table_col_index_mapping, - None, - None, - updated_sink_catalogs, + SinkIntoTableContext { + creating_sink_id: None, + dropping_sink_id: None, + updated_sink_catalogs, + }, ) .await?; Ok(version) @@ -1463,7 +1459,7 @@ impl DdlController { &self, job_id: StreamingJobId, drop_mode: DropMode, - target_replace_info: Option, + target_replace_info: Option, ) -> MetaResult { let (object_id, object_type) = match job_id { StreamingJobId::MaterializedView(id) => (id as _, ObjectType::Table), @@ -1637,25 +1633,24 @@ impl DdlController { } let replace_table_job_info = match affected_table_replace_info { - Some((streaming_job, fragment_graph)) => { + Some((table_stream_job, fragment_graph)) => { if snapshot_backfill_info.is_some() { return Err(anyhow!( - "snapshot backfill should not have replace table info: {streaming_job:?}" + "snapshot backfill should not have replace table info: {table_stream_job:?}" ) .into()); } - let StreamingJob::Sink(s, target_table) = &mut stream_job else { + let StreamingJob::Sink(sink, target_table) = &mut stream_job else { bail!("additional replace table event only occurs when sinking into table"); }; - let table = streaming_job.table().unwrap(); + table_stream_job.table().expect("should be table job"); let tmp_id = self .metadata_manager .catalog_controller .create_job_catalog_for_replace( - &streaming_job, + &table_stream_job, &stream_ctx, - table.get_version()?, &fragment_graph.specified_parallelism(), fragment_graph.max_parallelism(), ) @@ -1666,22 +1661,22 @@ impl DdlController { tmp_id, &self.metadata_manager, stream_ctx, - Some(s), + Some(sink), Some(&stream_job_fragments), None, - &streaming_job, + &table_stream_job, fragment_graph, ) .await?; // When sinking into table occurs, some variables of the target table may be modified, // such as `fragment_id` being altered by `prepare_replace_table`. // At this point, it’s necessary to update the table info carried with the sink. - must_match!(&streaming_job, StreamingJob::Table(source, table, _) => { + must_match!(&table_stream_job, StreamingJob::Table(source, table, _) => { // The StreamingJob in ReplaceTableInfo must be StreamingJob::Table *target_table = Some((table.clone(), source.clone())); }); - Some((streaming_job, context, table_fragments)) + Some((table_stream_job, context, table_fragments)) } None => None, }; @@ -1717,7 +1712,7 @@ impl DdlController { mut fragment_graph: StreamFragmentGraph, table_col_index_mapping: Option, tmp_table_id: TableId, - ) -> MetaResult<(ReplaceTableContext, StreamJobFragments)> { + ) -> MetaResult<(ReplaceStreamJobContext, StreamJobFragments)> { let id = stream_job.id(); let expr_context = stream_ctx.to_expr_context(); @@ -1837,7 +1832,7 @@ impl DdlController { // Note: no need to set `vnode_count` as it's already set by the frontend. // See `get_replace_table_plan`. - let ctx = ReplaceTableContext { + let ctx = ReplaceStreamJobContext { old_fragments, merge_updates, dispatchers, diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index b1bbc04fe5635..20eceaffce935 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -31,7 +31,7 @@ use tracing::Instrument; use super::{Locations, RescheduleOptions, ScaleControllerRef, TableResizePolicy}; use crate::barrier::{ BarrierScheduler, Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, - ReplaceTablePlan, SnapshotBackfillInfo, + ReplaceStreamJobPlan, SnapshotBackfillInfo, }; use crate::error::bail_invalid_parameter; use crate::manager::{DdlType, MetaSrvEnv, MetadataManager, NotificationVersion, StreamingJob}; @@ -77,7 +77,7 @@ pub struct CreateStreamingJobContext { pub ddl_type: DdlType, /// Context provided for potential replace table, typically used when sinking into a table. - pub replace_table_job_info: Option<(StreamingJob, ReplaceTableContext, StreamJobFragments)>, + pub replace_table_job_info: Option<(StreamingJob, ReplaceStreamJobContext, StreamJobFragments)>, pub snapshot_backfill_info: Option, @@ -164,10 +164,10 @@ impl CreatingStreamingJobInfo { type CreatingStreamingJobInfoRef = Arc; -/// [`ReplaceTableContext`] carries one-time infos for replacing the plan of an existing table. +/// [`ReplaceStreamJobContext`] carries one-time infos for replacing the plan of an existing stream job. /// /// Note: for better readability, keep this struct complete and immutable once created. -pub struct ReplaceTableContext { +pub struct ReplaceStreamJobContext { /// The old job fragments to be replaced. pub old_fragments: StreamJobFragments, @@ -177,7 +177,7 @@ pub struct ReplaceTableContext { /// New dispatchers to add from upstream actors to downstream actors. pub dispatchers: HashMap>, - /// The locations of the actors to build in the new table to replace. + /// The locations of the actors to build in the new job to replace. pub building_locations: Locations, /// The locations of the existing actors, essentially the downstream chain actors to update. @@ -359,7 +359,7 @@ impl GlobalStreamManager { replace_table_id = Some(tmp_table_id); - replace_table_command = Some(ReplaceTablePlan { + replace_table_command = Some(ReplaceStreamJobPlan { old_fragments: context.old_fragments, new_fragments: stream_job_fragments, merge_updates: context.merge_updates, @@ -457,32 +457,33 @@ impl GlobalStreamManager { } } - pub async fn replace_table( + /// Send replace job command to barrier scheduler. + pub async fn replace_stream_job( &self, - stream_job_fragments: StreamJobFragments, - ReplaceTableContext { + new_fragments: StreamJobFragments, + ReplaceStreamJobContext { old_fragments, merge_updates, dispatchers, tmp_id, streaming_job, .. - }: ReplaceTableContext, + }: ReplaceStreamJobContext, ) -> MetaResult<()> { - let tmp_table_id = stream_job_fragments.stream_job_id(); + let tmp_table_id = new_fragments.stream_job_id(); let init_split_assignment = self.source_manager.allocate_splits(&tmp_table_id).await?; self.barrier_scheduler .run_config_change_command_with_pause( streaming_job.database_id().into(), - Command::ReplaceTable(ReplaceTablePlan { + Command::ReplaceStreamJob(ReplaceStreamJobPlan { old_fragments, - new_fragments: stream_job_fragments, + new_fragments, merge_updates, dispatchers, init_split_assignment, - tmp_id, streaming_job, + tmp_id, }), ) .await?; From 519c95db4c18a3741e3f6ab4334af99c6e552531 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Tue, 26 Nov 2024 15:07:14 +0800 Subject: [PATCH 015/163] refactor(common): consolidate `StructType` constructors (Part 1/2: trivial ones) (#19569) --- src/batch/src/executor/insert.rs | 7 +++--- src/batch/src/executor/order_by.rs | 7 +++--- src/batch/src/executor/values.rs | 10 ++------- src/common/src/array/struct_array.rs | 22 +++++++++---------- src/common/src/types/mod.rs | 16 +++++++------- src/common/src/util/sort_util.rs | 4 ++-- .../src/sink/formatter/debezium_json.rs | 11 +++++----- src/frontend/src/binder/expr/value.rs | 6 +++-- src/frontend/src/catalog/table_catalog.rs | 9 ++++---- src/frontend/src/expr/literal.rs | 8 +++---- src/frontend/src/handler/create_mv.rs | 18 +++++++-------- src/frontend/src/handler/create_source.rs | 18 +++++++-------- src/frontend/src/handler/create_table.rs | 9 ++++---- src/stream/src/executor/values.rs | 5 +---- 14 files changed, 69 insertions(+), 81 deletions(-) diff --git a/src/batch/src/executor/insert.rs b/src/batch/src/executor/insert.rs index 5674e17557e6b..704182175b660 100644 --- a/src/batch/src/executor/insert.rs +++ b/src/batch/src/executor/insert.rs @@ -290,10 +290,9 @@ mod tests { let store = MemoryStateStore::new(); // Make struct field - let struct_field = Field::unnamed(DataType::new_struct( - vec![DataType::Int32, DataType::Int32, DataType::Int32], - vec![], - )); + let struct_field = Field::unnamed( + StructType::unnamed(vec![DataType::Int32, DataType::Int32, DataType::Int32]).into(), + ); // Schema for mock executor. let mut schema = schema_test_utils::ii(); diff --git a/src/batch/src/executor/order_by.rs b/src/batch/src/executor/order_by.rs index ad7cc13992346..95019c226ffda 100644 --- a/src/batch/src/executor/order_by.rs +++ b/src/batch/src/executor/order_by.rs @@ -851,10 +851,9 @@ mod tests { async fn test_encoding_for_struct_list() { let schema = Schema { fields: vec![ - Field::unnamed(DataType::new_struct( - vec![DataType::Varchar, DataType::Float32], - vec![], - )), + Field::unnamed( + StructType::unnamed(vec![DataType::Varchar, DataType::Float32]).into(), + ), Field::unnamed(DataType::List(Box::new(DataType::Int64))), ], }; diff --git a/src/batch/src/executor/values.rs b/src/batch/src/executor/values.rs index 7eb1a6e9893b5..3d506ff8b6fb0 100644 --- a/src/batch/src/executor/values.rs +++ b/src/batch/src/executor/values.rs @@ -167,10 +167,7 @@ mod tests { Some(ScalarImpl::Int64(3)), )), Box::new(LiteralExpression::new( - DataType::new_struct( - vec![DataType::Int32, DataType::Int32, DataType::Int32], - vec![], - ), + StructType::unnamed(vec![DataType::Int32, DataType::Int32, DataType::Int32]).into(), Some(ScalarImpl::Struct(value)), )) as BoxedExpression, ]; @@ -193,10 +190,7 @@ mod tests { assert_eq!(fields[2].data_type, DataType::Int64); assert_eq!( fields[3].data_type, - DataType::new_struct( - vec![DataType::Int32, DataType::Int32, DataType::Int32], - vec![], - ) + StructType::unnamed(vec![DataType::Int32, DataType::Int32, DataType::Int32]).into() ); let mut stream = values_executor.execute(); diff --git a/src/common/src/array/struct_array.rs b/src/common/src/array/struct_array.rs index 10ded3a64d66c..e0b254bfcad75 100644 --- a/src/common/src/array/struct_array.rs +++ b/src/common/src/array/struct_array.rs @@ -663,19 +663,17 @@ mod tests { let fields = [ DataType::Float32, DataType::Varchar, - DataType::new_struct( - vec![ - DataType::Float64, - DataType::Varchar, - DataType::Varchar, - DataType::new_struct(vec![], vec![]), - ], - vec![], - ), + StructType::unnamed(vec![ + DataType::Float64, + DataType::Varchar, + DataType::Varchar, + StructType::unnamed(vec![]).into(), + ]) + .into(), DataType::Int64, DataType::Varchar, DataType::Int16, - DataType::new_struct(vec![], vec![]), + StructType::unnamed(vec![]).into(), DataType::Int32, ]; let struct_ref = StructRef::ValueRef { val: &value }; @@ -747,7 +745,7 @@ mod tests { ]), vec![ DataType::Varchar, - DataType::new_struct(vec![DataType::Varchar], vec![]), + StructType::unnamed(vec![DataType::Varchar]).into(), ], Ordering::Greater, ), @@ -762,7 +760,7 @@ mod tests { ]), vec![ DataType::Varchar, - DataType::new_struct(vec![DataType::Varchar], vec![]), + StructType::unnamed(vec![DataType::Varchar]).into(), ], Ordering::Equal, ), diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index ad516eab101c6..5e6264cd933d0 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -409,10 +409,6 @@ impl DataType { Self::Struct(StructType::from_parts(field_names, fields)) } - pub fn new_unnamed_struct(fields: Vec) -> Self { - Self::Struct(StructType::unnamed(fields)) - } - pub fn as_struct(&self) -> &StructType { match self { DataType::Struct(t) => t, @@ -484,6 +480,12 @@ impl DataType { } } +impl From for DataType { + fn from(value: StructType) -> Self { + Self::Struct(value) + } +} + impl From for PbDataType { fn from(data_type: DataType) -> Self { data_type.to_protobuf() @@ -1156,10 +1158,8 @@ mod tests { #[test] fn test_data_type_display() { - let d: DataType = DataType::new_struct( - vec![DataType::Int32, DataType::Varchar], - vec!["i".to_string(), "j".to_string()], - ); + let d: DataType = + StructType::new(vec![("i", DataType::Int32), ("j", DataType::Varchar)]).into(); assert_eq!( format!("{}", d), "struct".to_string() diff --git a/src/common/src/util/sort_util.rs b/src/common/src/util/sort_util.rs index ff4bde652496d..3915208f7ed23 100644 --- a/src/common/src/util/sort_util.rs +++ b/src/common/src/util/sort_util.rs @@ -579,7 +579,7 @@ mod tests { use super::*; use crate::array::{ListValue, StructValue}; use crate::row::OwnedRow; - use crate::types::{DataType, Datum, ScalarImpl}; + use crate::types::{DataType, Datum, ScalarImpl, StructType}; #[test] fn test_order_type() { @@ -720,7 +720,7 @@ mod tests { DataType::Date, DataType::Timestamp, DataType::Time, - DataType::new_struct(vec![DataType::Int32, DataType::Float32], vec![]), + StructType::unnamed(vec![DataType::Int32, DataType::Float32]).into(), DataType::List(Box::new(DataType::Int32)), ], ); diff --git a/src/connector/src/sink/formatter/debezium_json.rs b/src/connector/src/sink/formatter/debezium_json.rs index 9fff8a9b8ba6b..8a3c7397e01bd 100644 --- a/src/connector/src/sink/formatter/debezium_json.rs +++ b/src/connector/src/sink/formatter/debezium_json.rs @@ -336,7 +336,7 @@ pub(crate) fn field_to_json(field: &Field) -> Value { #[cfg(test)] mod tests { use risingwave_common::test_prelude::StreamChunkTestExt; - use risingwave_common::types::DataType; + use risingwave_common::types::{DataType, StructType}; use super::*; use crate::sink::utils::chunk_to_json; @@ -373,10 +373,11 @@ mod tests { type_name: "".into(), }, Field { - data_type: DataType::new_struct( - vec![DataType::Int32, DataType::Float32], - vec!["v4".to_string(), "v5".to_string()], - ), + data_type: StructType::new(vec![ + ("v4", DataType::Int32), + ("v5", DataType::Float32), + ]) + .into(), name: "v3".into(), sub_fields: vec![ Field { diff --git a/src/frontend/src/binder/expr/value.rs b/src/frontend/src/binder/expr/value.rs index 961306408a43e..fe27ec38f51fa 100644 --- a/src/frontend/src/binder/expr/value.rs +++ b/src/frontend/src/binder/expr/value.rs @@ -14,7 +14,9 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; -use risingwave_common::types::{DataType, DateTimeField, Decimal, Interval, MapType, ScalarImpl}; +use risingwave_common::types::{ + DataType, DateTimeField, Decimal, Interval, MapType, ScalarImpl, StructType, +}; use risingwave_sqlparser::ast::{DateTimeField as AstDateTimeField, Expr, Value}; use thiserror_ext::AsReport; @@ -283,7 +285,7 @@ impl Binder { .map(|e| self.bind_expr_inner(e)) .collect::>>()?; let data_type = - DataType::new_unnamed_struct(exprs.iter().map(|e| e.return_type()).collect_vec()); + StructType::unnamed(exprs.iter().map(|e| e.return_type()).collect_vec()).into(); let expr: ExprImpl = FunctionCall::new_unchecked(ExprType::Row, exprs, data_type).into(); Ok(expr) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 6ae880ca14830..8835b655e06fa 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -761,10 +761,11 @@ mod tests { ColumnCatalog::row_id_column(), ColumnCatalog { column_desc: ColumnDesc { - data_type: DataType::new_struct( - vec![DataType::Varchar, DataType::Varchar], - vec!["address".to_string(), "zipcode".to_string()] - ), + data_type: StructType::new(vec![ + ("address", DataType::Varchar), + ("zipcode", DataType::Varchar) + ],) + .into(), column_id: ColumnId::new(1), name: "country".to_string(), field_descs: vec![ diff --git a/src/frontend/src/expr/literal.rs b/src/frontend/src/expr/literal.rs index ed76d1a9bf998..02bcb6544cf58 100644 --- a/src/frontend/src/expr/literal.rs +++ b/src/frontend/src/expr/literal.rs @@ -148,7 +148,7 @@ fn value_encoding_to_literal( #[cfg(test)] mod tests { use risingwave_common::array::{ListValue, StructValue}; - use risingwave_common::types::{DataType, Datum, ScalarImpl}; + use risingwave_common::types::{DataType, Datum, ScalarImpl, StructType}; use risingwave_common::util::value_encoding::DatumFromProtoExt; use risingwave_pb::expr::expr_node::RexNode; @@ -166,10 +166,8 @@ mod tests { if let RexNode::Constant(prost) = node { let data2 = Datum::from_protobuf( &prost, - &DataType::new_struct( - vec![DataType::Varchar, DataType::Int32, DataType::Int32], - vec![], - ), + &StructType::unnamed(vec![DataType::Varchar, DataType::Int32, DataType::Int32]) + .into(), ) .unwrap() .unwrap(); diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 66105099432fc..1d026b75944f0 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -272,7 +272,7 @@ pub mod tests { use risingwave_common::catalog::{ DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, ROWID_PREFIX, RW_TIMESTAMP_COLUMN_NAME, }; - use risingwave_common::types::DataType; + use risingwave_common::types::{DataType, StructType}; use crate::catalog::root_catalog::SchemaPath; use crate::test_utils::{create_proto_file, LocalFrontend, PROTO_FILE_DATA}; @@ -314,16 +314,16 @@ pub mod tests { .map(|col| (col.name(), col.data_type().clone())) .collect::>(); - let city_type = DataType::new_struct( - vec![DataType::Varchar, DataType::Varchar], - vec!["address".to_string(), "zipcode".to_string()], - ); + let city_type = StructType::new(vec![ + ("address", DataType::Varchar), + ("zipcode", DataType::Varchar), + ]) + .into(); let expected_columns = maplit::hashmap! { ROWID_PREFIX => DataType::Serial, - "country" => DataType::new_struct( - vec![DataType::Varchar,city_type,DataType::Varchar], - vec!["address".to_string(), "city".to_string(), "zipcode".to_string()], - ), + "country" => StructType::new( + vec![("address", DataType::Varchar),("city", city_type),("zipcode", DataType::Varchar)], + ).into(), RW_TIMESTAMP_COLUMN_NAME => DataType::Timestamptz, }; assert_eq!(columns, expected_columns); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 438e3cea1a736..1f89494f64ce6 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -1780,7 +1780,7 @@ pub mod tests { CDC_SOURCE_COLUMN_NUM, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, OFFSET_COLUMN_NAME, ROWID_PREFIX, TABLE_NAME_COLUMN_NAME, }; - use risingwave_common::types::DataType; + use risingwave_common::types::{DataType, StructType}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::source_catalog::SourceCatalog; @@ -1820,19 +1820,19 @@ pub mod tests { let columns = GET_COLUMN_FROM_CATALOG(source); - let city_type = DataType::new_struct( - vec![DataType::Varchar, DataType::Varchar], - vec!["address".to_string(), "zipcode".to_string()], - ); + let city_type = StructType::new(vec![ + ("address", DataType::Varchar), + ("zipcode", DataType::Varchar), + ]) + .into(); let expected_columns = maplit::hashmap! { ROWID_PREFIX => DataType::Serial, "id" => DataType::Int32, "zipcode" => DataType::Int64, "rate" => DataType::Float32, - "country" => DataType::new_struct( - vec![DataType::Varchar,city_type,DataType::Varchar], - vec!["address".to_string(), "city".to_string(), "zipcode".to_string()], - ), + "country" => StructType::new( + vec![("address", DataType::Varchar),("city", city_type),("zipcode", DataType::Varchar)], + ).into(), }; assert_eq!(columns, expected_columns); } diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index ff2e410370782..cc6929a19d4ff 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1454,7 +1454,7 @@ mod tests { use risingwave_common::catalog::{ Field, DEFAULT_DATABASE_NAME, ROWID_PREFIX, RW_TIMESTAMP_COLUMN_NAME, }; - use risingwave_common::types::DataType; + use risingwave_common::types::{DataType, StructType}; use super::*; use crate::test_utils::{create_proto_file, LocalFrontend, PROTO_FILE_DATA}; @@ -1519,10 +1519,9 @@ mod tests { let expected_columns = maplit::hashmap! { ROWID_PREFIX => DataType::Serial, "v1" => DataType::Int16, - "v2" => DataType::new_struct( - vec![DataType::Int64,DataType::Float64,DataType::Float64], - vec!["v3".to_string(), "v4".to_string(), "v5".to_string()], - ), + "v2" => StructType::new( + vec![("v3", DataType::Int64),("v4", DataType::Float64),("v5", DataType::Float64)], + ).into(), RW_TIMESTAMP_COLUMN_NAME => DataType::Timestamptz, }; diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 89946d9dc94e6..d53bcca24a1ec 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -173,10 +173,7 @@ mod tests { Some(ScalarImpl::Int64(3)), )), Box::new(LiteralExpression::new( - DataType::new_struct( - vec![DataType::Int32, DataType::Int32, DataType::Int32], - vec![], - ), + StructType::unnamed(vec![DataType::Int32, DataType::Int32, DataType::Int32]).into(), Some(ScalarImpl::Struct(value)), )), Box::new(LiteralExpression::new( From 184900ad9f4adad8504dafbfa86dbfd943dcc678 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Tue, 26 Nov 2024 15:10:37 +0800 Subject: [PATCH 016/163] chore(sqlsmith): ignore sqrt with negative input error (#19566) --- src/tests/sqlsmith/src/validation.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/tests/sqlsmith/src/validation.rs b/src/tests/sqlsmith/src/validation.rs index 99580616c15de..c49ee05cbdd97 100644 --- a/src/tests/sqlsmith/src/validation.rs +++ b/src/tests/sqlsmith/src/validation.rs @@ -104,6 +104,10 @@ pub fn is_neg_exp_error(db_error: &str) -> bool { db_error.contains("zero raised to a negative power is undefined") } +pub fn is_neg_input_error(db_error: &str) -> bool { + db_error.contains("input cannot be negative value") +} + /// Certain errors are permitted to occur. This is because: /// 1. It is more complex to generate queries without these errors. /// 2. These errors seldom occur, skipping them won't affect overall effectiveness of sqlsmith. @@ -121,4 +125,5 @@ pub fn is_permissible_error(db_error: &str) -> bool { || is_overlay_start_error(db_error) || is_broken_channel_error(db_error) || is_neg_exp_error(db_error) + || is_neg_input_error(db_error) } From a3c2648f6fe751d2eb17be8a56e6b524c2310a6b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 26 Nov 2024 16:19:55 +0800 Subject: [PATCH 017/163] chore(deps): Bump the all group across 1 directory with 52 updates (#19565) Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> Co-authored-by: Eric Fu --- .github/dependabot.yml | 16 +++++++++++ java/connector-node/assembly/pom.xml | 2 +- .../risingwave-source-cdc-test/pom.xml | 8 +----- .../risingwave-source-cdc/pom.xml | 1 + java/java-binding/pom.xml | 2 +- java/pom.xml | 27 ++++++++----------- java/proto/pom.xml | 2 +- java/tools/maven/checkstyle.xml | 2 +- 8 files changed, 33 insertions(+), 27 deletions(-) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 3ee3bea8bbabf..ca7a7920566f4 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -60,8 +60,24 @@ updates: - dependency-name: "io.debezium:*" update-types: ["version-update:semver-minor", "version-update:semver-major"] + # Don't upgrade protobuf to 4.x now. See https://github.com/grpc/grpc-java/issues/11015 + - dependency-name: "com.google.protobuf:*" + update-types: + ["version-update:semver-major"] + # Let's do major version updates manually + - dependency-name: "*" + update-types: + ["version-update:semver-major"] groups: # Group all dependenies together because Java libraries are quite stable all: patterns: - "*" + +# Don't touch risingwave-sink-deltalake-test. It's too complicated and it's only for testing +- package-ecosystem: maven + directory: /java/connector-node/risingwave-sink-deltalake-test/ + schedule: + interval: "weekly" + ignore: + - dependency-name: "*" \ No newline at end of file diff --git a/java/connector-node/assembly/pom.xml b/java/connector-node/assembly/pom.xml index bd24a75350335..26bd36dc29fcd 100644 --- a/java/connector-node/assembly/pom.xml +++ b/java/connector-node/assembly/pom.xml @@ -65,7 +65,7 @@ org.apache.maven.plugins maven-assembly-plugin - 3.4.2 + 3.7.1 UTF-8 ${assembly.name} diff --git a/java/connector-node/risingwave-source-cdc-test/pom.xml b/java/connector-node/risingwave-source-cdc-test/pom.xml index db7d207bb4a6f..0c7bb245c349e 100644 --- a/java/connector-node/risingwave-source-cdc-test/pom.xml +++ b/java/connector-node/risingwave-source-cdc-test/pom.xml @@ -57,7 +57,7 @@ com.zaxxer HikariCP - 6.0.0 + 6.2.1 test @@ -76,12 +76,6 @@ org.testcontainers mongodb - - - org.mongodb - mongodb-driver-sync - 5.2.0 - com.fasterxml.jackson.core jackson-databind diff --git a/java/connector-node/risingwave-source-cdc/pom.xml b/java/connector-node/risingwave-source-cdc/pom.xml index 26a2d5bf1edc0..a60f899f09ec4 100644 --- a/java/connector-node/risingwave-source-cdc/pom.xml +++ b/java/connector-node/risingwave-source-cdc/pom.xml @@ -68,6 +68,7 @@ org.mongodb mongodb-driver-sync + 4.11.5 diff --git a/java/java-binding/pom.xml b/java/java-binding/pom.xml index 8b7705ee46db3..58754054d7fa8 100644 --- a/java/java-binding/pom.xml +++ b/java/java-binding/pom.xml @@ -18,7 +18,7 @@ 11 11 0.1.0-SNAPSHOT - 1.0.0 + 1.2.0 false true diff --git a/java/pom.xml b/java/pom.xml index 70ac90e17be71..6f3ade3f36028 100644 --- a/java/pom.xml +++ b/java/pom.xml @@ -67,14 +67,14 @@ 3.25.5 1.68.1 - 2.24.1 + 2.24.2 2.0.16 2.6.2.Final 2.18.1 3.4.1 - 1.20.3 - 2.29.2 - 12.0.14 + 1.20.4 + 2.29.20 + 12.0.15 1.14.4 @@ -113,7 +113,7 @@ commons-io commons-io - 2.17.0 + 2.18.0 org.apache.commons @@ -173,11 +173,6 @@ mysql-connector-j 9.1.0 - - org.mongodb - mongodb-driver-sync - 4.11.5 - org.elasticsearch.client elasticsearch-rest-high-level-client @@ -186,7 +181,7 @@ org.opensearch.client opensearch-rest-high-level-client - 2.17.1 + 2.18.0 org.apache.httpcomponents @@ -419,7 +414,7 @@ com.puppycrawl.tools checkstyle - 8.29 + 10.20.1 @@ -472,7 +467,7 @@ org.apache.maven.plugins maven-surefire-plugin - 3.0.0-M7 + 3.5.2 true @@ -498,7 +493,7 @@ org.apache.maven.plugins maven-source-plugin - 3.3.0 + 3.3.1 attach-sources @@ -512,7 +507,7 @@ org.apache.maven.plugins maven-javadoc-plugin - 3.5.0 + 3.11.1 attach-javadocs @@ -547,7 +542,7 @@ org.sonatype.plugins nexus-staging-maven-plugin - 1.6.13 + 1.7.0 true ossrh diff --git a/java/proto/pom.xml b/java/proto/pom.xml index 642b4d378018e..29e49a65fa041 100644 --- a/java/proto/pom.xml +++ b/java/proto/pom.xml @@ -46,7 +46,7 @@ kr.motd.maven os-maven-plugin - 1.6.2 + 1.7.1 diff --git a/java/tools/maven/checkstyle.xml b/java/tools/maven/checkstyle.xml index b46efef4d313f..2196cfd1afc4e 100644 --- a/java/tools/maven/checkstyle.xml +++ b/java/tools/maven/checkstyle.xml @@ -221,7 +221,7 @@ This file is based on the checkstyle file of Apache Beam. - + From 4752db6d820a1af4af0a188f07933504dd9625a0 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Tue, 26 Nov 2024 16:56:34 +0800 Subject: [PATCH 018/163] fix: minor bug of `to_binary_with_type` of List (#19571) --- src/common/src/types/to_binary.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/common/src/types/to_binary.rs b/src/common/src/types/to_binary.rs index 294f96bc70453..63efcc47da69d 100644 --- a/src/common/src/types/to_binary.rs +++ b/src/common/src/types/to_binary.rs @@ -100,7 +100,7 @@ impl ToBinary for ListRef<'_> { buf.put_i32(1); // Has nulls? buf.put_i32(element_ty.to_oid()); // Element type buf.put_i32(self.len() as i32); // Length of 1st dimension - buf.put_i32(0); // Offset of 1st dimension + buf.put_i32(1); // Offset of 1st dimension, starting from 1 for element in self.iter() { match element { None => { From fc639feff477c8b97fee13afcf1de2687a4032b0 Mon Sep 17 00:00:00 2001 From: Croxx Date: Tue, 26 Nov 2024 17:11:27 +0800 Subject: [PATCH 019/163] perf(cache): reduce exclusive lock usage while dropping in-memory cache entry (#19436) Signed-off-by: MrCroxx --- Cargo.lock | 77 ++++++++-------- Cargo.toml | 6 +- src/batch/src/executor/insert.rs | 4 +- src/storage/Cargo.toml | 2 +- src/storage/benches/bench_block_cache.rs | 4 - src/storage/benches/bench_compactor.rs | 10 +-- .../benches/bench_hummock_iter.rs | 4 +- .../hummock_test/src/compactor_tests.rs | 10 +-- .../hummock_test/src/failpoint_tests.rs | 14 +-- .../hummock_test/src/hummock_storage_tests.rs | 90 +++++++++---------- .../hummock_test/src/snapshot_tests.rs | 4 +- .../hummock_test/src/state_store_tests.rs | 90 +++++++++---------- .../hummock_test/src/sync_point_tests.rs | 4 +- src/storage/hummock_trace/src/opts.rs | 16 ++-- .../compactor/shared_buffer_compact.rs | 4 +- .../sstable/backward_sstable_iterator.rs | 4 +- .../sstable/forward_sstable_iterator.rs | 4 +- src/storage/src/hummock/sstable/writer.rs | 4 +- src/storage/src/hummock/sstable/xor_filter.rs | 6 +- src/storage/src/hummock/sstable_store.rs | 15 ++-- src/storage/src/hummock/test_utils.rs | 4 +- src/storage/src/hummock/utils.rs | 8 +- src/storage/src/store_impl.rs | 2 - .../src/table/batch_table/storage_table.rs | 8 +- .../log_store_impl/kv_log_store/reader.rs | 6 +- src/stream/src/common/table/state_table.rs | 8 +- .../src/compaction_test_runner.rs | 4 +- 27 files changed, 206 insertions(+), 206 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index c08e776947cca..46ab0da13c40c 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -5044,6 +5044,12 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" +[[package]] +name = "foldhash" +version = "0.1.3" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f81ec6369c545a7d40e4589b5597581fa1c441fe1cce96dd1de43159910a36a2" + [[package]] name = "foreign-types" version = "0.3.2" @@ -5070,12 +5076,12 @@ dependencies = [ [[package]] name = "foyer" -version = "0.12.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "3ffa01d910407917a7c268fddf2be41163028693556b833a55f59351deffa011" +version = "0.13.0-dev" +source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" dependencies = [ "ahash 0.8.11", "anyhow", + "equivalent", "fastrace", "foyer-common", "foyer-memory", @@ -5088,13 +5094,11 @@ dependencies = [ [[package]] name = "foyer-common" -version = "0.12.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "797addbeabe5c42db9615e4d350a9b9d1597bebc07eefbd032d0e173430c8868" +version = "0.13.0-dev" +source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" dependencies = [ "bytes", "cfg-if", - "crossbeam", "fastrace", "futures", "hashbrown 0.14.5", @@ -5106,33 +5110,24 @@ dependencies = [ "serde", ] -[[package]] -name = "foyer-intrusive" -version = "0.12.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "b4ee0c1f19c7736f34e37e7cd4c1d5ceef3a32e24dbb14fcf2fcc836ce1cbfc4" -dependencies = [ - "foyer-common", - "itertools 0.13.0", -] - [[package]] name = "foyer-memory" -version = "0.12.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5b46b30275039721c05b390d4df00eff88814b1877ee06d56b61a859a5cf9577" +version = "0.13.0-dev" +source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" dependencies = [ "ahash 0.8.11", "bitflags 2.6.0", "cmsketch", + "equivalent", "fastrace", "foyer-common", - "foyer-intrusive", "futures", "hashbrown 0.14.5", + "intrusive-collections", "itertools 0.13.0", "madsim-tokio", "parking_lot 0.12.1", + "paste", "pin-project", "serde", "tracing", @@ -5140,9 +5135,8 @@ dependencies = [ [[package]] name = "foyer-storage" -version = "0.12.2" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "038ecf3211a5f984d534a5654a41613be7334fe8f5aab529708866da2de62d82" +version = "0.13.0-dev" +source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" dependencies = [ "ahash 0.8.11", "allocator-api2", @@ -5155,6 +5149,7 @@ dependencies = [ "bytes", "clap", "either", + "equivalent", "fastrace", "flume", "foyer-common", @@ -5873,6 +5868,9 @@ name = "hashbrown" version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e087f84d4f86bf4b218b927129862374b72199ae7d8657835f1e89000eea4fb" +dependencies = [ + "foldhash", +] [[package]] name = "hashlink" @@ -6557,6 +6555,14 @@ version = "3.0.4" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "8bb03732005da905c88227371639bf1ad885cc712789c011c31c5fb3ab3ccf02" +[[package]] +name = "intrusive-collections" +version = "0.9.7" +source = "git+https://github.com/foyer-rs/intrusive-rs?rev=94cfac4701dbc0033b7bc27e31c46bf3a12d96d7#94cfac4701dbc0033b7bc27e31c46bf3a12d96d7" +dependencies = [ + "memoffset", +] + [[package]] name = "io-extras" version = "0.18.1" @@ -7483,9 +7489,9 @@ dependencies = [ [[package]] name = "metrics" -version = "0.23.0" +version = "0.24.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "884adb57038347dfbaf2d5065887b6cf4312330dc8e94bc30a1a839bd79d3261" +checksum = "8ae428771d17306715c5091d446327d1cfdedc82185c65ba8423ab404e45bf10" dependencies = [ "ahash 0.8.11", "portable-atomic", @@ -7493,31 +7499,29 @@ dependencies = [ [[package]] name = "metrics-prometheus" -version = "0.7.0" +version = "0.8.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "51557a875fdbd5b953b698ecd6cd06efef47618e02d95ad912e2392e5b5617ff" +checksum = "ba3115d0e859470455e1587af70aed7935163339f959f01d6cbd245605fe7e67" dependencies = [ "arc-swap", "metrics", "metrics-util", - "once_cell", "prometheus", - "sealed 0.5.0", + "sealed 0.6.0", "smallvec", "thiserror 1.0.63", ] [[package]] name = "metrics-util" -version = "0.17.0" +version = "0.18.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "4259040465c955f9f2f1a4a8a16dc46726169bca0f88e8fb2dbeced487c3e828" +checksum = "15b482df36c13dd1869d73d14d28cd4855fbd6cfc32294bee109908a9f4a4ed7" dependencies = [ "crossbeam-epoch", "crossbeam-utils", - "hashbrown 0.14.5", + "hashbrown 0.15.0", "metrics", - "num_cpus", ] [[package]] @@ -12966,11 +12970,10 @@ dependencies = [ [[package]] name = "sealed" -version = "0.5.0" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f4a8caec23b7800fb97971a1c6ae365b6239aaeddfb934d6265f8505e795699d" +checksum = "22f968c5ea23d555e670b449c1c5e7b2fc399fdaec1d304a17cd48e288abc107" dependencies = [ - "heck 0.4.1", "proc-macro2", "quote", "syn 2.0.87", diff --git a/Cargo.toml b/Cargo.toml index 6f9b85aa5c1ae..a2c47ea293c78 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -80,7 +80,11 @@ license = "Apache-2.0" repository = "https://github.com/risingwavelabs/risingwave" [workspace.dependencies] -foyer = { version = "0.12.2", features = ["tracing", "nightly"] } +# foyer = { version = "0.12.2", features = ["tracing", "nightly"] } +foyer = { git = "https://github.com/mrcroxx/foyer", rev = "99005412eac1bca6aa7c73d750efc9cdb8ae176e", features = [ + "tracing", + "nightly", +] } apache-avro = { git = "https://github.com/risingwavelabs/avro", rev = "25113ba88234a9ae23296e981d8302c290fdaa4b", features = [ "snappy", "zstandard", diff --git a/src/batch/src/executor/insert.rs b/src/batch/src/executor/insert.rs index 704182175b660..c007deaa9a8e0 100644 --- a/src/batch/src/executor/insert.rs +++ b/src/batch/src/executor/insert.rs @@ -267,7 +267,7 @@ mod tests { use std::ops::Bound; use assert_matches::assert_matches; - use foyer::CacheContext; + use foyer::CacheHint; use futures::StreamExt; use risingwave_common::array::{Array, ArrayImpl, I32Array, StructArray}; use risingwave_common::catalog::{ @@ -398,7 +398,7 @@ mod tests { epoch, None, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index a5ed7938df2e8..6b7314ce0c72e 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -35,7 +35,7 @@ itertools = { workspace = true } libc = "0.2" lz4 = "1.28.0" memcomparable = "0.2" -metrics-prometheus = "0.7" +metrics-prometheus = "0.8" moka = { version = "0.12.0", features = ["future", "sync"] } more-asserts = "0.3" num-integer = "0.1" diff --git a/src/storage/benches/bench_block_cache.rs b/src/storage/benches/bench_block_cache.rs index fbc9783f444c0..16a797c144d0f 100644 --- a/src/storage/benches/bench_block_cache.rs +++ b/src/storage/benches/bench_block_cache.rs @@ -130,7 +130,6 @@ impl FoyerCache { .with_eviction_config(foyer::LruConfig { high_priority_pool_ratio: 0.8, }) - .with_object_pool_capacity(8 * 1024) .build(); Self { inner, @@ -147,7 +146,6 @@ impl FoyerCache { cmsketch_eps: 0.001, cmsketch_confidence: 0.9, }) - .with_object_pool_capacity(8 * 1024) .build(); Self { inner, @@ -187,7 +185,6 @@ impl FoyerHybridCache { .with_eviction_config(foyer::LruConfig { high_priority_pool_ratio: 0.8, }) - .with_object_pool_capacity(8 * 1024) .storage(Engine::Large) .build() .await @@ -208,7 +205,6 @@ impl FoyerHybridCache { cmsketch_eps: 0.001, cmsketch_confidence: 0.9, }) - .with_object_pool_capacity(8 * 1024) .storage(Engine::Large) .build() .await diff --git a/src/storage/benches/bench_compactor.rs b/src/storage/benches/bench_compactor.rs index f09c5c3fa15b9..77a22c88bea24 100644 --- a/src/storage/benches/bench_compactor.rs +++ b/src/storage/benches/bench_compactor.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use criterion::async_executor::FuturesExecutor; use criterion::{criterion_group, criterion_main, Criterion}; -use foyer::{CacheContext, Engine, HybridCacheBuilder}; +use foyer::{CacheHint, Engine, HybridCacheBuilder}; use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; use risingwave_common::config::{MetricLevel, ObjectStoreConfig}; use risingwave_common::hash::VirtualNode; @@ -95,7 +95,7 @@ pub fn default_writer_opts() -> SstableWriterOptions { SstableWriterOptions { capacity_hint: None, tracker: None, - policy: CachePolicy::Fill(CacheContext::Default), + policy: CachePolicy::Fill(CacheHint::Normal), } } @@ -132,7 +132,7 @@ async fn build_table( SstableWriterOptions { capacity_hint: None, tracker: None, - policy: CachePolicy::Fill(CacheContext::Default), + policy: CachePolicy::Fill(CacheHint::Normal), }, ); let table_id_to_vnode = HashMap::from_iter(vec![(0, VirtualNode::COUNT_FOR_TEST)]); @@ -181,7 +181,7 @@ async fn build_table_2( SstableWriterOptions { capacity_hint: None, tracker: None, - policy: CachePolicy::Fill(CacheContext::Default), + policy: CachePolicy::Fill(CacheHint::Normal), }, ); @@ -345,7 +345,7 @@ fn bench_merge_iterator_compactor(c: &mut Criterion) { }); let level2 = vec![info1, info2]; let read_options = Arc::new(SstableIteratorReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), prefetch_for_large_query: false, must_iterated_end_user_key: None, max_preload_retry_times: 0, diff --git a/src/storage/hummock_test/benches/bench_hummock_iter.rs b/src/storage/hummock_test/benches/bench_hummock_iter.rs index adb41aef460cc..1480cf14cddac 100644 --- a/src/storage/hummock_test/benches/bench_hummock_iter.rs +++ b/src/storage/hummock_test/benches/bench_hummock_iter.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use bytes::Bytes; use criterion::{criterion_group, criterion_main, Criterion}; -use foyer::CacheContext; +use foyer::CacheHint; use futures::pin_mut; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::key::TableKey; @@ -109,7 +109,7 @@ fn criterion_benchmark(c: &mut Criterion) { epoch, ReadOptions { prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, )) diff --git a/src/storage/hummock_test/src/compactor_tests.rs b/src/storage/hummock_test/src/compactor_tests.rs index d0ba14195ed2a..468d872bd4e5c 100644 --- a/src/storage/hummock_test/src/compactor_tests.rs +++ b/src/storage/hummock_test/src/compactor_tests.rs @@ -19,7 +19,7 @@ pub(crate) mod tests { use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; - use foyer::CacheContext; + use foyer::CacheHint; use itertools::Itertools; use rand::{Rng, RngCore, SeedableRng}; use risingwave_common::bitmap::BitmapBuilder; @@ -338,7 +338,7 @@ pub(crate) mod tests { TableKey(key.clone()), get_epoch, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -668,7 +668,7 @@ pub(crate) mod tests { ReadOptions { table_id: TableId::from(existing_table_id), prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -874,7 +874,7 @@ pub(crate) mod tests { ReadOptions { table_id: TableId::from(existing_table_id), prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1088,7 +1088,7 @@ pub(crate) mod tests { prefix_hint: Some(Bytes::from(bloom_filter_key)), table_id: TableId::from(existing_table_id), prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) diff --git a/src/storage/hummock_test/src/failpoint_tests.rs b/src/storage/hummock_test/src/failpoint_tests.rs index f7802380ea28f..9ed816f914154 100644 --- a/src/storage/hummock_test/src/failpoint_tests.rs +++ b/src/storage/hummock_test/src/failpoint_tests.rs @@ -17,7 +17,7 @@ use std::ops::Bound; use std::sync::Arc; use bytes::{BufMut, Bytes}; -use foyer::CacheContext; +use foyer::CacheHint; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::TABLE_PREFIX_LEN; @@ -116,7 +116,7 @@ async fn test_failpoints_state_store_read_upload() { 1, ReadOptions { prefix_hint: Some(Bytes::from(anchor_prefix_hint)), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -172,7 +172,7 @@ async fn test_failpoints_state_store_read_upload() { 2, ReadOptions { prefix_hint: Some(Bytes::from(anchor_prefix_hint)), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -187,7 +187,7 @@ async fn test_failpoints_state_store_read_upload() { 2, ReadOptions { table_id: Default::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -206,7 +206,7 @@ async fn test_failpoints_state_store_read_upload() { 2, ReadOptions { prefix_hint: Some(Bytes::from(bee_prefix_hint)), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -248,7 +248,7 @@ async fn test_failpoints_state_store_read_upload() { 5, ReadOptions { prefix_hint: Some(Bytes::from(anchor_prefix_hint)), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -265,7 +265,7 @@ async fn test_failpoints_state_store_read_upload() { 5, ReadOptions { prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 0b216e84c4960..d85354636a850 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -18,7 +18,7 @@ use std::ops::Range; use std::sync::Arc; use bytes::{BufMut, Bytes}; -use foyer::CacheContext; +use foyer::CacheHint; use futures::TryStreamExt; use itertools::Itertools; use risingwave_common::bitmap::BitmapBuilder; @@ -133,7 +133,7 @@ async fn test_storage_basic() { ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -148,7 +148,7 @@ async fn test_storage_basic() { epoch1, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -165,7 +165,7 @@ async fn test_storage_basic() { epoch1, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -198,7 +198,7 @@ async fn test_storage_basic() { ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -232,7 +232,7 @@ async fn test_storage_basic() { epoch3, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -249,7 +249,7 @@ async fn test_storage_basic() { ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -269,7 +269,7 @@ async fn test_storage_basic() { ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -310,7 +310,7 @@ async fn test_storage_basic() { ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -328,7 +328,7 @@ async fn test_storage_basic() { ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -347,7 +347,7 @@ async fn test_storage_basic() { epoch2, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -401,7 +401,7 @@ async fn test_storage_basic() { epoch3, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -597,7 +597,7 @@ async fn test_state_store_sync() { epoch1, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -643,7 +643,7 @@ async fn test_state_store_sync() { epoch2, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -666,7 +666,7 @@ async fn test_state_store_sync() { epoch1, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), read_committed: true, ..Default::default() }, @@ -686,7 +686,7 @@ async fn test_state_store_sync() { epoch1, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), read_committed: true, ..Default::default() }, @@ -748,7 +748,7 @@ async fn test_state_store_sync() { epoch2, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -768,7 +768,7 @@ async fn test_state_store_sync() { epoch2, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -925,7 +925,7 @@ async fn test_delete_get() { epoch2, ReadOptions { prefix_hint: None, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() } ) @@ -1034,7 +1034,7 @@ async fn test_multiple_epoch_sync() { ReadOptions { table_id: TEST_TABLE_ID, read_committed, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1050,7 +1050,7 @@ async fn test_multiple_epoch_sync() { ReadOptions { table_id: TEST_TABLE_ID, read_committed, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1065,7 +1065,7 @@ async fn test_multiple_epoch_sync() { ReadOptions { table_id: TEST_TABLE_ID, read_committed, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1213,7 +1213,7 @@ async fn test_iter_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1239,7 +1239,7 @@ async fn test_iter_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1264,7 +1264,7 @@ async fn test_iter_with_min_epoch() { table_id: TEST_TABLE_ID, retention_seconds: Some(0), prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1316,7 +1316,7 @@ async fn test_iter_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), read_committed: true, ..Default::default() }, @@ -1343,7 +1343,7 @@ async fn test_iter_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1370,7 +1370,7 @@ async fn test_iter_with_min_epoch() { table_id: TEST_TABLE_ID, retention_seconds: Some(0), prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1507,7 +1507,7 @@ async fn test_hummock_version_reader() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1542,7 +1542,7 @@ async fn test_hummock_version_reader() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1578,7 +1578,7 @@ async fn test_hummock_version_reader() { table_id: TEST_TABLE_ID, retention_seconds: Some(0), prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1650,7 +1650,7 @@ async fn test_hummock_version_reader() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1698,7 +1698,7 @@ async fn test_hummock_version_reader() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1734,7 +1734,7 @@ async fn test_hummock_version_reader() { table_id: TEST_TABLE_ID, retention_seconds: Some(0), prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1769,7 +1769,7 @@ async fn test_hummock_version_reader() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1807,7 +1807,7 @@ async fn test_hummock_version_reader() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1839,7 +1839,7 @@ async fn test_hummock_version_reader() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, read_snapshot, @@ -1945,7 +1945,7 @@ async fn test_get_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefetch_options: Default::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1963,7 +1963,7 @@ async fn test_get_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefix_hint: Some(Bytes::from(prefix_hint.clone())), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1981,7 +1981,7 @@ async fn test_get_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefix_hint: Some(Bytes::from(prefix_hint.clone())), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -2001,7 +2001,7 @@ async fn test_get_with_min_epoch() { retention_seconds: Some(0), prefix_hint: Some(Bytes::from(prefix_hint.clone())), prefetch_options: Default::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -2051,7 +2051,7 @@ async fn test_get_with_min_epoch() { epoch1, ReadOptions { table_id: TEST_TABLE_ID, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), read_committed: true, ..Default::default() }, @@ -2071,7 +2071,7 @@ async fn test_get_with_min_epoch() { table_id: TEST_TABLE_ID, read_committed: true, prefix_hint: Some(Bytes::from(prefix_hint.clone())), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -2090,7 +2090,7 @@ async fn test_get_with_min_epoch() { ReadOptions { table_id: TEST_TABLE_ID, prefix_hint: Some(Bytes::from(prefix_hint.clone())), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -2111,7 +2111,7 @@ async fn test_get_with_min_epoch() { retention_seconds: Some(0), prefix_hint: Some(Bytes::from(prefix_hint.clone())), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) diff --git a/src/storage/hummock_test/src/snapshot_tests.rs b/src/storage/hummock_test/src/snapshot_tests.rs index d8b77df0dbe92..11bfec2b435d6 100644 --- a/src/storage/hummock_test/src/snapshot_tests.rs +++ b/src/storage/hummock_test/src/snapshot_tests.rs @@ -16,7 +16,7 @@ use std::ops::Bound; use std::sync::Arc; use bytes::Bytes; -use foyer::CacheContext; +use foyer::CacheHint; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_hummock_sdk::key::prefixed_range_with_vnode; @@ -59,7 +59,7 @@ macro_rules! assert_count_range_scan { $epoch, ReadOptions { prefetch_options: PrefetchOptions::prefetch_for_large_range_scan(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), read_committed: $read_committed, ..Default::default() }, diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 1fd6f6ae0de25..7124d25020fd4 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use bytes::Bytes; use expect_test::expect; -use foyer::CacheContext; +use foyer::CacheHint; use futures::{pin_mut, StreamExt}; use itertools::Itertools; use risingwave_common::bitmap::Bitmap; @@ -55,7 +55,7 @@ async fn test_empty_read() { u64::MAX, ReadOptions { table_id: TableId { table_id: 2333 }, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -71,7 +71,7 @@ async fn test_empty_read() { u64::MAX, ReadOptions { table_id: TableId { table_id: 2333 }, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -171,7 +171,7 @@ async fn test_basic() { anchor.clone(), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -184,7 +184,7 @@ async fn test_basic() { gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -199,7 +199,7 @@ async fn test_basic() { gen_key_from_str(VirtualNode::ZERO, "ab"), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -229,7 +229,7 @@ async fn test_basic() { anchor.clone(), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -259,7 +259,7 @@ async fn test_basic() { anchor.clone(), epoch3, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -273,7 +273,7 @@ async fn test_basic() { gen_key_from_str(VirtualNode::ZERO, "ff"), epoch3, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -293,7 +293,7 @@ async fn test_basic() { ), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -308,7 +308,7 @@ async fn test_basic() { anchor.clone(), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -323,7 +323,7 @@ async fn test_basic() { anchor.clone(), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -344,7 +344,7 @@ async fn test_basic() { epoch2, ReadOptions { prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -366,7 +366,7 @@ async fn test_basic() { epoch3, ReadOptions { prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -395,7 +395,7 @@ async fn test_basic() { gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -408,7 +408,7 @@ async fn test_basic() { gen_key_from_str(VirtualNode::ZERO, "dd"), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -605,7 +605,7 @@ async fn test_reload_storage() { anchor.clone(), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -620,7 +620,7 @@ async fn test_reload_storage() { gen_key_from_str(VirtualNode::ZERO, "ab"), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -648,7 +648,7 @@ async fn test_reload_storage() { anchor.clone(), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -670,7 +670,7 @@ async fn test_reload_storage() { epoch1, ReadOptions { prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -685,7 +685,7 @@ async fn test_reload_storage() { anchor.clone(), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -700,7 +700,7 @@ async fn test_reload_storage() { anchor.clone(), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -721,7 +721,7 @@ async fn test_reload_storage() { epoch2, ReadOptions { prefetch_options: PrefetchOptions::default(), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -757,7 +757,7 @@ async fn test_reload_storage() { // gen_key_from_str(VirtualNode::ZERO, "aa"), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // } // ) @@ -772,7 +772,7 @@ async fn test_reload_storage() { // gen_key_from_str(VirtualNode::ZERO, "bb"), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // } // ) @@ -787,7 +787,7 @@ async fn test_reload_storage() { // gen_key_from_str(VirtualNode::ZERO, "cc"), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // } // ) @@ -804,7 +804,7 @@ async fn test_reload_storage() { // ), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // }, // ) @@ -889,7 +889,7 @@ async fn test_reload_storage() { // gen_key_from_str(VirtualNode::ZERO, "aa"), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // } // ) @@ -903,7 +903,7 @@ async fn test_reload_storage() { // gen_key_from_str(VirtualNode::ZERO, "bb"), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // } // ) @@ -917,7 +917,7 @@ async fn test_reload_storage() { // gen_key_from_str(VirtualNode::ZERO, "cc"), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // } // ) @@ -933,7 +933,7 @@ async fn test_reload_storage() { // ), // epoch, // ReadOptions { -// cache_policy: CachePolicy::Fill(CacheContext::Default), +// cache_policy: CachePolicy::Fill(CacheHint::Normal), // ..Default::default() // }, // ) @@ -1112,7 +1112,7 @@ async fn test_delete_get() { gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() } ) @@ -1207,7 +1207,7 @@ async fn test_multiple_epoch_sync() { epoch1, ReadOptions { read_committed, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() } ) @@ -1222,7 +1222,7 @@ async fn test_multiple_epoch_sync() { epoch2, ReadOptions { read_committed, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() } ) @@ -1236,7 +1236,7 @@ async fn test_multiple_epoch_sync() { epoch3, ReadOptions { read_committed, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() } ) @@ -1393,7 +1393,7 @@ async fn test_replicated_local_hummock_storage() { table_id: TableId { table_id: TEST_TABLE_ID.table_id, }, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; @@ -1774,7 +1774,7 @@ async fn test_get_keyed_row() { anchor.clone(), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1788,7 +1788,7 @@ async fn test_get_keyed_row() { gen_key_from_str(VirtualNode::ZERO, "bb"), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1807,7 +1807,7 @@ async fn test_get_keyed_row() { gen_key_from_str(VirtualNode::ZERO, "ab"), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1837,7 +1837,7 @@ async fn test_get_keyed_row() { anchor.clone(), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1867,7 +1867,7 @@ async fn test_get_keyed_row() { anchor.clone(), epoch3, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1881,7 +1881,7 @@ async fn test_get_keyed_row() { anchor.clone(), epoch1, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1897,7 +1897,7 @@ async fn test_get_keyed_row() { anchor.clone(), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1928,7 +1928,7 @@ async fn test_get_keyed_row() { gen_key_from_str(VirtualNode::ZERO, "bb"), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) @@ -1945,7 +1945,7 @@ async fn test_get_keyed_row() { gen_key_from_str(VirtualNode::ZERO, "dd"), epoch2, ReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) diff --git a/src/storage/hummock_test/src/sync_point_tests.rs b/src/storage/hummock_test/src/sync_point_tests.rs index 065c9536006bb..c31597c256351 100644 --- a/src/storage/hummock_test/src/sync_point_tests.rs +++ b/src/storage/hummock_test/src/sync_point_tests.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use std::time::Duration; use bytes::Bytes; -use foyer::CacheContext; +use foyer::CacheHint; use risingwave_common::catalog::hummock::CompactionFilterFlag; use risingwave_common::catalog::TableId; use risingwave_common::hash::VirtualNode; @@ -462,7 +462,7 @@ async fn test_syncpoints_get_in_delete_range_boundary() { storage.wait_version(version).await; let read_options = ReadOptions { table_id: TableId::from(existing_table_id), - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; let get_result = storage diff --git a/src/storage/hummock_trace/src/opts.rs b/src/storage/hummock_trace/src/opts.rs index 480f0315eb396..aede95fbd25f6 100644 --- a/src/storage/hummock_trace/src/opts.rs +++ b/src/storage/hummock_trace/src/opts.rs @@ -13,7 +13,7 @@ // limitations under the License. use bincode::{Decode, Encode}; -use foyer::CacheContext; +use foyer::CacheHint; use risingwave_common::bitmap::Bitmap; use risingwave_common::cache::CachePriority; use risingwave_common::catalog::{TableId, TableOption}; @@ -60,20 +60,20 @@ impl From for CachePriority { } } -impl From for TracedCachePriority { - fn from(value: CacheContext) -> Self { +impl From for TracedCachePriority { + fn from(value: CacheHint) -> Self { match value { - CacheContext::Default => Self::High, - CacheContext::LowPriority => Self::Low, + CacheHint::Normal => Self::High, + CacheHint::Low => Self::Low, } } } -impl From for CacheContext { +impl From for CacheHint { fn from(value: TracedCachePriority) -> Self { match value { - TracedCachePriority::High => Self::Default, - TracedCachePriority::Low => Self::LowPriority, + TracedCachePriority::High => Self::Normal, + TracedCachePriority::Low => Self::Low, } } } diff --git a/src/storage/src/hummock/compactor/shared_buffer_compact.rs b/src/storage/src/hummock/compactor/shared_buffer_compact.rs index b7ae13b2c1067..3ec310ec32da5 100644 --- a/src/storage/src/hummock/compactor/shared_buffer_compact.rs +++ b/src/storage/src/hummock/compactor/shared_buffer_compact.rs @@ -20,7 +20,7 @@ use std::sync::{Arc, LazyLock}; use await_tree::InstrumentAwait; use bytes::Bytes; -use foyer::CacheContext; +use foyer::CacheHint; use futures::future::try_join; use futures::{stream, FutureExt, StreamExt, TryFutureExt}; use itertools::Itertools; @@ -526,7 +526,7 @@ impl SharedBufferCompactRunner { options, super::TaskConfig { key_range, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), gc_delete_keys: GC_DELETE_KEYS_FOR_FLUSH, retain_multiple_version: true, stats_target_table_ids: None, diff --git a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs index d923750de7859..b15693aacd56b 100644 --- a/src/storage/src/hummock/sstable/backward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/backward_sstable_iterator.rs @@ -15,7 +15,7 @@ use std::cmp::Ordering::{Equal, Less}; use std::sync::Arc; -use foyer::CacheContext; +use foyer::CacheHint; use risingwave_hummock_sdk::key::FullKey; use crate::hummock::iterator::{Backward, HummockIterator, ValueMeta}; @@ -67,7 +67,7 @@ impl BackwardSstableIterator { .get( &self.sst, idx as usize, - crate::hummock::CachePolicy::Fill(CacheContext::Default), + crate::hummock::CachePolicy::Fill(CacheHint::Normal), &mut self.stats, ) .await?; diff --git a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs index 287c387dd3270..b9da631fc4927 100644 --- a/src/storage/src/hummock/sstable/forward_sstable_iterator.rs +++ b/src/storage/src/hummock/sstable/forward_sstable_iterator.rs @@ -312,7 +312,7 @@ mod tests { use std::collections::Bound; use bytes::Bytes; - use foyer::CacheContext; + use foyer::CacheHint; use itertools::Itertools; use rand::prelude::*; use risingwave_common::catalog::TableId; @@ -477,7 +477,7 @@ mod tests { TableKey(Bytes::from(end_key.user_key.table_key.0)), ); let options = Arc::new(SstableIteratorReadOptions { - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), must_iterated_end_user_key: Some(Bound::Included(uk.clone())), max_preload_retry_times: 0, prefetch_for_large_query: false, diff --git a/src/storage/src/hummock/sstable/writer.rs b/src/storage/src/hummock/sstable/writer.rs index 9c0f3f06f1c28..e446649570927 100644 --- a/src/storage/src/hummock/sstable/writer.rs +++ b/src/storage/src/hummock/sstable/writer.rs @@ -225,7 +225,7 @@ impl SstableWriter for BatchUploadWriter { // The `block_info` may be empty when there is only range-tombstones, because we // store them in meta-block. for (block_idx, block) in self.block_info.into_iter().enumerate() { - self.sstable_store.block_cache().insert_with_context( + self.sstable_store.block_cache().insert_with_hint( SstableBlockIndex { sst_id: self.object_id, block_idx: block_idx as _, @@ -336,7 +336,7 @@ impl SstableWriter for StreamingUploadWriter { && !self.blocks.is_empty() { for (block_idx, block) in self.blocks.into_iter().enumerate() { - self.sstable_store.block_cache().insert_with_context( + self.sstable_store.block_cache().insert_with_hint( SstableBlockIndex { sst_id: self.object_id, block_idx: block_idx as _, diff --git a/src/storage/src/hummock/sstable/xor_filter.rs b/src/storage/src/hummock/sstable/xor_filter.rs index 8120184b17247..7fc439f2a8265 100644 --- a/src/storage/src/hummock/sstable/xor_filter.rs +++ b/src/storage/src/hummock/sstable/xor_filter.rs @@ -444,7 +444,7 @@ impl Clone for XorFilterReader { mod tests { use std::collections::HashMap; - use foyer::CacheContext; + use foyer::CacheHint; use rand::RngCore; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::test_epoch; @@ -467,7 +467,7 @@ mod tests { let writer_opts = SstableWriterOptions { capacity_hint: None, tracker: None, - policy: CachePolicy::Fill(CacheContext::Default), + policy: CachePolicy::Fill(CacheHint::Normal), }; let opts = SstableBuilderOptions { capacity: 0, @@ -522,7 +522,7 @@ mod tests { .get_block_response( &sstable, idx, - CachePolicy::Fill(CacheContext::Default), + CachePolicy::Fill(CacheHint::Normal), &mut stat, ) .await diff --git a/src/storage/src/hummock/sstable_store.rs b/src/storage/src/hummock/sstable_store.rs index 402e3e37594d0..24c3acfef9948 100644 --- a/src/storage/src/hummock/sstable_store.rs +++ b/src/storage/src/hummock/sstable_store.rs @@ -21,8 +21,7 @@ use await_tree::InstrumentAwait; use bytes::Bytes; use fail::fail_point; use foyer::{ - CacheContext, Engine, EventListener, FetchState, HybridCache, HybridCacheBuilder, - HybridCacheEntry, + CacheHint, Engine, EventListener, FetchState, HybridCache, HybridCacheBuilder, HybridCacheEntry, }; use futures::{future, StreamExt}; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -67,7 +66,7 @@ impl EventListener for BlockCacheEventListener { type Key = SstableBlockIndex; type Value = Box; - fn on_memory_release(&self, _key: Self::Key, value: Self::Value) + fn on_leave(&self, _reason: foyer::Event, _key: &Self::Key, value: &Self::Value) where Self::Key: foyer::Key, Self::Value: foyer::Value, @@ -84,14 +83,14 @@ pub enum CachePolicy { /// Disable read cache and not fill the cache afterwards. Disable, /// Try reading the cache and fill the cache afterwards. - Fill(CacheContext), + Fill(CacheHint), /// Read the cache but not fill the cache afterwards. NotFill, } impl Default for CachePolicy { fn default() -> Self { - CachePolicy::Fill(CacheContext::Default) + CachePolicy::Fill(CacheHint::Normal) } } @@ -354,9 +353,9 @@ impl SstableStore { let cache_priority = if idx == block_index { priority } else { - CacheContext::LowPriority + CacheHint::Low }; - let entry = self.block_cache.insert_with_context( + let entry = self.block_cache.insert_with_hint( SstableBlockIndex { sst_id: object_id, block_idx: idx as _, @@ -440,7 +439,7 @@ impl SstableStore { match policy { CachePolicy::Fill(context) => { - let entry = self.block_cache.fetch_with_context( + let entry = self.block_cache.fetch_with_hint( SstableBlockIndex { sst_id: object_id, block_idx: block_index as _, diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index 45d18027c1ebe..148810f9b016d 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use bytes::Bytes; use foyer::{ - CacheContext, Engine, HybridCache, HybridCacheBuilder, StorageKey as HybridKey, + CacheHint, Engine, HybridCache, HybridCacheBuilder, StorageKey as HybridKey, StorageValue as HybridValue, }; use itertools::Itertools; @@ -322,7 +322,7 @@ pub async fn gen_test_sstable_with_range_tombstone( object_id, kv_iter, sstable_store.clone(), - CachePolicy::Fill(CacheContext::Default), + CachePolicy::Fill(CacheHint::Normal), ) .await } diff --git a/src/storage/src/hummock/utils.rs b/src/storage/src/hummock/utils.rs index 2092aba24dec5..f73d434cd8c59 100644 --- a/src/storage/src/hummock/utils.rs +++ b/src/storage/src/hummock/utils.rs @@ -22,7 +22,7 @@ use std::sync::Arc; use std::time::{Duration, Instant}; use bytes::Bytes; -use foyer::CacheContext; +use foyer::CacheHint; use parking_lot::Mutex; use risingwave_common::catalog::{TableId, TableOption}; use risingwave_common::config::StorageMemoryConfig; @@ -387,7 +387,7 @@ pub(crate) async fn do_insert_sanity_check( let read_options = ReadOptions { retention_seconds: table_option.retention_seconds, table_id, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; let stored_value = inner.get(key.clone(), epoch, read_options).await?; @@ -423,7 +423,7 @@ pub(crate) async fn do_delete_sanity_check( let read_options = ReadOptions { retention_seconds: table_option.retention_seconds, table_id, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; match inner.get(key.clone(), epoch, read_options).await? { @@ -469,7 +469,7 @@ pub(crate) async fn do_update_sanity_check( let read_options = ReadOptions { retention_seconds: table_option.retention_seconds, table_id, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index 8ce2906ea3e75..d0f7280ca3d84 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -625,7 +625,6 @@ impl StateStoreImpl { .memory(opts.meta_cache_capacity_mb * MB) .with_shards(opts.meta_cache_shard_num) .with_eviction_config(opts.meta_cache_eviction_config.clone()) - .with_object_pool_capacity(1024 * opts.meta_cache_shard_num) .with_weighter(|_: &HummockSstableObjectId, value: &Box| { u64::BITS as usize / 8 + value.estimate_size() }) @@ -674,7 +673,6 @@ impl StateStoreImpl { .memory(opts.block_cache_capacity_mb * MB) .with_shards(opts.block_cache_shard_num) .with_eviction_config(opts.block_cache_eviction_config.clone()) - .with_object_pool_capacity(1024 * opts.block_cache_shard_num) .with_weighter(|_: &SstableBlockIndex, value: &Box| { // FIXME(MrCroxx): Calculate block weight more accurately. u64::BITS as usize * 2 / 8 + value.raw().len() diff --git a/src/storage/src/table/batch_table/storage_table.rs b/src/storage/src/table/batch_table/storage_table.rs index 65e6d2cc95d31..ef3813dbaa15a 100644 --- a/src/storage/src/table/batch_table/storage_table.rs +++ b/src/storage/src/table/batch_table/storage_table.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use auto_enums::auto_enum; use await_tree::InstrumentAwait; use bytes::Bytes; -use foyer::CacheContext; +use foyer::CacheHint; use futures::future::try_join_all; use futures::{Stream, StreamExt}; use futures_async_stream::try_stream; @@ -401,7 +401,7 @@ impl StorageTableInner { table_id: self.table_id, read_version_from_backup: read_backup, read_committed, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; if let Some((full_key, value)) = self @@ -521,8 +521,8 @@ impl StorageTableInner { ) { // To prevent unbounded range scan queries from polluting the block cache, use the // low priority fill policy. - (Unbounded, _) | (_, Unbounded) => CachePolicy::Fill(CacheContext::LowPriority), - _ => CachePolicy::Fill(CacheContext::Default), + (Unbounded, _) | (_, Unbounded) => CachePolicy::Fill(CacheHint::Low), + _ => CachePolicy::Fill(CacheHint::Normal), }; let table_key_ranges = { diff --git a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs index 9dcc0453bc7db..c51630e184cdd 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/reader.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/reader.rs @@ -21,7 +21,7 @@ use std::time::Duration; use anyhow::anyhow; use await_tree::InstrumentAwait; use bytes::Bytes; -use foyer::CacheContext; +use foyer::CacheHint; use futures::future::{try_join_all, BoxFuture}; use futures::{FutureExt, TryFutureExt}; use risingwave_common::array::StreamChunk; @@ -377,7 +377,7 @@ impl KvLogStoreReader { ReadOptions { // This stream lives too long, the connection of prefetch object may break. So use a short connection prefetch. prefetch_options: PrefetchOptions::prefetch_for_small_range_scan(), - cache_policy: CachePolicy::Fill(CacheContext::LowPriority), + cache_policy: CachePolicy::Fill(CacheHint::Low), table_id, ..Default::default() }, @@ -529,7 +529,7 @@ impl LogReader for KvLogStoreReader { ReadOptions { prefetch_options: PrefetchOptions::prefetch_for_large_range_scan(), - cache_policy: CachePolicy::Fill(CacheContext::LowPriority), + cache_policy: CachePolicy::Fill(CacheHint::Low), table_id, ..Default::default() }, diff --git a/src/stream/src/common/table/state_table.rs b/src/stream/src/common/table/state_table.rs index 7979bc9f4354f..c39fb37300134 100644 --- a/src/stream/src/common/table/state_table.rs +++ b/src/stream/src/common/table/state_table.rs @@ -19,7 +19,7 @@ use std::sync::Arc; use bytes::{BufMut, Bytes, BytesMut}; use either::Either; -use foyer::CacheContext; +use foyer::CacheHint; use futures::{pin_mut, Stream, StreamExt, TryStreamExt}; use futures_async_stream::for_await; use itertools::{izip, Itertools}; @@ -650,7 +650,7 @@ where prefix_hint, retention_seconds: self.table_option.retention_seconds, table_id: self.table_id, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; @@ -1221,7 +1221,7 @@ where retention_seconds: self.table_option.retention_seconds, table_id: self.table_id, prefetch_options, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; @@ -1239,7 +1239,7 @@ where retention_seconds: self.table_option.retention_seconds, table_id: self.table_id, prefetch_options, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }; diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index 37d38c88947ee..cba5565a0906c 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -23,7 +23,7 @@ use std::time::Duration; use anyhow::anyhow; use bytes::{BufMut, Bytes, BytesMut}; use clap::Parser; -use foyer::CacheContext; +use foyer::CacheHint; use risingwave_common::catalog::TableId; use risingwave_common::config::{ extract_storage_memory_config, load_config, MetaConfig, NoOverride, @@ -627,7 +627,7 @@ async fn open_hummock_iters( epoch, ReadOptions { table_id: TableId { table_id }, - cache_policy: CachePolicy::Fill(CacheContext::Default), + cache_policy: CachePolicy::Fill(CacheHint::Normal), ..Default::default() }, ) From 759ea19eda7cd72afcacb186f79e4e17ed603669 Mon Sep 17 00:00:00 2001 From: StrikeW Date: Tue, 26 Nov 2024 17:25:30 +0800 Subject: [PATCH 020/163] feat(cdc): init default value for cdc table columns (#19354) Co-authored-by: Kexiang Wang --- .../source_legacy/cdc/cdc.share_stream.slt | 46 ++++ e2e_test/source_legacy/cdc/mysql_create.sql | 7 + e2e_test/source_legacy/cdc/postgres_cdc.sql | 8 + .../sql_server_cdc/sql_server_cdc.slt | 8 +- .../src/source/cdc/external/sql_server.rs | 10 + .../tests/testdata/input/create_source.yaml | 36 --- .../tests/testdata/output/create_source.yaml | 42 --- src/frontend/src/handler/create_table.rs | 240 ++++++++++-------- 8 files changed, 215 insertions(+), 182 deletions(-) diff --git a/e2e_test/source_legacy/cdc/cdc.share_stream.slt b/e2e_test/source_legacy/cdc/cdc.share_stream.slt index cf1000957b6fb..60b5f7cd52893 100644 --- a/e2e_test/source_legacy/cdc/cdc.share_stream.slt +++ b/e2e_test/source_legacy/cdc/cdc.share_stream.slt @@ -134,6 +134,33 @@ create table orders_test ( PRIMARY KEY (order_id) ) from mysql_mytest table 'mytest.orders'; +statement error +CREATE TABLE test_my_default_value ( + id int, + name varchar, + city varchar DEFAULT 'Beijing', + PRIMARY KEY (id) +) FROM mysql_mytest TABLE 'mytest.test_my_default_value'; + +statement ok +CREATE TABLE test_my_default_value ( + id int, + name varchar, + city varchar, + PRIMARY KEY (id) +) FROM mysql_mytest TABLE 'mytest.test_my_default_value'; + +statement ok +SET RW_IMPLICIT_FLUSH=true; + +statement ok +INSERT INTO test_my_default_value VALUES (2, 'jack'); + +query II +select * from test_my_default_value; +---- +2 jack Shanghai + statement ok create table kt1 (*) from mysql_source table 'kdb.kt1'; @@ -575,3 +602,22 @@ query II select * from upper_orders_shared order by id; ---- 1 happy + +statement ok +CREATE TABLE test_pg_default_value ( + id int, + name varchar, + city varchar, + PRIMARY KEY (id) +) FROM pg_source TABLE 'public.test_default_value'; + +statement ok +SET RW_IMPLICIT_FLUSH=true; + +statement ok +INSERT INTO test_pg_default_value VALUES (1, 'noris'); + +query II +select * from test_pg_default_value; +---- +1 noris Shanghai diff --git a/e2e_test/source_legacy/cdc/mysql_create.sql b/e2e_test/source_legacy/cdc/mysql_create.sql index 2279742356939..cf21b222558ea 100644 --- a/e2e_test/source_legacy/cdc/mysql_create.sql +++ b/e2e_test/source_legacy/cdc/mysql_create.sql @@ -49,3 +49,10 @@ CREATE TABLE IF NOT EXISTS mysql_all_types( INSERT INTO mysql_all_types VALUES ( False, 0, null, null, -8388608, -2147483647, 9223372036854775806, -10.0, -9999.999999, -10000.0, 'c', 'd', '', '', '1001-01-01', '-838:59:59.000000', '2000-01-01 00:00:00.000000', null); INSERT INTO mysql_all_types VALUES ( True, 1, -128, -32767, -8388608, -2147483647, -9223372036854775807, -10.0, -9999.999999, -10000.0, 'a', 'b', '', '', '1001-01-01', '00:00:00', '1998-01-01 00:00:00.000000', '1970-01-01 00:00:01'); + +CREATE TABLE test_my_default_value ( + id int, + name varchar(64), + city varchar(200) default 'Shanghai', + PRIMARY KEY (id) +); diff --git a/e2e_test/source_legacy/cdc/postgres_cdc.sql b/e2e_test/source_legacy/cdc/postgres_cdc.sql index e3e40ab6514c9..1b338ddefe902 100644 --- a/e2e_test/source_legacy/cdc/postgres_cdc.sql +++ b/e2e_test/source_legacy/cdc/postgres_cdc.sql @@ -149,3 +149,11 @@ INSERT INTO partitioned_timestamp_table (c_int, c_boolean, c_timestamp) VALUES -- Here we create this publication without `WITH ( publish_via_partition_root = true )` only for tests. Normally, it should be added. create publication rw_publication_pubviaroot_false for TABLE partitioned_timestamp_table; + + +CREATE TABLE test_default_value ( + "id" int, + "name" varchar(64), + "city" varchar(200) default 'Shanghai', + PRIMARY KEY ("id") +); diff --git a/e2e_test/source_legacy/cdc_inline/sql_server_cdc/sql_server_cdc.slt b/e2e_test/source_legacy/cdc_inline/sql_server_cdc/sql_server_cdc.slt index c8a5afc110302..dfbc0dae25d8b 100644 --- a/e2e_test/source_legacy/cdc_inline/sql_server_cdc/sql_server_cdc.slt +++ b/e2e_test/source_legacy/cdc_inline/sql_server_cdc/sql_server_cdc.slt @@ -171,7 +171,7 @@ CREATE TABLE shared_orders ( ) from mssql_source table 'orders'; # invalid table name -statement error Sql Server table 'dbo'.'wrong_orders' doesn't exist +statement error Sql Server table 'dbo'.'wrong_orders' not found CREATE TABLE shared_orders ( order_id INT, order_date BIGINT, @@ -183,7 +183,7 @@ CREATE TABLE shared_orders ( ) from mssql_source table 'mydb.dbo.wrong_orders'; # invalid schema name -statement error Sql Server table 'wrong_dbo'.'orders' doesn't exist +statement error Sql Server table 'wrong_dbo'.'orders' not found CREATE TABLE shared_orders ( order_id INT, order_date BIGINT, @@ -295,7 +295,7 @@ CREATE TABLE shared_sqlserver_all_data_types ( PRIMARY KEY (id) ) from mssql_source table 'mydb.dbo.sqlserver_all_data_types'; -statement error Sql Server table 'UpperSchema'.'UpperTable' doesn't exist in 'mydb' +statement error Sql Server table 'UpperSchema'.'UpperTable' not found in 'mydb' CREATE TABLE upper_table ( "ID" INT, "Name" VARCHAR, @@ -309,7 +309,7 @@ CREATE TABLE upper_table ( PRIMARY KEY ("ID") ) from upper_mssql_source table 'UpperDB.UpperSchema.UpperTable'; -statement error Sql Server table 'upperSchema'.'upperTable' doesn't exist in 'UpperDB' +statement error Sql Server table 'upperSchema'.'upperTable' not found in 'UpperDB' CREATE TABLE upper_table ( "ID" INT, "Name" VARCHAR, diff --git a/src/connector/src/source/cdc/external/sql_server.rs b/src/connector/src/source/cdc/external/sql_server.rs index 4350f3125fcd3..7ad908a5b3ea6 100644 --- a/src/connector/src/source/cdc/external/sql_server.rs +++ b/src/connector/src/source/cdc/external/sql_server.rs @@ -158,6 +158,16 @@ impl SqlServerExternalTable { } } + // The table does not exist + if column_descs.is_empty() { + bail!( + "Sql Server table '{}'.'{}' not found in '{}'", + config.schema, + config.table, + config.database + ); + } + Ok(Self { column_descs, pk_names, diff --git a/src/frontend/planner_test/tests/testdata/input/create_source.yaml b/src/frontend/planner_test/tests/testdata/input/create_source.yaml index 9d57c828c41d3..204d0fdfe0b1f 100644 --- a/src/frontend/planner_test/tests/testdata/input/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/input/create_source.yaml @@ -43,39 +43,3 @@ ) FORMAT PLAIN ENCODE CSV (delimiter = E'\t', without_header = true); expected_outputs: - explain_output -- id: create_source_with_cdc_backfill - sql: | - create source mysql_mydb with ( - connector = 'mysql-cdc', - hostname = '127.0.0.1', - port = '8306', - username = 'root', - password = '123456', - database.name = 'mydb', - server.id = 5888 - ); - explain (logical) create table t1_rw ( - v1 int, - v2 int, - primary key(v1) - ) from mysql_mydb table 'mydb.t1'; - expected_outputs: - - explain_output -- id: create_source_with_cdc_backfill - sql: | - create source mysql_mydb with ( - connector = 'mysql-cdc', - hostname = '127.0.0.1', - port = '8306', - username = 'root', - password = '123456', - database.name = 'mydb', - server.id = 5888 - ); - explain create table t1_rw ( - v1 int, - v2 int, - primary key(v1) - ) from mysql_mydb table 'mydb.t1'; - expected_outputs: - - explain_output diff --git a/src/frontend/planner_test/tests/testdata/output/create_source.yaml b/src/frontend/planner_test/tests/testdata/output/create_source.yaml index 887b068cc9638..77bcc43aa5733 100644 --- a/src/frontend/planner_test/tests/testdata/output/create_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/create_source.yaml @@ -62,45 +62,3 @@ └─StreamExchange { dist: HashShard(_row_id) } └─StreamDml { columns: [v1, v2, _row_id] } └─StreamSource -- id: create_source_with_cdc_backfill - sql: | - create source mysql_mydb with ( - connector = 'mysql-cdc', - hostname = '127.0.0.1', - port = '8306', - username = 'root', - password = '123456', - database.name = 'mydb', - server.id = 5888 - ); - explain (logical) create table t1_rw ( - v1 int, - v2 int, - primary key(v1) - ) from mysql_mydb table 'mydb.t1'; - explain_output: | - LogicalCdcScan { table: mydb.t1, columns: [v1, v2] } -- id: create_source_with_cdc_backfill - sql: | - create source mysql_mydb with ( - connector = 'mysql-cdc', - hostname = '127.0.0.1', - port = '8306', - username = 'root', - password = '123456', - database.name = 'mydb', - server.id = 5888 - ); - explain create table t1_rw ( - v1 int, - v2 int, - primary key(v1) - ) from mysql_mydb table 'mydb.t1'; - explain_output: | - StreamMaterialize { columns: [v1, v2], stream_key: [v1], pk_columns: [v1], pk_conflict: Overwrite } - └─StreamUnion { all: true } - ├─StreamExchange { dist: HashShard(mydb.t1.v1) } - │ └─StreamCdcTableScan { table: mydb.t1, columns: [v1, v2] } - └─StreamExchange { dist: HashShard(v1) } - └─StreamDml { columns: [v1, v2] } - └─StreamSource diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index cc6929a19d4ff..a34c1cf462e0f 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -956,116 +956,156 @@ pub(super) async fn handle_create_table_plan( &cdc_table_info, )?; - let ((plan, source, table), job_type) = - match (format_encode, cdc_table_info.as_ref()) { - (Some(format_encode), None) => ( - gen_create_table_plan_with_source( - handler_args, - explain_options, - table_name.clone(), - column_defs, - wildcard_idx, - constraints, - format_encode, - source_watermarks, - col_id_gen, - append_only, - on_conflict, - with_version_column, - include_column_options, - ) - .await?, - TableJobType::General, - ), - (None, None) => { - let context = OptimizerContext::new(handler_args, explain_options); - let (plan, table) = gen_create_table_plan( - context, - table_name.clone(), - column_defs, - constraints, - col_id_gen, - source_watermarks, - append_only, - on_conflict, - with_version_column, - )?; + let ((plan, source, table), job_type) = match (format_encode, cdc_table_info.as_ref()) { + (Some(format_encode), None) => ( + gen_create_table_plan_with_source( + handler_args, + explain_options, + table_name.clone(), + column_defs, + wildcard_idx, + constraints, + format_encode, + source_watermarks, + col_id_gen, + append_only, + on_conflict, + with_version_column, + include_column_options, + ) + .await?, + TableJobType::General, + ), + (None, None) => { + let context = OptimizerContext::new(handler_args, explain_options); + let (plan, table) = gen_create_table_plan( + context, + table_name.clone(), + column_defs, + constraints, + col_id_gen, + source_watermarks, + append_only, + on_conflict, + with_version_column, + )?; - ((plan, None, table), TableJobType::General) - } + ((plan, None, table), TableJobType::General) + } - (None, Some(cdc_table)) => { - sanity_check_for_cdc_table( - append_only, - &column_defs, - &wildcard_idx, - &constraints, - &source_watermarks, - )?; + (None, Some(cdc_table)) => { + sanity_check_for_cdc_table( + append_only, + &column_defs, + &wildcard_idx, + &constraints, + &source_watermarks, + )?; - let session = &handler_args.session; - let db_name = session.database(); - let (schema_name, resolved_table_name) = - Binder::resolve_schema_qualified_name(db_name, table_name.clone())?; - let (database_id, schema_id) = - session.get_database_and_schema_id_for_create(schema_name.clone())?; - - // cdc table cannot be append-only - let (format_encode, source_name) = - Binder::resolve_schema_qualified_name(db_name, cdc_table.source_name.clone())?; - - let source = { - let catalog_reader = session.env().catalog_reader().read_guard(); - let schema_name = format_encode - .clone() - .unwrap_or(DEFAULT_SCHEMA_NAME.to_string()); - let (source, _) = catalog_reader.get_source_by_name( - db_name, - SchemaPath::Name(schema_name.as_str()), - source_name.as_str(), - )?; - source.clone() - }; - let cdc_with_options: WithOptionsSecResolved = derive_with_options_for_cdc_table( - &source.with_properties, - cdc_table.external_table_name.clone(), + let session = &handler_args.session; + let db_name = session.database(); + let (schema_name, resolved_table_name) = + Binder::resolve_schema_qualified_name(db_name, table_name.clone())?; + let (database_id, schema_id) = + session.get_database_and_schema_id_for_create(schema_name.clone())?; + + // cdc table cannot be append-only + let (format_encode, source_name) = + Binder::resolve_schema_qualified_name(db_name, cdc_table.source_name.clone())?; + + let source = { + let catalog_reader = session.env().catalog_reader().read_guard(); + let schema_name = format_encode + .clone() + .unwrap_or(DEFAULT_SCHEMA_NAME.to_string()); + let (source, _) = catalog_reader.get_source_by_name( + db_name, + SchemaPath::Name(schema_name.as_str()), + source_name.as_str(), )?; + source.clone() + }; + let cdc_with_options: WithOptionsSecResolved = derive_with_options_for_cdc_table( + &source.with_properties, + cdc_table.external_table_name.clone(), + )?; - let (columns, pk_names) = match wildcard_idx { - Some(_) => bind_cdc_table_schema_externally(cdc_with_options.clone()).await?, - None => bind_cdc_table_schema(&column_defs, &constraints, None)?, - }; - - let context: OptimizerContextRef = - OptimizerContext::new(handler_args, explain_options).into(); - let (plan, table) = gen_create_table_plan_for_cdc_table( - context, - source, - cdc_table.external_table_name.clone(), - column_defs, - columns, - pk_names, - cdc_with_options, - col_id_gen, - on_conflict, - with_version_column, - include_column_options, - table_name, - resolved_table_name, - database_id, - schema_id, - TableId::placeholder(), - )?; + let (columns, pk_names) = match wildcard_idx { + Some(_) => bind_cdc_table_schema_externally(cdc_with_options.clone()).await?, + None => { + for column_def in &column_defs { + for option_def in &column_def.options { + if let ColumnOption::DefaultColumns(_) = option_def.option { + return Err(ErrorCode::NotSupported( + "Default value for columns defined on the table created from a CDC source".into(), + "Remove the default value expression in the column definitions".into(), + ) + .into()); + } + } + } - ((plan, None, table), TableJobType::SharedCdcSource) - } - (Some(_), Some(_)) => return Err(ErrorCode::NotSupported( + let (mut columns, pk_names) = + bind_cdc_table_schema(&column_defs, &constraints, None)?; + // read default value definition from external db + let (options, secret_refs) = cdc_with_options.clone().into_parts(); + let config = ExternalTableConfig::try_from_btreemap(options, secret_refs) + .context("failed to extract external table config")?; + + let table = ExternalTableImpl::connect(config) + .await + .context("failed to auto derive table schema")?; + let external_columns: Vec<_> = table + .column_descs() + .iter() + .cloned() + .map(|column_desc| ColumnCatalog { + column_desc, + is_hidden: false, + }) + .collect(); + for (col, external_col) in + columns.iter_mut().zip_eq_fast(external_columns.into_iter()) + { + col.column_desc.generated_or_default_column = + external_col.column_desc.generated_or_default_column; + } + (columns, pk_names) + } + }; + + let context: OptimizerContextRef = + OptimizerContext::new(handler_args, explain_options).into(); + let (plan, table) = gen_create_table_plan_for_cdc_table( + context, + source, + cdc_table.external_table_name.clone(), + column_defs, + columns, + pk_names, + cdc_with_options, + col_id_gen, + on_conflict, + with_version_column, + include_column_options, + table_name, + resolved_table_name, + database_id, + schema_id, + TableId::placeholder(), + )?; + + ((plan, None, table), TableJobType::SharedCdcSource) + } + (Some(_), Some(_)) => { + return Err(ErrorCode::NotSupported( "Data format and encoding format doesn't apply to table created from a CDC source" .into(), "Remove the FORMAT and ENCODE specification".into(), ) - .into()), - }; + .into()) + } + }; Ok((plan, source, table, job_type)) } From 5d6aaf47710b449cde54081e987ba8d457adb6c6 Mon Sep 17 00:00:00 2001 From: stonepage <40830455+st1page@users.noreply.github.com> Date: Tue, 26 Nov 2024 17:48:37 +0800 Subject: [PATCH 021/163] feat(sink): use mysql dialect for jdbc url start with mariadb (#19523) --- .../src/main/java/com/risingwave/connector/JdbcUtils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java index 2360ac96585dd..c18107c1328f5 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JdbcUtils.java @@ -29,7 +29,7 @@ public abstract class JdbcUtils { static final int SOCKET_TIMEOUT = 300; public static Optional getDialectFactory(String jdbcUrl) { - if (jdbcUrl.startsWith("jdbc:mysql")) { + if (jdbcUrl.startsWith("jdbc:mysql") || jdbcUrl.startsWith("jdbc:mariadb")) { return Optional.of(new MySqlDialectFactory()); } else if (jdbcUrl.startsWith("jdbc:postgresql")) { return Optional.of(new PostgresDialectFactory()); From 507085657ca54769cb83aafe2e5d6278b9a985eb Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Wed, 27 Nov 2024 00:28:10 +0800 Subject: [PATCH 022/163] feat(secret): alter secret in catalog (#19495) --- ci/workflows/pull-request.yml | 2 +- e2e_test/ddl/secret.slt | 20 ++++ e2e_test/sink/iceberg_sink.slt | 5 + .../source_legacy/cdc/cdc.share_stream.slt | 57 +++++++++- proto/ddl_service.proto | 14 +++ src/common/secret/src/secret_manager.rs | 31 +++++- src/compute/src/observer/observer_manager.rs | 3 + src/frontend/src/catalog/catalog_service.rs | 33 ++++++ src/frontend/src/catalog/secret_catalog.rs | 4 +- src/frontend/src/handler/alter_secret.rs | 101 ++++++++++++++++++ src/frontend/src/handler/create_secret.rs | 87 ++++++++------- src/frontend/src/handler/drop_secret.rs | 74 ++++++++----- src/frontend/src/handler/mod.rs | 6 ++ src/frontend/src/observer/observer_manager.rs | 3 + src/frontend/src/test_utils.rs | 12 +++ src/meta/service/src/ddl_service.rs | 21 ++++ src/meta/src/controller/catalog.rs | 39 ++++++- src/meta/src/rpc/ddl_controller.rs | 45 +++++--- src/rpc_client/src/meta_client.rs | 24 +++++ src/sqlparser/src/ast/ddl.rs | 18 +++- src/sqlparser/src/ast/mod.rs | 18 +++- src/sqlparser/src/parser.rs | 17 ++- src/utils/pgwire/src/pg_response.rs | 1 + 23 files changed, 538 insertions(+), 97 deletions(-) create mode 100644 src/frontend/src/handler/alter_secret.rs diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 8b852b6bb0b0a..45502ab3037c0 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -818,7 +818,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 30 + timeout_in_minutes: 32 retry: *auto-retry # FIXME(kwannoel): Let the github PR labeller label it, if sqlsmith source files has changes. diff --git a/e2e_test/ddl/secret.slt b/e2e_test/ddl/secret.slt index 6522565986a18..cf9d16214da38 100644 --- a/e2e_test/ddl/secret.slt +++ b/e2e_test/ddl/secret.slt @@ -40,6 +40,26 @@ create secret secret_1 with ( backend = 'meta' ) as 'demo_secret'; +statement ok +alter secret secret_1 with ( + backend = 'meta' +) as 'demo_secret_altered'; + +statement ok +alter secret secret_1 as 'demo_secret_altered_again'; + +statement error +alter secret secret_2 with ( + backend = 'meta' +) as 'demo_secret_altered'; +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Catalog error + 2: secret not found: secret_2 + + # wait for support for hashicorp_vault backend # statement ok # create secret secret_2 with ( diff --git a/e2e_test/sink/iceberg_sink.slt b/e2e_test/sink/iceberg_sink.slt index b08abd8a4918c..ced999e546d14 100644 --- a/e2e_test/sink/iceberg_sink.slt +++ b/e2e_test/sink/iceberg_sink.slt @@ -10,6 +10,11 @@ CREATE MATERIALIZED VIEW mv6 AS SELECT * FROM t6; statement ok CREATE SECRET iceberg_s3_access_key WITH ( backend = 'meta' +) as 'hummockadmin_wrong'; + +statement ok +ALTER SECRET iceberg_s3_access_key WITH ( + backend = 'meta' ) as 'hummockadmin'; statement ok diff --git a/e2e_test/source_legacy/cdc/cdc.share_stream.slt b/e2e_test/source_legacy/cdc/cdc.share_stream.slt index 60b5f7cd52893..db301c3afc6e5 100644 --- a/e2e_test/source_legacy/cdc/cdc.share_stream.slt +++ b/e2e_test/source_legacy/cdc/cdc.share_stream.slt @@ -22,7 +22,31 @@ mysql --protocol=tcp -u root mytest < e2e_test/source_legacy/cdc/mysql_init_data statement ok create secret mysql_pwd with ( backend = 'meta' -) as '${MYSQL_PWD:}'; +) as 'incorrect_password'; + +# create a cdc source job, with incorrct password +statement error +create source mysql_mytest with ( + connector = 'mysql-cdc', + hostname = '${MYSQL_HOST:localhost}', + port = '${MYSQL_TCP_PORT:8306}', + username = 'rwcdc', + password = secret mysql_pwd, + database.name = 'mytest', + server.id = '5601' +); +# The detailed error message is commented out because the user IP in error message may vary in different environments. +# ---- +# db error: ERROR: Failed to run the query +# Caused by these errors (recent errors listed first): +# 1: gRPC request to meta service failed: Internal error +# 2: failed to create source worker +# 3: failed to create SplitEnumerator +# 4: source cannot pass validation +# 5: Internal error: Access denied for user 'rwcdc'@'172.17.0.1' (using password: YES) + +statement ok +alter secret mysql_pwd as '${MYSQL_PWD:}'; # create a cdc source job, which format fixed to `FORMAT PLAIN ENCODE JSON` statement ok @@ -36,7 +60,6 @@ create source mysql_mytest with ( server.id = '5601' ); - statement error Should not create MATERIALIZED VIEW or SELECT directly on shared CDC source create materialized view mv as select * from mysql_mytest; @@ -62,6 +85,22 @@ from mysql_mytest table 'mytest.products'; # sleep to ensure (default,'Milk','Milk is a white liquid food') is consumed from Debezium message instead of backfill. sleep 10s +### BEGIN test change secret in MySQL and ALTER SECRET in Risingwave +onlyif can-use-recover +statement ok +alter secret mysql_pwd with ( + backend = 'meta' +) as 'new_password'; + +onlyif can-use-recover +system ok +mysql --protocol=tcp -u root -e "ALTER USER 'rwcdc'@'%' IDENTIFIED BY 'new_password';" + +onlyif can-use-recover +statement ok +recover; +### END + statement error Permission denied drop secret mysql_pwd; @@ -621,3 +660,17 @@ query II select * from test_pg_default_value; ---- 1 noris Shanghai + +### BEGIN reset the password to the original one +onlyif can-use-recover +statement ok +alter secret mysql_pwd as '${MYSQL_PWD:}'; + +onlyif can-use-recover +system ok +mysql --protocol=tcp -u root -e "ALTER USER 'rwcdc'@'%' IDENTIFIED BY '${MYSQL_PWD:}';" + +onlyif can-use-recover +statement ok +recover; +### END \ No newline at end of file diff --git a/proto/ddl_service.proto b/proto/ddl_service.proto index 59d8b2f999371..8daa21a3085ad 100644 --- a/proto/ddl_service.proto +++ b/proto/ddl_service.proto @@ -419,6 +419,19 @@ message DropSecretResponse { WaitVersion version = 1; } +message AlterSecretRequest { + uint32 secret_id = 1; + string name = 2; + bytes value = 3; + uint32 database_id = 4; + uint32 schema_id = 5; + uint32 owner_id = 6; +} + +message AlterSecretResponse { + WaitVersion version = 1; +} + message CreateConnectionRequest { message PrivateLink { catalog.Connection.PrivateLinkService.PrivateLinkProvider provider = 1; @@ -515,6 +528,7 @@ service DdlService { rpc CreateTable(CreateTableRequest) returns (CreateTableResponse); rpc CreateSecret(CreateSecretRequest) returns (CreateSecretResponse); rpc DropSecret(DropSecretRequest) returns (DropSecretResponse); + rpc AlterSecret(AlterSecretRequest) returns (AlterSecretResponse); rpc AlterName(AlterNameRequest) returns (AlterNameResponse); rpc AlterSource(AlterSourceRequest) returns (AlterSourceResponse); rpc AlterOwner(AlterOwnerRequest) returns (AlterOwnerResponse); diff --git a/src/common/secret/src/secret_manager.rs b/src/common/secret/src/secret_manager.rs index b6a71a4c3ebe8..723c0f5791d4d 100644 --- a/src/common/secret/src/secret_manager.rs +++ b/src/common/secret/src/secret_manager.rs @@ -74,7 +74,23 @@ impl LocalSecretManager { pub fn add_secret(&self, secret_id: SecretId, secret: Vec) { let mut secret_guard = self.secrets.write(); - secret_guard.insert(secret_id, secret); + if secret_guard.insert(secret_id, secret).is_some() { + tracing::error!( + secret_id = secret_id, + "adding a secret but it already exists, overwriting it" + ); + }; + } + + pub fn update_secret(&self, secret_id: SecretId, secret: Vec) { + let mut secret_guard = self.secrets.write(); + if secret_guard.insert(secret_id, secret).is_none() { + tracing::error!( + secret_id = secret_id, + "updating a secret but it does not exist, adding it" + ); + } + self.remove_secret_file_if_exist(&secret_id); } pub fn init_secrets(&self, secrets: Vec) { @@ -174,9 +190,7 @@ impl LocalSecretManager { } fn get_secret_value(pb_secret_bytes: &[u8]) -> SecretResult> { - let pb_secret = risingwave_pb::secret::Secret::decode(pb_secret_bytes) - .context("failed to decode secret")?; - let secret_value = match pb_secret.get_secret_backend().unwrap() { + let secret_value = match Self::get_pb_secret_backend(pb_secret_bytes)? { risingwave_pb::secret::secret::SecretBackend::Meta(backend) => backend.value.clone(), risingwave_pb::secret::secret::SecretBackend::HashicorpVault(_) => { return Err(anyhow!("hashicorp_vault backend is not implemented yet").into()) @@ -184,4 +198,13 @@ impl LocalSecretManager { }; Ok(secret_value) } + + /// Get the secret backend from the given decrypted secret bytes. + pub fn get_pb_secret_backend( + pb_secret_bytes: &[u8], + ) -> SecretResult { + let pb_secret = risingwave_pb::secret::Secret::decode(pb_secret_bytes) + .context("failed to decode secret")?; + Ok(pb_secret.get_secret_backend().unwrap().clone()) + } } diff --git a/src/compute/src/observer/observer_manager.rs b/src/compute/src/observer/observer_manager.rs index c028c1e851613..3f3a57c363930 100644 --- a/src/compute/src/observer/observer_manager.rs +++ b/src/compute/src/observer/observer_manager.rs @@ -38,6 +38,9 @@ impl ObserverState for ComputeObserverNode { Operation::Delete => { LocalSecretManager::global().remove_secret(s.id); } + Operation::Update => { + LocalSecretManager::global().update_secret(s.id, s.value); + } _ => { panic!("error type notification"); } diff --git a/src/frontend/src/catalog/catalog_service.rs b/src/frontend/src/catalog/catalog_service.rs index cc7cac86570c5..f7dcb919a6ad7 100644 --- a/src/frontend/src/catalog/catalog_service.rs +++ b/src/frontend/src/catalog/catalog_service.rs @@ -178,6 +178,16 @@ pub trait CatalogWriter: Send + Sync { async fn drop_secret(&self, secret_id: SecretId) -> Result<()>; + async fn alter_secret( + &self, + secret_id: u32, + secret_name: String, + database_id: u32, + schema_id: u32, + owner_id: u32, + payload: Vec, + ) -> Result<()>; + async fn alter_name( &self, object_id: alter_name_request::Object, @@ -511,6 +521,29 @@ impl CatalogWriter for CatalogWriterImpl { let version = self.meta_client.alter_swap_rename(object).await?; self.wait_version(version).await } + + async fn alter_secret( + &self, + secret_id: u32, + secret_name: String, + database_id: u32, + schema_id: u32, + owner_id: u32, + payload: Vec, + ) -> Result<()> { + let version = self + .meta_client + .alter_secret( + secret_id, + secret_name, + database_id, + schema_id, + owner_id, + payload, + ) + .await?; + self.wait_version(version).await + } } impl CatalogWriterImpl { diff --git a/src/frontend/src/catalog/secret_catalog.rs b/src/frontend/src/catalog/secret_catalog.rs index d1f9048baf0e7..db50fbc201bdb 100644 --- a/src/frontend/src/catalog/secret_catalog.rs +++ b/src/frontend/src/catalog/secret_catalog.rs @@ -14,7 +14,7 @@ use risingwave_pb::catalog::PbSecret; -use crate::catalog::{DatabaseId, OwnedByUserCatalog, SecretId}; +use crate::catalog::{DatabaseId, OwnedByUserCatalog, SchemaId, SecretId}; use crate::user::UserId; #[derive(Clone, Debug, PartialEq, Eq, Hash)] @@ -22,6 +22,7 @@ pub struct SecretCatalog { pub id: SecretId, pub name: String, pub database_id: DatabaseId, + pub schema_id: SchemaId, pub value: Vec, pub owner: UserId, } @@ -34,6 +35,7 @@ impl From<&PbSecret> for SecretCatalog { owner: value.owner, name: value.name.clone(), value: value.value.clone(), + schema_id: value.schema_id, } } } diff --git a/src/frontend/src/handler/alter_secret.rs b/src/frontend/src/handler/alter_secret.rs new file mode 100644 index 0000000000000..3ab5ece2ee518 --- /dev/null +++ b/src/frontend/src/handler/alter_secret.rs @@ -0,0 +1,101 @@ +// Copyright 2024 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 pgwire::pg_response::StatementType; +use prost::Message; +use risingwave_common::bail_not_implemented; +use risingwave_common::license::Feature; +use risingwave_common::secret::LocalSecretManager; +use risingwave_pb::secret::secret; +use risingwave_sqlparser::ast::{AlterSecretOperation, ObjectName, SqlOption}; + +use super::create_secret::{get_secret_payload, secret_to_str}; +use super::drop_secret::fetch_secret_catalog_with_db_schema_id; +use crate::error::Result; +use crate::handler::{HandlerArgs, RwPgResponse}; +use crate::WithOptions; + +pub async fn handle_alter_secret( + handler_args: HandlerArgs, + secret_name: ObjectName, + sql_options: Vec, + operation: AlterSecretOperation, +) -> Result { + Feature::SecretManagement + .check_available() + .map_err(|e| anyhow::anyhow!(e))?; + + let session = handler_args.session; + + if let Some((secret_catalog, _, _)) = + fetch_secret_catalog_with_db_schema_id(&session, &secret_name, false)? + { + let AlterSecretOperation::ChangeCredential { new_credential } = operation; + + let secret_id = secret_catalog.id.secret_id(); + let secret_payload = if sql_options.is_empty() { + let original_pb_secret_bytes = LocalSecretManager::global() + .get_secret(secret_id) + .ok_or(anyhow!( + "Failed to get secret in secret manager, secret_id: {}", + secret_id + ))?; + let original_secret_backend = + LocalSecretManager::get_pb_secret_backend(&original_pb_secret_bytes)?; + match original_secret_backend { + secret::SecretBackend::Meta(_) => { + let new_secret_value_bytes = + secret_to_str(&new_credential)?.as_bytes().to_vec(); + let secret_payload = risingwave_pb::secret::Secret { + secret_backend: Some(risingwave_pb::secret::secret::SecretBackend::Meta( + risingwave_pb::secret::SecretMetaBackend { + value: new_secret_value_bytes, + }, + )), + }; + secret_payload.encode_to_vec() + } + secret::SecretBackend::HashicorpVault(_) => { + bail_not_implemented!("hashicorp_vault backend is not implemented yet") + } + } + } else { + let with_options = WithOptions::try_from(sql_options.as_ref() as &[SqlOption])?; + get_secret_payload(new_credential, with_options)? + }; + + let catalog_writer = session.catalog_writer()?; + + catalog_writer + .alter_secret( + secret_id, + secret_catalog.name.clone(), + secret_catalog.database_id, + secret_catalog.schema_id, + secret_catalog.owner, + secret_payload, + ) + .await?; + + Ok(RwPgResponse::empty_result(StatementType::ALTER_SECRET)) + } else { + Ok(RwPgResponse::builder(StatementType::ALTER_SECRET) + .notice(format!( + "secret \"{}\" does not exist, skipping", + secret_name + )) + .into()) + } +} diff --git a/src/frontend/src/handler/create_secret.rs b/src/frontend/src/handler/create_secret.rs index 9810751361be3..6d5a6283e3288 100644 --- a/src/frontend/src/handler/create_secret.rs +++ b/src/frontend/src/handler/create_secret.rs @@ -37,60 +37,21 @@ pub async fn handle_create_secret( let session = handler_args.session.clone(); let db_name = session.database(); - let (schema_name, connection_name) = + let (schema_name, secret_name) = Binder::resolve_schema_qualified_name(db_name, stmt.secret_name.clone())?; if let Err(e) = session.check_secret_name_duplicated(stmt.secret_name.clone()) { return if stmt.if_not_exists { Ok(PgResponse::builder(StatementType::CREATE_SECRET) - .notice(format!("secret \"{}\" exists, skipping", connection_name)) + .notice(format!("secret \"{}\" exists, skipping", secret_name)) .into()) } else { Err(e) }; } + let with_options = WithOptions::try_from(stmt.with_properties.0.as_ref() as &[SqlOption])?; - let secret = secret_to_str(&stmt.credential)?.as_bytes().to_vec(); - - // check if the secret backend is supported - let with_props = WithOptions::try_from(stmt.with_properties.0.as_ref() as &[SqlOption])?; - let secret_payload: Vec = { - if let Some(backend) = with_props.get(SECRET_BACKEND_KEY) { - match backend.to_lowercase().as_ref() { - SECRET_BACKEND_META => { - let backend = risingwave_pb::secret::Secret { - secret_backend: Some(risingwave_pb::secret::secret::SecretBackend::Meta( - risingwave_pb::secret::SecretMetaBackend { value: secret }, - )), - }; - backend.encode_to_vec() - } - SECRET_BACKEND_HASHICORP_VAULT => { - if stmt.credential != Value::Null { - return Err(ErrorCode::InvalidParameterValue( - "credential must be null for hashicorp_vault backend".to_string(), - ) - .into()); - } - bail_not_implemented!("hashicorp_vault backend is not implemented yet") - } - _ => { - return Err(ErrorCode::InvalidParameterValue(format!( - "secret backend \"{}\" is not supported. Supported backends are: {}", - backend, - [SECRET_BACKEND_META, SECRET_BACKEND_HASHICORP_VAULT].join(",") - )) - .into()); - } - } - } else { - return Err(ErrorCode::InvalidParameterValue(format!( - "secret backend is not specified in with clause. Supported backends are: {}", - [SECRET_BACKEND_META, SECRET_BACKEND_HASHICORP_VAULT].join(",") - )) - .into()); - } - }; + let secret_payload = get_secret_payload(stmt.credential, with_options)?; let (database_id, schema_id) = session.get_database_and_schema_id_for_create(schema_name)?; @@ -108,7 +69,7 @@ pub async fn handle_create_secret( Ok(PgResponse::empty_result(StatementType::CREATE_SECRET)) } -fn secret_to_str(value: &Value) -> Result { +pub fn secret_to_str(value: &Value) -> Result { match value { Value::DoubleQuotedString(s) | Value::SingleQuotedString(s) => Ok(s.to_string()), _ => Err(ErrorCode::InvalidInputSyntax( @@ -117,3 +78,41 @@ fn secret_to_str(value: &Value) -> Result { .into()), } } + +pub(crate) fn get_secret_payload(credential: Value, with_options: WithOptions) -> Result> { + let secret = secret_to_str(&credential)?.as_bytes().to_vec(); + + if let Some(backend) = with_options.get(SECRET_BACKEND_KEY) { + match backend.to_lowercase().as_ref() { + SECRET_BACKEND_META => { + let backend = risingwave_pb::secret::Secret { + secret_backend: Some(risingwave_pb::secret::secret::SecretBackend::Meta( + risingwave_pb::secret::SecretMetaBackend { value: secret }, + )), + }; + Ok(backend.encode_to_vec()) + } + SECRET_BACKEND_HASHICORP_VAULT => { + if credential != Value::Null { + return Err(ErrorCode::InvalidParameterValue( + "credential must be null for hashicorp_vault backend".to_string(), + ) + .into()); + } + bail_not_implemented!("hashicorp_vault backend is not implemented yet") + } + _ => Err(ErrorCode::InvalidParameterValue(format!( + "secret backend \"{}\" is not supported. Supported backends are: {}", + backend, + [SECRET_BACKEND_META, SECRET_BACKEND_HASHICORP_VAULT].join(",") + )) + .into()), + } + } else { + Err(ErrorCode::InvalidParameterValue(format!( + "secret backend is not specified in with clause. Supported backends are: {}", + [SECRET_BACKEND_META, SECRET_BACKEND_HASHICORP_VAULT].join(",") + )) + .into()) + } +} diff --git a/src/frontend/src/handler/drop_secret.rs b/src/frontend/src/handler/drop_secret.rs index eff4b35224b8b..4720d73bfa7e6 100644 --- a/src/frontend/src/handler/drop_secret.rs +++ b/src/frontend/src/handler/drop_secret.rs @@ -12,13 +12,18 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::sync::Arc; + use pgwire::pg_response::StatementType; use risingwave_common::license::Feature; use risingwave_sqlparser::ast::ObjectName; use crate::catalog::root_catalog::SchemaPath; +use crate::catalog::secret_catalog::SecretCatalog; +use crate::catalog::{DatabaseId, SchemaId}; use crate::error::Result; use crate::handler::{HandlerArgs, RwPgResponse}; +use crate::session::SessionImpl; use crate::Binder; pub async fn handle_drop_secret( @@ -31,37 +36,54 @@ pub async fn handle_drop_secret( .map_err(|e| anyhow::anyhow!(e))?; let session = handler_args.session; + + if let Some((secret_catalog, _, _)) = + fetch_secret_catalog_with_db_schema_id(&session, &secret_name, if_exists)? + { + let catalog_writer = session.catalog_writer()?; + catalog_writer.drop_secret(secret_catalog.id).await?; + + Ok(RwPgResponse::empty_result(StatementType::DROP_SECRET)) + } else { + Ok(RwPgResponse::builder(StatementType::DROP_SECRET) + .notice(format!( + "secret \"{}\" does not exist, skipping", + secret_name + )) + .into()) + } +} + +/// Fetch the secret catalog and the `database/schema_id` of the source. +pub fn fetch_secret_catalog_with_db_schema_id( + session: &SessionImpl, + secret_name: &ObjectName, + if_exists: bool, +) -> Result, DatabaseId, SchemaId)>> { let db_name = session.database(); - let (schema_name, secret_name) = Binder::resolve_schema_qualified_name(db_name, secret_name)?; + let (schema_name, secret_name) = + Binder::resolve_schema_qualified_name(db_name, secret_name.clone())?; let search_path = session.config().search_path(); let user_name = &session.auth_context().user_name; - let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name); - let secret_id = { - let reader = session.env().catalog_reader().read_guard(); - let (secret, schema_name) = - match reader.get_secret_by_name(db_name, schema_path, secret_name.as_str()) { - Ok((c, s)) => (c, s), - Err(e) => { - return if if_exists { - Ok(RwPgResponse::builder(StatementType::DROP_SECRET) - .notice(format!( - "secret \"{}\" does not exist, skipping", - secret_name - )) - .into()) - } else { - Err(e.into()) - }; - } - }; - session.check_privilege_for_drop_alter(schema_name, &**secret)?; + let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name); - secret.id - }; + let reader = session.env().catalog_reader().read_guard(); + match reader.get_secret_by_name(db_name, schema_path, &secret_name) { + Ok((catalog, schema_name)) => { + session.check_privilege_for_drop_alter(schema_name, &**catalog)?; - let catalog_writer = session.catalog_writer()?; - catalog_writer.drop_secret(secret_id).await?; + let db = reader.get_database_by_name(db_name)?; + let schema = db.get_schema_by_name(schema_name).unwrap(); - Ok(RwPgResponse::empty_result(StatementType::DROP_SECRET)) + Ok(Some((Arc::clone(catalog), db.id(), schema.id()))) + } + Err(e) => { + if if_exists { + Ok(None) + } else { + Err(e.into()) + } + } + } } diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index 9cf94a37c65b0..cfa6dc10277e8 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -42,6 +42,7 @@ use crate::utils::WithOptions; mod alter_owner; mod alter_parallelism; mod alter_rename; +mod alter_secret; mod alter_set_schema; mod alter_source_column; mod alter_source_with_sr; @@ -1076,6 +1077,11 @@ pub async fn handle( Statement::AlterSystem { param, value } => { alter_system::handle_alter_system(handler_args, param, value).await } + Statement::AlterSecret { + name, + with_options, + operation, + } => alter_secret::handle_alter_secret(handler_args, name, with_options, operation).await, Statement::StartTransaction { modes } => { transaction::handle_begin(handler_args, START_TRANSACTION, modes).await } diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index 92408c2b03885..270c467b04754 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -481,6 +481,9 @@ impl FrontendObserverNode { Operation::Delete => { LocalSecretManager::global().remove_secret(secret.id); } + Operation::Update => { + LocalSecretManager::global().update_secret(secret.id, secret.value); + } _ => { panic!("error type notification"); } diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index 810724d2bac22..77a9bc23d5a53 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -654,6 +654,18 @@ impl CatalogWriter for MockCatalogWriter { async fn alter_swap_rename(&self, _object: alter_swap_rename_request::Object) -> Result<()> { todo!() } + + async fn alter_secret( + &self, + _secret_id: u32, + _secret_name: String, + _database_id: u32, + _schema_id: u32, + _owner_id: u32, + _payload: Vec, + ) -> Result<()> { + unreachable!() + } } impl MockCatalogWriter { diff --git a/src/meta/service/src/ddl_service.rs b/src/meta/service/src/ddl_service.rs index 3556890db04cc..553aaeb986e78 100644 --- a/src/meta/service/src/ddl_service.rs +++ b/src/meta/service/src/ddl_service.rs @@ -181,6 +181,27 @@ impl DdlService for DdlServiceImpl { Ok(Response::new(DropSecretResponse { version })) } + async fn alter_secret( + &self, + request: Request, + ) -> Result, Status> { + let req = request.into_inner(); + let pb_secret = Secret { + id: req.get_secret_id(), + name: req.get_name().clone(), + database_id: req.get_database_id(), + value: req.get_value().clone(), + owner: req.get_owner_id(), + schema_id: req.get_schema_id(), + }; + let version = self + .ddl_controller + .run_command(DdlCommand::AlterSecret(pb_secret)) + .await?; + + Ok(Response::new(AlterSecretResponse { version })) + } + async fn create_schema( &self, request: Request, diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index f3dac5c39e2fb..5950f0357eee5 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -1397,6 +1397,43 @@ impl CatalogController { Ok(version) } + pub async fn alter_secret( + &self, + pb_secret: PbSecret, + secret_plain_payload: Vec, + ) -> MetaResult { + let inner = self.inner.write().await; + let owner_id = pb_secret.owner as _; + let txn = inner.db.begin().await?; + ensure_user_id(owner_id, &txn).await?; + ensure_object_id(ObjectType::Database, pb_secret.database_id as _, &txn).await?; + ensure_object_id(ObjectType::Schema, pb_secret.schema_id as _, &txn).await?; + + ensure_object_id(ObjectType::Secret, pb_secret.id as _, &txn).await?; + let secret: secret::ActiveModel = pb_secret.clone().into(); + Secret::update(secret).exec(&txn).await?; + + txn.commit().await?; + + // Notify the compute and frontend node plain secret + let mut secret_plain = pb_secret; + secret_plain.value.clone_from(&secret_plain_payload); + + LocalSecretManager::global().update_secret(secret_plain.id, secret_plain_payload); + self.env + .notification_manager() + .notify_compute_without_version(Operation::Update, Info::Secret(secret_plain.clone())); + + let version = self + .notify_frontend( + NotificationOperation::Update, + NotificationInfo::Secret(secret_plain), + ) + .await; + + Ok(version) + } + pub async fn get_secret_by_id(&self, secret_id: SecretId) -> MetaResult { let inner = self.inner.read().await; let (secret, obj) = Secret::find_by_id(secret_id) @@ -1417,7 +1454,7 @@ impl CatalogController { .ok_or_else(|| MetaError::catalog_id_not_found("secret", secret_id))?; ensure_object_not_refer(ObjectType::Secret, secret_id, &txn).await?; - // Find affect users with privileges on the connection. + // Find affect users with privileges on the secret. let to_update_user_ids: Vec = UserPrivilege::find() .select_only() .distinct() diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index bb14ecefc1298..ffb46ea03a6ac 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -151,6 +151,7 @@ pub enum DdlCommand { CreateConnection(Connection), DropConnection(ConnectionId), CreateSecret(Secret), + AlterSecret(Secret), DropSecret(SecretId), CommentOn(Comment), CreateSubscription(Subscription), @@ -179,6 +180,7 @@ impl DdlCommand { | DdlCommand::CreateConnection(_) | DdlCommand::CommentOn(_) | DdlCommand::CreateSecret(_) + | DdlCommand::AlterSecret(_) | DdlCommand::AlterSwapRename(_) => true, DdlCommand::CreateStreamingJob(_, _, _, _, _) | DdlCommand::CreateSourceWithoutStreamingJob(_) @@ -350,6 +352,7 @@ impl DdlController { } DdlCommand::CreateSecret(secret) => ctrl.create_secret(secret).await, DdlCommand::DropSecret(secret_id) => ctrl.drop_secret(secret_id).await, + DdlCommand::AlterSecret(secret) => ctrl.alter_secret(secret).await, DdlCommand::AlterSourceColumn(source) => ctrl.alter_source(source).await, DdlCommand::CommentOn(comment) => ctrl.comment_on(comment).await, DdlCommand::CreateSubscription(subscription) => { @@ -520,10 +523,9 @@ impl DdlController { .await } - async fn create_secret(&self, mut secret: Secret) -> MetaResult { - // The 'secret' part of the request we receive from the frontend is in plaintext; - // here, we need to encrypt it before storing it in the catalog. - let secret_plain_payload = secret.value.clone(); + // The 'secret' part of the request we receive from the frontend is in plaintext; + // here, we need to encrypt it before storing it in the catalog. + fn get_encrypted_payload(&self, secret: &Secret) -> MetaResult> { let secret_store_private_key = self .env .opts @@ -531,16 +533,21 @@ impl DdlController { .clone() .ok_or_else(|| anyhow!("secret_store_private_key is not configured"))?; - let encrypted_payload = { - let encrypted_secret = SecretEncryption::encrypt( - secret_store_private_key.as_slice(), - secret.get_value().as_slice(), - ) - .context(format!("failed to encrypt secret {}", secret.name))?; - encrypted_secret - .serialize() - .context(format!("failed to serialize secret {}", secret.name))? - }; + let encrypted_payload = SecretEncryption::encrypt( + secret_store_private_key.as_slice(), + secret.get_value().as_slice(), + ) + .context(format!("failed to encrypt secret {}", secret.name))?; + Ok(encrypted_payload + .serialize() + .context(format!("failed to serialize secret {}", secret.name))?) + } + + async fn create_secret(&self, mut secret: Secret) -> MetaResult { + // The 'secret' part of the request we receive from the frontend is in plaintext; + // here, we need to encrypt it before storing it in the catalog. + let secret_plain_payload = secret.value.clone(); + let encrypted_payload = self.get_encrypted_payload(&secret)?; secret.value = encrypted_payload; self.metadata_manager @@ -556,6 +563,16 @@ impl DdlController { .await } + async fn alter_secret(&self, mut secret: Secret) -> MetaResult { + let secret_plain_payload = secret.value.clone(); + let encrypted_payload = self.get_encrypted_payload(&secret)?; + secret.value = encrypted_payload; + self.metadata_manager + .catalog_controller + .alter_secret(secret, secret_plain_payload) + .await + } + async fn create_subscription( &self, mut subscription: Subscription, diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 8f4e6779b8e56..6efac3f414564 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -590,6 +590,29 @@ impl MetaClient { .ok_or_else(|| anyhow!("wait version not set"))?) } + pub async fn alter_secret( + &self, + secret_id: u32, + secret_name: String, + database_id: u32, + schema_id: u32, + owner_id: u32, + value: Vec, + ) -> Result { + let request = AlterSecretRequest { + secret_id, + name: secret_name, + database_id, + schema_id, + owner_id, + value, + }; + let resp = self.inner.alter_secret(request).await?; + Ok(resp + .version + .ok_or_else(|| anyhow!("wait version not set"))?) + } + pub async fn replace_table( &self, source: Option, @@ -2095,6 +2118,7 @@ macro_rules! for_all_meta_rpc { ,{ ddl_client, wait, WaitRequest, WaitResponse } ,{ ddl_client, auto_schema_change, AutoSchemaChangeRequest, AutoSchemaChangeResponse } ,{ ddl_client, alter_swap_rename, AlterSwapRenameRequest, AlterSwapRenameResponse } + ,{ ddl_client, alter_secret, AlterSecretRequest, AlterSecretResponse } ,{ hummock_client, unpin_version_before, UnpinVersionBeforeRequest, UnpinVersionBeforeResponse } ,{ hummock_client, get_current_version, GetCurrentVersionRequest, GetCurrentVersionResponse } ,{ hummock_client, replay_version_delta, ReplayVersionDeltaRequest, ReplayVersionDeltaResponse } diff --git a/src/sqlparser/src/ast/ddl.rs b/src/sqlparser/src/ast/ddl.rs index d94cf80cb9f2d..35eac31cefa72 100644 --- a/src/sqlparser/src/ast/ddl.rs +++ b/src/sqlparser/src/ast/ddl.rs @@ -20,7 +20,7 @@ use core::fmt; #[cfg(feature = "serde")] use serde::{Deserialize, Serialize}; -use super::FormatEncodeOptions; +use super::{FormatEncodeOptions, Value}; use crate::ast::{ display_comma_separated, display_separated, DataType, Expr, Ident, ObjectName, SetVariableValue, }; @@ -214,6 +214,12 @@ pub enum AlterConnectionOperation { SetSchema { new_schema_name: ObjectName }, } +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub enum AlterSecretOperation { + ChangeCredential { new_credential: Value }, +} + impl fmt::Display for AlterDatabaseOperation { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { match self { @@ -481,6 +487,16 @@ impl fmt::Display for AlterConnectionOperation { } } +impl fmt::Display for AlterSecretOperation { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + match self { + AlterSecretOperation::ChangeCredential { new_credential } => { + write!(f, "AS {new_credential}") + } + } + } +} + /// An `ALTER COLUMN` (`Statement::AlterTable`) operation #[derive(Debug, Clone, PartialEq, Eq, Hash)] #[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index be9d4ae489f53..9007613f30cee 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -38,8 +38,8 @@ use winnow::PResult; pub use self::data_type::{DataType, StructField}; pub use self::ddl::{ AlterColumnOperation, AlterConnectionOperation, AlterDatabaseOperation, AlterFunctionOperation, - AlterSchemaOperation, AlterTableOperation, ColumnDef, ColumnOption, ColumnOptionDef, - ReferentialAction, SourceWatermark, TableConstraint, + AlterSchemaOperation, AlterSecretOperation, AlterTableOperation, ColumnDef, ColumnOption, + ColumnOptionDef, ReferentialAction, SourceWatermark, TableConstraint, }; pub use self::legacy_source::{ get_delimiter, AvroSchema, CompatibleFormatEncode, DebeziumAvroSchema, ProtobufSchema, @@ -1438,6 +1438,13 @@ pub enum Statement { name: ObjectName, operation: AlterConnectionOperation, }, + /// ALTER SECRET + AlterSecret { + /// Secret name + name: ObjectName, + with_options: Vec, + operation: AlterSecretOperation, + }, /// DESCRIBE TABLE OR SOURCE Describe { /// Table or Source name @@ -1967,6 +1974,13 @@ impl fmt::Display for Statement { Statement::AlterConnection { name, operation } => { write!(f, "ALTER CONNECTION {} {}", name, operation) } + Statement::AlterSecret { name, with_options, operation } => { + write!(f, "ALTER SECRET {}", name)?; + if !with_options.is_empty() { + write!(f, " WITH ({})", display_comma_separated(with_options))?; + } + write!(f, " {}", operation) + } Statement::Discard(t) => write!(f, "DISCARD {}", t), Statement::Drop(stmt) => write!(f, "DROP {}", stmt), Statement::DropFunction { diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index cf7d8e4394574..81dbc54e84c80 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -3026,9 +3026,11 @@ impl Parser<'_> { self.parse_alter_system() } else if self.parse_keyword(Keyword::SUBSCRIPTION) { self.parse_alter_subscription() + } else if self.parse_keyword(Keyword::SECRET) { + self.parse_alter_secret() } else { self.expected( - "DATABASE, SCHEMA, TABLE, INDEX, MATERIALIZED, VIEW, SINK, SUBSCRIPTION, SOURCE, FUNCTION, USER or SYSTEM after ALTER" + "DATABASE, SCHEMA, TABLE, INDEX, MATERIALIZED, VIEW, SINK, SUBSCRIPTION, SOURCE, FUNCTION, USER, SECRET or SYSTEM after ALTER" ) } } @@ -3541,6 +3543,19 @@ impl Parser<'_> { Ok(Statement::AlterSystem { param, value }) } + pub fn parse_alter_secret(&mut self) -> PResult { + let secret_name = self.parse_object_name()?; + let with_options = self.parse_with_properties()?; + self.expect_keyword(Keyword::AS)?; + let new_credential = self.parse_value()?; + let operation = AlterSecretOperation::ChangeCredential { new_credential }; + Ok(Statement::AlterSecret { + name: secret_name, + with_options, + operation, + }) + } + /// Parse a copy statement pub fn parse_copy(&mut self) -> PResult { let table_name = self.parse_object_name()?; diff --git a/src/utils/pgwire/src/pg_response.rs b/src/utils/pgwire/src/pg_response.rs index 5718533f67097..33e1ec9c5c22b 100644 --- a/src/utils/pgwire/src/pg_response.rs +++ b/src/utils/pgwire/src/pg_response.rs @@ -89,6 +89,7 @@ pub enum StatementType { ALTER_FUNCTION, ALTER_CONNECTION, ALTER_SYSTEM, + ALTER_SECRET, REVOKE_PRIVILEGE, // Introduce ORDER_BY statement type cuz Calcite unvalidated AST has SqlKind.ORDER_BY. Note // that Statement Type is not designed to be one to one mapping with SqlKind. From 21ea2bc4d92714d867d44d5d7c6efb37a6e4c3d4 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Wed, 27 Nov 2024 00:35:04 +0800 Subject: [PATCH 023/163] feat(risedev): support configuring meta store by url from env var (#19560) Signed-off-by: Bugen Zhao --- Cargo.lock | 3 + Makefile.toml | 1 + src/common/src/config.rs | 2 +- src/risedevtool/Cargo.toml | 5 +- src/risedevtool/src/bin/risedev-dev.rs | 12 ++ src/risedevtool/src/service_config.rs | 1 + src/risedevtool/src/task/meta_node_service.rs | 133 +++++++++++++++++- 7 files changed, 153 insertions(+), 4 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index 46ab0da13c40c..ad3a78617ec56 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -10407,6 +10407,7 @@ dependencies = [ "google-cloud-pubsub", "indicatif", "itertools 0.13.0", + "log", "madsim-rdkafka", "madsim-tokio", "redis", @@ -10416,10 +10417,12 @@ dependencies = [ "serde_json", "serde_with 3.8.1", "serde_yaml", + "sqlx", "tempfile", "thiserror-ext", "tracing", "tracing-subscriber", + "url", "workspace-hack", "yaml-rust", ] diff --git a/Makefile.toml b/Makefile.toml index a7c45d363e8fd..f772ba27e8bb1 100644 --- a/Makefile.toml +++ b/Makefile.toml @@ -1282,6 +1282,7 @@ echo If you still feel this is not enough, you may copy $(tput setaf 4)risedev$( category = "RiseDev - CI" dependencies = ["clean-data", "pre-start-dev"] command = "target/debug/risedev-dev" # `risedev-dev` is always built in dev profile +env = { RISEDEV_CLEAN_START = true } args = ["${@}"] description = "Clean data and start a full RisingWave dev cluster using risedev-dev" diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 361f524ce775c..8163cd359ff28 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -173,7 +173,7 @@ serde_with::with_prefix!(batch_prefix "batch_"); pub enum MetaBackend { #[default] Mem, - Sql, // keep for backward compatibility + Sql, // any database url Sqlite, Postgres, Mysql, diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index a3854a2ba2726..2c415d9f5da78 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -26,6 +26,7 @@ glob = "0.3" google-cloud-pubsub = "0.29" indicatif = "0.17" itertools = { workspace = true } +log = "0.4" rdkafka = { workspace = true } redis = "0.25" regex = "1" @@ -34,6 +35,7 @@ serde = { version = "1", features = ["derive"] } serde_json = "1" serde_with = "3" serde_yaml = "0.9" +sqlx = { workspace = true, features = ["any"] } tempfile = "3" thiserror-ext = { workspace = true } tokio = { version = "0.2", package = "madsim-tokio", features = [ @@ -46,7 +48,8 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ "fs", ] } tracing = "0.1" -tracing-subscriber = "0.3" +tracing-subscriber = { version = "0.3", features = ["env-filter"] } +url = "2" workspace-hack = { path = "../workspace-hack" } yaml-rust = "0.4" diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 1420ec9e6bf72..8bf9aa52fe8fc 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -32,6 +32,8 @@ use risedev::{ }; use tempfile::tempdir; use thiserror_ext::AsReport; +use tracing::level_filters::LevelFilter; +use tracing_subscriber::EnvFilter; use yaml_rust::YamlEmitter; #[derive(Default)] @@ -378,6 +380,16 @@ fn main() -> Result<()> { // Backtraces for RisingWave components are enabled in `Task::execute`. std::env::set_var("RUST_BACKTRACE", "0"); + // Init logger from a customized env var. + tracing_subscriber::fmt() + .with_env_filter( + EnvFilter::builder() + .with_default_directive(LevelFilter::WARN.into()) + .with_env_var("RISEDEV_RUST_LOG") + .from_env_lossy(), + ) + .init(); + preflight_check()?; let task_name = std::env::args() diff --git a/src/risedevtool/src/service_config.rs b/src/risedevtool/src/service_config.rs index 9c47828a61b2b..6d8277c3396f8 100644 --- a/src/risedevtool/src/service_config.rs +++ b/src/risedevtool/src/service_config.rs @@ -51,6 +51,7 @@ pub enum MetaBackend { Sqlite, Postgres, Mysql, + Env, } #[derive(Clone, Debug, PartialEq, Serialize, Deserialize)] diff --git a/src/risedevtool/src/task/meta_node_service.rs b/src/risedevtool/src/task/meta_node_service.rs index 3d80efc5a70de..752dd87cd0cda 100644 --- a/src/risedevtool/src/task/meta_node_service.rs +++ b/src/risedevtool/src/task/meta_node_service.rs @@ -15,17 +15,48 @@ use std::env; use std::path::{Path, PathBuf}; use std::process::Command; +use std::sync::LazyLock; -use anyhow::{anyhow, Result}; +use anyhow::{anyhow, bail, Context, Result}; use itertools::Itertools; +use sqlx::{ConnectOptions, Database}; +use tempfile::NamedTempFile; +use url::Url; use super::{risingwave_cmd, ExecuteContext, Task}; -use crate::util::{get_program_args, get_program_env_cmd, get_program_name}; +use crate::util::{get_program_args, get_program_env_cmd, get_program_name, is_env_set}; use crate::{ add_hummock_backend, add_tempo_endpoint, Application, HummockInMemoryStrategy, MetaBackend, MetaNodeConfig, }; +/// URL for connecting to the SQL meta store, retrieved from the env var `RISEDEV_SQL_ENDPOINT`. +/// If it is not set, a temporary sqlite file is created and used. +/// +/// # Examples +/// +/// - `mysql://root:my-secret-pw@127.0.0.1:3306/metastore` +/// - `postgresql://localhost:5432/metastore` +/// - `sqlite:///path/to/file.db` +/// - `sqlite::memory:` +fn sql_endpoint_from_env() -> String { + static SQL_ENDPOINT: LazyLock = LazyLock::new(|| { + if let Ok(endpoint) = env::var("RISEDEV_SQL_ENDPOINT") { + endpoint + } else { + let temp_path = NamedTempFile::with_suffix(".db").unwrap().into_temp_path(); + let temp_sqlite_endpoint = format!("sqlite://{}?mode=rwc", temp_path.to_string_lossy()); + tracing::warn!( + "env RISEDEV_SQL_ENDPOINT not set, use temporary sqlite `{}`", + temp_sqlite_endpoint + ); + temp_sqlite_endpoint + } + }); + + SQL_ENDPOINT.to_owned() +} + pub struct MetaNodeService { config: MetaNodeConfig, } @@ -139,6 +170,15 @@ impl MetaNodeService { .arg("--sql-database") .arg(&mysql_store_config.database); } + MetaBackend::Env => { + let endpoint = sql_endpoint_from_env(); + is_persistent_meta_store = true; + + cmd.arg("--backend") + .arg("sql") + .arg("--sql-endpoint") + .arg(endpoint); + } } let provide_minio = config.provide_minio.as_ref().unwrap(); @@ -245,6 +285,13 @@ impl Task for MetaNodeService { cmd.arg("--config-path") .arg(Path::new(&prefix_config).join("risingwave.toml")); + if let MetaBackend::Env = self.config.meta_backend { + if is_env_set("RISEDEV_CLEAN_START") { + ctx.pb.set_message("initializing meta store from env..."); + initialize_meta_store()?; + } + } + if !self.config.user_managed { ctx.run_command(ctx.tmux_run(cmd)?)?; ctx.pb.set_message("started"); @@ -266,3 +313,85 @@ impl Task for MetaNodeService { self.config.id.clone() } } + +fn initialize_meta_store() -> Result<(), anyhow::Error> { + let rt = tokio::runtime::Builder::new_current_thread() + .enable_all() + .build()?; + + let endpoint: Url = sql_endpoint_from_env() + .parse() + .context("invalid url for SQL endpoint")?; + let scheme = endpoint.scheme(); + + // Retrieve the database name to use for the meta store. + // Modify the URL to establish a temporary connection to initialize that database. + let (db, init_url) = if sqlx::Postgres::URL_SCHEMES.contains(&scheme) { + let options = sqlx::postgres::PgConnectOptions::from_url(&endpoint) + .context("invalid database url for Postgres meta backend")?; + + let db = options + .get_database() + .unwrap_or_else(|| options.get_username()) // PG defaults to username if no database is specified + .to_owned(); + // https://www.postgresql.org/docs/current/manage-ag-templatedbs.html + let init_options = options.database("template1"); + let init_url = init_options.to_url_lossy(); + + (db, init_url) + } else if sqlx::MySql::URL_SCHEMES.contains(&scheme) { + let options = sqlx::mysql::MySqlConnectOptions::from_url(&endpoint) + .context("invalid database url for MySQL meta backend")?; + + let db = options + .get_database() + .context("database not specified for MySQL meta backend")? + .to_owned(); + // Effectively unset the database field when converting back to URL, meaning connect to no database. + let init_options = options.database(""); + let init_url = init_options.to_url_lossy(); + + (db, init_url) + } else if sqlx::Sqlite::URL_SCHEMES.contains(&scheme) { + // For SQLite, simply empty the file. + let options = sqlx::sqlite::SqliteConnectOptions::from_url(&endpoint) + .context("invalid database url for SQLite meta backend")?; + + if endpoint.as_str().contains(":memory:") || endpoint.as_str().contains("mode=memory") { + // SQLite in-memory database does not need initialization. + } else { + let filename = options.get_filename(); + fs_err::write(filename, b"").context("failed to empty SQLite file")?; + } + + return Ok(()); + } else { + bail!("unsupported SQL scheme for meta backend: {}", scheme); + }; + + rt.block_on(async move { + use sqlx::any::*; + install_default_drivers(); + + let options = sqlx::any::AnyConnectOptions::from_url(&init_url)? + .log_statements(log::LevelFilter::Debug); + + let mut conn = options + .connect() + .await + .context("failed to connect to a template database for meta store")?; + + // Intentionally not executing in a transaction because Postgres does not allow it. + sqlx::raw_sql(&format!("DROP DATABASE IF EXISTS {};", db)) + .execute(&mut conn) + .await?; + sqlx::raw_sql(&format!("CREATE DATABASE {};", db)) + .execute(&mut conn) + .await?; + + Ok::<_, anyhow::Error>(()) + }) + .context("failed to initialize database for meta store")?; + + Ok(()) +} From 641cf65c4218d632cf17ac155eb91e65cfb9ff17 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Wed, 27 Nov 2024 01:39:03 +0800 Subject: [PATCH 024/163] fix: add RW_HEAP_PROFILING_DIR for frontend (#19323) --- src/frontend/src/lib.rs | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 6424da42a1510..986d4cfb35660 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -149,6 +149,11 @@ pub struct FrontendOpts { #[override_opts(path = server.metrics_level)] pub metrics_level: Option, + /// Enable heap profile dump when memory usage is high. + #[clap(long, hide = true, env = "RW_HEAP_PROFILING_DIR")] + #[override_opts(path = server.heap_profiling.dir)] + pub heap_profiling_dir: Option, + #[clap(long, hide = true, env = "ENABLE_BARRIER_READ")] #[override_opts(path = batch.enable_barrier_read)] pub enable_barrier_read: Option, From d17b4027ccbcf18304a9b9ee78fb819a514ca80a Mon Sep 17 00:00:00 2001 From: congyi wang <58715567+wcy-fdu@users.noreply.github.com> Date: Wed, 27 Nov 2024 09:41:13 +0800 Subject: [PATCH 025/163] feat(connector): support more parquet data types (#19561) --- e2e_test/s3/file_sink.py | 39 ++++- src/common/src/array/arrow/arrow_impl.rs | 153 ++++++++++++++++++ .../source/iceberg/parquet_file_handler.rs | 77 ++++++++- 3 files changed, 264 insertions(+), 5 deletions(-) diff --git a/e2e_test/s3/file_sink.py b/e2e_test/s3/file_sink.py index 091c24b710b32..a64f40d0692df 100644 --- a/e2e_test/s3/file_sink.py +++ b/e2e_test/s3/file_sink.py @@ -11,6 +11,7 @@ from minio import Minio from random import uniform from time import sleep +import numpy as np import time def gen_data(file_num, item_num_per_file): @@ -23,6 +24,12 @@ def gen_data(file_num, item_num_per_file): 'sex': item_id % 2, 'mark': (-1) ** (item_id % 2), 'test_int': pa.scalar(1, type=pa.int32()), + 'test_int8': pa.scalar(1, type=pa.int8()), + 'test_uint8': pa.scalar(item_id % 256, type=pa.uint8()), # UInt8 + 'test_uint16': pa.scalar(item_id % 65536, type=pa.uint16()), # UInt16 + 'test_uint32': pa.scalar(item_id % (2**32), type=pa.uint32()), # UInt32 + 'test_uint64': pa.scalar(item_id % (2**64), type=pa.uint64()), # UInt64 + 'test_float_16': pa.scalar(np.float16(4.0), type=pa.float16()), 'test_real': pa.scalar(4.0, type=pa.float32()), 'test_double_precision': pa.scalar(5.0, type=pa.float64()), 'test_varchar': pa.scalar('7', type=pa.string()), @@ -62,6 +69,12 @@ def _table(): sex bigint, mark bigint, test_int int, + test_int8 smallint, + test_uint8 smallint, + test_uint16 int, + test_uint32 bigint, + test_uint64 decimal, + test_float_16 real, test_real real, test_double_precision double precision, test_varchar varchar, @@ -137,6 +150,12 @@ def _table(): sex, mark, test_int, + test_int8, + test_uint8, + test_uint16, + test_uint32, + test_uint64, + test_float_16, test_real, test_double_precision, test_varchar, @@ -172,6 +191,12 @@ def _table(): sex bigint, mark bigint, test_int int, + test_int8 smallint, + test_uint8 smallint, + test_uint16 int, + test_uint32 bigint, + test_uint64 decimal, + test_float_16 real, test_real real, test_double_precision double precision, test_varchar varchar, @@ -209,7 +234,7 @@ def _table(): stmt = f'select count(*), sum(id) from test_parquet_sink_table' print(f'Execute reading sink files: {stmt}') - print(f'Create snowflake s3 sink ') + print(f'Create s3 sink json format') # Execute a SELECT statement cur.execute(f'''CREATE sink test_file_sink_json as select id, @@ -217,6 +242,12 @@ def _table(): sex, mark, test_int, + test_int8, + test_uint8, + test_uint16, + test_uint32, + test_uint64, + test_float_16, test_real, test_double_precision, test_varchar, @@ -252,6 +283,12 @@ def _table(): sex bigint, mark bigint, test_int int, + test_int8 smallint, + test_uint8 smallint, + test_uint16 int, + test_uint32 bigint, + test_uint64 decimal, + test_float_16 real, test_real real, test_double_precision double precision, test_varchar varchar, diff --git a/src/common/src/array/arrow/arrow_impl.rs b/src/common/src/array/arrow/arrow_impl.rs index 2e35a1403781f..f72a1d8739be5 100644 --- a/src/common/src/array/arrow/arrow_impl.rs +++ b/src/common/src/array/arrow/arrow_impl.rs @@ -505,6 +505,12 @@ pub trait FromArrow { Int16 => DataType::Int16, Int32 => DataType::Int32, Int64 => DataType::Int64, + Int8 => DataType::Int16, + UInt8 => DataType::Int16, + UInt16 => DataType::Int32, + UInt32 => DataType::Int64, + UInt64 => DataType::Decimal, + Float16 => DataType::Float32, Float32 => DataType::Float32, Float64 => DataType::Float64, Decimal128(_, _) => DataType::Decimal, @@ -581,11 +587,18 @@ pub trait FromArrow { } match array.data_type() { Boolean => self.from_bool_array(array.as_any().downcast_ref().unwrap()), + Int8 => self.from_int8_array(array.as_any().downcast_ref().unwrap()), Int16 => self.from_int16_array(array.as_any().downcast_ref().unwrap()), Int32 => self.from_int32_array(array.as_any().downcast_ref().unwrap()), Int64 => self.from_int64_array(array.as_any().downcast_ref().unwrap()), + UInt8 => self.from_uint8_array(array.as_any().downcast_ref().unwrap()), + UInt16 => self.from_uint16_array(array.as_any().downcast_ref().unwrap()), + UInt32 => self.from_uint32_array(array.as_any().downcast_ref().unwrap()), + + UInt64 => self.from_uint64_array(array.as_any().downcast_ref().unwrap()), Decimal128(_, _) => self.from_decimal128_array(array.as_any().downcast_ref().unwrap()), Decimal256(_, _) => self.from_int256_array(array.as_any().downcast_ref().unwrap()), + Float16 => self.from_float16_array(array.as_any().downcast_ref().unwrap()), Float32 => self.from_float32_array(array.as_any().downcast_ref().unwrap()), Float64 => self.from_float64_array(array.as_any().downcast_ref().unwrap()), Date32 => self.from_date32_array(array.as_any().downcast_ref().unwrap()), @@ -669,6 +682,22 @@ pub trait FromArrow { Ok(ArrayImpl::Int16(array.into())) } + fn from_int8_array(&self, array: &arrow_array::Int8Array) -> Result { + Ok(ArrayImpl::Int16(array.into())) + } + + fn from_uint8_array(&self, array: &arrow_array::UInt8Array) -> Result { + Ok(ArrayImpl::Int16(array.into())) + } + + fn from_uint16_array(&self, array: &arrow_array::UInt16Array) -> Result { + Ok(ArrayImpl::Int32(array.into())) + } + + fn from_uint32_array(&self, array: &arrow_array::UInt32Array) -> Result { + Ok(ArrayImpl::Int64(array.into())) + } + fn from_int32_array(&self, array: &arrow_array::Int32Array) -> Result { Ok(ArrayImpl::Int32(array.into())) } @@ -691,6 +720,17 @@ pub trait FromArrow { Ok(ArrayImpl::Decimal(array.try_into()?)) } + fn from_uint64_array(&self, array: &arrow_array::UInt64Array) -> Result { + Ok(ArrayImpl::Decimal(array.try_into()?)) + } + + fn from_float16_array( + &self, + array: &arrow_array::Float16Array, + ) -> Result { + Ok(ArrayImpl::Float32(array.try_into()?)) + } + fn from_float32_array( &self, array: &arrow_array::Float32Array, @@ -908,6 +948,37 @@ macro_rules! converts { }; } +/// Used to convert different types. +macro_rules! converts_with_type { + ($ArrayType:ty, $ArrowType:ty, $FromType:ty, $ToType:ty) => { + impl From<&$ArrayType> for $ArrowType { + fn from(array: &$ArrayType) -> Self { + let values: Vec> = + array.iter().map(|x| x.map(|v| v as $ToType)).collect(); + <$ArrowType>::from_iter(values) + } + } + + impl From<&$ArrowType> for $ArrayType { + fn from(array: &$ArrowType) -> Self { + let values: Vec> = + array.iter().map(|x| x.map(|v| v as $FromType)).collect(); + <$ArrayType>::from_iter(values) + } + } + + impl From<&[$ArrowType]> for $ArrayType { + fn from(arrays: &[$ArrowType]) -> Self { + let values: Vec> = arrays + .iter() + .flat_map(|a| a.iter().map(|x| x.map(|v| v as $FromType))) + .collect(); + <$ArrayType>::from_iter(values) + } + } + }; +} + macro_rules! converts_with_timeunit { ($ArrayType:ty, $ArrowType:ty, $time_unit:expr, @map) => { @@ -960,6 +1031,11 @@ converts!(TimeArray, arrow_array::Time64MicrosecondArray, @map); converts!(IntervalArray, arrow_array::IntervalMonthDayNanoArray, @map); converts!(SerialArray, arrow_array::Int64Array, @map); +converts_with_type!(I16Array, arrow_array::Int8Array, i16, i8); +converts_with_type!(I16Array, arrow_array::UInt8Array, i16, u8); +converts_with_type!(I32Array, arrow_array::UInt16Array, i32, u16); +converts_with_type!(I64Array, arrow_array::UInt32Array, i64, u32); + converts_with_timeunit!(TimestampArray, arrow_array::TimestampSecondArray, TimeUnit::Second, @map); converts_with_timeunit!(TimestampArray, arrow_array::TimestampMillisecondArray, TimeUnit::Millisecond, @map); converts_with_timeunit!(TimestampArray, arrow_array::TimestampMicrosecondArray, TimeUnit::Microsecond, @map); @@ -1171,6 +1247,38 @@ impl TryFrom<&arrow_array::Decimal128Array> for DecimalArray { } } +// Since RisingWave does not support UInt type, convert UInt64Array to Decimal. +impl TryFrom<&arrow_array::UInt64Array> for DecimalArray { + type Error = ArrayError; + + fn try_from(array: &arrow_array::UInt64Array) -> Result { + let from_arrow = |value| { + // Convert the value to a Decimal with scale 0 + let res = Decimal::from(value); + Ok(res) + }; + + // Map over the array and convert each value + array + .iter() + .map(|o| o.map(from_arrow).transpose()) + .collect::>() + } +} + +impl TryFrom<&arrow_array::Float16Array> for F32Array { + type Error = ArrayError; + + fn try_from(array: &arrow_array::Float16Array) -> Result { + let from_arrow = |value| Ok(f32::from(value)); + + array + .iter() + .map(|o| o.map(from_arrow).transpose()) + .collect::>() + } +} + impl TryFrom<&arrow_array::LargeBinaryArray> for DecimalArray { type Error = ArrayError; @@ -1311,6 +1419,7 @@ impl From<&arrow_array::Decimal256Array> for Int256Array { #[cfg(test)] mod tests { + use super::*; #[test] @@ -1355,6 +1464,50 @@ mod tests { assert_eq!(F64Array::from(&arrow), array); } + #[test] + fn int8() { + let array: PrimitiveArray = I16Array::from_iter([None, Some(-128), Some(127)]); + let arr = arrow_array::Int8Array::from(vec![None, Some(-128), Some(127)]); + let converted: PrimitiveArray = (&arr).into(); + assert_eq!(converted, array); + } + + #[test] + fn uint8() { + let array: PrimitiveArray = I16Array::from_iter([None, Some(7), Some(25)]); + let arr = arrow_array::UInt8Array::from(vec![None, Some(7), Some(25)]); + let converted: PrimitiveArray = (&arr).into(); + assert_eq!(converted, array); + } + + #[test] + fn uint16() { + let array: PrimitiveArray = I32Array::from_iter([None, Some(7), Some(65535)]); + let arr = arrow_array::UInt16Array::from(vec![None, Some(7), Some(65535)]); + let converted: PrimitiveArray = (&arr).into(); + assert_eq!(converted, array); + } + + #[test] + fn uint32() { + let array: PrimitiveArray = I64Array::from_iter([None, Some(7), Some(4294967295)]); + let arr = arrow_array::UInt32Array::from(vec![None, Some(7), Some(4294967295)]); + let converted: PrimitiveArray = (&arr).into(); + assert_eq!(converted, array); + } + + #[test] + fn uint64() { + let array: PrimitiveArray = DecimalArray::from_iter([ + None, + Some(Decimal::Normalized("7".parse().unwrap())), + Some(Decimal::Normalized("18446744073709551615".parse().unwrap())), + ]); + let arr = arrow_array::UInt64Array::from(vec![None, Some(7), Some(18446744073709551615)]); + let converted: PrimitiveArray = (&arr).try_into().unwrap(); + assert_eq!(converted, array); + } + #[test] fn date() { let array = DateArray::from_iter([ diff --git a/src/connector/src/source/iceberg/parquet_file_handler.rs b/src/connector/src/source/iceberg/parquet_file_handler.rs index 146348545fbc0..c4bfceb1ed897 100644 --- a/src/connector/src/source/iceberg/parquet_file_handler.rs +++ b/src/connector/src/source/iceberg/parquet_file_handler.rs @@ -33,9 +33,11 @@ use opendal::Operator; use parquet::arrow::async_reader::AsyncFileReader; use parquet::arrow::{parquet_to_arrow_schema, ParquetRecordBatchStreamBuilder, ProjectionMask}; use parquet::file::metadata::{FileMetaData, ParquetMetaData, ParquetMetaDataReader}; +use risingwave_common::array::arrow::arrow_schema_udf::{DataType as ArrowDateType, IntervalUnit}; use risingwave_common::array::arrow::IcebergArrowConvert; use risingwave_common::array::StreamChunk; use risingwave_common::catalog::ColumnId; +use risingwave_common::types::DataType as RwDataType; use risingwave_common::util::tokio_util::compat::FuturesAsyncReadCompatExt; use url::Url; @@ -218,10 +220,10 @@ pub fn extract_valid_column_indices( .iter() .position(|&name| name == column.name) .and_then(|pos| { - let arrow_field = IcebergArrowConvert - .to_arrow_field(&column.name, &column.data_type) - .ok()?; - if &arrow_field == converted_arrow_schema.field(pos) { + let arrow_data_type: &risingwave_common::array::arrow::arrow_schema_udf::DataType = converted_arrow_schema.field(pos).data_type(); + let rw_data_type: &risingwave_common::types::DataType = &column.data_type; + + if is_parquet_schema_match_source_schema(arrow_data_type, rw_data_type) { Some(pos) } else { None @@ -318,3 +320,70 @@ pub async fn get_parquet_fields( converted_arrow_schema.fields; Ok(fields) } + +/// This function checks whether the schema of a Parquet file matches the user defined schema. +/// It handles the following special cases: +/// - Arrow's `timestamp(_, None)` types (all four time units) match with RisingWave's `TimeStamp` type. +/// - Arrow's `timestamp(_, Some)` matches with RisingWave's `TimeStamptz` type. +/// - Since RisingWave does not have an `UInt` type: +/// - Arrow's `UInt8` matches with RisingWave's `Int16`. +/// - Arrow's `UInt16` matches with RisingWave's `Int32`. +/// - Arrow's `UInt32` matches with RisingWave's `Int64`. +/// - Arrow's `UInt64` matches with RisingWave's `Decimal`. +/// - Arrow's `Float16` matches with RisingWave's `Float32`. +fn is_parquet_schema_match_source_schema( + arrow_data_type: &ArrowDateType, + rw_data_type: &RwDataType, +) -> bool { + matches!( + (arrow_data_type, rw_data_type), + (ArrowDateType::Boolean, RwDataType::Boolean) + | ( + ArrowDateType::Int8 | ArrowDateType::Int16 | ArrowDateType::UInt8, + RwDataType::Int16 + ) + | ( + ArrowDateType::Int32 | ArrowDateType::UInt16, + RwDataType::Int32 + ) + | ( + ArrowDateType::Int64 | ArrowDateType::UInt32, + RwDataType::Int64 + ) + | ( + ArrowDateType::UInt64 | ArrowDateType::Decimal128(_, _), + RwDataType::Decimal + ) + | (ArrowDateType::Decimal256(_, _), RwDataType::Int256) + | ( + ArrowDateType::Float16 | ArrowDateType::Float32, + RwDataType::Float32 + ) + | (ArrowDateType::Float64, RwDataType::Float64) + | (ArrowDateType::Timestamp(_, None), RwDataType::Timestamp) + | ( + ArrowDateType::Timestamp(_, Some(_)), + RwDataType::Timestamptz + ) + | (ArrowDateType::Date32, RwDataType::Date) + | ( + ArrowDateType::Time32(_) | ArrowDateType::Time64(_), + RwDataType::Time + ) + | ( + ArrowDateType::Interval(IntervalUnit::MonthDayNano), + RwDataType::Interval + ) + | ( + ArrowDateType::Utf8 | ArrowDateType::LargeUtf8, + RwDataType::Varchar + ) + | ( + ArrowDateType::Binary | ArrowDateType::LargeBinary, + RwDataType::Bytea + ) + | (ArrowDateType::List(_), RwDataType::List(_)) + | (ArrowDateType::Struct(_), RwDataType::Struct(_)) + | (ArrowDateType::Map(_, _), RwDataType::Map(_)) + ) +} From ba76431f6bee416ee05ed144df58eefe516c9792 Mon Sep 17 00:00:00 2001 From: Eric Fu Date: Wed, 27 Nov 2024 10:26:27 +0800 Subject: [PATCH 026/163] feat(batch): add `batch_expr_strict_mode` to ignore expression error in batch query (#19562) --- e2e_test/batch/basic/strict_mode.slt.part | 38 +++++++++++++++++++ e2e_test/batch/catalog/pg_settings.slt.part | 1 + proto/plan_common.proto | 1 + src/batch/src/executor/filter.rs | 4 +- src/batch/src/executor/project.rs | 4 +- src/batch/src/task/task_manager.rs | 1 + src/common/src/session_config/mod.rs | 6 +++ src/expr/core/src/expr/build.rs | 17 +++++++++ src/expr/core/src/expr_context.rs | 23 ++++++++++- .../src/scheduler/distributed/stage.rs | 1 + src/frontend/src/scheduler/local.rs | 5 ++- src/meta/src/controller/fragment.rs | 2 + src/meta/src/model/stream.rs | 1 + src/meta/src/stream/test_fragmenter.rs | 1 + src/stream/src/executor/integration_tests.rs | 1 + 15 files changed, 99 insertions(+), 7 deletions(-) create mode 100644 e2e_test/batch/basic/strict_mode.slt.part diff --git a/e2e_test/batch/basic/strict_mode.slt.part b/e2e_test/batch/basic/strict_mode.slt.part new file mode 100644 index 0000000000000..0f5081058b54b --- /dev/null +++ b/e2e_test/batch/basic/strict_mode.slt.part @@ -0,0 +1,38 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table t (v int); + +statement ok +insert into t values(-1), (0), (1); + +statement ok +SET batch_expr_strict_mode = false; + +query I +SELECT 1/v FROM unnest(ARRAY[-1, 0, 1]) v; +---- +-1 +NULL +1 + +# This plan consists of a BatchExchange. +query I +SELECT 1/v FROM t order by v; +---- +-1 +NULL +1 + +statement ok +SET batch_expr_strict_mode = DEFAULT; + +statement error Division by zero +SELECT 1/v FROM unnest(ARRAY[-1, 0, 1]) v; + +statement error Division by zero +SELECT 1/v FROM t order by v; + +statement ok +drop table t; diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 73e84f371c35e..32dbd17b7b48a 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -23,6 +23,7 @@ user background_ddl user batch_enable_distributed_dml user batch_enable_lookup_join user batch_enable_sort_agg +user batch_expr_strict_mode user batch_parallelism user bypass_cluster_limits user bytea_output diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 487ab54e2a666..4a93a84a9265b 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -236,6 +236,7 @@ message Cardinality { // Provide statement-local context, e.g. session info like time zone, for execution. message ExprContext { string time_zone = 1; + bool strict_mode = 2; } message AdditionalColumnKey {} diff --git a/src/batch/src/executor/filter.rs b/src/batch/src/executor/filter.rs index 96049b96a4179..0291582b95b59 100644 --- a/src/batch/src/executor/filter.rs +++ b/src/batch/src/executor/filter.rs @@ -17,7 +17,7 @@ use risingwave_common::array::ArrayImpl::Bool; use risingwave_common::array::DataChunk; use risingwave_common::catalog::Schema; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; -use risingwave_expr::expr::{build_from_prost, BoxedExpression}; +use risingwave_expr::expr::{build_batch_expr_from_prost, BoxedExpression}; use risingwave_pb::batch_plan::plan_node::NodeBody; use crate::error::{BatchError, Result}; @@ -90,7 +90,7 @@ impl BoxedExecutorBuilder for FilterExecutor { )?; let expr_node = filter_node.get_search_condition()?; - let expr = build_from_prost(expr_node)?; + let expr = build_batch_expr_from_prost(expr_node)?; Ok(Box::new(Self::new( expr, input, diff --git a/src/batch/src/executor/project.rs b/src/batch/src/executor/project.rs index 7fbc5540b975b..4e23aaa587985 100644 --- a/src/batch/src/executor/project.rs +++ b/src/batch/src/executor/project.rs @@ -18,7 +18,7 @@ use futures::{Stream, StreamExt}; use itertools::Itertools; use risingwave_common::array::DataChunk; use risingwave_common::catalog::{Field, Schema}; -use risingwave_expr::expr::{build_from_prost, BoxedExpression, Expression}; +use risingwave_expr::expr::{build_batch_expr_from_prost, BoxedExpression, Expression}; use risingwave_pb::batch_plan::plan_node::NodeBody; use crate::error::{BatchError, Result}; @@ -89,7 +89,7 @@ impl BoxedExecutorBuilder for ProjectExecutor { let project_exprs: Vec<_> = project_node .get_select_list() .iter() - .map(build_from_prost) + .map(build_batch_expr_from_prost) .try_collect()?; let fields = project_exprs diff --git a/src/batch/src/task/task_manager.rs b/src/batch/src/task/task_manager.rs index 4db15df2dbe85..c9fbde6369ef5 100644 --- a/src/batch/src/task/task_manager.rs +++ b/src/batch/src/task/task_manager.rs @@ -148,6 +148,7 @@ impl BatchManager { TracingContext::none(), ExprContext { time_zone: "UTC".to_string(), + strict_mode: false, }, ) .await diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index d452c51ad3539..7595ceeb21a70 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -109,6 +109,12 @@ pub struct SessionConfig { #[parameter(default = false, rename = "batch_enable_distributed_dml")] batch_enable_distributed_dml: bool, + /// Evaluate expression in strict mode for batch queries. + /// If set to false, an expression failure will not cause an error but leave a null value + /// on the result set. + #[parameter(default = true)] + batch_expr_strict_mode: bool, + /// The max gap allowed to transform small range scan into multi point lookup. #[parameter(default = 8)] max_split_range_gap: i32, diff --git a/src/expr/core/src/expr/build.rs b/src/expr/core/src/expr/build.rs index 988adbb5d8342..3b87a78b70984 100644 --- a/src/expr/core/src/expr/build.rs +++ b/src/expr/core/src/expr/build.rs @@ -16,6 +16,7 @@ use std::iter::Peekable; use itertools::Itertools; use risingwave_common::types::{DataType, ScalarImpl}; +use risingwave_expr::expr::LogReport; use risingwave_pb::expr::expr_node::{PbType, RexNode}; use risingwave_pb::expr::ExprNode; @@ -29,6 +30,7 @@ use super::NonStrictExpression; use crate::expr::{ BoxedExpression, Expression, ExpressionBoxExt, InputRefExpression, LiteralExpression, }; +use crate::expr_context::strict_mode; use crate::sig::FUNCTION_REGISTRY; use crate::{bail, Result}; @@ -48,6 +50,21 @@ pub fn build_non_strict_from_prost( .map(NonStrictExpression) } +/// Build a strict or non-strict expression according to expr context. +/// +/// When strict mode is off, the expression will not fail but leave a null value as result. +/// +/// Unlike [`build_non_strict_from_prost`], the returning value here can be either non-strict or +/// strict. Thus, the caller is supposed to handle potential errors under strict mode. +pub fn build_batch_expr_from_prost(prost: &ExprNode) -> Result { + if strict_mode()? { + build_from_prost(prost) + } else { + // TODO(eric): report errors to users via psql notice + Ok(ExprBuilder::new_non_strict(LogReport).build(prost)?.boxed()) + } +} + /// Build an expression from protobuf with possibly some wrappers attached to each node. struct ExprBuilder { /// The error reporting for non-strict mode. diff --git a/src/expr/core/src/expr_context.rs b/src/expr/core/src/expr_context.rs index e66f6322706f9..67d74f2a71a0d 100644 --- a/src/expr/core/src/expr_context.rs +++ b/src/expr/core/src/expr_context.rs @@ -22,11 +22,16 @@ define_context! { pub TIME_ZONE: String, pub FRAGMENT_ID: u32, pub VNODE_COUNT: usize, + pub STRICT_MODE: bool, } pub fn capture_expr_context() -> ExprResult { let time_zone = TIME_ZONE::try_with(ToOwned::to_owned)?; - Ok(ExprContext { time_zone }) + let strict_mode = STRICT_MODE::try_with(|v| *v)?; + Ok(ExprContext { + time_zone, + strict_mode, + }) } /// Get the vnode count from the context. @@ -36,9 +41,23 @@ pub fn vnode_count() -> ExprResult { VNODE_COUNT::try_with(|&x| x) } +/// Get the strict mode from expr context +/// +/// The return value depends on session variable. Default is true for batch query. +/// +/// Conceptually, streaming always use non-strict mode. Our implementation doesn't read this value, +/// although it's set to false as a placeholder. +pub fn strict_mode() -> ExprResult { + STRICT_MODE::try_with(|&v| v) +} + pub async fn expr_context_scope(expr_context: ExprContext, future: Fut) -> Fut::Output where Fut: Future, { - TIME_ZONE::scope(expr_context.time_zone.to_owned(), future).await + TIME_ZONE::scope( + expr_context.time_zone.to_owned(), + STRICT_MODE::scope(expr_context.strict_mode, future), + ) + .await } diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 83d85b378a114..087ba189770df 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -721,6 +721,7 @@ impl StageRunner { async fn schedule_tasks_for_all(&mut self, shutdown_rx: ShutdownToken) -> SchedulerResult<()> { let expr_context = ExprContext { time_zone: self.ctx.session().config().timezone().to_owned(), + strict_mode: self.ctx.session().config().batch_expr_strict_mode(), }; // If root, we execute it locally. if !self.is_root_stage() { diff --git a/src/frontend/src/scheduler/local.rs b/src/frontend/src/scheduler/local.rs index b11d462b151c7..633dac2d2d562 100644 --- a/src/frontend/src/scheduler/local.rs +++ b/src/frontend/src/scheduler/local.rs @@ -118,6 +118,7 @@ impl LocalQueryExecution { self.batch_query_epoch, self.shutdown_rx().clone(), ); + let executor = executor.build().await?; #[for_await] @@ -146,6 +147,7 @@ impl LocalQueryExecution { let db_name = self.session.database().to_string(); let search_path = self.session.config().search_path(); let time_zone = self.session.config().timezone(); + let strict_mode = self.session.config().batch_expr_strict_mode(); let timeout = self.timeout; let meta_client = self.front_env.meta_client_ref(); @@ -166,7 +168,7 @@ impl LocalQueryExecution { } }; - use risingwave_expr::expr_context::TIME_ZONE; + use risingwave_expr::expr_context::*; use crate::expr::function_impl::context::{ AUTH_CONTEXT, CATALOG_READER, DB_NAME, META_CLIENT, SEARCH_PATH, USER_INFO_READER, @@ -179,6 +181,7 @@ impl LocalQueryExecution { let exec = async move { SEARCH_PATH::scope(search_path, exec).await }.boxed(); let exec = async move { AUTH_CONTEXT::scope(auth_context, exec).await }.boxed(); let exec = async move { TIME_ZONE::scope(time_zone, exec).await }.boxed(); + let exec = async move { STRICT_MODE::scope(strict_mode, exec).await }.boxed(); let exec = async move { META_CLIENT::scope(meta_client, exec).await }.boxed(); if let Some(timeout) = timeout { diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 672fef180afde..a2d0094ef2d0a 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1678,6 +1678,7 @@ mod tests { mview_definition: "".to_string(), expr_context: Some(PbExprContext { time_zone: String::from("America/New_York"), + strict_mode: false, }), } }) @@ -1798,6 +1799,7 @@ mod tests { .map(VnodeBitmap::from), expr_context: ExprContext::from(&PbExprContext { time_zone: String::from("America/New_York"), + strict_mode: false, }), } }) diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index b87d06fdd0898..b252beb39012d 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -144,6 +144,7 @@ impl StreamContext { PbExprContext { // `self.timezone` must always be set; an invalid value is used here for debugging if it's not. time_zone: self.timezone.clone().unwrap_or("Empty Time Zone".into()), + strict_mode: false, } } diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index cfde9187abc66..7d435a6b0257d 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -453,6 +453,7 @@ async fn test_graph_builder() -> MetaResult<()> { let graph = make_stream_graph(); let expr_context = ExprContext { time_zone: graph.ctx.as_ref().unwrap().timezone.clone(), + strict_mode: false, }; let fragment_graph = StreamFragmentGraph::new(&env, graph, &job)?; let internal_tables = fragment_graph.incomplete_internal_tables(); diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 41495198632f9..01d4ced06805c 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -46,6 +46,7 @@ use crate::task::barrier_test_utils::LocalBarrierTestEnv; async fn test_merger_sum_aggr() { let expr_context = ExprContext { time_zone: String::from("UTC"), + strict_mode: false, }; let barrier_test_env = LocalBarrierTestEnv::for_test().await; From 066978338838a0a1f98f93bc440d8ac30ab46126 Mon Sep 17 00:00:00 2001 From: chagelo Date: Wed, 27 Nov 2024 10:55:44 +0800 Subject: [PATCH 027/163] feat: add builtin function `timezone` (#19524) Signed-off-by: chagelo --- e2e_test/batch/functions/timezone.slt.part | 112 ++++++++++++++++++ src/expr/impl/src/scalar/timestamptz.rs | 16 +-- .../binder/expr/function/builtin_scalar.rs | 12 +- src/tests/regress/data/schedule | 2 +- 4 files changed, 131 insertions(+), 11 deletions(-) create mode 100644 e2e_test/batch/functions/timezone.slt.part diff --git a/e2e_test/batch/functions/timezone.slt.part b/e2e_test/batch/functions/timezone.slt.part new file mode 100644 index 0000000000000..344e00e735dc0 --- /dev/null +++ b/e2e_test/batch/functions/timezone.slt.part @@ -0,0 +1,112 @@ +statement ok +SET TimeZone to 'UTC'; + +query T +SELECT timezone('Europe/Moscow', '2011-03-27 02:00:00'::timestamp); +---- +2011-03-26 23:00:00+00:00 + +query T +SELECT timezone('Europe/Moscow', '2011-03-27 02:59:59'::timestamp); +---- +2011-03-26 23:59:59+00:00 + +query T +SELECT timezone('Europe/Moscow', '2011-03-27 03:00:00'::timestamp); +---- +2011-03-26 23:00:00+00:00 + +query T +SELECT timezone('Europe/Moscow', '2014-10-26 00:59:59'::timestamp); +---- +2014-10-25 20:59:59+00:00 + +query T +SELECT timezone('Europe/Moscow', '2014-10-26 01:00:00'::timestamp); +---- +2014-10-25 22:00:00+00:00 + +query T +SELECT timezone('Europe/Moscow', '2014-10-26 01:00:01'::timestamp); +---- +2014-10-25 22:00:01+00:00 + +query T +SELECT timezone('Asia/Shanghai', '2024-11-20 20:00:00 UTC'::timestamptz); +---- +2024-11-21 04:00:00 + +query T +SELECT timezone('Asia/Shanghai', '2023-02-28 20:00:00 UTC'::timestamptz); +---- +2023-03-01 04:00:00 + +query T +SELECT timezone('Asia/Shanghai', '2024-02-28 20:00:00 UTC'::timestamptz); +---- +2024-02-29 04:00:00 + +query T +SELECT timezone('Europe/Moscow', '2011-03-26 22:00:00 UTC'::timestamptz); +---- +2011-03-27 01:00:00 + +query T +SELECT timezone('Europe/Moscow', '2011-03-26 23:00:00 UTC'::timestamptz); +---- +2011-03-27 03:00:00 + +query T +SELECT timezone('Europe/Moscow', '2011-03-26 23:00:01 UTC'::timestamptz); +---- +2011-03-27 03:00:01 + +query T +SELECT timezone('Europe/Moscow', '2011-03-27 01:00:00 UTC'::timestamptz); +---- +2011-03-27 05:00:00 + +query T +SELECT timezone('Europe/Moscow', '2011-03-27 02:00:00 UTC'::timestamptz); +---- +2011-03-27 06:00:00 + +query T +SELECT timezone('Europe/Moscow', '2011-03-27 02:59:59 UTC'::timestamptz); +---- +2011-03-27 06:59:59 + +query T +SELECT timezone('Europe/Moscow', '2011-03-27 03:00:00 UTC'::timestamptz); +---- +2011-03-27 07:00:00 + +query T +SELECT timezone('Europe/Moscow', '2014-10-25 21:59:59 UTC'::timestamptz); +---- +2014-10-26 01:59:59 + +query T +SELECT timezone('Europe/Moscow', '2014-10-25 22:00:00 UTC'::timestamptz); +---- +2014-10-26 01:00:00 + +query T +SELECT timezone('Europe/Moscow', '2014-10-25 22:59:59 UTC'::timestamptz); +---- +2014-10-26 01:59:59 + +query T +SELECT timezone('Europe/Moscow', '2014-10-25 23:00:00 UTC'::timestamptz); +---- +2014-10-26 02:00:00 + +query T +SELECT timezone('Europe/Moscow', '2014-10-25 23:00:01 UTC'::timestamptz); +---- +2014-10-26 02:00:01 + +query T +SELECT timezone('Europe/Moscow', '2014-10-26 02:00:01 UTC'::timestamptz); +---- +2014-10-26 05:00:01 diff --git a/src/expr/impl/src/scalar/timestamptz.rs b/src/expr/impl/src/scalar/timestamptz.rs index 7dcc718d728ef..06f9f2a7ffba4 100644 --- a/src/expr/impl/src/scalar/timestamptz.rs +++ b/src/expr/impl/src/scalar/timestamptz.rs @@ -41,6 +41,14 @@ pub fn f64_sec_to_timestamptz(elem: F64) -> Result { Ok(Timestamptz::from_micros(micros)) } +#[function("at_time_zone(timestamptz, varchar) -> timestamp")] +pub fn timestamptz_at_time_zone(input: Timestamptz, time_zone: &str) -> Result { + let time_zone = Timestamptz::lookup_time_zone(time_zone).map_err(time_zone_err)?; + let instant_local = input.to_datetime_in_zone(time_zone); + let naive = instant_local.naive_local(); + Ok(Timestamp(naive)) +} + #[function("at_time_zone(timestamp, varchar) -> timestamptz")] pub fn timestamp_at_time_zone(input: Timestamp, time_zone: &str) -> Result { let time_zone = Timestamptz::lookup_time_zone(time_zone).map_err(time_zone_err)?; @@ -96,14 +104,6 @@ pub fn str_to_timestamptz(elem: &str, time_zone: &str) -> Result { }) } -#[function("at_time_zone(timestamptz, varchar) -> timestamp")] -pub fn timestamptz_at_time_zone(input: Timestamptz, time_zone: &str) -> Result { - let time_zone = Timestamptz::lookup_time_zone(time_zone).map_err(time_zone_err)?; - let instant_local = input.to_datetime_in_zone(time_zone); - let naive = instant_local.naive_local(); - Ok(Timestamp(naive)) -} - /// This operation is zone agnostic. #[function("subtract(timestamptz, timestamptz) -> interval")] pub fn timestamptz_timestamptz_sub(l: Timestamptz, r: Timestamptz) -> Result { diff --git a/src/frontend/src/binder/expr/function/builtin_scalar.rs b/src/frontend/src/binder/expr/function/builtin_scalar.rs index 66c28b0ba24d6..7570f72c5d095 100644 --- a/src/frontend/src/binder/expr/function/builtin_scalar.rs +++ b/src/frontend/src/binder/expr/function/builtin_scalar.rs @@ -210,7 +210,7 @@ impl Binder { ("scale", raw_call(ExprType::Scale)), ("min_scale", raw_call(ExprType::MinScale)), ("trim_scale", raw_call(ExprType::TrimScale)), - + // date and time ( "to_timestamp", dispatch_by_len(vec![ @@ -223,8 +223,16 @@ impl Binder { ("make_date", raw_call(ExprType::MakeDate)), ("make_time", raw_call(ExprType::MakeTime)), ("make_timestamp", raw_call(ExprType::MakeTimestamp)), - ("to_date", raw_call(ExprType::CharToDate)), ("make_timestamptz", raw_call(ExprType::MakeTimestamptz)), + ("timezone", rewrite(ExprType::AtTimeZone, |mut inputs|{ + if inputs.len() == 2 { + inputs.swap(0, 1); + Ok(inputs) + } else { + Err(ErrorCode::ExprError("unexpected arguments number".into()).into()) + } + })), + ("to_date", raw_call(ExprType::CharToDate)), // string ("substr", raw_call(ExprType::Substr)), ("length", raw_call(ExprType::Length)), diff --git a/src/tests/regress/data/schedule b/src/tests/regress/data/schedule index d1cfd9ab86253..54cf97a5ed4d6 100644 --- a/src/tests/regress/data/schedule +++ b/src/tests/regress/data/schedule @@ -12,4 +12,4 @@ test: strings date time timestamp interval test: case arrays delete test: jsonb jsonb_jsonpath test: regex -test: contrib-pgcrypto-rijndael +test: contrib-pgcrypto-rijndael \ No newline at end of file From 3faa0bf346665379bfa2e1295c71aee9f0e29c21 Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Wed, 27 Nov 2024 12:18:38 +0800 Subject: [PATCH 028/163] refactor(common): consolidate `StructType` constructors (Part 2/2) (#19570) --- src/common/src/array/arrow/arrow_impl.rs | 2 +- src/common/src/catalog/column.rs | 8 +++---- src/common/src/types/mod.rs | 11 +++++---- src/common/src/types/struct_type.rs | 18 ++++----------- .../codec/src/decoder/avro/schema.rs | 23 ++++++++++--------- .../codec/src/decoder/protobuf/parser.rs | 15 ++++++++---- src/frontend/src/binder/expr/mod.rs | 10 ++++---- src/frontend/src/binder/update.rs | 12 ++++++---- src/frontend/src/expr/subquery.rs | 4 +--- src/frontend/src/handler/create_function.rs | 8 +++---- .../src/handler/create_sql_function.rs | 12 ++++------ src/tests/sqlsmith/src/sql_gen/expr.rs | 3 +-- 12 files changed, 59 insertions(+), 67 deletions(-) diff --git a/src/common/src/array/arrow/arrow_impl.rs b/src/common/src/array/arrow/arrow_impl.rs index f72a1d8739be5..a341e218878c9 100644 --- a/src/common/src/array/arrow/arrow_impl.rs +++ b/src/common/src/array/arrow/arrow_impl.rs @@ -485,7 +485,7 @@ pub trait FromArrow { fields .iter() .map(|f| Ok((f.name().clone(), self.from_field(f)?))) - .try_collect::<_, _, ArrayError>()?, + .try_collect::<_, Vec<_>, ArrayError>()?, )) } diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index f7c9260e0d1ab..7b1387a5feffc 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -25,7 +25,7 @@ use risingwave_pb::plan_common::{ use super::{row_id_column_desc, rw_timestamp_column_desc, USER_COLUMN_ID_OFFSET}; use crate::catalog::{cdc_table_name_column_desc, offset_column_desc, Field, ROW_ID_COLUMN_ID}; -use crate::types::DataType; +use crate::types::{DataType, StructType}; use crate::util::value_encoding::DatumToProtoExt; /// Column ID is the unique identifier of a column in a table. Different from table ID, column ID is @@ -270,10 +270,8 @@ impl ColumnDesc { type_name: &str, fields: Vec, ) -> Self { - let data_type = DataType::new_struct( - fields.iter().map(|f| f.data_type.clone()).collect_vec(), - fields.iter().map(|f| f.name.clone()).collect_vec(), - ); + let data_type = + StructType::new(fields.iter().map(|f| (&f.name, f.data_type.clone()))).into(); Self { data_type, column_id: ColumnId::new(column_id), diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index 5e6264cd933d0..0b0d63aa4b300 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -30,6 +30,7 @@ use postgres_types::{FromSql, IsNull, ToSql, Type}; use risingwave_common_estimate_size::{EstimateSize, ZeroHeapSize}; use risingwave_pb::data::data_type::PbTypeName; use risingwave_pb::data::PbDataType; +use rw_iter_util::ZipEqFast as _; use serde::{Deserialize, Serialize, Serializer}; use strum_macros::EnumDiscriminants; use thiserror_ext::AsReport; @@ -241,7 +242,11 @@ impl From<&PbDataType> for DataType { PbTypeName::Struct => { let fields: Vec = proto.field_type.iter().map(|f| f.into()).collect_vec(); let field_names: Vec = proto.field_names.iter().cloned().collect_vec(); - DataType::new_struct(fields, field_names) + if proto.field_names.is_empty() { + StructType::unnamed(fields).into() + } else { + StructType::new(field_names.into_iter().zip_eq_fast(fields)).into() + } } PbTypeName::List => DataType::List( // The first (and only) item is the list element type. @@ -405,10 +410,6 @@ impl DataType { } } - pub fn new_struct(fields: Vec, field_names: Vec) -> Self { - Self::Struct(StructType::from_parts(field_names, fields)) - } - pub fn as_struct(&self) -> &StructType { match self { DataType::Struct(t) => t, diff --git a/src/common/src/types/struct_type.rs b/src/common/src/types/struct_type.rs index edc4b73311533..cc1980b34f830 100644 --- a/src/common/src/types/struct_type.rs +++ b/src/common/src/types/struct_type.rs @@ -48,10 +48,11 @@ struct StructTypeInner { impl StructType { /// Creates a struct type with named fields. - pub fn new(named_fields: Vec<(impl Into, DataType)>) -> Self { - let mut field_types = Vec::with_capacity(named_fields.len()); - let mut field_names = Vec::with_capacity(named_fields.len()); - for (name, ty) in named_fields { + pub fn new(named_fields: impl IntoIterator, DataType)>) -> Self { + let iter = named_fields.into_iter(); + let mut field_types = Vec::with_capacity(iter.size_hint().0); + let mut field_names = Vec::with_capacity(iter.size_hint().0); + for (name, ty) in iter { field_names.push(name.into()); field_types.push(ty); } @@ -70,15 +71,6 @@ impl StructType { })) } - pub(super) fn from_parts(field_names: Vec, field_types: Vec) -> Self { - // TODO: enable this assertion - // debug_assert!(field_names.len() == field_types.len()); - Self(Arc::new(StructTypeInner { - field_types: field_types.into(), - field_names: field_names.into(), - })) - } - /// Creates a struct type with unnamed fields. pub fn unnamed(fields: Vec) -> Self { Self(Arc::new(StructTypeInner { diff --git a/src/connector/codec/src/decoder/avro/schema.rs b/src/connector/codec/src/decoder/avro/schema.rs index 7e86a1cc11dd1..f523147d8175d 100644 --- a/src/connector/codec/src/decoder/avro/schema.rs +++ b/src/connector/codec/src/decoder/avro/schema.rs @@ -20,7 +20,7 @@ use apache_avro::AvroResult; use itertools::Itertools; use risingwave_common::error::NotImplemented; use risingwave_common::log::LogSuppresser; -use risingwave_common::types::{DataType, Decimal, MapType}; +use risingwave_common::types::{DataType, Decimal, MapType, StructType}; use risingwave_common::{bail, bail_not_implemented}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; @@ -190,12 +190,13 @@ fn avro_type_mapping( return Ok(DataType::Decimal); } - let struct_fields = fields - .iter() - .map(|f| avro_type_mapping(&f.schema, map_handling)) - .collect::>()?; - let struct_names = fields.iter().map(|f| f.name.clone()).collect_vec(); - DataType::new_struct(struct_fields, struct_names) + StructType::new( + fields + .iter() + .map(|f| Ok((&f.name, avro_type_mapping(&f.schema, map_handling)?))) + .collect::>>()?, + ) + .into() } Schema::Array(item_schema) => { let item_type = avro_type_mapping(item_schema.as_ref(), map_handling)?; @@ -225,7 +226,7 @@ fn avro_type_mapping( // Note: Avro union's variant tag is type name, not field name (unlike Rust enum, or Protobuf oneof). // XXX: do we need to introduce union.handling.mode? - let (fields, field_names) = union_schema + let fields = union_schema .variants() .iter() // null will mean the whole struct is null @@ -233,13 +234,13 @@ fn avro_type_mapping( .map(|variant| { avro_type_mapping(variant, map_handling).and_then(|t| { let name = avro_schema_to_struct_field_name(variant)?; - Ok((t, name)) + Ok((name, t)) }) }) - .process_results(|it| it.unzip::<_, _, Vec<_>, Vec<_>>()) + .try_collect::<_, Vec<_>, _>() .context("failed to convert Avro union to struct")?; - DataType::new_struct(fields, field_names) + StructType::new(fields).into() } } } diff --git a/src/connector/codec/src/decoder/protobuf/parser.rs b/src/connector/codec/src/decoder/protobuf/parser.rs index 852fa9cca48d6..f249d7db72e13 100644 --- a/src/connector/codec/src/decoder/protobuf/parser.rs +++ b/src/connector/codec/src/decoder/protobuf/parser.rs @@ -17,7 +17,8 @@ use itertools::Itertools; use prost_reflect::{Cardinality, FieldDescriptor, Kind, MessageDescriptor, ReflectMessage, Value}; use risingwave_common::array::{ListValue, StructValue}; use risingwave_common::types::{ - DataType, DatumCow, Decimal, JsonbVal, MapType, MapValue, ScalarImpl, ToOwnedDatum, F32, F64, + DataType, DatumCow, Decimal, JsonbVal, MapType, MapValue, ScalarImpl, StructType, ToOwnedDatum, + F32, F64, }; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; use thiserror::Error; @@ -257,10 +258,14 @@ fn protobuf_type_mapping( } else { let fields = m .fields() - .map(|f| protobuf_type_mapping(&f, parse_trace)) - .try_collect()?; - let field_names = m.fields().map(|f| f.name().to_string()).collect_vec(); - DataType::new_struct(fields, field_names) + .map(|f| { + Ok(( + f.name().to_string(), + protobuf_type_mapping(&f, parse_trace)?, + )) + }) + .try_collect::<_, Vec<_>, _>()?; + StructType::new(fields).into() } } Kind::Enum(_) => DataType::Varchar, diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 86631767998bd..59aa42be0e79b 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -14,7 +14,7 @@ use itertools::Itertools; use risingwave_common::catalog::{ColumnDesc, ColumnId, PG_CATALOG_SCHEMA_NAME}; -use risingwave_common::types::{DataType, MapType}; +use risingwave_common::types::{DataType, MapType, StructType}; use risingwave_common::util::iter_util::zip_eq_fast; use risingwave_common::{bail_no_function, bail_not_implemented, not_implemented}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDescVersion}; @@ -1008,13 +1008,13 @@ pub fn bind_data_type(data_type: &AstDataType) -> Result { AstDataType::Char(..) => { bail_not_implemented!("CHAR is not supported, please use VARCHAR instead") } - AstDataType::Struct(types) => DataType::new_struct( + AstDataType::Struct(types) => StructType::new( types .iter() - .map(|f| bind_data_type(&f.data_type)) + .map(|f| Ok((f.name.real_value(), bind_data_type(&f.data_type)?))) .collect::>>()?, - types.iter().map(|f| f.name.real_value()).collect_vec(), - ), + ) + .into(), AstDataType::Map(kv) => { let key = bind_data_type(&kv.0)?; let value = bind_data_type(&kv.1)?; diff --git a/src/frontend/src/binder/update.rs b/src/frontend/src/binder/update.rs index a2038a4d471d9..d16ece284fa3b 100644 --- a/src/frontend/src/binder/update.rs +++ b/src/frontend/src/binder/update.rs @@ -17,7 +17,7 @@ use std::collections::{BTreeMap, HashMap}; use fixedbitset::FixedBitSet; use itertools::Itertools; use risingwave_common::catalog::{Schema, TableVersionId}; -use risingwave_common::types::DataType; +use risingwave_common::types::StructType; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_sqlparser::ast::{Assignment, AssignmentValue, Expr, ObjectName, SelectItem}; @@ -206,10 +206,12 @@ impl Binder { bail_bind_error!("number of columns does not match number of values"); } - let target_type = DataType::new_struct( - ids.iter().map(|id| id.return_type()).collect(), - id.iter().map(|id| id.real_value()).collect(), - ); + let target_type = StructType::new( + id.iter() + .zip_eq_fast(ids) + .map(|(id, expr)| (id.real_value(), expr.return_type())), + ) + .into(); let expr = expr.cast_assign(target_type)?; exprs.push(expr); diff --git a/src/frontend/src/expr/subquery.rs b/src/frontend/src/expr/subquery.rs index fc904638790e2..6f5b419a46d64 100644 --- a/src/frontend/src/expr/subquery.rs +++ b/src/frontend/src/expr/subquery.rs @@ -97,9 +97,7 @@ impl Expr for Subquery { StructType::unnamed(self.query.data_types()) } else { StructType::new( - (schema.fields().iter().cloned()) - .map(|f| (f.name, f.data_type)) - .collect(), + (schema.fields().iter().cloned()).map(|f| (f.name, f.data_type)), ) }; DataType::Struct(struct_type) diff --git a/src/frontend/src/handler/create_function.rs b/src/frontend/src/handler/create_function.rs index b81d2b4514edf..6d529e9ccb356 100644 --- a/src/frontend/src/handler/create_function.rs +++ b/src/frontend/src/handler/create_function.rs @@ -14,7 +14,7 @@ use anyhow::Context; use risingwave_common::catalog::FunctionId; -use risingwave_common::types::DataType; +use risingwave_common::types::StructType; use risingwave_expr::sig::{CreateFunctionOptions, UdfKind}; use risingwave_pb::catalog::function::{Kind, ScalarFunction, TableFunction}; use risingwave_pb::catalog::Function; @@ -83,9 +83,9 @@ pub async fn handle_create_function( // return type is a struct for multiple columns let it = columns .into_iter() - .map(|c| bind_data_type(&c.data_type).map(|ty| (ty, c.name.real_value()))); - let (datatypes, names) = itertools::process_results(it, |it| it.unzip())?; - return_type = DataType::new_struct(datatypes, names); + .map(|c| bind_data_type(&c.data_type).map(|ty| (c.name.real_value(), ty))); + let fields = it.try_collect::<_, Vec<_>, _>()?; + return_type = StructType::new(fields).into(); } Kind::Table(TableFunction {}) } diff --git a/src/frontend/src/handler/create_sql_function.rs b/src/frontend/src/handler/create_sql_function.rs index c733f603a3c44..b48b06942005e 100644 --- a/src/frontend/src/handler/create_sql_function.rs +++ b/src/frontend/src/handler/create_sql_function.rs @@ -16,7 +16,7 @@ use std::collections::HashMap; use fancy_regex::Regex; use risingwave_common::catalog::FunctionId; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, StructType}; use risingwave_pb::catalog::function::{Kind, ScalarFunction, TableFunction}; use risingwave_pb::catalog::Function; use risingwave_sqlparser::parser::{Parser, ParserError}; @@ -188,15 +188,11 @@ pub async fn handle_create_sql_function( return_type = bind_data_type(&columns[0].data_type)?; } else { // return type is a struct for multiple columns - let datatypes = columns + let fields = columns .iter() - .map(|c| bind_data_type(&c.data_type)) + .map(|c| Ok((c.name.real_value(), bind_data_type(&c.data_type)?))) .collect::>>()?; - let names = columns - .iter() - .map(|c| c.name.real_value()) - .collect::>(); - return_type = DataType::new_struct(datatypes, names); + return_type = StructType::new(fields).into(); } Kind::Table(TableFunction {}) } diff --git a/src/tests/sqlsmith/src/sql_gen/expr.rs b/src/tests/sqlsmith/src/sql_gen/expr.rs index 4625727f67dca..f5d93d2b6d6f9 100644 --- a/src/tests/sqlsmith/src/sql_gen/expr.rs +++ b/src/tests/sqlsmith/src/sql_gen/expr.rs @@ -168,8 +168,7 @@ impl SqlGenerator<'_, R> { DataType::Struct(StructType::new( STRUCT_FIELD_NAMES[0..num_fields] .iter() - .map(|s| (s.to_string(), self.gen_data_type_inner(depth))) - .collect(), + .map(|s| (s.to_string(), self.gen_data_type_inner(depth))), )) } From c121fa7ff0d51b668616c19638db028b1f56e2fa Mon Sep 17 00:00:00 2001 From: Croxx Date: Wed, 27 Nov 2024 13:46:10 +0800 Subject: [PATCH 029/163] refactor(metrics): adapt to the new foyer metrics framework (#19580) Signed-off-by: MrCroxx --- Cargo.lock | 74 +++++------------------------------ Cargo.toml | 3 +- src/storage/Cargo.toml | 1 - src/storage/src/store_impl.rs | 17 ++++---- 4 files changed, 19 insertions(+), 76 deletions(-) diff --git a/Cargo.lock b/Cargo.lock index ad3a78617ec56..48298a8f5f08a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -2684,7 +2684,7 @@ dependencies = [ "hyper 0.14.27", "hyper-tls 0.5.0", "lz4", - "sealed 0.4.0", + "sealed", "serde", "static_assertions", "thiserror 1.0.63", @@ -5044,12 +5044,6 @@ version = "1.0.7" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "3f9eec918d3f24069decb9af1554cad7c880e2da24a9afd88aca000531ab82c1" -[[package]] -name = "foldhash" -version = "0.1.3" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "f81ec6369c545a7d40e4589b5597581fa1c441fe1cce96dd1de43159910a36a2" - [[package]] name = "foreign-types" version = "0.3.2" @@ -5077,7 +5071,7 @@ dependencies = [ [[package]] name = "foyer" version = "0.13.0-dev" -source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" +source = "git+https://github.com/mrcroxx/foyer?rev=19bc2def14dfa8bb7d6581fb625584694bc7f4a0#19bc2def14dfa8bb7d6581fb625584694bc7f4a0" dependencies = [ "ahash 0.8.11", "anyhow", @@ -5095,8 +5089,9 @@ dependencies = [ [[package]] name = "foyer-common" version = "0.13.0-dev" -source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" +source = "git+https://github.com/mrcroxx/foyer?rev=19bc2def14dfa8bb7d6581fb625584694bc7f4a0#19bc2def14dfa8bb7d6581fb625584694bc7f4a0" dependencies = [ + "ahash 0.8.11", "bytes", "cfg-if", "fastrace", @@ -5104,16 +5099,16 @@ dependencies = [ "hashbrown 0.14.5", "itertools 0.13.0", "madsim-tokio", - "metrics", "parking_lot 0.12.1", "pin-project", + "prometheus", "serde", ] [[package]] name = "foyer-memory" version = "0.13.0-dev" -source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" +source = "git+https://github.com/mrcroxx/foyer?rev=19bc2def14dfa8bb7d6581fb625584694bc7f4a0#19bc2def14dfa8bb7d6581fb625584694bc7f4a0" dependencies = [ "ahash 0.8.11", "bitflags 2.6.0", @@ -5130,13 +5125,14 @@ dependencies = [ "paste", "pin-project", "serde", + "thiserror 2.0.3", "tracing", ] [[package]] name = "foyer-storage" version = "0.13.0-dev" -source = "git+https://github.com/mrcroxx/foyer?rev=99005412eac1bca6aa7c73d750efc9cdb8ae176e#99005412eac1bca6aa7c73d750efc9cdb8ae176e" +source = "git+https://github.com/mrcroxx/foyer?rev=19bc2def14dfa8bb7d6581fb625584694bc7f4a0#19bc2def14dfa8bb7d6581fb625584694bc7f4a0" dependencies = [ "ahash 0.8.11", "allocator-api2", @@ -5167,7 +5163,7 @@ dependencies = [ "pin-project", "rand", "serde", - "thiserror 1.0.63", + "thiserror 2.0.3", "tracing", "twox-hash", "zstd 0.13.0", @@ -5868,9 +5864,6 @@ name = "hashbrown" version = "0.15.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "1e087f84d4f86bf4b218b927129862374b72199ae7d8657835f1e89000eea4fb" -dependencies = [ - "foldhash", -] [[package]] name = "hashlink" @@ -7487,43 +7480,6 @@ dependencies = [ "autocfg", ] -[[package]] -name = "metrics" -version = "0.24.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8ae428771d17306715c5091d446327d1cfdedc82185c65ba8423ab404e45bf10" -dependencies = [ - "ahash 0.8.11", - "portable-atomic", -] - -[[package]] -name = "metrics-prometheus" -version = "0.8.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ba3115d0e859470455e1587af70aed7935163339f959f01d6cbd245605fe7e67" -dependencies = [ - "arc-swap", - "metrics", - "metrics-util", - "prometheus", - "sealed 0.6.0", - "smallvec", - "thiserror 1.0.63", -] - -[[package]] -name = "metrics-util" -version = "0.18.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "15b482df36c13dd1869d73d14d28cd4855fbd6cfc32294bee109908a9f4a4ed7" -dependencies = [ - "crossbeam-epoch", - "crossbeam-utils", - "hashbrown 0.15.0", - "metrics", -] - [[package]] name = "miette" version = "7.2.0" @@ -12092,7 +12048,6 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "memcomparable", - "metrics-prometheus", "moka", "more-asserts", "nix 0.29.0", @@ -12971,17 +12926,6 @@ dependencies = [ "syn 1.0.109", ] -[[package]] -name = "sealed" -version = "0.6.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "22f968c5ea23d555e670b449c1c5e7b2fc399fdaec1d304a17cd48e288abc107" -dependencies = [ - "proc-macro2", - "quote", - "syn 2.0.87", -] - [[package]] name = "sec1" version = "0.3.0" diff --git a/Cargo.toml b/Cargo.toml index a2c47ea293c78..98f37686db325 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -81,9 +81,10 @@ repository = "https://github.com/risingwavelabs/risingwave" [workspace.dependencies] # foyer = { version = "0.12.2", features = ["tracing", "nightly"] } -foyer = { git = "https://github.com/mrcroxx/foyer", rev = "99005412eac1bca6aa7c73d750efc9cdb8ae176e", features = [ +foyer = { git = "https://github.com/mrcroxx/foyer", rev = "19bc2def14dfa8bb7d6581fb625584694bc7f4a0", features = [ "tracing", "nightly", + "prometheus", ] } apache-avro = { git = "https://github.com/risingwavelabs/avro", rev = "25113ba88234a9ae23296e981d8302c290fdaa4b", features = [ "snappy", diff --git a/src/storage/Cargo.toml b/src/storage/Cargo.toml index 6b7314ce0c72e..a5ccd017bdd75 100644 --- a/src/storage/Cargo.toml +++ b/src/storage/Cargo.toml @@ -35,7 +35,6 @@ itertools = { workspace = true } libc = "0.2" lz4 = "1.28.0" memcomparable = "0.2" -metrics-prometheus = "0.8" moka = { version = "0.12.0", features = ["future", "sync"] } more-asserts = "0.3" num-integer = "0.1" diff --git a/src/storage/src/store_impl.rs b/src/storage/src/store_impl.rs index d0f7280ca3d84..397c9065261b1 100644 --- a/src/storage/src/store_impl.rs +++ b/src/storage/src/store_impl.rs @@ -13,13 +13,13 @@ // limitations under the License. use std::fmt::Debug; -use std::ops::Deref; -use std::sync::Arc; +use std::sync::{Arc, LazyLock}; use std::time::Duration; use enum_as_inner::EnumAsInner; use foyer::{ - DirectFsDeviceOptions, Engine, HybridCacheBuilder, LargeEngineOptions, RateLimitPicker, + DirectFsDeviceOptions, Engine, HybridCacheBuilder, LargeEngineOptions, + PrometheusMetricsRegistry, RateLimitPicker, }; use risingwave_common::monitor::GLOBAL_METRICS_REGISTRY; use risingwave_common_service::RpcNotificationClient; @@ -42,6 +42,9 @@ use crate::monitor::{ use crate::opts::StorageOpts; use crate::StateStore; +static FOYER_METRICS_REGISTRY: LazyLock = + LazyLock::new(|| PrometheusMetricsRegistry::new(GLOBAL_METRICS_REGISTRY.clone())); + mod opaque_type { use super::*; @@ -613,15 +616,10 @@ impl StateStoreImpl { ) -> StorageResult { const MB: usize = 1 << 20; - if cfg!(not(madsim)) { - metrics_prometheus::Recorder::builder() - .with_registry(GLOBAL_METRICS_REGISTRY.deref().clone()) - .build_and_install(); - } - let meta_cache = { let mut builder = HybridCacheBuilder::new() .with_name("foyer.meta") + .with_metrics_registry(FOYER_METRICS_REGISTRY.clone()) .memory(opts.meta_cache_capacity_mb * MB) .with_shards(opts.meta_cache_shard_num) .with_eviction_config(opts.meta_cache_eviction_config.clone()) @@ -667,6 +665,7 @@ impl StateStoreImpl { let block_cache = { let mut builder = HybridCacheBuilder::new() .with_name("foyer.data") + .with_metrics_registry(FOYER_METRICS_REGISTRY.clone()) .with_event_listener(Arc::new(BlockCacheEventListener::new( state_store_metrics.clone(), ))) From ac5cb4036ed5d1d28483ab296a3a8811f6addc90 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Wed, 27 Nov 2024 17:05:57 +0800 Subject: [PATCH 030/163] feat(barrier): maintain per database scheduled command queue (#19592) --- src/meta/src/barrier/context/context_impl.rs | 13 +- src/meta/src/barrier/context/mod.rs | 9 +- src/meta/src/barrier/context/recovery.rs | 4 +- src/meta/src/barrier/schedule.rs | 176 +++++++++++++------ src/meta/src/barrier/worker.rs | 4 +- src/meta/src/stream/stream_manager.rs | 5 +- 6 files changed, 143 insertions(+), 68 deletions(-) diff --git a/src/meta/src/barrier/context/context_impl.rs b/src/meta/src/barrier/context/context_impl.rs index 4d652494ffd36..1f8d19f690054 100644 --- a/src/meta/src/barrier/context/context_impl.rs +++ b/src/meta/src/barrier/context/context_impl.rs @@ -15,6 +15,7 @@ use std::sync::Arc; use futures::future::try_join_all; +use risingwave_common::catalog::DatabaseId; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::meta::PausedReason; @@ -42,16 +43,20 @@ impl GlobalBarrierWorkerContext for GlobalBarrierWorkerContextImpl { self.scheduled_barriers.next_scheduled().await } - fn abort_and_mark_blocked(&self, recovery_reason: RecoveryReason) { + fn abort_and_mark_blocked( + &self, + database_id: Option, + recovery_reason: RecoveryReason, + ) { self.set_status(BarrierManagerStatus::Recovering(recovery_reason)); // Mark blocked and abort buffered schedules, they might be dirty already. self.scheduled_barriers - .abort_and_mark_blocked("cluster is under recovering"); + .abort_and_mark_blocked(database_id, "cluster is under recovering"); } - fn mark_ready(&self) { - self.scheduled_barriers.mark_ready(); + fn mark_ready(&self, database_id: Option) { + self.scheduled_barriers.mark_ready(database_id); self.set_status(BarrierManagerStatus::Running); } diff --git a/src/meta/src/barrier/context/mod.rs b/src/meta/src/barrier/context/mod.rs index 7306c16171621..61136121c67af 100644 --- a/src/meta/src/barrier/context/mod.rs +++ b/src/meta/src/barrier/context/mod.rs @@ -19,6 +19,7 @@ use std::future::Future; use std::sync::Arc; use arc_swap::ArcSwap; +use risingwave_common::catalog::DatabaseId; use risingwave_pb::common::WorkerNode; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_service::streaming_control_stream_request::PbInitRequest; @@ -42,8 +43,12 @@ pub(super) trait GlobalBarrierWorkerContext: Send + Sync + 'static { ) -> impl Future> + Send + '_; async fn next_scheduled(&self) -> Scheduled; - fn abort_and_mark_blocked(&self, recovery_reason: RecoveryReason); - fn mark_ready(&self); + fn abort_and_mark_blocked( + &self, + database_id: Option, + recovery_reason: RecoveryReason, + ); + fn mark_ready(&self, database_id: Option); fn post_collect_command<'a>( &'a self, diff --git a/src/meta/src/barrier/context/recovery.rs b/src/meta/src/barrier/context/recovery.rs index b352ece5012a1..4bac51823ca52 100644 --- a/src/meta/src/barrier/context/recovery.rs +++ b/src/meta/src/barrier/context/recovery.rs @@ -134,7 +134,7 @@ impl GlobalBarrierWorkerContextImpl { tracing::info!("recovered mview progress"); // This is a quick path to accelerate the process of dropping and canceling streaming jobs. - let _ = self.scheduled_barriers.pre_apply_drop_cancel(); + let _ = self.scheduled_barriers.pre_apply_drop_cancel(None); let mut active_streaming_nodes = ActiveStreamingWorkerNodes::new_snapshot(self.metadata_manager.clone()) @@ -178,7 +178,7 @@ impl GlobalBarrierWorkerContextImpl { })? }; - if self.scheduled_barriers.pre_apply_drop_cancel() { + if self.scheduled_barriers.pre_apply_drop_cancel(None) { info = self.resolve_graph_info().await.inspect_err(|err| { warn!(error = %err.as_report(), "resolve actor info failed"); })? diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 4fc25d25b04f2..bbe34897fbe8f 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -12,13 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::VecDeque; +use std::collections::{HashMap, VecDeque}; use std::iter::once; use std::sync::Arc; use std::time::{Duration, Instant}; use anyhow::{anyhow, Context}; use assert_matches::assert_matches; +use itertools::Itertools; use parking_lot::Mutex; use prometheus::HistogramTimer; use risingwave_common::catalog::{DatabaseId, TableId}; @@ -64,22 +65,27 @@ enum QueueStatus { } struct ScheduledQueueItem { - database_id: DatabaseId, command: Command, notifiers: Vec, send_latency_timer: HistogramTimer, span: tracing::Span, } -pub(super) struct ScheduledQueue { - queue: VecDeque, +struct StatusQueue { + queue: T, status: QueueStatus, } -impl ScheduledQueue { - fn new() -> Self { +type DatabaseScheduledQueue = StatusQueue>; +type ScheduledQueue = StatusQueue>; + +impl StatusQueue { + fn new() -> Self + where + T: Default, + { Self { - queue: VecDeque::new(), + queue: T::default(), status: QueueStatus::Ready, } } @@ -92,11 +98,7 @@ impl ScheduledQueue { self.status = QueueStatus::Ready; } - fn len(&self) -> usize { - self.queue.len() - } - - fn push_back(&mut self, scheduled: ScheduledQueueItem) -> MetaResult<()> { + fn validate_item(&mut self, scheduled: &ScheduledQueueItem) -> MetaResult<()> { // We don't allow any command to be scheduled when the queue is blocked, except for dropping streaming jobs. // Because we allow dropping streaming jobs when the cluster is under recovery, so we have to buffer the drop // command and execute it when the cluster is ready to clean up it. @@ -110,7 +112,6 @@ impl ScheduledQueue { { return Err(MetaError::unavailable(reason)); } - self.queue.push_back(scheduled); Ok(()) } } @@ -131,7 +132,6 @@ impl Inner { /// Create a new scheduled barrier with the given `checkpoint`, `command` and `notifiers`. fn new_scheduled( &self, - database_id: DatabaseId, command: Command, notifiers: impl IntoIterator, ) -> ScheduledQueueItem { @@ -139,7 +139,6 @@ impl Inner { let span = tracing_span(); ScheduledQueueItem { - database_id, command, notifiers: notifiers.into_iter().collect(), send_latency_timer: self.metrics.barrier_send_latency.start_timer(), @@ -181,20 +180,38 @@ impl BarrierScheduler { } /// Push a scheduled barrier into the queue. - fn push(&self, scheduleds: impl IntoIterator) -> MetaResult<()> { + fn push( + &self, + database_id: DatabaseId, + scheduleds: impl IntoIterator, + ) -> MetaResult<()> { let mut queue = self.inner.queue.lock(); + let scheduleds = scheduleds.into_iter().collect_vec(); + scheduleds + .iter() + .try_for_each(|scheduled| queue.validate_item(scheduled))?; + let queue = queue + .queue + .entry(database_id) + .or_insert_with(DatabaseScheduledQueue::new); + scheduleds + .iter() + .try_for_each(|scheduled| queue.validate_item(scheduled))?; for scheduled in scheduleds { - queue.push_back(scheduled)?; - if queue.len() == 1 { + queue.queue.push_back(scheduled); + if queue.queue.len() == 1 { self.inner.changed_tx.send(()).ok(); } } Ok(()) } - /// Try to cancel scheduled cmd for create streaming job, return true if cancelled. - pub fn try_cancel_scheduled_create(&self, table_id: TableId) -> bool { + /// Try to cancel scheduled cmd for create streaming job, return true if the command exists previously and get cancelled. + pub fn try_cancel_scheduled_create(&self, database_id: DatabaseId, table_id: TableId) -> bool { let queue = &mut self.inner.queue.lock(); + let Some(queue) = queue.queue.get_mut(&database_id) else { + return false; + }; if let Some(idx) = queue.queue.iter().position(|scheduled| { if let Command::CreateStreamingJob { info, .. } = &scheduled.command @@ -232,7 +249,6 @@ impl BarrierScheduler { contexts.push((started_rx, collect_rx)); scheduleds.push(self.inner.new_scheduled( - database_id, command, once(Notifier { started: Some(started_tx), @@ -241,7 +257,7 @@ impl BarrierScheduler { )); } - self.push(scheduleds)?; + self.push(database_id, scheduleds)?; for (injected_rx, collect_rx) in contexts { // Wait for this command to be injected, and record the result. @@ -375,18 +391,20 @@ impl PeriodicBarriers { impl ScheduledBarriers { pub(super) async fn next_scheduled(&self) -> Scheduled { - loop { + 'outer: loop { let mut rx = self.inner.changed_tx.subscribe(); { let mut queue = self.inner.queue.lock(); - if let Some(item) = queue.queue.pop_front() { - item.send_latency_timer.observe_duration(); - break Scheduled { - database_id: item.database_id, - command: item.command, - notifiers: item.notifiers, - span: item.span, - }; + for (database_id, queue) in &mut queue.queue { + if let Some(item) = queue.queue.pop_front() { + item.send_latency_timer.observe_duration(); + break 'outer Scheduled { + database_id: *database_id, + command: item.command, + notifiers: item.notifiers, + span: item.span, + }; + } } } rx.changed().await.unwrap(); @@ -396,52 +414,96 @@ impl ScheduledBarriers { impl ScheduledBarriers { /// Pre buffered drop and cancel command, return true if any. - pub(super) fn pre_apply_drop_cancel(&self) -> bool { - self.pre_apply_drop_cancel_scheduled() + pub(super) fn pre_apply_drop_cancel(&self, database_id: Option) -> bool { + self.pre_apply_drop_cancel_scheduled(database_id) } /// Mark command scheduler as blocked and abort all queued scheduled command and notify with /// specific reason. - pub(super) fn abort_and_mark_blocked(&self, reason: impl Into + Copy) { + pub(super) fn abort_and_mark_blocked( + &self, + database_id: Option, + reason: impl Into + Copy, + ) { let mut queue = self.inner.queue.lock(); - queue.mark_blocked(reason.into()); - while let Some(ScheduledQueueItem { notifiers, .. }) = queue.queue.pop_front() { - notifiers - .into_iter() - .for_each(|notify| notify.notify_collection_failed(anyhow!(reason.into()).into())) + let mark_blocked_and_notify_failed = |queue: &mut DatabaseScheduledQueue| { + queue.mark_blocked(reason.into()); + while let Some(ScheduledQueueItem { notifiers, .. }) = queue.queue.pop_front() { + notifiers.into_iter().for_each(|notify| { + notify.notify_collection_failed(anyhow!(reason.into()).into()) + }) + } + }; + if let Some(database_id) = database_id { + let queue = queue + .queue + .entry(database_id) + .or_insert_with(DatabaseScheduledQueue::new); + mark_blocked_and_notify_failed(queue); + } else { + queue.mark_blocked(reason.into()); + for queue in queue.queue.values_mut() { + mark_blocked_and_notify_failed(queue); + } } } /// Mark command scheduler as ready to accept new command. - pub(super) fn mark_ready(&self) { + pub(super) fn mark_ready(&self, database_id: Option) { let mut queue = self.inner.queue.lock(); - queue.mark_ready(); + if let Some(database_id) = database_id { + queue + .queue + .entry(database_id) + .or_insert_with(DatabaseScheduledQueue::new) + .mark_ready(); + } else { + queue.mark_ready(); + for queue in queue.queue.values_mut() { + queue.mark_ready(); + } + } } /// Try to pre apply drop and cancel scheduled command and return them if any. /// It should only be called in recovery. - pub(super) fn pre_apply_drop_cancel_scheduled(&self) -> bool { + pub(super) fn pre_apply_drop_cancel_scheduled(&self, database_id: Option) -> bool { let mut queue = self.inner.queue.lock(); - assert_matches!(queue.status, QueueStatus::Blocked(_)); let mut applied = false; - while let Some(ScheduledQueueItem { - notifiers, command, .. - }) = queue.queue.pop_front() - { - match command { - Command::DropStreamingJobs { .. } => { - applied = true; - } - Command::DropSubscription { .. } => {} - _ => { - unreachable!("only drop and cancel streaming jobs should be buffered"); + let mut pre_apply_drop_cancel = |queue: &mut DatabaseScheduledQueue| { + while let Some(ScheduledQueueItem { + notifiers, command, .. + }) = queue.queue.pop_front() + { + match command { + Command::DropStreamingJobs { .. } => { + applied = true; + } + Command::DropSubscription { .. } => {} + _ => { + unreachable!("only drop and cancel streaming jobs should be buffered"); + } } + notifiers.into_iter().for_each(|notify| { + notify.notify_collected(); + }); + } + }; + + if let Some(database_id) = database_id { + assert_matches!(queue.status, QueueStatus::Ready); + if let Some(queue) = queue.queue.get_mut(&database_id) { + assert_matches!(queue.status, QueueStatus::Blocked(_)); + pre_apply_drop_cancel(queue); + } + } else { + assert_matches!(queue.status, QueueStatus::Blocked(_)); + for queue in queue.queue.values_mut() { + pre_apply_drop_cancel(queue); } - notifiers.into_iter().for_each(|notify| { - notify.notify_collected(); - }); } + applied } } diff --git a/src/meta/src/barrier/worker.rs b/src/meta/src/barrier/worker.rs index f6999e5ce9235..e69acda8f4651 100644 --- a/src/meta/src/barrier/worker.rs +++ b/src/meta/src/barrier/worker.rs @@ -512,12 +512,12 @@ impl GlobalBarrierWorker { err: Option, recovery_reason: RecoveryReason, ) { - self.context.abort_and_mark_blocked(recovery_reason); + self.context.abort_and_mark_blocked(None, recovery_reason); // Clear all control streams to release resources (connections to compute nodes) first. self.control_stream_manager.clear(); self.recovery_inner(paused_reason, err).await; - self.context.mark_ready(); + self.context.mark_ready(None); } async fn recovery_inner( diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 20eceaffce935..3a0abf9bc665c 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -287,7 +287,10 @@ impl GlobalStreamManager { .await { // try to cancel buffered creating command. - if self.barrier_scheduler.try_cancel_scheduled_create(table_id) { + if self + .barrier_scheduler + .try_cancel_scheduled_create(database_id, table_id) + { tracing::debug!("cancelling streaming job {table_id} in buffer queue."); } else if !table_fragments.is_created() { tracing::debug!( From 200ed05c35ce57a178c30def0380eba7d4e5dc4a Mon Sep 17 00:00:00 2001 From: Dylan Date: Wed, 27 Nov 2024 17:35:19 +0800 Subject: [PATCH 031/163] feat(meta): add a rpc to fetch meta store endpoint (#19594) --- proto/meta.proto | 7 +++++++ src/meta/node/src/server.rs | 6 +++--- src/meta/service/src/cluster_service.rs | 18 +++++++++++++++--- src/meta/src/backup_restore/utils.rs | 6 +++--- src/meta/src/controller/cluster.rs | 4 ++++ src/meta/src/controller/mod.rs | 10 +++++++--- src/rpc_client/src/meta_client.rs | 7 +++++++ 7 files changed, 46 insertions(+), 12 deletions(-) diff --git a/proto/meta.proto b/proto/meta.proto index e5dda6b83a922..8b385370c5b2a 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -413,6 +413,12 @@ message GetClusterRecoveryStatusResponse { RecoveryStatus status = 1; } +message GetMetaStoreInfoRequest {} + +message GetMetaStoreInfoResponse { + string meta_store_endpoint = 1; +} + service ClusterService { rpc AddWorkerNode(AddWorkerNodeRequest) returns (AddWorkerNodeResponse); rpc ActivateWorkerNode(ActivateWorkerNodeRequest) returns (ActivateWorkerNodeResponse); @@ -420,6 +426,7 @@ service ClusterService { rpc UpdateWorkerNodeSchedulability(UpdateWorkerNodeSchedulabilityRequest) returns (UpdateWorkerNodeSchedulabilityResponse); rpc ListAllNodes(ListAllNodesRequest) returns (ListAllNodesResponse); rpc GetClusterRecoveryStatus(GetClusterRecoveryStatusRequest) returns (GetClusterRecoveryStatusResponse); + rpc GetMetaStoreInfo(GetMetaStoreInfoRequest) returns (GetMetaStoreInfoResponse); } enum SubscribeType { diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index b929e4b326907..77a2aecc030a7 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -135,7 +135,7 @@ pub async fn rpc_serve( )); let conn = sea_orm::Database::connect(IN_MEMORY_STORE).await?; rpc_serve_with_store( - SqlMetaStore::new(conn), + SqlMetaStore::new(conn, IN_MEMORY_STORE.to_string()), dummy_election_client, address_info, max_cluster_heartbeat_interval, @@ -149,7 +149,7 @@ pub async fn rpc_serve( } MetaStoreBackend::Sql { endpoint, config } => { let is_sqlite = DbBackend::Sqlite.is_prefix_of(&endpoint); - let mut options = sea_orm::ConnectOptions::new(endpoint); + let mut options = sea_orm::ConnectOptions::new(endpoint.clone()); options .max_connections(config.max_connections) .min_connections(config.min_connections) @@ -164,7 +164,7 @@ pub async fn rpc_serve( } let conn = sea_orm::Database::connect(options).await?; - let meta_store_sql = SqlMetaStore::new(conn); + let meta_store_sql = SqlMetaStore::new(conn, endpoint); // Init election client. let id = address_info.advertise_addr.clone(); diff --git a/src/meta/service/src/cluster_service.rs b/src/meta/service/src/cluster_service.rs index e346ebd446ea9..d506e7826cbf6 100644 --- a/src/meta/service/src/cluster_service.rs +++ b/src/meta/service/src/cluster_service.rs @@ -21,9 +21,9 @@ use risingwave_pb::meta::cluster_service_server::ClusterService; use risingwave_pb::meta::{ ActivateWorkerNodeRequest, ActivateWorkerNodeResponse, AddWorkerNodeRequest, AddWorkerNodeResponse, DeleteWorkerNodeRequest, DeleteWorkerNodeResponse, - GetClusterRecoveryStatusRequest, GetClusterRecoveryStatusResponse, ListAllNodesRequest, - ListAllNodesResponse, UpdateWorkerNodeSchedulabilityRequest, - UpdateWorkerNodeSchedulabilityResponse, + GetClusterRecoveryStatusRequest, GetClusterRecoveryStatusResponse, GetMetaStoreInfoRequest, + GetMetaStoreInfoResponse, ListAllNodesRequest, ListAllNodesResponse, + UpdateWorkerNodeSchedulabilityRequest, UpdateWorkerNodeSchedulabilityResponse, }; use tonic::{Request, Response, Status}; @@ -167,4 +167,16 @@ impl ClusterService for ClusterServiceImpl { status: self.barrier_manager.get_recovery_status() as _, })) } + + async fn get_meta_store_info( + &self, + _request: Request, + ) -> Result, Status> { + Ok(Response::new(GetMetaStoreInfoResponse { + meta_store_endpoint: self + .metadata_manager + .cluster_controller + .meta_store_endpoint(), + })) + } } diff --git a/src/meta/src/backup_restore/utils.rs b/src/meta/src/backup_restore/utils.rs index 0946ce74e5fbf..6dcb5dac8e018 100644 --- a/src/meta/src/backup_restore/utils.rs +++ b/src/meta/src/backup_restore/utils.rs @@ -56,7 +56,7 @@ pub async fn get_meta_store(opts: RestoreOpts) -> BackupResult { match meta_store_backend { MetaStoreBackend::Mem => { let conn = sea_orm::Database::connect(IN_MEMORY_STORE).await.unwrap(); - Ok(SqlMetaStore::new(conn)) + Ok(SqlMetaStore::new(conn, IN_MEMORY_STORE.to_string())) } MetaStoreBackend::Sql { endpoint, config } => { let max_connection = if DbBackend::Sqlite.is_prefix_of(&endpoint) { @@ -66,7 +66,7 @@ pub async fn get_meta_store(opts: RestoreOpts) -> BackupResult { } else { config.max_connections }; - let mut options = sea_orm::ConnectOptions::new(endpoint); + let mut options = sea_orm::ConnectOptions::new(endpoint.clone()); options .max_connections(max_connection) .min_connections(config.min_connections) @@ -76,7 +76,7 @@ pub async fn get_meta_store(opts: RestoreOpts) -> BackupResult { let conn = sea_orm::Database::connect(options) .await .map_err(|e| BackupError::MetaStorage(e.into()))?; - Ok(SqlMetaStore::new(conn)) + Ok(SqlMetaStore::new(conn, endpoint)) } } } diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 242cbbafddd41..3926be84edbe4 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -379,6 +379,10 @@ impl ClusterController { pub fn cluster_id(&self) -> &ClusterId { self.env.cluster_id() } + + pub fn meta_store_endpoint(&self) -> String { + self.env.meta_store_ref().endpoint.clone() + } } /// The cluster info used for scheduling a streaming job. diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index c01c34bc9927c..a8397b4f03526 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -60,20 +60,24 @@ impl From for MetaError { #[derive(Clone)] pub struct SqlMetaStore { pub conn: DatabaseConnection, + pub endpoint: String, } pub const IN_MEMORY_STORE: &str = "sqlite::memory:"; impl SqlMetaStore { - pub fn new(conn: DatabaseConnection) -> Self { - Self { conn } + pub fn new(conn: DatabaseConnection, endpoint: String) -> Self { + Self { conn, endpoint } } #[cfg(any(test, feature = "test"))] pub async fn for_test() -> Self { let conn = sea_orm::Database::connect(IN_MEMORY_STORE).await.unwrap(); Migrator::up(&conn, None).await.unwrap(); - Self { conn } + Self { + conn, + endpoint: IN_MEMORY_STORE.to_string(), + } } /// Check whether the cluster, which uses SQL as the backend, is a new cluster. diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 6efac3f414564..edf83355651aa 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -1252,6 +1252,12 @@ impl MetaClient { Ok(resp.params.unwrap().into()) } + pub async fn get_meta_store_endpoint(&self) -> Result { + let req = GetMetaStoreInfoRequest {}; + let resp = self.inner.get_meta_store_info(req).await?; + Ok(resp.meta_store_endpoint) + } + pub async fn set_system_param( &self, param: String, @@ -2066,6 +2072,7 @@ macro_rules! for_all_meta_rpc { ,{ cluster_client, update_worker_node_schedulability, UpdateWorkerNodeSchedulabilityRequest, UpdateWorkerNodeSchedulabilityResponse } ,{ cluster_client, list_all_nodes, ListAllNodesRequest, ListAllNodesResponse } ,{ cluster_client, get_cluster_recovery_status, GetClusterRecoveryStatusRequest, GetClusterRecoveryStatusResponse } + ,{ cluster_client, get_meta_store_info, GetMetaStoreInfoRequest, GetMetaStoreInfoResponse } ,{ heartbeat_client, heartbeat, HeartbeatRequest, HeartbeatResponse } ,{ stream_client, flush, FlushRequest, FlushResponse } ,{ stream_client, pause, PauseRequest, PauseResponse } From c45327bae82869699328a47e70919d38e783df2d Mon Sep 17 00:00:00 2001 From: xxchan Date: Wed, 27 Nov 2024 17:38:53 +0800 Subject: [PATCH 032/163] fix(meta): locate upstream source fragment correctly for SourceBackfill (#19564) Signed-off-by: xxchan --- .../kafka/issue_19563.slt.serial | 73 ++++++ proto/stream_plan.proto | 6 + src/ctl/src/cmd_impl/meta/cluster_info.rs | 14 +- src/meta/model/src/fragment.rs | 2 + src/meta/src/controller/fragment.rs | 53 +++-- src/meta/src/manager/metadata.rs | 17 +- src/meta/src/model/stream.rs | 19 +- src/meta/src/stream/scale.rs | 22 +- src/meta/src/stream/source_manager.rs | 42 ++-- src/prost/src/lib.rs | 21 +- src/tests/simulation/src/ctl_ext.rs | 9 +- .../integration_tests/scale/shared_source.rs | 207 +++++++++++++++--- 12 files changed, 370 insertions(+), 115 deletions(-) create mode 100644 e2e_test/source_inline/kafka/issue_19563.slt.serial diff --git a/e2e_test/source_inline/kafka/issue_19563.slt.serial b/e2e_test/source_inline/kafka/issue_19563.slt.serial new file mode 100644 index 0000000000000..91b795d6acf34 --- /dev/null +++ b/e2e_test/source_inline/kafka/issue_19563.slt.serial @@ -0,0 +1,73 @@ +control substitution on + +# Note: rw_fragments is not isolated by schema so we make the test serial. + +system ok +rpk topic create test-topic-19563 -p 6 + +statement ok +CREATE SOURCE kafkasource ( + v1 timestamp with time zone +) +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'test-topic-19563', + scan.startup.mode = 'earliest' +) FORMAT PLAIN ENCODE JSON ( + timestamptz.handling.mode = 'utc_without_suffix' +); + +# Note that StreamSourceScan is in the StreamDynamicFilter fragment, which has 3 upstream fragments. +query T +explain create materialized view mv1 as select v1 from kafkasource where v1 between now() and now() + interval '1 day' * 365 * 2000; +---- +StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } +└─StreamDynamicFilter { predicate: (v1 <= $expr1), output: [v1, _row_id], cleaned_by_watermark: true } + ├─StreamProject { exprs: [v1, _row_id], output_watermarks: [v1] } + │ └─StreamDynamicFilter { predicate: (v1 >= now), output_watermarks: [v1], output: [v1, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset], cleaned_by_watermark: true } + │ ├─StreamRowIdGen { row_id_index: 2 } + │ │ └─StreamSourceScan { columns: [v1, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + │ └─StreamExchange { dist: Broadcast } + │ └─StreamNow + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [AddWithTimeZone(now, '730000 days':Interval, 'UTC':Varchar) as $expr1], output_watermarks: [$expr1] } + └─StreamNow + + +# The following test is adapted from `temporal_filter.slt`. + +# This statement should be correct for the next ~1000 years +# We cannot have a variable interval for now, so we use 2000 year's worth of days as the upper bound. +statement ok +create materialized view mv1 as select v1 from kafkasource where v1 between now() and now() + interval '1 day' * 365 * 2000; + +query I +select array_length(upstream_fragment_ids) from rw_fragments where array_contains(flags, Array['SOURCE_SCAN']); +---- +3 + +system ok +cat < anyhow: for actor in &fragment.actors { if let Some(ConnectorSplits { splits }) = actor_splits.remove(&actor.actor_id) { + let num_splits = splits.len(); let splits = splits .iter() .map(|split| SplitImpl::try_from(split).unwrap()) .map(|split| split.id()) .collect_vec() .join(","); - actor_splits_map.insert(actor.actor_id, (splits.len(), splits)); + actor_splits_map.insert(actor.actor_id, (num_splits, splits)); } } } @@ -123,12 +124,11 @@ pub async fn source_split_info(context: &CtlContext, ignore_id: bool) -> anyhow: split_count, splits, if !actor.upstream_actor_id.is_empty() { - assert!( - actor.upstream_actor_id.len() == 1, - "should have only one upstream actor, got {actor:?}" - ); - let upstream_splits = - actor_splits_map.get(&actor.upstream_actor_id[0]).unwrap(); + let upstream_splits = actor + .upstream_actor_id + .iter() + .find_map(|id| actor_splits_map.get(id)) + .expect("should have one upstream source actor"); format!( " <- Upstream Actor{}: [{}]", if ignore_id { diff --git a/src/meta/model/src/fragment.rs b/src/meta/model/src/fragment.rs index 45f59ed3c0aec..5f7768d19176a 100644 --- a/src/meta/model/src/fragment.rs +++ b/src/meta/model/src/fragment.rs @@ -26,6 +26,8 @@ pub struct Model { pub job_id: ObjectId, pub fragment_type_mask: i32, pub distribution_type: DistributionType, + /// Note: the `StreamNode` is different from the final plan node used by actors. + /// Specifically, `Merge` nodes' `upstream_actor_id` will be filled. (See `compose_fragment`) pub stream_node: StreamNode, pub state_table_ids: I32Array, pub upstream_fragment_id: I32Array, diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index a2d0094ef2d0a..c6f423f229631 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1292,12 +1292,22 @@ impl CatalogController { Ok(actors) } - /// Get the actor ids, and each actor's upstream actor ids of the fragment with `fragment_id` with `Running` status. - pub async fn get_running_actors_and_upstream_of_fragment( + /// Get the actor ids, and each actor's upstream source actor ids of the fragment with `fragment_id` with `Running` status. + pub async fn get_running_actors_for_source_backfill( &self, fragment_id: FragmentId, - ) -> MetaResult> { + ) -> MetaResult> { let inner = self.inner.read().await; + let txn = inner.db.begin().await?; + let fragment = Fragment::find_by_id(fragment_id) + .one(&txn) + .await? + .context(format!("fragment {} not found", fragment_id))?; + let (_source_id, upstream_source_fragment_id) = fragment + .stream_node + .to_protobuf() + .find_source_backfill() + .unwrap(); let actors: Vec<(ActorId, ActorUpstreamActors)> = Actor::find() .select_only() .column(actor::Column::ActorId) @@ -1305,9 +1315,24 @@ impl CatalogController { .filter(actor::Column::FragmentId.eq(fragment_id)) .filter(actor::Column::Status.eq(ActorStatus::Running)) .into_tuple() - .all(&inner.db) + .all(&txn) .await?; - Ok(actors) + Ok(actors + .into_iter() + .map(|(actor_id, upstream_actor_ids)| { + let upstream_source_actors = + &upstream_actor_ids.0[&(upstream_source_fragment_id as i32)]; + assert_eq!( + upstream_source_actors.len(), + 1, + "expect only one upstream source actor, but got {:?}, actor_id: {}, fragment_id: {}", + upstream_source_actors, + actor_id, + fragment_id + ); + (actor_id, upstream_source_actors[0]) + }) + .collect()) } pub async fn get_actors_by_job_ids(&self, job_ids: Vec) -> MetaResult> { @@ -1453,31 +1478,29 @@ impl CatalogController { pub async fn load_backfill_fragment_ids( &self, - ) -> MetaResult>> { + ) -> MetaResult>> { let inner = self.inner.read().await; - let mut fragments: Vec<(FragmentId, I32Array, i32, StreamNode)> = Fragment::find() + let mut fragments: Vec<(FragmentId, i32, StreamNode)> = Fragment::find() .select_only() .columns([ fragment::Column::FragmentId, - fragment::Column::UpstreamFragmentId, fragment::Column::FragmentTypeMask, fragment::Column::StreamNode, ]) .into_tuple() .all(&inner.db) .await?; - fragments.retain(|(_, _, mask, _)| *mask & PbFragmentTypeFlag::SourceScan as i32 != 0); + fragments.retain(|(_, mask, _)| *mask & PbFragmentTypeFlag::SourceScan as i32 != 0); let mut source_fragment_ids = HashMap::new(); - for (fragment_id, upstream_fragment_id, _, stream_node) in fragments { - if let Some(source_id) = stream_node.to_protobuf().find_source_backfill() { - if upstream_fragment_id.inner_ref().len() != 1 { - bail!("SourceBackfill should have only one upstream fragment, found {} for fragment {}", upstream_fragment_id.inner_ref().len(), fragment_id); - } + for (fragment_id, _, stream_node) in fragments { + if let Some((source_id, upstream_source_fragment_id)) = + stream_node.to_protobuf().find_source_backfill() + { source_fragment_ids .entry(source_id as SourceId) .or_insert_with(BTreeSet::new) - .insert((fragment_id, upstream_fragment_id.inner_ref()[0])); + .insert((fragment_id, upstream_source_fragment_id)); } } Ok(source_fragment_ids) diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index f2510a5b75486..c54d4f4a9db4c 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -571,26 +571,17 @@ impl MetadataManager { Ok(actor_ids.into_iter().map(|id| id as ActorId).collect()) } - pub async fn get_running_actors_and_upstream_actors_of_fragment( + pub async fn get_running_actors_for_source_backfill( &self, id: FragmentId, - ) -> MetaResult)>> { + ) -> MetaResult> { let actor_ids = self .catalog_controller - .get_running_actors_and_upstream_of_fragment(id as _) + .get_running_actors_for_source_backfill(id as _) .await?; Ok(actor_ids .into_iter() - .map(|(id, actors)| { - ( - id as ActorId, - actors - .into_inner() - .into_iter() - .flat_map(|(_, ids)| ids.into_iter().map(|id| id as ActorId)) - .collect(), - ) - }) + .map(|(id, upstream)| (id as ActorId, upstream as ActorId)) .collect()) } diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index b252beb39012d..12c06325fd99e 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -424,27 +424,30 @@ impl StreamJobFragments { source_fragments } + /// Returns (`source_id`, -> (`source_backfill_fragment_id`, `upstream_source_fragment_id`)). + /// + /// Note: the fragment `source_backfill_fragment_id` may actually have multiple upstream fragments, + /// but only one of them is the upstream source fragment, which is what we return. pub fn source_backfill_fragments( &self, ) -> MetadataModelResult>> { - let mut source_fragments = HashMap::new(); + let mut source_backfill_fragments = HashMap::new(); for fragment in self.fragments() { for actor in &fragment.actors { - if let Some(source_id) = actor.nodes.as_ref().unwrap().find_source_backfill() { - if fragment.upstream_fragment_ids.len() != 1 { - return Err(anyhow::anyhow!("SourceBackfill should have only one upstream fragment, found {:?} for fragment {}", fragment.upstream_fragment_ids, fragment.fragment_id).into()); - } - source_fragments + if let Some((source_id, upstream_source_fragment_id)) = + actor.nodes.as_ref().unwrap().find_source_backfill() + { + source_backfill_fragments .entry(source_id as SourceId) .or_insert(BTreeSet::new()) - .insert((fragment.fragment_id, fragment.upstream_fragment_ids[0])); + .insert((fragment.fragment_id, upstream_source_fragment_id)); break; } } } - Ok(source_fragments) + Ok(source_backfill_fragments) } /// Resolve dependent table diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index c4a97d25d7392..db4e54168766c 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -158,8 +158,8 @@ pub struct RescheduleContext { upstream_dispatchers: HashMap>, /// Fragments with `StreamSource` stream_source_fragment_ids: HashSet, - /// Fragments with `StreamSourceBackfill` - stream_source_backfill_fragment_ids: HashSet, + /// Fragments with `StreamSourceBackfill` and the corresponding upstream source fragment + stream_source_backfill_fragment_ids: HashMap, /// Target fragments in `NoShuffle` relation no_shuffle_target_fragment_ids: HashSet, /// Source fragments in `NoShuffle` relation @@ -696,7 +696,7 @@ impl ScaleController { } let mut stream_source_fragment_ids = HashSet::new(); - let mut stream_source_backfill_fragment_ids = HashSet::new(); + let mut stream_source_backfill_fragment_ids = HashMap::new(); let mut no_shuffle_reschedule = HashMap::new(); for (fragment_id, WorkerReschedule { worker_actor_diff }) in &*reschedule { let fragment = fragment_map @@ -821,8 +821,11 @@ impl ScaleController { // SourceScan is always a NoShuffle downstream, rescheduled together with the upstream Source. if (fragment.get_fragment_type_mask() & FragmentTypeFlag::SourceScan as u32) != 0 { let stream_node = fragment.actor_template.nodes.as_ref().unwrap(); - if stream_node.find_source_backfill().is_some() { - stream_source_backfill_fragment_ids.insert(fragment.fragment_id); + if let Some((_source_id, upstream_source_fragment_id)) = + stream_node.find_source_backfill() + { + stream_source_backfill_fragment_ids + .insert(fragment.fragment_id, upstream_source_fragment_id); } } } @@ -1257,17 +1260,14 @@ impl ScaleController { for fragment_id in reschedules.keys() { let actors_after_reschedule = &fragment_actors_after_reschedule[fragment_id]; - if ctx - .stream_source_backfill_fragment_ids - .contains(fragment_id) + if let Some(upstream_source_fragment_id) = + ctx.stream_source_backfill_fragment_ids.get(fragment_id) { - let fragment = &ctx.fragment_map[fragment_id]; - let curr_actor_ids = actors_after_reschedule.keys().cloned().collect_vec(); let actor_splits = self.source_manager.migrate_splits_for_backfill_actors( *fragment_id, - &fragment.upstream_fragment_ids, + *upstream_source_fragment_id, &curr_actor_ids, &fragment_actor_splits, &no_shuffle_upstream_actor_map, diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index f0fc22aadd92c..ddd65bf9352fc 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -392,7 +392,7 @@ impl SourceManagerCore { }; let actors = match self .metadata_manager - .get_running_actors_and_upstream_actors_of_fragment(*fragment_id) + .get_running_actors_for_source_backfill(*fragment_id) .await { Ok(actors) => { @@ -653,19 +653,16 @@ where } fn align_backfill_splits( - backfill_actors: impl IntoIterator)>, + backfill_actors: impl IntoIterator, upstream_assignment: &HashMap>, fragment_id: FragmentId, - upstream_fragment_id: FragmentId, + upstream_source_fragment_id: FragmentId, ) -> anyhow::Result>> { backfill_actors .into_iter() .map(|(actor_id, upstream_actor_id)| { - let err = || anyhow::anyhow!("source backfill actor should have one upstream actor, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_fragment_id}, actor_id: {actor_id}, upstream_assignment: {upstream_assignment:?}, upstream_actor_id: {upstream_actor_id:?}"); - if upstream_actor_id.len() != 1 { - return Err(err()); - } - let Some(splits) = upstream_assignment.get(&upstream_actor_id[0]) else { + let err = || anyhow::anyhow!("source backfill actor should have one upstream source actor, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_source_fragment_id}, actor_id: {actor_id}, upstream_assignment: {upstream_assignment:?}, upstream_actor_id: {upstream_actor_id:?}"); + let Some(splits) = upstream_assignment.get(&upstream_actor_id) else { return Err(err()); }; Ok(( @@ -844,28 +841,29 @@ impl SourceManager { pub fn migrate_splits_for_backfill_actors( &self, fragment_id: FragmentId, - upstream_fragment_ids: &Vec, + upstream_source_fragment_id: FragmentId, curr_actor_ids: &[ActorId], fragment_actor_splits: &HashMap>>, no_shuffle_upstream_actor_map: &HashMap>, ) -> MetaResult>> { // align splits for backfill fragments with its upstream source fragment - debug_assert!(upstream_fragment_ids.len() == 1); - let upstream_fragment_id = upstream_fragment_ids[0]; let actors = no_shuffle_upstream_actor_map .iter() .filter(|(id, _)| curr_actor_ids.contains(id)) .map(|(id, upstream_fragment_actors)| { - debug_assert!(upstream_fragment_actors.len() == 1); ( *id, - vec![*upstream_fragment_actors.get(&upstream_fragment_id).unwrap()], + *upstream_fragment_actors + .get(&upstream_source_fragment_id) + .unwrap(), ) }); - let upstream_assignment = fragment_actor_splits.get(&upstream_fragment_id).unwrap(); + let upstream_assignment = fragment_actor_splits + .get(&upstream_source_fragment_id) + .unwrap(); tracing::info!( fragment_id, - upstream_fragment_id, + upstream_source_fragment_id, ?upstream_assignment, "migrate_splits_for_backfill_actors" ); @@ -873,7 +871,7 @@ impl SourceManager { actors, upstream_assignment, fragment_id, - upstream_fragment_id, + upstream_source_fragment_id, )?) } @@ -960,19 +958,19 @@ impl SourceManager { let mut assigned = HashMap::new(); for (_source_id, fragments) in source_backfill_fragments { - for (fragment_id, upstream_fragment_id) in fragments { + for (fragment_id, upstream_source_fragment_id) in fragments { let upstream_actors = core .metadata_manager - .get_running_actors_of_fragment(upstream_fragment_id) + .get_running_actors_of_fragment(upstream_source_fragment_id) .await?; let mut backfill_actors = vec![]; for upstream_actor in upstream_actors { if let Some(dispatchers) = dispatchers.get(&upstream_actor) { let err = || { anyhow::anyhow!( - "source backfill fragment's upstream fragment should have one dispatcher, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_fragment_id}, upstream_actor: {upstream_actor}, dispatchers: {dispatchers:?}", + "source backfill fragment's upstream fragment should have one dispatcher, fragment_id: {fragment_id}, upstream_fragment_id: {upstream_source_fragment_id}, upstream_actor: {upstream_actor}, dispatchers: {dispatchers:?}", fragment_id = fragment_id, - upstream_fragment_id = upstream_fragment_id, + upstream_source_fragment_id = upstream_source_fragment_id, upstream_actor = upstream_actor, dispatchers = dispatchers ) @@ -982,7 +980,7 @@ impl SourceManager { } backfill_actors - .push((dispatchers[0].downstream_actor_id[0], vec![upstream_actor])); + .push((dispatchers[0].downstream_actor_id[0], upstream_actor)); } } assigned.insert( @@ -991,7 +989,7 @@ impl SourceManager { backfill_actors, upstream_assignment, fragment_id, - upstream_fragment_id, + upstream_source_fragment_id, )?, ); } diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index 15a0d4b4ff1ba..e204600729898 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -289,12 +289,27 @@ impl stream_plan::StreamNode { /// Find the external stream source info inside the stream node, if any. /// - /// Returns `source_id`. - pub fn find_source_backfill(&self) -> Option { + /// Returns (`source_id`, `upstream_source_fragment_id`). + /// + /// Note: we must get upstream fragment id from the merge node, not from the fragment's + /// `upstream_fragment_ids`. e.g., DynamicFilter may have 2 upstream fragments, but only + /// one is the upstream source fragment. + pub fn find_source_backfill(&self) -> Option<(u32, u32)> { if let Some(crate::stream_plan::stream_node::NodeBody::SourceBackfill(source)) = self.node_body.as_ref() { - return Some(source.upstream_source_id); + if let crate::stream_plan::stream_node::NodeBody::Merge(merge) = + self.input[0].node_body.as_ref().unwrap() + { + // Note: avoid using `merge.upstream_actor_id` to prevent misuse. + // See comments there for details. + return Some((source.upstream_source_id, merge.upstream_fragment_id)); + } else { + unreachable!( + "source backfill must have a merge node as its input: {:?}", + self + ); + } } for child in &self.input { diff --git a/src/tests/simulation/src/ctl_ext.rs b/src/tests/simulation/src/ctl_ext.rs index 2a022165c8525..acb49a0df4d09 100644 --- a/src/tests/simulation/src/ctl_ext.rs +++ b/src/tests/simulation/src/ctl_ext.rs @@ -361,14 +361,12 @@ impl Cluster { Ok(response) } - /// `table_id -> actor_id -> splits` - pub async fn list_source_splits(&self) -> Result>> { + /// `actor_id -> splits` + pub async fn list_source_splits(&self) -> Result> { let info = self.get_cluster_info().await?; let mut res = BTreeMap::new(); for table in info.table_fragments { - let mut table_actor_splits = BTreeMap::new(); - for (actor_id, splits) in table.actor_splits { let splits = splits .splits @@ -377,9 +375,8 @@ impl Cluster { .map(|split| split.id()) .collect_vec() .join(","); - table_actor_splits.insert(actor_id, splits); + res.insert(actor_id, splits); } - res.insert(table.table_id, table_actor_splits); } Ok(res) diff --git a/src/tests/simulation/tests/integration_tests/scale/shared_source.rs b/src/tests/simulation/tests/integration_tests/scale/shared_source.rs index 76aaae6a716a8..8d9e626c4119f 100644 --- a/src/tests/simulation/tests/integration_tests/scale/shared_source.rs +++ b/src/tests/simulation/tests/integration_tests/scale/shared_source.rs @@ -12,13 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeMap; +use std::collections::HashMap; use anyhow::Result; use itertools::Itertools; use maplit::{convert_args, hashmap}; use risingwave_common::hash::WorkerSlotId; use risingwave_pb::meta::table_fragments::Fragment; +use risingwave_pb::stream_plan::DispatcherType; use risingwave_simulation::cluster::{Cluster, Configuration}; use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; @@ -29,11 +30,38 @@ CREATE SOURCE s(v1 int, v2 varchar) WITH ( topic='shared_source' ) FORMAT PLAIN ENCODE JSON;"#; -fn actor_upstream(fragment: &Fragment) -> Vec<(u32, Vec)> { - fragment +/// Returns `Vec<(backfill_actor_id, source_actor_id)>` +fn source_backfill_upstream( + source_backfill_fragment: &Fragment, + source_fragment: &Fragment, +) -> Vec<(u32, u32)> { + let mut no_shuffle_downstream_to_upstream = HashMap::new(); + for source_actor in &source_fragment.actors { + for dispatcher in &source_actor.dispatcher { + if dispatcher.r#type == DispatcherType::NoShuffle as i32 { + assert_eq!(dispatcher.downstream_actor_id.len(), 1); + let downstream_actor_id = dispatcher.downstream_actor_id[0]; + no_shuffle_downstream_to_upstream + .insert(downstream_actor_id, source_actor.actor_id); + } + } + } + + source_backfill_fragment .actors .iter() - .map(|actor| (actor.actor_id, actor.upstream_actor_id.clone())) + .map(|backfill_actor| { + let (_, source_fragment_id) = backfill_actor + .get_nodes() + .unwrap() + .find_source_backfill() + .unwrap(); + assert_eq!(source_fragment.fragment_id, source_fragment_id); + ( + backfill_actor.actor_id, + no_shuffle_downstream_to_upstream[&backfill_actor.actor_id], + ) + }) .collect_vec() } @@ -41,9 +69,16 @@ async fn validate_splits_aligned(cluster: &mut Cluster) -> Result<()> { let source_backfill_fragment = cluster .locate_one_fragment([identity_contains("StreamSourceScan")]) .await?; + let source_fragment = cluster + .locate_one_fragment([ + identity_contains("Source"), + no_identity_contains("StreamSourceScan"), + ]) + .await?; // The result of scaling is non-deterministic. // So we just print the result here, instead of asserting with a fixed value. - let actor_upstream = actor_upstream(&source_backfill_fragment.inner); + let actor_upstream = + source_backfill_upstream(&source_backfill_fragment.inner, &source_fragment.inner); tracing::info!( "{}", actor_upstream @@ -53,18 +88,12 @@ async fn validate_splits_aligned(cluster: &mut Cluster) -> Result<()> { actor_id, upstream ))) ); - let splits = cluster.list_source_splits().await?; - tracing::info!("{:#?}", splits); - let actor_splits: BTreeMap = splits - .values() - .flat_map(|m| m.clone().into_iter()) - .collect(); + let actor_splits = cluster.list_source_splits().await?; + tracing::info!("{:#?}", actor_splits); for (actor, upstream) in actor_upstream { - assert!(upstream.len() == 1, "invalid upstream: {:?}", upstream); - let upstream_actor = upstream[0]; assert_eq!( actor_splits.get(&actor).unwrap(), - actor_splits.get(&upstream_actor).unwrap() + actor_splits.get(&upstream).unwrap() ); } Ok(()) @@ -112,10 +141,10 @@ async fn test_shared_source() -> Result<()> { .collect_vec(); validate_splits_aligned(&mut cluster).await?; expect_test::expect![[r#" - 1 6 HASH {7} {} {SOURCE} 6 256 - 2 8 HASH {9,8} {3} {MVIEW} 6 256 - 3 8 HASH {10} {1} {SOURCE_SCAN} 6 256"#]] - .assert_eq(&cluster.run("select * from rw_fragments;").await?); + 1 6 HASH {SOURCE} 6 256 + 2 8 HASH {MVIEW} 6 256 + 3 8 HASH {SOURCE_SCAN} 6 256"#]] + .assert_eq(&cluster.run("select fragment_id, table_id, distribution_type, flags, parallelism, max_parallelism from rw_fragments;").await?); expect_test::expect![[r#" 6 CREATED ADAPTIVE 256 8 CREATED ADAPTIVE 256"#]] @@ -137,10 +166,10 @@ async fn test_shared_source() -> Result<()> { .await .unwrap(); expect_test::expect![[r#" - 1 6 HASH {7} {} {SOURCE} 6 256 - 2 8 HASH {9,8} {3} {MVIEW} 5 256 - 3 8 HASH {10} {1} {SOURCE_SCAN} 6 256"#]] - .assert_eq(&cluster.run("select * from rw_fragments;").await?); + 1 6 HASH {SOURCE} 6 256 + 2 8 HASH {MVIEW} 5 256 + 3 8 HASH {SOURCE_SCAN} 6 256"#]] + .assert_eq(&cluster.run("select fragment_id, table_id, distribution_type, flags, parallelism, max_parallelism from rw_fragments;").await?); // source is the NoShuffle upstream. It can be scaled, and the downstream SourceBackfill will be scaled together. cluster @@ -156,10 +185,10 @@ async fn test_shared_source() -> Result<()> { .unwrap(); validate_splits_aligned(&mut cluster).await?; expect_test::expect![[r#" - 1 6 HASH {7} {} {SOURCE} 3 256 - 2 8 HASH {9,8} {3} {MVIEW} 5 256 - 3 8 HASH {10} {1} {SOURCE_SCAN} 3 256"#]] - .assert_eq(&cluster.run("select * from rw_fragments;").await?); + 1 6 HASH {SOURCE} 3 256 + 2 8 HASH {MVIEW} 5 256 + 3 8 HASH {SOURCE_SCAN} 3 256"#]] + .assert_eq(&cluster.run("select fragment_id, table_id, distribution_type, flags, parallelism, max_parallelism from rw_fragments;").await?); expect_test::expect![[r#" 6 CREATED CUSTOM 256 8 CREATED CUSTOM 256"#]] @@ -180,13 +209,131 @@ async fn test_shared_source() -> Result<()> { .unwrap(); validate_splits_aligned(&mut cluster).await?; expect_test::expect![[r#" - 1 6 HASH {7} {} {SOURCE} 7 256 - 2 8 HASH {9,8} {3} {MVIEW} 5 256 - 3 8 HASH {10} {1} {SOURCE_SCAN} 7 256"#]] - .assert_eq(&cluster.run("select * from rw_fragments;").await?); + 1 6 HASH {SOURCE} 7 256 + 2 8 HASH {MVIEW} 5 256 + 3 8 HASH {SOURCE_SCAN} 7 256"#]] + .assert_eq(&cluster.run("select fragment_id, table_id, distribution_type, flags, parallelism, max_parallelism from rw_fragments;").await?); expect_test::expect![[r#" 6 CREATED CUSTOM 256 8 CREATED CUSTOM 256"#]] .assert_eq(&cluster.run("select * from rw_table_fragments;").await?); Ok(()) } + +#[tokio::test] +async fn test_issue_19563() -> Result<()> { + tracing_subscriber::fmt::Subscriber::builder() + .with_max_level(tracing::Level::ERROR) + .with_env_filter("risingwave_stream::executor::source::source_backfill_executor=DEBUG,integration_tests=DEBUG") + .init(); + + let mut cluster = Cluster::start(Configuration::for_scale_shared_source()).await?; + cluster.create_kafka_topics(convert_args!(hashmap!( + "shared_source" => 4, + ))); + let mut session = cluster.start_session(); + + session.run("set rw_implicit_flush = true;").await?; + + session + .run( + r#" +CREATE SOURCE s(v1 timestamp with time zone) WITH ( + connector='kafka', + properties.bootstrap.server='192.168.11.1:29092', + topic='shared_source' +) FORMAT PLAIN ENCODE JSON;"#, + ) + .await?; + session + .run("create materialized view mv1 as select v1 from s where v1 between now() and now() + interval '1 day' * 365 * 2000;") + .await?; + let source_fragment = cluster + .locate_one_fragment([ + identity_contains("Source"), + no_identity_contains("StreamSourceScan"), + ]) + .await?; + let source_workers = source_fragment.all_worker_count().into_keys().collect_vec(); + let source_backfill_dynamic_filter_fragment = cluster + .locate_one_fragment([ + identity_contains("StreamSourceScan"), + identity_contains("StreamDynamicFilter"), + ]) + .await?; + let source_backfill_workers = source_backfill_dynamic_filter_fragment + .all_worker_count() + .into_keys() + .collect_vec(); + validate_splits_aligned(&mut cluster).await?; + expect_test::expect![[r#" + 1 6 HASH {SOURCE} 6 256 + 2 8 HASH {MVIEW,SOURCE_SCAN} 6 256 + 3 8 SINGLE {NOW} 1 1 + 4 8 SINGLE {NOW} 1 1"#]] + .assert_eq(&cluster.run("select fragment_id, table_id, distribution_type, flags, parallelism, max_parallelism from rw_fragments;").await?); + expect_test::expect![[r#" + 6 CREATED ADAPTIVE 256 + 8 CREATED ADAPTIVE 256"#]] + .assert_eq(&cluster.run("select * from rw_table_fragments;").await?); + + // SourceBackfill/DynamicFilter cannot be scaled because of NoShuffle. + assert!( + &cluster + .reschedule( + source_backfill_dynamic_filter_fragment + .reschedule([WorkerSlotId::new(source_backfill_workers[0], 0)], []), + ) + .await.unwrap_err().to_string().contains("rescheduling NoShuffle downstream fragment (maybe Chain fragment) is forbidden, please use NoShuffle upstream fragment (like Materialized fragment) to scale"), + ); + + // source is the NoShuffle upstream. It can be scaled, and the downstream SourceBackfill/DynamicFilter will be scaled together. + cluster + .reschedule(source_fragment.reschedule( + [ + WorkerSlotId::new(source_workers[0], 0), + WorkerSlotId::new(source_workers[0], 1), + WorkerSlotId::new(source_workers[2], 0), + ], + [], + )) + .await + .unwrap(); + validate_splits_aligned(&mut cluster).await?; + expect_test::expect![[r#" + 1 6 HASH {SOURCE} 3 256 + 2 8 HASH {MVIEW,SOURCE_SCAN} 3 256 + 3 8 SINGLE {NOW} 1 1 + 4 8 SINGLE {NOW} 1 1"#]] + .assert_eq(&cluster.run("select fragment_id, table_id, distribution_type, flags, parallelism, max_parallelism from rw_fragments;").await?); + expect_test::expect![[r#" + 6 CREATED CUSTOM 256 + 8 CREATED ADAPTIVE 256"#]] + .assert_eq(&cluster.run("select * from rw_table_fragments;").await?); + + // resolve_no_shuffle for backfill fragment is OK, which will scale the upstream together. + cluster + .reschedule_resolve_no_shuffle(source_backfill_dynamic_filter_fragment.reschedule( + [], + [ + WorkerSlotId::new(source_workers[0], 0), + WorkerSlotId::new(source_workers[0], 1), + WorkerSlotId::new(source_workers[2], 0), + WorkerSlotId::new(source_workers[2], 1), + ], + )) + .await + .unwrap(); + validate_splits_aligned(&mut cluster).await?; + expect_test::expect![[r#" + 1 6 HASH {SOURCE} 7 256 + 2 8 HASH {MVIEW,SOURCE_SCAN} 7 256 + 3 8 SINGLE {NOW} 1 1 + 4 8 SINGLE {NOW} 1 1"#]] + .assert_eq(&cluster.run("select fragment_id, table_id, distribution_type, flags, parallelism, max_parallelism from rw_fragments;").await?); + expect_test::expect![[r#" + 6 CREATED CUSTOM 256 + 8 CREATED ADAPTIVE 256"#]] + .assert_eq(&cluster.run("select * from rw_table_fragments;").await?); + Ok(()) +} From 9643ff9097fd7ea6a1d0ed47c1f0ce15d2f9e9f0 Mon Sep 17 00:00:00 2001 From: Yuhao Su <31772373+yuhao-su@users.noreply.github.com> Date: Wed, 27 Nov 2024 22:31:29 +0800 Subject: [PATCH 033/163] fix(secret): the name of a secret in the catalog should not contains the schema name (#19584) --- e2e_test/ddl/secret.slt | 8 ++++++++ src/frontend/src/handler/create_secret.rs | 2 +- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/e2e_test/ddl/secret.slt b/e2e_test/ddl/secret.slt index cf9d16214da38..493fba90283c5 100644 --- a/e2e_test/ddl/secret.slt +++ b/e2e_test/ddl/secret.slt @@ -73,3 +73,11 @@ secret_1 statement ok drop secret secret_1; + +statement ok +create secret public.secret_1 with ( + backend = 'meta' +) as 'demo_secret'; + +statement ok +drop secret secret_1; \ No newline at end of file diff --git a/src/frontend/src/handler/create_secret.rs b/src/frontend/src/handler/create_secret.rs index 6d5a6283e3288..27a6b37fca277 100644 --- a/src/frontend/src/handler/create_secret.rs +++ b/src/frontend/src/handler/create_secret.rs @@ -58,7 +58,7 @@ pub async fn handle_create_secret( let catalog_writer = session.catalog_writer()?; catalog_writer .create_secret( - stmt.secret_name.real_value(), + secret_name, database_id, schema_id, session.user_id(), From ba225b21dbdf6f24d3f3086c03a18859b9949d73 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 28 Nov 2024 00:49:06 +0800 Subject: [PATCH 034/163] fix(planner): correctly handle hidden columns for SourceBackfill (#19578) Signed-off-by: xxchan --- .pre-commit-config.yaml | 1 + ci/scripts/e2e-source-test.sh | 2 +- .../kafka/issue_19563.slt.serial | 6 +- .../src/parser/additional_columns.rs | 14 ++- src/connector/src/source/reader/desc.rs | 2 +- .../tests/testdata/output/batch_source.yaml | 4 +- .../tests/testdata/output/shared_source.yml | 12 +- src/frontend/src/handler/create_source.rs | 113 +++++++++++++----- src/frontend/src/handler/create_table.rs | 1 + src/frontend/src/handler/show.rs | 79 +++++++++--- .../src/optimizer/optimizer_context.rs | 2 +- .../src/optimizer/plan_node/stream_source.rs | 22 +--- .../optimizer/plan_node/stream_source_scan.rs | 22 +--- 13 files changed, 176 insertions(+), 104 deletions(-) diff --git a/.pre-commit-config.yaml b/.pre-commit-config.yaml index cb54c1606356e..78574872d8f01 100644 --- a/.pre-commit-config.yaml +++ b/.pre-commit-config.yaml @@ -6,6 +6,7 @@ repos: rev: v2.3.0 hooks: - id: end-of-file-fixer + exclude: 'src/frontend/planner_test/tests/testdata/.*' - id: trailing-whitespace - repo: https://github.com/crate-ci/typos rev: v1.23.1 diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 4491db5633ea8..b6ea8267a1690 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -36,7 +36,7 @@ python3 -m pip install --break-system-packages requests protobuf fastavro conflu apt-get -y install jq echo "--- e2e, inline test" -RUST_LOG="debug,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ +RUST_LOG="debug,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info,risingwave_meta=info" \ risedev ci-start ci-inline-source-test risedev slt './e2e_test/source_inline/**/*.slt' -j16 risedev slt './e2e_test/source_inline/**/*.slt.serial' diff --git a/e2e_test/source_inline/kafka/issue_19563.slt.serial b/e2e_test/source_inline/kafka/issue_19563.slt.serial index 91b795d6acf34..528fac0735767 100644 --- a/e2e_test/source_inline/kafka/issue_19563.slt.serial +++ b/e2e_test/source_inline/kafka/issue_19563.slt.serial @@ -24,9 +24,9 @@ explain create materialized view mv1 as select v1 from kafkasource where v1 betw StreamMaterialize { columns: [v1, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamDynamicFilter { predicate: (v1 <= $expr1), output: [v1, _row_id], cleaned_by_watermark: true } ├─StreamProject { exprs: [v1, _row_id], output_watermarks: [v1] } - │ └─StreamDynamicFilter { predicate: (v1 >= now), output_watermarks: [v1], output: [v1, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset], cleaned_by_watermark: true } - │ ├─StreamRowIdGen { row_id_index: 2 } - │ │ └─StreamSourceScan { columns: [v1, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + │ └─StreamDynamicFilter { predicate: (v1 >= now), output_watermarks: [v1], output: [v1, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], cleaned_by_watermark: true } + │ ├─StreamRowIdGen { row_id_index: 4 } + │ │ └─StreamSourceScan { columns: [v1, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } │ └─StreamExchange { dist: Broadcast } │ └─StreamNow └─StreamExchange { dist: Broadcast } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index e5712d95066cb..62febde16243a 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -281,9 +281,20 @@ pub fn build_additional_column_desc( pub fn source_add_partition_offset_cols( columns: &[ColumnCatalog], connector_name: &str, + skip_col_id: bool, ) -> ([bool; 2], [ColumnDesc; 2]) { let mut columns_exist = [false; 2]; + let mut last_column_id = max_column_id(columns); + let mut assign_col_id = || { + if skip_col_id { + // col id will be filled outside later. Here just use a placeholder. + ColumnId::placeholder() + } else { + last_column_id = last_column_id.next(); + last_column_id + } + }; let additional_columns: Vec<_> = { let compat_col_types = COMPATIBLE_ADDITIONAL_COLUMNS @@ -292,11 +303,10 @@ pub fn source_add_partition_offset_cols( ["partition", "file", "offset"] .iter() .filter_map(|col_type| { - last_column_id = last_column_id.next(); if compat_col_types.contains(col_type) { Some( build_additional_column_desc( - last_column_id, + assign_col_id(), connector_name, col_type, None, diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index af607d2537ea6..d64435ef3a3cc 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -91,7 +91,7 @@ impl SourceDescBuilder { .map(|s| s.to_lowercase()) .unwrap(); let (columns_exist, additional_columns) = - source_add_partition_offset_cols(&self.columns, &connector_name); + source_add_partition_offset_cols(&self.columns, &connector_name, false); let mut columns: Vec<_> = self .columns diff --git a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml index 63daa9c43b2db..2b7f23dff932e 100644 --- a/src/frontend/planner_test/tests/testdata/output/batch_source.yaml +++ b/src/frontend/planner_test/tests/testdata/output/batch_source.yaml @@ -3,11 +3,11 @@ select * from s logical_plan: |- LogicalProject { exprs: [id, value] } - └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id] } + └─LogicalSource { source: s, columns: [id, value, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [id, value] } - └─BatchKafkaScan { source: s, columns: [id, value, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [id, value, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } create_source: format: plain encode: protobuf diff --git a/src/frontend/planner_test/tests/testdata/output/shared_source.yml b/src/frontend/planner_test/tests/testdata/output/shared_source.yml index 83fde26bfc7d2..fa75610ff09e3 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -68,12 +68,12 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } with_config_map: streaming_use_shared_source: 'true' - before: @@ -84,11 +84,11 @@ batch_plan: |- BatchExchange { order: [], dist: Single } └─BatchProject { exprs: [x, y] } - └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id], filter: (None, None) } + └─BatchKafkaScan { source: s, columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } stream_plan: |- StreamMaterialize { columns: [x, y, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } └─StreamProject { exprs: [x, y, _row_id] } - └─StreamRowIdGen { row_id_index: 3 } - └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } + └─StreamRowIdGen { row_id_index: 5 } + └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } with_config_map: streaming_use_shared_source: 'true' diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 1f89494f64ce6..d9f4fcce5ff85 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -31,8 +31,10 @@ use risingwave_common::catalog::{ use risingwave_common::license::Feature; use risingwave_common::secret::LocalSecretManager; use risingwave_common::types::DataType; +use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::parser::additional_columns::{ build_additional_column_desc, get_supported_additional_columns, + source_add_partition_offset_cols, }; use risingwave_connector::parser::{ fetch_json_schema_and_map_to_columns, AvroParserConfig, DebeziumAvroParserConfig, @@ -1493,6 +1495,7 @@ pub async fn bind_create_source_or_table_with_connector( col_id_gen: &mut ColumnIdGenerator, // `true` for "create source", `false` for "create table with connector" is_create_source: bool, + is_shared_non_cdc: bool, source_rate_limit: Option, ) -> Result<(SourceCatalog, DatabaseId, SchemaId)> { let session = &handler_args.session; @@ -1554,6 +1557,21 @@ pub async fn bind_create_source_or_table_with_connector( if is_create_source { // must behind `handle_addition_columns` check_and_add_timestamp_column(&with_properties, &mut columns); + + // For shared sources, we will include partition and offset cols in the SourceExecutor's *output*, to be used by the SourceBackfillExecutor. + // For shared CDC source, the schema is different. See debezium_cdc_source_schema, CDC_BACKFILL_TABLE_ADDITIONAL_COLUMNS + if is_shared_non_cdc { + let (columns_exist, additional_columns) = source_add_partition_offset_cols( + &columns, + &with_properties.get_connector().unwrap(), + true, // col_id filled below at col_id_gen.generate + ); + for (existed, c) in columns_exist.into_iter().zip_eq_fast(additional_columns) { + if !existed { + columns.push(ColumnCatalog::hidden(c)); + } + } + } } // resolve privatelink connection for Kafka @@ -1670,14 +1688,14 @@ pub async fn handle_create_source( let with_properties = bind_connector_props(&handler_args, &format_encode, true)?; let create_cdc_source_job = with_properties.is_shareable_cdc_connector(); - let is_shared = create_cdc_source_job - || (with_properties.is_shareable_non_cdc_connector() - && session - .env() - .streaming_config() - .developer - .enable_shared_source - && session.config().streaming_use_shared_source()); + let is_shared_non_cdc = with_properties.is_shareable_non_cdc_connector() + && session + .env() + .streaming_config() + .developer + .enable_shared_source + && session.config().streaming_use_shared_source(); + let is_shared = create_cdc_source_job || is_shared_non_cdc; let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job { bind_columns_from_source_for_cdc(&session, &format_encode)? @@ -1705,6 +1723,7 @@ pub async fn handle_create_source( stmt.include_column_options, &mut col_id_gen, true, + is_shared_non_cdc, overwrite_options.source_rate_limit, ) .await?; @@ -1777,8 +1796,7 @@ pub mod tests { use std::sync::Arc; use risingwave_common::catalog::{ - CDC_SOURCE_COLUMN_NUM, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, OFFSET_COLUMN_NAME, - ROWID_PREFIX, TABLE_NAME_COLUMN_NAME, + CDC_SOURCE_COLUMN_NUM, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, ROWID_PREFIX, }; use risingwave_common::types::{DataType, StructType}; @@ -1932,15 +1950,29 @@ pub mod tests { .columns .iter() .map(|col| (col.name(), col.data_type().clone())) - .collect::>(); + .collect::>(); - let expected_columns = maplit::hashmap! { - ROWID_PREFIX => DataType::Serial, - "payload" => DataType::Jsonb, - OFFSET_COLUMN_NAME => DataType::Varchar, - TABLE_NAME_COLUMN_NAME => DataType::Varchar, - }; - assert_eq!(columns, expected_columns); + expect_test::expect![[r#" + [ + ( + "payload", + Jsonb, + ), + ( + "_rw_offset", + Varchar, + ), + ( + "_rw_table_name", + Varchar, + ), + ( + "_row_id", + Serial, + ), + ] + "#]] + .assert_debug_eq(&columns); } #[tokio::test] @@ -1969,16 +2001,41 @@ pub mod tests { .unwrap(); assert_eq!(source.name, "s"); - let columns = GET_COLUMN_FROM_CATALOG(source); - let expect_columns = maplit::hashmap! { - ROWID_PREFIX => DataType::Serial, - "v1" => DataType::Int32, - "_rw_kafka_key" => DataType::Bytea, - // todo: kafka connector will automatically derive the column - // will change to a required field in the include clause - "_rw_kafka_timestamp" => DataType::Timestamptz, - }; - assert_eq!(columns, expect_columns); + let columns = source + .columns + .iter() + .map(|col| (col.name(), col.data_type().clone())) + .collect::>(); + + expect_test::expect![[r#" + [ + ( + "v1", + Int32, + ), + ( + "_rw_kafka_key", + Bytea, + ), + ( + "_rw_kafka_timestamp", + Timestamptz, + ), + ( + "_rw_kafka_partition", + Varchar, + ), + ( + "_rw_kafka_offset", + Varchar, + ), + ( + "_row_id", + Serial, + ), + ] + "#]] + .assert_debug_eq(&columns); let sql = "CREATE SOURCE s3 (v1 int) include timestamp 'header1' as header_col with (connector = 'kafka') format plain encode json" diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index a34c1cf462e0f..961be381b5f8c 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -506,6 +506,7 @@ pub(crate) async fn gen_create_table_plan_with_source( include_column_options, &mut col_id_gen, false, + false, rate_limit, ) .await?; diff --git a/src/frontend/src/handler/show.rs b/src/frontend/src/handler/show.rs index cbd2effb54b8c..248636addee3a 100644 --- a/src/frontend/src/handler/show.rs +++ b/src/frontend/src/handler/show.rs @@ -684,7 +684,6 @@ pub fn handle_show_create_object( #[cfg(test)] mod tests { - use std::collections::HashMap; use std::ops::Index; use futures_async_stream::for_await; @@ -720,36 +719,78 @@ mod tests { let sql = "show columns from t"; let mut pg_response = frontend.run_sql(sql).await.unwrap(); - let mut columns = HashMap::new(); + let mut columns = Vec::new(); #[for_await] for row_set in pg_response.values_stream() { let row_set = row_set.unwrap(); for row in row_set { - columns.insert( + columns.push(( std::str::from_utf8(row.index(0).as_ref().unwrap()) .unwrap() .to_string(), std::str::from_utf8(row.index(1).as_ref().unwrap()) .unwrap() .to_string(), - ); + )); } } - let expected_columns: HashMap = maplit::hashmap! { - "id".into() => "integer".into(), - "country.zipcode".into() => "character varying".into(), - "zipcode".into() => "bigint".into(), - "country.city.address".into() => "character varying".into(), - "country.address".into() => "character varying".into(), - "country.city".into() => "test.City".into(), - "country.city.zipcode".into() => "character varying".into(), - "rate".into() => "real".into(), - "country".into() => "test.Country".into(), - "_rw_kafka_timestamp".into() => "timestamp with time zone".into(), - "_row_id".into() => "serial".into(), - }; - - assert_eq!(columns, expected_columns); + expect_test::expect![[r#" + [ + ( + "id", + "integer", + ), + ( + "country", + "test.Country", + ), + ( + "country.address", + "character varying", + ), + ( + "country.city", + "test.City", + ), + ( + "country.city.address", + "character varying", + ), + ( + "country.city.zipcode", + "character varying", + ), + ( + "country.zipcode", + "character varying", + ), + ( + "zipcode", + "bigint", + ), + ( + "rate", + "real", + ), + ( + "_rw_kafka_timestamp", + "timestamp with time zone", + ), + ( + "_rw_kafka_partition", + "character varying", + ), + ( + "_rw_kafka_offset", + "character varying", + ), + ( + "_row_id", + "serial", + ), + ] + "#]] + .assert_debug_eq(&columns); } } diff --git a/src/frontend/src/optimizer/optimizer_context.rs b/src/frontend/src/optimizer/optimizer_context.rs index 75af6a3b3da69..2dfe766224ecb 100644 --- a/src/frontend/src/optimizer/optimizer_context.rs +++ b/src/frontend/src/optimizer/optimizer_context.rs @@ -191,7 +191,7 @@ impl OptimizerContext { } let mut optimizer_trace = self.optimizer_trace.borrow_mut(); let string = str.into(); - tracing::trace!(target: "explain_trace", "{}", string); + tracing::trace!(target: "explain_trace", "\n{}", string); optimizer_trace.push(string); optimizer_trace.push("\n".to_string()); } diff --git a/src/frontend/src/optimizer/plan_node/stream_source.rs b/src/frontend/src/optimizer/plan_node/stream_source.rs index d7808e4be51ce..909fa1e0d3009 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source.rs @@ -17,9 +17,6 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::ColumnCatalog; -use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_connector::parser::additional_columns::source_add_partition_offset_cols; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use risingwave_pb::stream_plan::{PbStreamSource, SourceNode}; @@ -40,24 +37,7 @@ pub struct StreamSource { } impl StreamSource { - pub fn new(mut core: generic::Source) -> Self { - // For shared sources, we will include partition and offset cols in the *output*, to be used by the SourceBackfillExecutor. - // XXX: If we don't add here, these cols are also added in source reader, but pruned in the SourceExecutor's output. - // Should we simply add them here for all sources for consistency? - if let Some(source_catalog) = &core.catalog - && source_catalog.info.is_shared() - { - let (columns_exist, additional_columns) = source_add_partition_offset_cols( - &core.column_catalog, - &source_catalog.connector_name(), - ); - for (existed, c) in columns_exist.into_iter().zip_eq_fast(additional_columns) { - if !existed { - core.column_catalog.push(ColumnCatalog::hidden(c)); - } - } - } - + pub fn new(core: generic::Source) -> Self { let base = PlanBase::new_stream_with_core( &core, Distribution::SomeShard, diff --git a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs index 83c79259952b2..ade1a4b6f2fee 100644 --- a/src/frontend/src/optimizer/plan_node/stream_source_scan.rs +++ b/src/frontend/src/optimizer/plan_node/stream_source_scan.rs @@ -17,11 +17,9 @@ use std::rc::Rc; use fixedbitset::FixedBitSet; use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_common::catalog::{ColumnCatalog, Field}; +use risingwave_common::catalog::Field; use risingwave_common::types::DataType; -use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::OrderType; -use risingwave_connector::parser::additional_columns::source_add_partition_offset_cols; use risingwave_pb::stream_plan::stream_node::{NodeBody, PbNodeBody}; use risingwave_pb::stream_plan::PbStreamNode; @@ -52,23 +50,7 @@ pub struct StreamSourceScan { impl_plan_tree_node_for_leaf! { StreamSourceScan } impl StreamSourceScan { - pub fn new(mut core: generic::Source) -> Self { - // XXX: do we need to include partition and offset cols here? It's needed by Backfill's input, but maybe not output? - // But the source's "schema" contains the hidden columns. - if let Some(source_catalog) = &core.catalog - && source_catalog.info.is_shared() - { - let (columns_exist, additional_columns) = source_add_partition_offset_cols( - &core.column_catalog, - &source_catalog.connector_name(), - ); - for (existed, c) in columns_exist.into_iter().zip_eq_fast(additional_columns) { - if !existed { - core.column_catalog.push(ColumnCatalog::hidden(c)); - } - } - } - + pub fn new(core: generic::Source) -> Self { let base = PlanBase::new_stream_with_core( &core, Distribution::SomeShard, From 42f1e1b1698c2f2a8bf617c824325293a6bb1f41 Mon Sep 17 00:00:00 2001 From: xxchan Date: Thu, 28 Nov 2024 08:31:54 +0800 Subject: [PATCH 035/163] test: add tpch/nexmark tests for shared kafka source (#19589) --- .../nexmark/create_sources_kafka.slt.part | 48 + e2e_test/nexmark/drop_sources_kafka.slt.part | 8 + e2e_test/nexmark/produce_kafka.slt.part | 58 + e2e_test/source_inline/kafka/nexmark.slt | 9 + e2e_test/source_inline/kafka/tpch.slt | 30 + e2e_test/tpch/create_sources_kafka.slt.part | 118 + e2e_test/tpch/drop_sources_kafka.slt.part | 23 + e2e_test/tpch/produce_kafka.slt.part | 131 + .../testdata/input/nexmark_source_kafka.yaml | 718 +++++ .../tests/testdata/input/tpch_kafka.yaml | 902 ++++++ .../testdata/output/nexmark_source_kafka.yaml | 2470 +++++++++++++++++ .../tests/testdata/output/tpch_kafka.yaml | 1513 ++++++++++ 12 files changed, 6028 insertions(+) create mode 100644 e2e_test/nexmark/create_sources_kafka.slt.part create mode 100644 e2e_test/nexmark/drop_sources_kafka.slt.part create mode 100644 e2e_test/nexmark/produce_kafka.slt.part create mode 100644 e2e_test/source_inline/kafka/nexmark.slt create mode 100644 e2e_test/source_inline/kafka/tpch.slt create mode 100644 e2e_test/tpch/create_sources_kafka.slt.part create mode 100644 e2e_test/tpch/drop_sources_kafka.slt.part create mode 100644 e2e_test/tpch/produce_kafka.slt.part create mode 100644 src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml create mode 100644 src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml create mode 100644 src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml diff --git a/e2e_test/nexmark/create_sources_kafka.slt.part b/e2e_test/nexmark/create_sources_kafka.slt.part new file mode 100644 index 0000000000000..71e05c9134b80 --- /dev/null +++ b/e2e_test/nexmark/create_sources_kafka.slt.part @@ -0,0 +1,48 @@ +control substitution on + +statement ok +CREATE SOURCE person ( + "id" BIGINT, + "name" VARCHAR, + "email_address" VARCHAR, + "credit_card" VARCHAR, + "city" VARCHAR, + "state" VARCHAR, + "date_time" TIMESTAMP, + "extra" VARCHAR, +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-person' +) FORMAT PLAIN ENCODE JSON; + + +statement ok +CREATE SOURCE auction ( + "id" BIGINT, + "item_name" VARCHAR, + "description" VARCHAR, + "initial_bid" BIGINT, + "reserve" BIGINT, + "date_time" TIMESTAMP, + "expires" TIMESTAMP, + "seller" BIGINT, + "category" BIGINT, + "extra" VARCHAR, +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-auction' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE bid ( + "auction" BIGINT, + "bidder" BIGINT, + "price" BIGINT, + "channel" VARCHAR, + "url" VARCHAR, + "date_time" TIMESTAMP, + "extra" VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-bid' +) FORMAT PLAIN ENCODE JSON; diff --git a/e2e_test/nexmark/drop_sources_kafka.slt.part b/e2e_test/nexmark/drop_sources_kafka.slt.part new file mode 100644 index 0000000000000..998b1b07c12d1 --- /dev/null +++ b/e2e_test/nexmark/drop_sources_kafka.slt.part @@ -0,0 +1,8 @@ +statement ok +DROP SOURCE person CASCADE; + +statement ok +DROP SOURCE auction CASCADE; + +statement ok +DROP SOURCE bid CASCADE; diff --git a/e2e_test/nexmark/produce_kafka.slt.part b/e2e_test/nexmark/produce_kafka.slt.part new file mode 100644 index 0000000000000..f922a39831e1c --- /dev/null +++ b/e2e_test/nexmark/produce_kafka.slt.part @@ -0,0 +1,58 @@ +control substitution on + +system ok +rpk topic delete -r nexmark-* || true + +system ok +rpk topic create nexmark-auction -p 4 && +rpk topic create nexmark-bid -p 4 && +rpk topic create nexmark-person -p 4 + +include ./create_tables.slt.part + +include ./insert_auction.slt.part +include ./insert_bid.slt.part +include ./insert_person.slt.part + +statement ok +flush; + +statement ok +create sink nexmark_auction FROM auction +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-auction' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink nexmark_bid FROM bid +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-bid' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink nexmark_person FROM person +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'nexmark-person' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +sleep 5s + +statement ok +DROP SINK nexmark_auction; + +statement ok +DROP SINK nexmark_bid; + +statement ok +DROP SINK nexmark_person; + +include ./drop_tables.slt.part diff --git a/e2e_test/source_inline/kafka/nexmark.slt b/e2e_test/source_inline/kafka/nexmark.slt new file mode 100644 index 0000000000000..9460a0190b534 --- /dev/null +++ b/e2e_test/source_inline/kafka/nexmark.slt @@ -0,0 +1,9 @@ +include ../../nexmark/produce_kafka.slt.part +include ../../nexmark/create_sources_kafka.slt.part + +control substitution off + +include ../../streaming/nexmark/create_views.slt.part +include ../../streaming/nexmark/test_mv_result.slt.part + +include ../../nexmark/drop_sources_kafka.slt.part diff --git a/e2e_test/source_inline/kafka/tpch.slt b/e2e_test/source_inline/kafka/tpch.slt new file mode 100644 index 0000000000000..85fed5034af4d --- /dev/null +++ b/e2e_test/source_inline/kafka/tpch.slt @@ -0,0 +1,30 @@ +include ../../tpch/produce_kafka.slt.part +include ../../tpch/create_sources_kafka.slt.part + +control substitution off + +include ../../streaming/tpch/create_views.slt.part +include ../../streaming/tpch/q1.slt.part +include ../../streaming/tpch/q2.slt.part +include ../../streaming/tpch/q3.slt.part +include ../../streaming/tpch/q4.slt.part +include ../../streaming/tpch/q5.slt.part +include ../../streaming/tpch/q6.slt.part +include ../../streaming/tpch/q7.slt.part +include ../../streaming/tpch/q8.slt.part +include ../../streaming/tpch/q9.slt.part +include ../../streaming/tpch/q10.slt.part +include ../../streaming/tpch/q11.slt.part +include ../../streaming/tpch/q12.slt.part +include ../../streaming/tpch/q13.slt.part +include ../../streaming/tpch/q14.slt.part +include ../../streaming/tpch/q15.slt.part +include ../../streaming/tpch/q16.slt.part +include ../../streaming/tpch/q17.slt.part +include ../../streaming/tpch/q18.slt.part +include ../../streaming/tpch/q19.slt.part +include ../../streaming/tpch/q20.slt.part +include ../../streaming/tpch/q21.slt.part +include ../../streaming/tpch/q22.slt.part + +include ../../tpch/drop_sources_kafka.slt.part diff --git a/e2e_test/tpch/create_sources_kafka.slt.part b/e2e_test/tpch/create_sources_kafka.slt.part new file mode 100644 index 0000000000000..9dcc50e7d1f0e --- /dev/null +++ b/e2e_test/tpch/create_sources_kafka.slt.part @@ -0,0 +1,118 @@ +control substitution on + +statement ok +CREATE SOURCE supplier ( + s_suppkey INTEGER, + s_name VARCHAR, + s_address VARCHAR, + s_nationkey INTEGER, + s_phone VARCHAR, + s_acctbal NUMERIC, + s_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-supplier' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE part ( + p_partkey INTEGER, + p_name VARCHAR, + p_mfgr VARCHAR, + p_brand VARCHAR, + p_type VARCHAR, + p_size INTEGER, + p_container VARCHAR, + p_retailprice NUMERIC, + p_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-part' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE partsupp ( + ps_partkey INTEGER, + ps_suppkey INTEGER, + ps_availqty INTEGER, + ps_supplycost NUMERIC, + ps_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-partsupp' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE customer ( + c_custkey INTEGER, + c_name VARCHAR, + c_address VARCHAR, + c_nationkey INTEGER, + c_phone VARCHAR, + c_acctbal NUMERIC, + c_mktsegment VARCHAR, + c_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-customer' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE orders ( + o_orderkey BIGINT, + o_custkey INTEGER, + o_orderstatus VARCHAR, + o_totalprice NUMERIC, + o_orderdate DATE, + o_orderpriority VARCHAR, + o_clerk VARCHAR, + o_shippriority INTEGER, + o_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-orders' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE lineitem ( + l_orderkey BIGINT, + l_partkey INTEGER, + l_suppkey INTEGER, + l_linenumber INTEGER, + l_quantity NUMERIC, + l_extendedprice NUMERIC, + l_discount NUMERIC, + l_tax NUMERIC, + l_returnflag VARCHAR, + l_linestatus VARCHAR, + l_shipdate DATE, + l_commitdate DATE, + l_receiptdate DATE, + l_shipinstruct VARCHAR, + l_shipmode VARCHAR, + l_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-lineitem' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE nation ( + n_nationkey INTEGER, + n_name VARCHAR, + n_regionkey INTEGER, + n_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-nation' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SOURCE region ( + r_regionkey INTEGER, + r_name VARCHAR, + r_comment VARCHAR +) WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-region' +) FORMAT PLAIN ENCODE JSON; diff --git a/e2e_test/tpch/drop_sources_kafka.slt.part b/e2e_test/tpch/drop_sources_kafka.slt.part new file mode 100644 index 0000000000000..2c239fcbb422f --- /dev/null +++ b/e2e_test/tpch/drop_sources_kafka.slt.part @@ -0,0 +1,23 @@ +statement ok +DROP SOURCE supplier CASCADE; + +statement ok +DROP SOURCE region CASCADE; + +statement ok +DROP SOURCE nation CASCADE; + +statement ok +DROP SOURCE lineitem CASCADE; + +statement ok +DROP SOURCE orders CASCADE; + +statement ok +DROP SOURCE customer CASCADE; + +statement ok +DROP SOURCE partsupp CASCADE; + +statement ok +DROP SOURCE part CASCADE; diff --git a/e2e_test/tpch/produce_kafka.slt.part b/e2e_test/tpch/produce_kafka.slt.part new file mode 100644 index 0000000000000..07b476a3a9f30 --- /dev/null +++ b/e2e_test/tpch/produce_kafka.slt.part @@ -0,0 +1,131 @@ +control substitution on + +system ok +rpk topic delete -r tpch-* || true + +system ok +rpk topic create tpch-supplier -p 4 && +rpk topic create tpch-part -p 4 && +rpk topic create tpch-partsupp -p 4 && +rpk topic create tpch-customer -p 4 && +rpk topic create tpch-orders -p 4 && +rpk topic create tpch-lineitem -p 4 && +rpk topic create tpch-nation -p 4 && +rpk topic create tpch-region -p 4 + +include ./create_tables.slt.part + +include ./insert_supplier.slt.part +include ./insert_part.slt.part +include ./insert_partsupp.slt.part +include ./insert_customer.slt.part +include ./insert_orders.slt.part +include ./insert_lineitem.slt.part +include ./insert_nation.slt.part +include ./insert_region.slt.part + +statement ok +flush; + +statement ok +create sink kafka_supplier FROM supplier +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-supplier' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_part FROM part +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-part' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_partsupp FROM partsupp +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-partsupp' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_customer FROM customer +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-customer' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +# note: In source, Date format is days_since_unix_epoch. In sink, it's num_days_from_ce. +# https://github.com/risingwavelabs/risingwave/issues/16467 + +statement ok +create sink kafka_orders AS select * except(o_orderdate), o_orderdate::varchar as o_orderdate FROM orders +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-orders' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_lineitem AS select * except(l_shipdate, l_commitdate, l_receiptdate), l_shipdate::varchar as l_shipdate, l_commitdate::varchar as l_commitdate, l_receiptdate::varchar as l_receiptdate FROM lineitem +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-lineitem' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_nation FROM nation +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-nation' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +statement ok +create sink kafka_region FROM region +WITH ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'tpch-region' +) FORMAT PLAIN ENCODE JSON ( + force_append_only='true' +); + +sleep 5s + +statement ok +DROP SINK kafka_supplier; + +statement ok +DROP SINK kafka_part; + +statement ok +DROP SINK kafka_partsupp; + +statement ok +DROP SINK kafka_customer; + +statement ok +DROP SINK kafka_orders; + +statement ok +DROP SINK kafka_lineitem; + +statement ok +DROP SINK kafka_nation; + +statement ok +DROP SINK kafka_region; + +include ./drop_tables.slt.part diff --git a/src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml b/src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml new file mode 100644 index 0000000000000..7087bff1b785b --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/nexmark_source_kafka.yaml @@ -0,0 +1,718 @@ +- id: create_sources + sql: | + create source auction ( + id BIGINT, + item_name VARCHAR, + description VARCHAR, + initial_bid BIGINT, + reserve BIGINT, + date_time TIMESTAMP, + expires TIMESTAMP, + seller BIGINT, + category BIGINT, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'auction' + ) FORMAT PLAIN ENCODE JSON; + + create source bid ( + auction BIGINT, + bidder BIGINT, + price BIGINT, + channel VARCHAR, + url VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'bid' + ) FORMAT PLAIN ENCODE JSON; + + create source person ( + id BIGINT, + name VARCHAR, + email_address VARCHAR, + credit_card VARCHAR, + city VARCHAR, + state VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'person' + ) FORMAT PLAIN ENCODE JSON; + expected_outputs: [] +- id: nexmark_q0 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time FROM bid; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q1 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + date_time + FROM bid; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q2 + before: + - create_sources + sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087; + expected_outputs: + - stream_dist_plan + - stream_plan + - batch_plan +- id: nexmark_q3 + before: + - create_sources + sql: | + SELECT + P.name, P.city, P.state, A.id + FROM + auction AS A INNER JOIN person AS P on A.seller = P.id + WHERE + A.category = 10 and (P.state = 'or' OR P.state = 'id' OR P.state = 'ca'); + expected_outputs: + - stream_plan + - batch_plan + - stream_dist_plan +- id: nexmark_q4 + before: + - create_sources + sql: | + SELECT + Q.category, + AVG(Q.final) as avg + FROM ( + SELECT MAX(B.price) AS final, A.category + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + GROUP BY A.id, A.category + ) Q + GROUP BY Q.category; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q5 + before: + - create_sources + sql: | + SELECT AuctionBids.auction, AuctionBids.num FROM ( + SELECT + bid.auction, + count(*) AS num, + window_start AS starttime + FROM + HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + window_start, + bid.auction + ) AS AuctionBids + JOIN ( + SELECT + max(CountBids.num) AS maxn, + CountBids.starttime_c + FROM ( + SELECT + count(*) AS num, + window_start AS starttime_c + FROM HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + bid.auction, + window_start + ) AS CountBids + GROUP BY + CountBids.starttime_c + ) AS MaxBids + ON AuctionBids.starttime = MaxBids.starttime_c AND AuctionBids.num >= MaxBids.maxn; + expected_outputs: + - logical_plan + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q6 + before: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT MAX(B.price) AS final, A.seller, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + GROUP BY A.id, A.seller + ) AS Q; + expected_outputs: + - planner_error +- id: nexmark_q6_group_top1 + before: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT ROW_NUMBER() OVER (PARTITION BY A.id, A.seller ORDER BY B.price) as rank, A.seller, B.price as final, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + ) AS Q + WHERE Q.rank <= 1; + expected_outputs: + - stream_dist_plan + # - batch_plan + - stream_plan +- id: nexmark_q7 + before: + - create_sources + sql: | + SELECT + B.auction, + B.price, + B.bidder, + B.date_time + FROM + bid B + JOIN ( + SELECT + MAX(price) AS maxprice, + window_end as date_time + FROM + TUMBLE(bid, date_time, INTERVAL '10' SECOND) + GROUP BY + window_end + ) B1 ON B.price = B1.maxprice + WHERE + B.date_time BETWEEN B1.date_time - INTERVAL '10' SECOND + AND B1.date_time; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q8 + before: + - create_sources + sql: | + SELECT + P.id, + P.name, + P.starttime + FROM ( + SELECT + id, + name, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(person, date_time, INTERVAL '10' SECOND) + GROUP BY + id, + name, + window_start, + window_end + ) P + JOIN ( + SELECT + seller, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(auction, date_time, INTERVAL '10' SECOND) + GROUP BY + seller, + window_start, + window_end + ) A ON P.id = A.seller + AND P.starttime = A.starttime + AND P.endtime = A.endtime; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q9 + before: + - create_sources + sql: | + SELECT + id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, + auction, bidder, price, bid_date_time + FROM ( + SELECT A.*, B.auction, B.bidder, B.price, B.date_time AS bid_date_time, + ROW_NUMBER() OVER (PARTITION BY A.id ORDER BY B.price DESC, B.date_time ASC) AS rownum + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + ) + WHERE rownum <= 1; + expected_outputs: + - logical_plan + - stream_plan + - stream_dist_plan + - optimized_logical_plan_for_batch + - batch_plan +- id: nexmark_q10 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time, TO_CHAR(date_time, 'YYYY-MM-DD') as date, TO_CHAR(date_time, 'HH:MI') as time FROM bid; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q11 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + SESSION_START(B.date_time, INTERVAL '10' SECOND) as starttime, + SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime + FROM bid B + GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); + expected_outputs: + - binder_error +- id: nexmark_q12 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + TUMBLE_START(B.p_time, INTERVAL '10' SECOND) as starttime, + TUMBLE_END(B.p_time, INTERVAL '10' SECOND) as endtime + FROM (SELECT *, PROCTIME() as p_time FROM bid) B + GROUP BY B.bidder, TUMBLE(B.p_time, INTERVAL '10' SECOND); + expected_outputs: + - binder_error +- id: nexmark_q13 + before: + - create_sources + sql: | + /* SELECT + B.auction, + B.bidder, + B.price, + B.date_time, + S.value + FROM (SELECT *, PROCTIME() as p_time FROM bid) B + JOIN side_input FOR SYSTEM_TIME AS OF B.p_time AS S + ON mod(B.auction, 10000) = S.key; */ + select 1; + expected_outputs: + - stream_error +- id: nexmark_q14 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + CASE + WHEN + extract(hour from date_time) >= 8 AND + extract(hour from date_time) <= 18 + THEN 'dayTime' + WHEN + extract(hour from date_time) <= 6 OR + extract(hour from date_time) >= 20 + THEN 'nightTime' + ELSE 'otherTime' + END AS bidTimeType, + date_time, + extra + -- ignore UDF in planner test + -- count_char(extra, 'c') AS c_counts + FROM bid + WHERE 0.908 * price > 1000000 AND 0.908 * price < 50000000; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q15 + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, 'yyyy-MM-dd') as day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q15_split_distinct_agg + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, 'yyyy-MM-dd') as day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q15_split_distinct_agg_and_force_two_phase + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, 'yyyy-MM-dd') as day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' + RW_FORCE_TWO_PHASE_AGG: 'true' +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, 'HH:mm')) AS minute, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, 'HH:mm')) AS minute, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); + expected_outputs: + - stream_plan + - stream_dist_plan + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q17 + before: + - create_sources + sql: | + SELECT + auction, + to_char(date_time, 'YYYY-MM-DD') AS day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + min(price) AS min_price, + max(price) AS max_price, + avg(price) AS avg_price, + sum(price) AS sum_price + FROM bid + GROUP BY auction, to_char(date_time, 'YYYY-MM-DD'); + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q18 + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, ROW_NUMBER() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + expected_outputs: + - stream_dist_plan + - logical_plan + - batch_plan + - stream_plan +- id: nexmark_q18_rank + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, RANK() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan + - logical_plan +- id: nexmark_q19 + before: + - create_sources + sql: | + SELECT * FROM + (SELECT *, ROW_NUMBER() OVER (PARTITION BY auction ORDER BY price DESC) AS rank_number FROM bid) + WHERE rank_number <= 10; + expected_outputs: + - logical_plan + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q20 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, url, B.date_time as date_timeB, + item_name, description, initial_bid, reserve, A.date_time as date_timeA, expires, seller, category + FROM + bid B INNER JOIN auction A on B.auction = A.id + WHERE A.category = 10; + expected_outputs: + - batch_plan + - stream_dist_plan + - stream_plan +- id: nexmark_q21 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + CASE + WHEN lower(channel) = 'apple' THEN '0' + WHEN lower(channel) = 'google' THEN '1' + WHEN lower(channel) = 'facebook' THEN '2' + WHEN lower(channel) = 'baidu' THEN '3' + ELSE REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] + END + AS channel_id FROM bid + where REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] is not null or + lower(channel) in ('apple', 'google', 'facebook', 'baidu'); + expected_outputs: + - batch_plan + - stream_dist_plan + - stream_plan +- id: nexmark_q22 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + SPLIT_PART(url, '/', 4) as dir1, + SPLIT_PART(url, '/', 5) as dir2, + SPLIT_PART(url, '/', 6) as dir3 FROM bid; + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q101 + before: + - create_sources + sql: | + -- A self-made query that covers outer join. + -- + -- Monitor ongoing auctions and track the current highest bid for each one in real-time. If + -- the auction has no bids, the highest bid will be NULL. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + b.max_price AS current_highest_bid + FROM auction a + LEFT OUTER JOIN ( + SELECT + b1.auction, + MAX(b1.price) max_price + FROM bid b1 + GROUP BY b1.auction + ) b ON a.id = b.auction; + expected_outputs: + - stream_dist_plan + - batch_plan + - stream_plan +- id: nexmark_q102 + before: + - create_sources + sql: | + -- A self-made query that covers dynamic filter. + -- + -- Show the auctions whose count of bids is greater than the overall average count of bids + -- per auction. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + HAVING COUNT(b.auction) >= ( + SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid + ) + expected_outputs: + - stream_plan + - stream_dist_plan + - batch_plan +- id: nexmark_q103 + before: + - create_sources + sql: | + -- A self-made query that covers semi join. + -- + -- Show the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) >= 20 + ); + expected_outputs: + - stream_dist_plan + - stream_plan + - batch_plan +- id: nexmark_q104 + before: + - create_sources + sql: | + -- A self-made query that covers anti join. + -- + -- This is the same as q103, which shows the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id NOT IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) < 20 + ); + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan +- id: nexmark_q105 + before: + - create_sources + sql: | + -- A self-made query that covers singleton top-n (and local-phase group top-n). + -- + -- Show the top 1000 auctions by the number of bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + ORDER BY bid_count DESC + LIMIT 1000; + expected_outputs: + - batch_plan + - stream_dist_plan + - stream_plan +- id: nexmark_q106 + before: + - create_sources + sql: | + -- A self-made query that covers two-phase stateful simple aggregation. + -- + -- Show the minimum final price of all auctions. + SELECT + MIN(final) AS min_final + FROM + ( + SELECT + auction.id, + MAX(price) AS final + FROM + auction, + bid + WHERE + bid.auction = auction.id + AND bid.date_time BETWEEN auction.date_time AND auction.expires + GROUP BY + auction.id + ) + expected_outputs: + - batch_plan + - stream_plan + - stream_dist_plan diff --git a/src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml b/src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml new file mode 100644 index 0000000000000..6372549b17573 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/tpch_kafka.yaml @@ -0,0 +1,902 @@ +- id: create_tables + sql: | + CREATE SOURCE supplier ( + s_suppkey INTEGER, + s_name VARCHAR, + s_address VARCHAR, + s_nationkey INTEGER, + s_phone VARCHAR, + s_acctbal NUMERIC, + s_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'supplier', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE part ( + p_partkey INTEGER, + p_name VARCHAR, + p_mfgr VARCHAR, + p_brand VARCHAR, + p_type VARCHAR, + p_size INTEGER, + p_container VARCHAR, + p_retailprice NUMERIC, + p_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'part', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE partsupp ( + ps_partkey INTEGER, + ps_suppkey INTEGER, + ps_availqty INTEGER, + ps_supplycost NUMERIC, + ps_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'partsupp', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE customer ( + c_custkey INTEGER, + c_name VARCHAR, + c_address VARCHAR, + c_nationkey INTEGER, + c_phone VARCHAR, + c_acctbal NUMERIC, + c_mktsegment VARCHAR, + c_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'customer', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE orders ( + o_orderkey BIGINT, + o_custkey INTEGER, + o_orderstatus VARCHAR, + o_totalprice NUMERIC, + o_orderdate DATE, + o_orderpriority VARCHAR, + o_clerk VARCHAR, + o_shippriority INTEGER, + o_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'orders', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE lineitem ( + l_orderkey BIGINT, + l_partkey INTEGER, + l_suppkey INTEGER, + l_linenumber INTEGER, + l_quantity NUMERIC, + l_extendedprice NUMERIC, + l_discount NUMERIC, + l_tax NUMERIC, + l_returnflag VARCHAR, + l_linestatus VARCHAR, + l_shipdate DATE, + l_commitdate DATE, + l_receiptdate DATE, + l_shipinstruct VARCHAR, + l_shipmode VARCHAR, + l_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'lineitem', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE nation ( + n_nationkey INTEGER, + n_name VARCHAR, + n_regionkey INTEGER, + n_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'nation', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE region ( + r_regionkey INTEGER, + r_name VARCHAR, + r_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'region', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + expected_outputs: [] +- id: tpch_q1 + before: + - create_tables + sql: | + select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order + from + lineitem + where + l_shipdate <= date '1998-12-01' - interval '71' day + group by + l_returnflag, + l_linestatus + order by + l_returnflag, + l_linestatus + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q2 + before: + - create_tables + sql: | + select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment + from + part, + supplier, + partsupp, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 4 + and p_type like '%TIN' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + ) + order by + s_acctbal desc, + n_name, + s_name, + p_partkey + LIMIT 100; + expected_outputs: + - stream_plan +- id: tpch_q3 + before: + - create_tables + sql: | + select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority + from + customer, + orders, + lineitem + where + c_mktsegment = 'FURNITURE' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-29' + and l_shipdate > date '1995-03-29' + group by + l_orderkey, + o_orderdate, + o_shippriority + order by + revenue desc, + o_orderdate + LIMIT 10; + expected_outputs: + - stream_plan +- id: tpch_q4 + before: + - create_tables + sql: | + select + o_orderpriority, + count(*) as order_count + from + orders + where + o_orderdate >= date '1997-07-01' + and o_orderdate < date '1997-07-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) + group by + o_orderpriority + order by + o_orderpriority + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q5 + before: + - create_tables + sql: | + select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue + from + customer, + orders, + lineitem, + supplier, + nation, + region + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'MIDDLE EAST' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year + group by + n_name + order by + revenue desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q6 + before: + - create_tables + sql: | + select + sum(l_extendedprice * l_discount) as revenue + from + lineitem + where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + and l_discount between 0.08 - 0.01 and 0.08 + 0.01 + and l_quantity < 24; + expected_outputs: + - stream_plan +- id: tpch_q7 + before: + - create_tables + sql: | + select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue + from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'ROMANIA' and n2.n_name = 'IRAN') + or (n1.n_name = 'IRAN' and n2.n_name = 'ROMANIA') + ) + and l_shipdate between date '1983-01-01' and date '2000-12-31' + ) as shipping + group by + supp_nation, + cust_nation, + l_year + order by + supp_nation, + cust_nation, + l_year + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q8 + before: + - create_tables + sql: | + select + o_year, + sum(case + when nation = 'IRAN' then volume + else 0 + end) / sum(volume) as mkt_share + from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'ASIA' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'PROMO ANODIZED STEEL' + ) as all_nations + group by + o_year + order by + o_year + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q9 + before: + - create_tables + sql: | + select + nation, + o_year, + sum(amount) as sum_profit + from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%yellow%' + ) as profit + group by + nation, + o_year + order by + nation, + o_year desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q10 + before: + - create_tables + sql: | + select + c_custkey, + c_name, + sum(l_extendedprice * (1.00 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment + from + customer, + orders, + lineitem, + nation + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey + group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment + order by + revenue desc + LIMIT 20; + expected_outputs: + - stream_plan +- id: tpch_q11 + before: + - create_tables + sql: | + select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + group by + ps_partkey + having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001000000 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + ) + order by + value desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q12 + before: + - create_tables + sql: | + select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count + from + orders, + lineitem + where + o_orderkey = l_orderkey + and l_shipmode in ('FOB', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1994-01-01' + and l_receiptdate < date '1994-01-01' + interval '1' year + group by + l_shipmode + order by + l_shipmode + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q13 + before: + - create_tables + sql: | + select + c_count, + count(*) as custdist + from + ( + select + c_custkey, + count(o_orderkey) as c_count + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%:1%:2%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) + group by + c_count + order by + custdist desc, + c_count desc + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q14 + before: + - create_tables + sql: | + select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue + from + lineitem, + part + where + l_partkey = p_partkey + and l_shipdate >= date '1995-09-01' + and l_shipdate < date '1995-09-01' + interval '1' month; + expected_outputs: + - stream_plan +- id: tpch_q15 + before: + - create_tables + sql: | + with revenue0 (supplier_no, total_revenue) as ( + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1993-01-01' + and l_shipdate < date '1993-01-01' + interval '3' month + group by + l_suppkey + ) + select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue + from + supplier, + revenue0 + where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) + order by + s_suppkey + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q16 + before: + - create_tables + sql: | + select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt + from + partsupp, + part + where + p_partkey = ps_partkey + and p_brand <> 'Brand#45' + and p_type not like 'SMALL PLATED%' + and p_size in (19, 17, 16, 23, 10, 4, 38, 11) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) + group by + p_brand, + p_type, + p_size + order by + supplier_cnt desc, + p_brand, + p_type, + p_size + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q17 + before: + - create_tables + sql: | + select + sum(l_extendedprice) / 7.0 as avg_yearly + from + lineitem, + part + where + p_partkey = l_partkey + and p_brand = 'Brand#13' + and p_container = 'JUMBO PKG' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); + expected_outputs: + - stream_plan +- id: tpch_q18 + before: + - create_tables + sql: | + select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) quantity + from + customer, + orders, + lineitem + where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey + having + sum(l_quantity) > 1 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey + group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice + order by + o_totalprice desc, + o_orderdate + LIMIT 100; + expected_outputs: + - stream_plan +- id: tpch_q19 + before: + - create_tables + sql: | + select + sum(l_extendedprice* (1 - l_discount)) as revenue + from + lineitem, + part + where + ( + p_partkey = l_partkey + and p_brand = 'Brand#52' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 1 and l_quantity <= 11 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#24' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 30 and l_quantity <= 40 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#32' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 10 and l_quantity <= 20 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); + expected_outputs: + - stream_plan +- id: tpch_q20 + before: + - create_tables + sql: | + select + s_name, + s_address + from + supplier, + nation + where + s_suppkey in ( + select + ps_suppkey + from + partsupp, + ( + select + l_partkey agg_partkey, + l_suppkey agg_suppkey, + 0.5 * sum(l_quantity) AS agg_quantity + from + lineitem + where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + group by + l_partkey, + l_suppkey + ) agg_lineitem + where + agg_partkey = ps_partkey + and agg_suppkey = ps_suppkey + and ps_partkey in ( + select + p_partkey + from + part + where + p_name like 'forest%' + ) + and ps_availqty > agg_quantity + ) + and s_nationkey = n_nationkey + and n_name = 'KENYA' + order by + s_name + LIMIT 1; + expected_outputs: + - stream_plan +- id: tpch_q21 + before: + - create_tables + sql: | + select + s_name, + count(*) as numwait + from + supplier, + lineitem l1, + orders, + nation + where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + group by + s_name + order by + numwait desc, + s_name + LIMIT 100; + expected_outputs: + - stream_plan +- id: tpch_q22 + before: + - create_tables + sql: | + select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal + from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00::numeric + and substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale + group by + cntrycode + order by + cntrycode + LIMIT 1; + expected_outputs: + - stream_plan diff --git a/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml new file mode 100644 index 0000000000000..469d94ca87ff3 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/nexmark_source_kafka.yaml @@ -0,0 +1,2470 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- id: create_sources + sql: | + create source auction ( + id BIGINT, + item_name VARCHAR, + description VARCHAR, + initial_bid BIGINT, + reserve BIGINT, + date_time TIMESTAMP, + expires TIMESTAMP, + seller BIGINT, + category BIGINT, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'auction' + ) FORMAT PLAIN ENCODE JSON; + + create source bid ( + auction BIGINT, + bidder BIGINT, + price BIGINT, + channel VARCHAR, + url VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'bid' + ) FORMAT PLAIN ENCODE JSON; + + create source person ( + id BIGINT, + name VARCHAR, + email_address VARCHAR, + credit_card VARCHAR, + city VARCHAR, + state VARCHAR, + date_time TIMESTAMP, + extra VARCHAR) + with ( + connector = 'kafka', + properties.bootstrap.server = 'fake', + topic = 'person' + ) FORMAT PLAIN ENCODE JSON; +- id: nexmark_q0 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time FROM bid; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, price, date_time] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 4 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q1 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + date_time + FROM bid; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, date_time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, date_time, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 4 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q2 + before: + - create_sources + sql: SELECT auction, price FROM bid WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) } + └─BatchProject { exprs: [auction, price] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, price, _row_id] } + └─StreamFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, price, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, price, _row_id] } + └── StreamFilter { predicate: ((((auction = 1007:Int32) OR (auction = 1020:Int32)) OR ((auction = 2001:Int32) OR (auction = 2019:Int32))) OR (auction = 2087:Int32)) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, price, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q3 + before: + - create_sources + sql: | + SELECT + P.name, P.city, P.state, A.id + FROM + auction AS A INNER JOIN person AS P on A.seller = P.id + WHERE + A.category = 10 and (P.state = 'or' OR P.state = 'id' OR P.state = 'ca'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: seller = id, output: [name, city, state, id] } + ├─BatchExchange { order: [], dist: HashShard(seller) } + │ └─BatchFilter { predicate: (category = 10:Int32) } + │ └─BatchProject { exprs: [id, seller, category] } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(id) } + └─BatchFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └─BatchProject { exprs: [id, name, city, state] } + └─BatchKafkaScan { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(_row_id, seller, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } + ├─StreamExchange { dist: HashShard(seller) } + │ └─StreamFilter { predicate: (category = 10:Int32) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └─StreamRowIdGen { row_id_index: 11 } + └─StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [name, city, state, id, _row_id(hidden), seller(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, seller], pk_columns: [_row_id, _row_id#1, seller], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([4, 5, 6]) from 1 + + Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: seller = id, output: [name, city, state, id, _row_id, seller, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([7]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamFilter { predicate: (category = 10:Int32) } + └── StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamFilter { predicate: (((state = 'or':Varchar) OR (state = 'id':Varchar)) OR (state = 'ca':Varchar)) } + └── StreamRowIdGen { row_id_index: 11 } + └── StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $7 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 7 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ seller, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 + ├── columns: [ id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $11 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ name, city, state, id, _row_id, seller, _row_id#1, _rw_timestamp ] + ├── primary key: [ $4 ASC, $6 ASC, $5 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 4, 5, 6 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q4 + before: + - create_sources + sql: | + SELECT + Q.category, + AVG(Q.final) as avg + FROM ( + SELECT MAX(B.price) AS final, A.category + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + GROUP BY A.id, A.category + ) Q + GROUP BY Q.category; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [category, (sum(max(price)) / count(max(price))::Decimal) as $expr1] } + └─BatchHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price))] } + └─BatchExchange { order: [], dist: HashShard(category) } + └─BatchHashAgg { group_key: [id, category], aggs: [max(price)] } + └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, category, price] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [category, avg], stream_key: [category], pk_columns: [category], pk_conflict: NoCheck } + └─StreamProject { exprs: [category, (sum(max(price)) / count(max(price))::Decimal) as $expr1] } + └─StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } + └─StreamExchange { dist: HashShard(category) } + └─StreamProject { exprs: [id, category, max(price)] } + └─StreamHashAgg [append_only] { group_key: [id, category], aggs: [max(price), count] } + └─StreamProject { exprs: [id, category, price, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [category, avg], stream_key: [category], pk_columns: [category], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [category, (sum(max(price)) / count(max(price))::Decimal) as $expr1] } + └── StreamHashAgg { group_key: [category], aggs: [sum(max(price)), count(max(price)), count] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([1]) from 1 + + Fragment 1 + StreamProject { exprs: [id, category, max(price)] } + └── StreamHashAgg [append_only] { group_key: [id, category], aggs: [max(price), count] } + ├── tables: [ HashAggState: 1 ] + └── StreamProject { exprs: [id, category, price, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├── tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 6 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } + └── Upstream + + Table 0 { columns: [ category, sum(max(price)), count(max(price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ id, category, max(price), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 2 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 5 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ category, avg, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + +- id: nexmark_q5 + before: + - create_sources + sql: | + SELECT AuctionBids.auction, AuctionBids.num FROM ( + SELECT + bid.auction, + count(*) AS num, + window_start AS starttime + FROM + HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + window_start, + bid.auction + ) AS AuctionBids + JOIN ( + SELECT + max(CountBids.num) AS maxn, + CountBids.starttime_c + FROM ( + SELECT + count(*) AS num, + window_start AS starttime_c + FROM HOP(bid, date_time, INTERVAL '2' SECOND, INTERVAL '10' SECOND) + GROUP BY + bid.auction, + window_start + ) AS CountBids + GROUP BY + CountBids.starttime_c + ) AS MaxBids + ON AuctionBids.starttime = MaxBids.starttime_c AND AuctionBids.num >= MaxBids.maxn; + logical_plan: |- + LogicalProject { exprs: [auction, count] } + └─LogicalJoin { type: Inner, on: (window_start = window_start) AND (count >= max(count)), output: all } + ├─LogicalProject { exprs: [auction, count, window_start] } + │ └─LogicalAgg { group_key: [window_start, auction], aggs: [count] } + │ └─LogicalProject { exprs: [window_start, auction] } + │ └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } + │ └─LogicalFilter { predicate: IsNotNull(date_time) } + │ └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalProject { exprs: [max(count), window_start] } + └─LogicalAgg { group_key: [window_start], aggs: [max(count)] } + └─LogicalProject { exprs: [window_start, count] } + └─LogicalProject { exprs: [count, window_start] } + └─LogicalAgg { group_key: [auction, window_start], aggs: [count] } + └─LogicalProject { exprs: [auction, window_start] } + └─LogicalHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: all } + └─LogicalFilter { predicate: IsNotNull(date_time) } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: window_start = window_start AND (count >= max(count)), output: [auction, count] } + ├─BatchExchange { order: [], dist: HashShard(window_start) } + │ └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } + │ └─BatchHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start] } + │ └─BatchExchange { order: [], dist: HashShard(auction) } + │ └─BatchProject { exprs: [auction, date_time] } + │ └─BatchFilter { predicate: IsNotNull(date_time) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchHashAgg { group_key: [window_start], aggs: [max(count)] } + └─BatchExchange { order: [], dist: HashShard(window_start) } + └─BatchHashAgg { group_key: [auction, window_start], aggs: [count] } + └─BatchHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchProject { exprs: [auction, date_time] } + └─BatchFilter { predicate: IsNotNull(date_time) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, 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 } + ├─StreamExchange { dist: HashShard(window_start) } + │ └─StreamShare { id: 7 } + │ └─StreamHashAgg [append_only] { group_key: [auction, window_start], aggs: [count] } + │ └─StreamExchange { dist: HashShard(auction, window_start) } + │ └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + │ └─StreamProject { exprs: [auction, date_time, _row_id] } + │ └─StreamFilter { predicate: IsNotNull(date_time) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [window_start, max(count)] } + └─StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } + └─StreamExchange { dist: HashShard(window_start) } + └─StreamShare { id: 7 } + └─StreamHashAgg [append_only] { group_key: [auction, window_start], aggs: [count] } + └─StreamExchange { dist: HashShard(auction, window_start) } + └─StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + └─StreamProject { exprs: [auction, date_time, _row_id] } + └─StreamFilter { predicate: IsNotNull(date_time) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, num, window_start(hidden), window_start#1(hidden)], stream_key: [auction, window_start], pk_columns: [auction, window_start], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, count, window_start, window_start] } + └── StreamFilter { predicate: (count >= max(count)) } + └── StreamHashJoin { type: Inner, predicate: window_start = window_start, output: all } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([1]) from 1 + └── StreamProject { exprs: [window_start, max(count)] } + └── StreamHashAgg { group_key: [window_start], aggs: [max(count), count] } { tables: [ HashAggState: 7, HashAggCall0: 6 ] } + └── StreamExchange Hash([1]) from 4 + + Fragment 1 + StreamNoOp + └── StreamExchange NoShuffle from 2 + + Fragment 2 + StreamHashAgg [append_only] { group_key: [auction, window_start], aggs: [count] } { tables: [ HashAggState: 4 ] } + └── StreamExchange Hash([0, 1]) from 3 + + Fragment 3 + StreamHopWindow { time_col: date_time, slide: 00:00:02, size: 00:00:10, output: [auction, window_start, _row_id] } + └── StreamProject { exprs: [auction, date_time, _row_id] } + └── StreamFilter { predicate: IsNotNull(date_time) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 5 ] + └── Upstream + + Fragment 4 + StreamNoOp + └── StreamExchange NoShuffle from 2 + + Table 0 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ window_start, auction, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ window_start, max(count), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ window_start, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ auction, window_start, count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 6 + ├── columns: [ window_start, count, auction, _rw_timestamp ] + ├── 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, _rw_timestamp ], 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, _rw_timestamp ] + ├── 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: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT MAX(B.price) AS final, A.seller, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + GROUP BY A.id, A.seller + ) AS Q; + planner_error: 'Invalid input syntax: column must appear in the GROUP BY clause or be used in an aggregate function' +- id: nexmark_q6_group_top1 + before: + - create_sources + sql: | + SELECT + Q.seller, + AVG(Q.final) OVER + (PARTITION BY Q.seller ORDER BY Q.date_time ROWS BETWEEN 10 PRECEDING AND CURRENT ROW) + as avg + FROM ( + SELECT ROW_NUMBER() OVER (PARTITION BY A.id, A.seller ORDER BY B.price) as rank, A.seller, B.price as final, B.date_time + FROM auction AS A, bid AS B + WHERE A.id = B.auction and B.date_time between A.date_time and A.expires + ) AS Q + WHERE Q.rank <= 1; + stream_plan: |- + StreamMaterialize { columns: [seller, avg, id(hidden)], stream_key: [id, seller], pk_columns: [id, seller], pk_conflict: NoCheck } + └─StreamProject { exprs: [seller, (sum / count::Decimal) as $expr1, id] } + └─StreamOverWindow { window_functions: [sum(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW), count(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW)] } + └─StreamExchange { dist: HashShard(seller) } + └─StreamProject { exprs: [seller, price, date_time, id] } + └─StreamGroupTopN [append_only] { order: [price ASC], limit: 1, offset: 0, group_key: [id, seller] } + └─StreamExchange { dist: HashShard(id, seller) } + └─StreamProject { exprs: [id, seller, price, date_time, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [seller, avg, id(hidden)], stream_key: [id, seller], pk_columns: [id, seller], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [seller, (sum / count::Decimal) as $expr1, id] } + └── StreamOverWindow { window_functions: [sum(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW), count(price) OVER(PARTITION BY seller ORDER BY date_time ASC ROWS BETWEEN 10 PRECEDING AND CURRENT ROW)] } + ├── tables: [ OverWindow: 0 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamProject { exprs: [seller, price, date_time, id] } + └── StreamGroupTopN [append_only] { order: [price ASC], limit: 1, offset: 0, group_key: [id, seller] } { tables: [ AppendOnlyGroupTopN: 1 ] } + └── StreamExchange Hash([0, 1]) from 2 + + Fragment 2 + StreamProject { exprs: [id, seller, price, date_time, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } { tables: [ HashJoinLeft: 2, HashJoinDegreeLeft: 3, HashJoinRight: 4, HashJoinDegreeRight: 5 ] } + ├── StreamExchange Hash([0]) from 3 + └── StreamExchange Hash([0]) from 4 + + Fragment 3 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Fragment 4 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } + └── Upstream + + Table 0 { columns: [ seller, price, date_time, id, sum, count, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ id, seller, price, date_time, _row_id, _row_id_0, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + Table 2 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 5 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ seller, avg, id, _rw_timestamp ], primary key: [ $2 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + +- id: nexmark_q7 + before: + - create_sources + sql: | + SELECT + B.auction, + B.price, + B.bidder, + B.date_time + FROM + bid B + JOIN ( + SELECT + MAX(price) AS maxprice, + window_end as date_time + FROM + TUMBLE(bid, date_time, INTERVAL '10' SECOND) + GROUP BY + window_end + ) B1 ON B.price = B1.maxprice + WHERE + B.date_time BETWEEN B1.date_time - INTERVAL '10' SECOND + AND B1.date_time; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: price = max(price) AND (date_time >= $expr2) AND (date_time <= $expr1), output: [auction, price, bidder, date_time] } + ├─BatchExchange { order: [], dist: HashShard(price) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(max(price)) } + └─BatchProject { exprs: [max(price), $expr1, ($expr1 - '00:00:10':Interval) as $expr2] } + └─BatchHashAgg { group_key: [$expr1], aggs: [max(price)] } + └─BatchExchange { order: [], dist: HashShard($expr1) } + └─BatchProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden)], stream_key: [_row_id, $expr1, price], pk_columns: [_row_id, $expr1, price], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1] } + └─StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } + └─StreamHashJoin { type: Inner, predicate: price = max(price), output: all } + ├─StreamExchange { dist: HashShard(price) } + │ └─StreamShare { id: 3 } + │ └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(max(price)) } + └─StreamProject { exprs: [$expr1, max(price), ($expr1 - '00:00:10':Interval) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [max(price), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, price, bidder, date_time, _row_id(hidden), $expr1(hidden)], stream_key: [_row_id, $expr1, price], pk_columns: [_row_id, $expr1, price], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, price, bidder, date_time, _row_id, $expr1] } + └── StreamFilter { predicate: (date_time >= $expr2) AND (date_time <= $expr1) } + └── StreamHashJoin { type: Inner, predicate: price = max(price), output: all } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } + ├── StreamExchange Hash([2]) from 1 + └── StreamExchange Hash([1]) from 3 + + Fragment 1 + StreamNoOp + └── StreamExchange NoShuffle from 2 + + Fragment 2 + StreamProject { exprs: [auction, bidder, price, date_time, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamProject { exprs: [$expr1, max(price), ($expr1 - '00:00:10':Interval) as $expr2] } + └── StreamHashAgg [append_only] { group_key: [$expr1], aggs: [max(price), count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 4 + + Fragment 4 + StreamProject { exprs: [(TumbleStart(date_time, '00:00:10':Interval) + '00:00:10':Interval) as $expr1, price, _row_id] } + └── StreamExchange NoShuffle from 2 + + Table 0 + ├── columns: [ auction, bidder, price, date_time, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $4 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 2 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ price, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ $expr1, max(price), $expr2, _rw_timestamp ], primary key: [ $1 ASC, $0 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 1 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ max(price), $expr1, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ $expr1, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4294967294 + ├── columns: [ auction, price, bidder, date_time, _row_id, $expr1, _rw_timestamp ] + ├── primary key: [ $4 ASC, $5 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5 ] + ├── distribution key: [ 1 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q8 + before: + - create_sources + sql: | + SELECT + P.id, + P.name, + P.starttime + FROM ( + SELECT + id, + name, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(person, date_time, INTERVAL '10' SECOND) + GROUP BY + id, + name, + window_start, + window_end + ) P + JOIN ( + SELECT + seller, + window_start AS starttime, + window_end AS endtime + FROM + TUMBLE(auction, date_time, INTERVAL '10' SECOND) + GROUP BY + seller, + window_start, + window_end + ) A ON P.id = A.seller + AND P.starttime = A.starttime + AND P.endtime = A.endtime; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: [id, name, $expr1] } + ├─BatchExchange { order: [], dist: HashShard(id, $expr1, $expr2) } + │ └─BatchHashAgg { group_key: [id, name, $expr1, $expr2], aggs: [] } + │ └─BatchExchange { order: [], dist: HashShard(id, name, $expr1, $expr2) } + │ └─BatchProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + │ └─BatchProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + │ └─BatchKafkaScan { source: person, columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchHashAgg { group_key: [seller, $expr3, $expr4], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(seller, $expr3, $expr4) } + └─BatchProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } + └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + 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 } + └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } + └─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] } + │ └─StreamExchange { dist: HashShard(id, name, $expr1, $expr2) } + │ └─StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + │ └─StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } + └─StreamExchange { dist: HashShard(seller, $expr3, $expr4) } + └─StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } + └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └─StreamRowIdGen { row_id_index: 13 } + └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + 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 } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 1, 2, 3]) from 1 + + Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: id = seller AND $expr1 = $expr3 AND $expr2 = $expr4, output: all } { tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] } + ├── StreamExchange Hash([0, 2, 3]) from 2 + └── StreamAppendOnlyDedup { dedup_cols: [seller, $expr3, $expr4] } { tables: [ AppendOnlyDedup: 6 ] } + └── StreamExchange Hash([0, 1, 2]) from 4 + + Fragment 2 + StreamAppendOnlyDedup { dedup_cols: [id, name, $expr1, $expr2] } { tables: [ AppendOnlyDedup: 4 ] } + └── StreamExchange Hash([0, 1, 2, 3]) from 3 + + Fragment 3 + StreamProject { exprs: [id, name, $expr1, ($expr1 + '00:00:10':Interval) as $expr2] } + └── StreamProject { exprs: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr1] } + └── StreamRowIdGen { row_id_index: 11 } + └── StreamSourceScan { columns: [id, name, email_address, credit_card, city, state, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Fragment 4 + StreamProject { exprs: [seller, $expr3, ($expr3 + '00:00:10':Interval) as $expr4] } + └── StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, TumbleStart(date_time, '00:00:10':Interval) as $expr3] } + └── StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 7 ] } + └── Upstream + + Table 0 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $2 ASC, $3 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 2, 3 ], read pk prefix len hint: 3 } + + Table 1 { columns: [ id, $expr1, $expr2, name, _degree, _rw_timestamp ], 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 2 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], 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 3 { columns: [ seller, $expr3, $expr4, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3 ], distribution key: [ 0, 1, 2 ], read pk prefix len hint: 3 } + + Table 4 { columns: [ id, name, $expr1, $expr2, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 0, 1, 2, 3 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 6 { columns: [ seller, $expr3, $expr4, _rw_timestamp ], 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 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ id, name, starttime, $expr2, seller, $expr3, $expr4, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 0, 1, 2, 3 ] + └── read pk prefix len hint: 4 + +- id: nexmark_q9 + before: + - create_sources + sql: | + SELECT + id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, + auction, bidder, price, bid_date_time + FROM ( + SELECT A.*, B.auction, B.bidder, B.price, B.date_time AS bid_date_time, + ROW_NUMBER() OVER (PARTITION BY A.id ORDER BY B.price DESC, B.date_time ASC) AS rownum + FROM auction A, bid B + WHERE A.id = B.auction AND B.date_time BETWEEN A.date_time AND A.expires + ) + WHERE rownum <= 1; + logical_plan: |- + LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } + └─LogicalFilter { predicate: (row_number <= 1:Int32) } + └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, auction, bidder, price, date_time, row_number] } + └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY id ORDER BY price DESC, date_time ASC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalFilter { predicate: (id = auction) AND (date_time >= date_time) AND (date_time <= expires) } + └─LogicalJoin { type: Inner, on: true, output: all } + ├─LogicalSource { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + optimized_logical_plan_for_batch: |- + LogicalTopN { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } + └─LogicalJoin { type: Inner, on: (id = auction) AND (date_time >= date_time) AND (date_time <= expires), output: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } + ├─LogicalKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], time_range: (Unbounded, Unbounded) } + └─LogicalKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], time_range: (Unbounded, Unbounded) } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchGroupTopN { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } + └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck } + └─StreamGroupTopN [append_only] { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } + └─StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id(hidden), _row_id#1(hidden)], stream_key: [id], pk_columns: [id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamGroupTopN [append_only] { order: [price DESC, date_time ASC], limit: 1, offset: 0, group_key: [id] } { tables: [ AppendOnlyGroupTopN: 0 ] } + └── StreamProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } { tables: [ HashJoinLeft: 1, HashJoinDegreeLeft: 2, HashJoinRight: 3, HashJoinDegreeRight: 4 ] } + ├── StreamExchange Hash([0]) from 1 + └── StreamExchange Hash([0]) from 2 + + Fragment 1 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Fragment 2 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, date_time_0, _row_id, _row_id_0, _rw_timestamp ] + ├── primary key: [ $0 ASC, $11 DESC, $12 ASC, $13 ASC, $14 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 2 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 4 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, auction, bidder, price, bid_date_time, _row_id, _row_id#1, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q10 + before: + - create_sources + sql: | + SELECT auction, bidder, price, date_time, TO_CHAR(date_time, 'YYYY-MM-DD') as date, TO_CHAR(date_time, 'HH:MI') as time FROM bid; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, date_time, date, time, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [auction, bidder, price, date_time, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, ToChar(date_time, 'HH:MI':Varchar) as $expr2, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 0 ] + └── Upstream + + Table 0 + ├── columns: [ partition_id, backfill_progress, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 0, 1 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 4294967294 + ├── columns: [ auction, bidder, price, date_time, date, time, _row_id, _rw_timestamp ] + ├── primary key: [ $6 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6 ] + ├── distribution key: [ 6 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q11 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + SESSION_START(B.date_time, INTERVAL '10' SECOND) as starttime, + SESSION_END(B.date_time, INTERVAL '10' SECOND) as endtime + FROM bid B + GROUP BY B.bidder, SESSION(B.date_time, INTERVAL '10' SECOND); + binder_error: | + Failed to bind expression: SESSION_START(B.date_time, INTERVAL '10' SECOND) + + Caused by: + function session_start(timestamp without time zone, interval) does not exist +- id: nexmark_q12 + before: + - create_sources + sql: | + SELECT + B.bidder, + count(*) as bid_count, + TUMBLE_START(B.p_time, INTERVAL '10' SECOND) as starttime, + TUMBLE_END(B.p_time, INTERVAL '10' SECOND) as endtime + FROM (SELECT *, PROCTIME() as p_time FROM bid) B + GROUP BY B.bidder, TUMBLE(B.p_time, INTERVAL '10' SECOND); + binder_error: | + Failed to bind expression: PROCTIME() + + Caused by: + Invalid input syntax: Function `PROCTIME()` is only allowed in CREATE TABLE/SOURCE. Is `NOW()` what you want? +- id: nexmark_q13 + before: + - create_sources + sql: | + /* SELECT + B.auction, + B.bidder, + B.price, + B.date_time, + S.value + FROM (SELECT *, PROCTIME() as p_time FROM bid) B + JOIN side_input FOR SYSTEM_TIME AS OF B.p_time AS S + ON mod(B.auction, 10000) = S.key; */ + select 1; + stream_error: 'Bind error: An alias must be specified for the 1st expression (counting from 1) in result relation' +- id: nexmark_q14 + before: + - create_sources + sql: | + SELECT + auction, + bidder, + 0.908 * price as price, + CASE + WHEN + extract(hour from date_time) >= 8 AND + extract(hour from date_time) <= 18 + THEN 'dayTime' + WHEN + extract(hour from date_time) <= 6 OR + extract(hour from date_time) >= 20 + THEN 'nightTime' + ELSE 'otherTime' + END AS bidTimeType, + date_time, + extra + -- ignore UDF in planner test + -- count_char(extra, 'c') AS c_counts + FROM bid + WHERE 0.908 * price > 1000000 AND 0.908 * price < 50000000; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra] } + └─BatchFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } + └─BatchProject { exprs: [auction, bidder, price, date_time, extra] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } + └─StreamFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, bidtimetype, date_time, extra, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [auction, bidder, (0.908:Decimal * price::Decimal) as $expr1, Case(((Extract('HOUR':Varchar, date_time) >= 8:Decimal) AND (Extract('HOUR':Varchar, date_time) <= 18:Decimal)), 'dayTime':Varchar, ((Extract('HOUR':Varchar, date_time) <= 6:Decimal) OR (Extract('HOUR':Varchar, date_time) >= 20:Decimal)), 'nightTime':Varchar, 'otherTime':Varchar) as $expr2, date_time, extra, _row_id] } + └── StreamFilter { predicate: ((0.908:Decimal * price::Decimal) > 1000000:Decimal) AND ((0.908:Decimal * price::Decimal) < 50000000:Decimal) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ auction, bidder, price, bidtimetype, date_time, extra, _row_id, _rw_timestamp ], primary key: [ $6 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6 ], distribution key: [ 6 ], read pk prefix len hint: 1 } + +- id: nexmark_q15 + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, 'yyyy-MM-dd') as day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY to_char(date_time, 'yyyy-MM-dd'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard($expr1_expanded) } + └─BatchHashAgg { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─BatchExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └─BatchProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } + └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamHashAgg [append_only] { group_key: [$expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + ├── tables: [ HashAggState: 0, HashAggDedupForCol2: 1, HashAggDedupForCol3: 2 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 + ├── columns: [ $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)), _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ $expr1, bidder, count_for_agg_call_4, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 2 { columns: [ $expr1, auction, count_for_agg_call_8, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3, 4, 5 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + +- id: nexmark_q15_split_distinct_agg + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, 'yyyy-MM-dd') as day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY to_char(date_time, 'yyyy-MM-dd'); + stream_plan: |- + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } + └─StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard($expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [$expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { tables: [ HashAggState: 1 ] } + └── StreamExchange Hash([0, 2, 3, 10]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └── StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 2 ] } + └── Upstream + + Table 0 + ├── columns: [ $expr1_expanded, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + + Table 2 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q15_split_distinct_agg_and_force_two_phase + before: + - create_sources + sql: | + SELECT + TO_CHAR(date_time, 'yyyy-MM-dd') as day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY to_char(date_time, 'yyyy-MM-dd'); + stream_plan: |- + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } + └─StreamProject { exprs: [$expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } + └─StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } + └─StreamExchange { dist: HashShard($expr1_expanded) } + └─StreamHashAgg { group_key: [$expr1_expanded, _vnode], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamProject { exprs: [$expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1_expanded, bidder_expanded, auction_expanded, flag) as _vnode] } + └─StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard($expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └─StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [day], pk_columns: [day], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [$expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)))] } + └── StreamHashAgg { group_key: [$expr1_expanded], aggs: [sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamHashAgg { group_key: [$expr1_expanded, _vnode], aggs: [sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } { tables: [ HashAggState: 1 ] } + └── StreamProject { exprs: [$expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), Vnode($expr1_expanded, bidder_expanded, auction_expanded, flag) as _vnode] } + └── StreamHashAgg [append_only] { group_key: [$expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { tables: [ HashAggState: 2 ] } + └── StreamExchange Hash([0, 2, 3, 10]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[$expr1], [$expr1, bidder], [$expr1, auction]] } + └── StreamProject { exprs: [ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 + ├── columns: [ $expr1_expanded, sum0(sum0(count) filter((flag = 0:Int64))), sum0(sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64))), sum0(sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64))), sum0(count(bidder_expanded) filter((flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64))), sum0(count(auction_expanded) filter((flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), sum0(count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))), count, _rw_timestamp ] + ├── primary key: [ $0 ASC ] + ├── value indices: [ 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 + ├── columns: [ $expr1_expanded, _vnode, sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [] + ├── read pk prefix len hint: 2 + └── vnode column idx: 1 + + Table 2 { columns: [ $expr1_expanded, bidder_expanded, auction_expanded, flag, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC ], value indices: [ 4, 5, 6, 7 ], distribution key: [ 0, 1, 2, 3 ], read pk prefix len hint: 4 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ day, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' + RW_FORCE_TWO_PHASE_AGG: 'true' +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, 'HH:mm')) AS minute, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─BatchExchange { order: [], dist: HashShard(channel_expanded, $expr1_expanded) } + └─BatchHashAgg { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─BatchExchange { order: [], dist: HashShard(channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─BatchExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } + └─BatchProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } + └─StreamHashAgg [append_only] { group_key: [channel, $expr1], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(channel, $expr1) } + └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamHashAgg [append_only] { group_key: [channel, $expr1], aggs: [max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32))] } + ├── tables: [ HashAggState: 0, HashAggDedupForCol4: 1, HashAggDedupForCol5: 2 ] + └── StreamExchange Hash([0, 1]) from 1 + + Fragment 1 + StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 + ├── columns: [ channel, $expr1, max($expr2), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), count(distinct bidder), count(distinct bidder) filter((price < 10000:Int32)), count(distinct bidder) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct bidder) filter((price >= 1000000:Int32)), count(distinct auction), count(distinct auction) filter((price < 10000:Int32)), count(distinct auction) filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count(distinct auction) filter((price >= 1000000:Int32)), _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ channel, $expr1, bidder, count_for_agg_call_5, count_for_agg_call_6, count_for_agg_call_7, count_for_agg_call_8, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4, 5, 6 ], distribution key: [ 0, 1 ], read pk prefix len hint: 3 } + + Table 2 { columns: [ channel, $expr1, auction, count_for_agg_call_9, count_for_agg_call_10, count_for_agg_call_11, count_for_agg_call_12, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC ], value indices: [ 3, 4, 5, 6 ], distribution key: [ 0, 1 ], read pk prefix len hint: 3 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + +- id: nexmark_q16 + before: + - create_sources + sql: | + SELECT + channel, + to_char(date_time, 'yyyy-MM-dd') AS day, + max(to_char(date_time, 'HH:mm')) AS minute, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + count(distinct bidder) AS total_bidders, + count(distinct bidder) filter (where price < 10000) AS rank1_bidders, + count(distinct bidder) filter (where price >= 10000 and price < 1000000) AS rank2_bidders, + count(distinct bidder) filter (where price >= 1000000) AS rank3_bidders, + count(distinct auction) AS total_auctions, + count(distinct auction) filter (where price < 10000) AS rank1_auctions, + count(distinct auction) filter (where price >= 10000 and price < 1000000) AS rank2_auctions, + count(distinct auction) filter (where price >= 1000000) AS rank3_auctions + FROM bid + GROUP BY channel, to_char(date_time, 'yyyy-MM-dd'); + stream_plan: |- + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } + └─StreamProject { exprs: [channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └─StreamHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + └─StreamExchange { dist: HashShard(channel_expanded, $expr1_expanded) } + └─StreamHashAgg [append_only] { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } + └─StreamExchange { dist: HashShard(channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag) } + └─StreamExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } + └─StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions], stream_key: [channel, day], pk_columns: [channel, day], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64))] } + └── StreamHashAgg { group_key: [channel_expanded, $expr1_expanded], aggs: [max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count] } + ├── tables: [ HashAggState: 1, HashAggCall0: 0 ] + └── StreamExchange Hash([0, 1]) from 1 + + Fragment 1 + StreamHashAgg [append_only] { group_key: [channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag], aggs: [max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32))] } { tables: [ HashAggState: 2 ] } + └── StreamExchange Hash([0, 1, 4, 5, 14]) from 2 + + Fragment 2 + StreamExpand { column_subsets: [[channel, $expr1, $expr2], [channel, $expr1, bidder], [channel, $expr1, auction]] } + └── StreamProject { exprs: [channel, ToChar(date_time, 'yyyy-MM-dd':Varchar) as $expr1, ToChar(date_time, 'HH:mm':Varchar) as $expr2, price, bidder, auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 3 ] } + └── Upstream + + Table 0 { columns: [ channel_expanded, $expr1_expanded, max($expr2_expanded), bidder_expanded, auction_expanded, flag, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 DESC, $3 ASC, $4 ASC, $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + Table 1 + ├── columns: [ channel_expanded, $expr1_expanded, max(max($expr2_expanded)) filter((flag = 0:Int64)), sum0(count) filter((flag = 0:Int64)), sum0(count filter((price < 10000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 10000:Int32) AND (price < 1000000:Int32))) filter((flag = 0:Int64)), sum0(count filter((price >= 1000000:Int32))) filter((flag = 0:Int64)), count(bidder_expanded) filter((flag = 1:Int64)), count(bidder_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(bidder_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 1:Int64)), count(auction_expanded) filter((flag = 2:Int64)), count(auction_expanded) filter((count filter((price < 10000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 10000:Int32) AND (price < 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count(auction_expanded) filter((count filter((price >= 1000000:Int32)) > 0:Int64) AND (flag = 2:Int64)), count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + + Table 2 { columns: [ channel_expanded, $expr1_expanded, bidder_expanded, auction_expanded, flag, max($expr2_expanded), count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), _rw_timestamp ], primary key: [ $0 ASC, $1 ASC, $2 ASC, $3 ASC, $4 ASC ], value indices: [ 5, 6, 7, 8, 9 ], distribution key: [ 0, 1, 2, 3, 4 ], read pk prefix len hint: 5 } + + Table 3 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ channel, day, minute, total_bids, rank1_bids, rank2_bids, rank3_bids, total_bidders, rank1_bidders, rank2_bidders, rank3_bidders, total_auctions, rank1_auctions, rank2_auctions, rank3_auctions, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14 ], distribution key: [ 0, 1 ], read pk prefix len hint: 2 } + + with_config_map: + RW_FORCE_SPLIT_DISTINCT_AGG: 'true' +- id: nexmark_q17 + before: + - create_sources + sql: | + SELECT + auction, + to_char(date_time, 'YYYY-MM-DD') AS day, + count(*) AS total_bids, + count(*) filter (where price < 10000) AS rank1_bids, + count(*) filter (where price >= 10000 and price < 1000000) AS rank2_bids, + count(*) filter (where price >= 1000000) AS rank3_bids, + min(price) AS min_price, + max(price) AS max_price, + avg(price) AS avg_price, + sum(price) AS sum_price + FROM bid + GROUP BY auction, to_char(date_time, 'YYYY-MM-DD'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price) / count(price)::Decimal) as $expr2, sum(price)] } + └─BatchHashAgg { group_key: [auction, $expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } + └─BatchExchange { order: [], dist: HashShard(auction, $expr1) } + └─BatchProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], stream_key: [auction, day], pk_columns: [auction, day], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price) / count(price)::Decimal) as $expr2, sum(price)] } + └─StreamHashAgg [append_only] { group_key: [auction, $expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } + └─StreamExchange { dist: HashShard(auction, $expr1) } + └─StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price], stream_key: [auction, day], pk_columns: [auction, day], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), (sum(price) / count(price)::Decimal) as $expr2, sum(price)] } + └── StreamHashAgg [append_only] { group_key: [auction, $expr1], aggs: [count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price)] } + ├── tables: [ HashAggState: 0 ] + └── StreamExchange Hash([0, 1]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, ToChar(date_time, 'YYYY-MM-DD':Varchar) as $expr1, price, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 1 ] } + └── Upstream + + Table 0 + ├── columns: [ auction, $expr1, count, count filter((price < 10000:Int32)), count filter((price >= 10000:Int32) AND (price < 1000000:Int32)), count filter((price >= 1000000:Int32)), min(price), max(price), sum(price), count(price), _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 2, 3, 4, 5, 6, 7, 8, 9 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, day, total_bids, rank1_bids, rank2_bids, rank3_bids, min_price, max_price, avg_price, sum_price, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9 ] + ├── distribution key: [ 0, 1 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q18 + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, ROW_NUMBER() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + logical_plan: |- + LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } + └─LogicalFilter { predicate: (row_number <= 1:Int32) } + └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } + └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchGroupTopN { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } + └─BatchExchange { order: [], dist: HashShard(bidder, auction) } + └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck } + └─StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } + └─StreamExchange { dist: HashShard(bidder, auction) } + └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction], pk_columns: [bidder, auction], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, group_key: [bidder, auction] } { tables: [ AppendOnlyGroupTopN: 0 ] } + └── StreamExchange Hash([1, 0]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 1 ] + └── Upstream + + Table 0 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q18_rank + before: + - create_sources + sql: | + SELECT auction, bidder, price, channel, url, date_time, extra + FROM (SELECT *, RANK() OVER (PARTITION BY bidder, auction ORDER BY date_time DESC) AS rank_number + FROM bid) + WHERE rank_number <= 1; + logical_plan: |- + LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } + └─LogicalFilter { predicate: (rank <= 1:Int32) } + └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, rank] } + └─LogicalOverWindow { window_functions: [rank() OVER(PARTITION BY bidder, auction ORDER BY date_time DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchGroupTopN { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } + └─BatchExchange { order: [], dist: HashShard(bidder, auction) } + └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction, _row_id], pk_columns: [bidder, auction, _row_id], pk_conflict: NoCheck } + └─StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } + └─StreamExchange { dist: HashShard(bidder, auction) } + └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden)], stream_key: [bidder, auction, _row_id], pk_columns: [bidder, auction, _row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamGroupTopN [append_only] { order: [date_time DESC], limit: 1, offset: 0, with_ties: true, group_key: [bidder, auction] } { tables: [ AppendOnlyGroupTopN: 0 ] } + └── StreamExchange Hash([1, 0]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 1 ] } + └── Upstream + + Table 0 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $5 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 2 + + Table 1 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $1 ASC, $0 ASC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 1, 0 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q19 + before: + - create_sources + sql: | + SELECT * FROM + (SELECT *, ROW_NUMBER() OVER (PARTITION BY auction ORDER BY price DESC) AS rank_number FROM bid) + WHERE rank_number <= 10; + logical_plan: |- + LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } + └─LogicalFilter { predicate: (row_number <= 10:Int32) } + └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, row_number] } + └─LogicalOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─LogicalProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─LogicalSource { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─BatchSort { order: [auction ASC, price DESC] } + └─BatchGroupTopN { order: [price DESC], limit: 10, offset: 0, group_key: [auction] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchProject { exprs: [auction, bidder, price, channel, url, date_time, extra] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden), rank_number], stream_key: [auction, _row_id], pk_columns: [auction, _row_id], pk_conflict: NoCheck } + └─StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } + └─StreamGroupTopN [append_only] { order: [price DESC], limit: 10, offset: 0, group_key: [auction] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_time, extra, _row_id(hidden), rank_number], stream_key: [auction, _row_id], pk_columns: [auction, _row_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamOverWindow { window_functions: [row_number() OVER(PARTITION BY auction ORDER BY price DESC ROWS BETWEEN UNBOUNDED PRECEDING AND CURRENT ROW)] } { tables: [ OverWindow: 0 ] } + └── StreamGroupTopN [append_only] { order: [price DESC], limit: 10, offset: 0, group_key: [auction] } { tables: [ AppendOnlyGroupTopN: 1 ] } + └── StreamExchange Hash([0]) from 1 + + Fragment 1 + StreamProject { exprs: [auction, bidder, price, channel, url, date_time, extra, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 2 ] } + └── Upstream + + Table 0 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, row_number, _rw_timestamp ] + ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $2 DESC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 2 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _row_id, rank_number, _rw_timestamp ] + ├── primary key: [ $0 ASC, $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q20 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, url, B.date_time as date_timeB, + item_name, description, initial_bid, reserve, A.date_time as date_timeA, expires, seller, category + FROM + bid B INNER JOIN auction A on B.auction = A.id + WHERE A.category = 10; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } + ├─BatchExchange { order: [], dist: HashShard(auction) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(id) } + └─BatchFilter { predicate: (category = 10:Int32) } + └─BatchProject { exprs: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category] } + └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(auction, _row_id, _row_id) } + └─StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(auction) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(id) } + └─StreamFilter { predicate: (category = 10:Int32) } + └─StreamRowIdGen { row_id_index: 13 } + └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + 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, _row_id(hidden), _row_id#1(hidden)], stream_key: [_row_id, _row_id#1, auction], pk_columns: [_row_id, _row_id#1, auction], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 14, 15]) from 1 + + Fragment 1 + StreamHashJoin [append_only] { type: Inner, predicate: auction = id, output: [auction, bidder, price, channel, url, date_time, item_name, description, initial_bid, reserve, date_time, expires, seller, category, _row_id, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 4 ] } + └── Upstream + + Fragment 3 + StreamFilter { predicate: (category = 10:Int32) } + └── StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 5 ] } + └── Upstream + + Table 0 { columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $10 ASC ], value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 3 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, url, date_timeb, item_name, description, initial_bid, reserve, date_timea, expires, seller, category, _row_id, _row_id#1, _rw_timestamp ] + ├── primary key: [ $14 ASC, $15 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15 ] + ├── distribution key: [ 0, 14, 15 ] + └── read pk prefix len hint: 3 + +- id: nexmark_q21 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + CASE + WHEN lower(channel) = 'apple' THEN '0' + WHEN lower(channel) = 'google' THEN '1' + WHEN lower(channel) = 'facebook' THEN '2' + WHEN lower(channel) = 'baidu' THEN '3' + ELSE REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] + END + AS channel_id FROM bid + where REGEXP_MATCH(url, '(&|^)channel_id=([^&]*)')[2] is not null or + lower(channel) in ('apple', 'google', 'facebook', 'baidu'); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1] } + └─BatchFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } + └─BatchProject { exprs: [auction, bidder, price, channel, url] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1, _row_id] } + └─StreamFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, channel_id, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [auction, bidder, price, channel, Case((Lower(channel) = 'apple':Varchar), '0':Varchar, (Lower(channel) = 'google':Varchar), '1':Varchar, (Lower(channel) = 'facebook':Varchar), '2':Varchar, (Lower(channel) = 'baidu':Varchar), '3':Varchar, ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) as $expr1, _row_id] } + └── StreamFilter { predicate: (IsNotNull(ArrayAccess(RegexpMatch(url, '(&|^)channel_id=([^&]*)':Varchar), 2:Int32)) OR In(Lower(channel), 'apple':Varchar, 'google':Varchar, 'facebook':Varchar, 'baidu':Varchar)) } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ auction, bidder, price, channel, channel_id, _row_id, _rw_timestamp ], primary key: [ $5 ASC ], value indices: [ 0, 1, 2, 3, 4, 5 ], distribution key: [ 5 ], read pk prefix len hint: 1 } + +- id: nexmark_q22 + before: + - create_sources + sql: | + SELECT + auction, bidder, price, channel, + SPLIT_PART(url, '/', 4) as dir1, + SPLIT_PART(url, '/', 5) as dir2, + SPLIT_PART(url, '/', 6) as dir3 FROM bid; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3] } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } + └─StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction, bidder, price, channel, dir1, dir2, dir3, _row_id(hidden)], stream_key: [_row_id], pk_columns: [_row_id], pk_conflict: NoCheck } { tables: [ Materialize: 4294967294 ] } + └── StreamProject { exprs: [auction, bidder, price, channel, SplitPart(url, '/':Varchar, 4:Int32) as $expr1, SplitPart(url, '/':Varchar, 5:Int32) as $expr2, SplitPart(url, '/':Varchar, 6:Int32) as $expr3, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 0 ] } + └── Upstream + + Table 0 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction, bidder, price, channel, dir1, dir2, dir3, _row_id, _rw_timestamp ] + ├── primary key: [ $7 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7 ] + ├── distribution key: [ 7 ] + └── read pk prefix len hint: 1 + +- id: nexmark_q101 + before: + - create_sources + sql: | + -- A self-made query that covers outer join. + -- + -- Monitor ongoing auctions and track the current highest bid for each one in real-time. If + -- the auction has no bids, the highest bid will be NULL. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + b.max_price AS current_highest_bid + FROM auction a + LEFT OUTER JOIN ( + SELECT + b1.auction, + MAX(b1.price) max_price + FROM bid b1 + GROUP BY b1.auction + ) b ON a.id = b.auction; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price)] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchHashAgg { group_key: [auction], aggs: [max(price)] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [auction, max(price)] } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction_id, auction_item_name, current_highest_bid, _row_id(hidden), auction(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 3]) from 1 + + Fragment 1 + StreamHashJoin { type: LeftOuter, predicate: id = auction, output: [id, item_name, max(price), _row_id, auction] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction, max(price)] } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [max(price), count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ auction, max(price), _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ auction, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, current_highest_bid, _row_id, auction, _rw_timestamp ] + ├── primary key: [ $3 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4 ] + ├── distribution key: [ 0, 3 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q102 + before: + - create_sources + sql: | + -- A self-made query that covers dynamic filter. + -- + -- Show the auctions whose count of bids is greater than the overall average count of bids + -- per auction. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + HAVING COUNT(b.auction) >= ( + SELECT COUNT(*) / COUNT(DISTINCT auction) FROM bid + ) + batch_plan: |- + BatchNestedLoopJoin { type: Inner, predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } + ├─BatchExchange { order: [], dist: Single } + │ └─BatchHashAgg { group_key: [id, item_name], aggs: [count(auction)] } + │ └─BatchHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction] } + │ ├─BatchExchange { order: [], dist: HashShard(id) } + │ │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + │ └─BatchExchange { order: [], dist: HashShard(auction) } + │ └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } + └─BatchSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [sum0(count), count(auction)] } + └─BatchHashAgg { group_key: [auction], aggs: [count] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [auction_id, auction_item_name], pk_conflict: NoCheck } + └─StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } + ├─StreamProject { exprs: [id, item_name, count(auction)] } + │ └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(id) } + │ │ └─StreamRowIdGen { row_id_index: 13 } + │ │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(auction) } + │ └─StreamShare { id: 5 } + │ └─StreamProject { exprs: [auction, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } + └─StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum0(count), count(auction)] } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamShare { id: 5 } + └─StreamProject { exprs: [auction, _row_id] } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [auction_id, auction_item_name], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamDynamicFilter { predicate: (count(auction) >= $expr1), output: [id, item_name, count(auction)] } { tables: [ DynamicFilterLeft: 0, DynamicFilterRight: 1 ] } + ├── StreamProject { exprs: [id, item_name, count(auction)] } + │ └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } + │ └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + │ ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] + │ ├── StreamExchange Hash([0]) from 1 + │ └── StreamExchange Hash([0]) from 2 + └── StreamExchange Broadcast from 4 + + Fragment 1 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 7 ] + └── Upstream + + Fragment 2 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Fragment 3 + StreamProject { exprs: [auction, _row_id] } + └── StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 8 ] } + └── Upstream + + Fragment 4 + StreamProject { exprs: [(sum0(sum0(count)) / sum0(count(auction))) as $expr1] } + └── StreamSimpleAgg { aggs: [sum0(sum0(count)), sum0(count(auction)), count] } { tables: [ SimpleAggState: 9 ] } + └── StreamExchange Single from 5 + + Fragment 5 + StreamStatelessSimpleAgg { aggs: [sum0(count), count(auction)] } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { tables: [ HashAggState: 10 ] } + └── StreamExchange Hash([0]) from 6 + + Fragment 6 + StreamNoOp + └── StreamExchange NoShuffle from 3 + + Table 0 { columns: [ id, item_name, count(auction), _rw_timestamp ], primary key: [ $2 ASC, $0 ASC, $1 ASC ], value indices: [ 0, 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 1 { columns: [ $expr1, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + + Table 2 { columns: [ id, item_name, count(auction), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 3 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 { columns: [ auction, _row_id, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 8 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 9 { columns: [ sum0(sum0(count)), sum0(count(auction)), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1, 2 ], distribution key: [], read pk prefix len hint: 0 } + + Table 10 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q103 + before: + - create_sources + sql: | + -- A self-made query that covers semi join. + -- + -- Show the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) >= 20 + ); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction] } + └─BatchFilter { predicate: (count >= 20:Int32) } + └─BatchHashAgg { group_key: [auction], aggs: [count] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count >= 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 2]) from 1 + + Fragment 1 + StreamHashJoin { type: LeftSemi, predicate: id = auction, output: [id, item_name, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count >= 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0, 2 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q104 + before: + - create_sources + sql: | + -- A self-made query that covers anti join. + -- + -- This is the same as q103, which shows the auctions that have at least 20 bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name + FROM auction a + WHERE a.id NOT IN ( + SELECT b.auction FROM bid b + GROUP BY b.auction + HAVING COUNT(*) < 20 + ); + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchProject { exprs: [auction] } + └─BatchFilter { predicate: (count < 20:Int32) } + └─BatchHashAgg { group_key: [auction], aggs: [count] } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } + └─StreamExchange { dist: HashShard(id, _row_id) } + └─StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [auction] } + └─StreamFilter { predicate: (count < 20:Int32) } + └─StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction_id, auction_item_name, _row_id(hidden)], stream_key: [_row_id, auction_id], pk_columns: [_row_id, auction_id], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamExchange Hash([0, 2]) from 1 + + Fragment 1 + StreamHashJoin { type: LeftAnti, predicate: id = auction, output: [id, item_name, _row_id] } + ├── tables: [ HashJoinLeft: 0, HashJoinDegreeLeft: 1, HashJoinRight: 2, HashJoinDegreeRight: 3 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamProject { exprs: [auction] } + └── StreamFilter { predicate: (count < 20:Int32) } + └── StreamHashAgg [append_only] { group_key: [auction], aggs: [count] } { tables: [ HashAggState: 5 ] } + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 4 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 6 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 1 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 2 { columns: [ auction, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 3 { columns: [ auction, _degree, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 4 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 5 { columns: [ auction, count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 6 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, _row_id, _rw_timestamp ] + ├── primary key: [ $2 ASC, $0 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [ 0, 2 ] + └── read pk prefix len hint: 2 + +- id: nexmark_q105 + before: + - create_sources + sql: | + -- A self-made query that covers singleton top-n (and local-phase group top-n). + -- + -- Show the top 1000 auctions by the number of bids. + SELECT + a.id AS auction_id, + a.item_name AS auction_item_name, + COUNT(b.auction) AS bid_count + FROM auction a + JOIN bid b ON a.id = b.auction + GROUP BY a.id, a.item_name + ORDER BY bid_count DESC + LIMIT 1000; + batch_plan: |- + BatchTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─BatchExchange { order: [], dist: Single } + └─BatchTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─BatchHashAgg { group_key: [id, item_name], aggs: [count(auction)] } + └─BatchHashJoin { type: Inner, predicate: id = auction, output: [id, item_name, auction] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } + └─StreamProject { exprs: [id, item_name, count(auction)] } + └─StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [auction_id, auction_item_name, bid_count], stream_key: [auction_id, auction_item_name], pk_columns: [bid_count, auction_id, auction_item_name], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [id, item_name, count(auction)] } + └── StreamTopN { order: [count(auction) DESC], limit: 1000, offset: 0 } { tables: [ TopN: 0 ] } + └── StreamExchange Single from 1 + + Fragment 1 + StreamGroupTopN { order: [count(auction) DESC], limit: 1000, offset: 0, group_key: [$expr1] } { tables: [ GroupTopN: 1 ] } + └── StreamProject { exprs: [id, item_name, count(auction), Vnode(id) as $expr1] } + └── StreamHashAgg [append_only] { group_key: [id, item_name], aggs: [count(auction), count] } { tables: [ HashAggState: 2 ] } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: [id, item_name, auction, _row_id, _row_id] } + ├── tables: [ HashJoinLeft: 3, HashJoinDegreeLeft: 4, HashJoinRight: 5, HashJoinDegreeRight: 6 ] + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 7 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 8 ] } + └── Upstream + + Table 0 + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [] + └── read pk prefix len hint: 0 + + Table 1 + ├── columns: [ id, item_name, count(auction), $expr1, _rw_timestamp ] + ├── primary key: [ $3 ASC, $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2, 3 ] + ├── distribution key: [ 0 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 3 + + Table 2 { columns: [ id, item_name, count(auction), count, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2, 3 ], distribution key: [ 0 ], read pk prefix len hint: 2 } + + Table 3 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 4 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 6 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 7 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 8 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 + ├── columns: [ auction_id, auction_item_name, bid_count, _rw_timestamp ] + ├── primary key: [ $2 DESC, $0 ASC, $1 ASC ] + ├── value indices: [ 0, 1, 2 ] + ├── distribution key: [] + └── read pk prefix len hint: 3 + +- id: nexmark_q106 + before: + - create_sources + sql: | + -- A self-made query that covers two-phase stateful simple aggregation. + -- + -- Show the minimum final price of all auctions. + SELECT + MIN(final) AS min_final + FROM + ( + SELECT + auction.id, + MAX(price) AS final + FROM + auction, + bid + WHERE + bid.auction = auction.id + AND bid.date_time BETWEEN auction.date_time AND auction.expires + GROUP BY + auction.id + ) + batch_plan: |- + BatchSimpleAgg { aggs: [min(min(max(price)))] } + └─BatchExchange { order: [], dist: Single } + └─BatchSimpleAgg { aggs: [min(max(price))] } + └─BatchHashAgg { group_key: [id], aggs: [max(price)] } + └─BatchHashJoin { type: Inner, predicate: id = auction AND (date_time >= date_time) AND (date_time <= expires), output: [id, price] } + ├─BatchExchange { order: [], dist: HashShard(id) } + │ └─BatchKafkaScan { source: auction, columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + └─BatchExchange { order: [], dist: HashShard(auction) } + └─BatchKafkaScan { source: bid, columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id], filter: (None, None) } + stream_plan: |- + StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [min(min(max(price)))] } + └─StreamSimpleAgg { aggs: [min(min(max(price))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [$expr1], aggs: [min(max(price)), count] } + └─StreamProject { exprs: [id, max(price), Vnode(id) as $expr1] } + └─StreamHashAgg [append_only] { group_key: [id], aggs: [max(price), count] } + └─StreamProject { exprs: [id, price, _row_id, _row_id] } + └─StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └─StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├─StreamExchange { dist: HashShard(id) } + │ └─StreamRowIdGen { row_id_index: 13 } + │ └─StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(auction) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + stream_dist_plan: |+ + Fragment 0 + StreamMaterialize { columns: [min_final], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + ├── tables: [ Materialize: 4294967294 ] + └── StreamProject { exprs: [min(min(max(price)))] } + └── StreamSimpleAgg { aggs: [min(min(max(price))), count] } + ├── tables: [ SimpleAggState: 1, SimpleAggCall0: 0 ] + └── StreamExchange Single from 1 + + Fragment 1 + StreamHashAgg { group_key: [$expr1], aggs: [min(max(price)), count] } + ├── tables: [ HashAggState: 3, HashAggCall0: 2 ] + └── StreamProject { exprs: [id, max(price), Vnode(id) as $expr1] } + └── StreamHashAgg [append_only] { group_key: [id], aggs: [max(price), count] } + ├── tables: [ HashAggState: 4 ] + └── StreamProject { exprs: [id, price, _row_id, _row_id] } + └── StreamFilter { predicate: (date_time >= date_time) AND (date_time <= expires) } + └── StreamHashJoin [append_only] { type: Inner, predicate: id = auction, output: all } + ├── tables: + │ ┌── HashJoinLeft: 5 + │ ├── HashJoinDegreeLeft: 6 + │ ├── HashJoinRight: 7 + │ └── HashJoinDegreeRight: 8 + ├── StreamExchange Hash([0]) from 2 + └── StreamExchange Hash([0]) from 3 + + Fragment 2 + StreamRowIdGen { row_id_index: 13 } + └── StreamSourceScan { columns: [id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + ├── tables: [ SourceBackfill: 9 ] + └── Upstream + + Fragment 3 + StreamRowIdGen { row_id_index: 10 } + └── StreamSourceScan { columns: [auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } { tables: [ SourceBackfill: 10 ] } + └── Upstream + + Table 0 { columns: [ min(max(price)), $expr1, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 1 { columns: [ min(min(max(price))), count, _rw_timestamp ], primary key: [], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 2 + ├── columns: [ $expr1, max(price), id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $1 ASC, $2 ASC ] + ├── value indices: [ 1, 2 ] + ├── distribution key: [ 2 ] + ├── read pk prefix len hint: 1 + └── vnode column idx: 0 + + Table 3 { columns: [ $expr1, min(max(price)), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [], read pk prefix len hint: 1, vnode column idx: 0 } + + Table 4 { columns: [ id, max(price), count, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 1, 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 5 + ├── columns: [ id, item_name, description, initial_bid, reserve, date_time, expires, seller, category, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $13 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 6 { columns: [ id, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 7 + ├── columns: [ auction, bidder, price, channel, url, date_time, extra, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, _rw_timestamp ] + ├── primary key: [ $0 ASC, $10 ASC ] + ├── value indices: [ 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10 ] + ├── distribution key: [ 0 ] + └── read pk prefix len hint: 1 + + Table 8 { columns: [ auction, _row_id, _degree, _rw_timestamp ], primary key: [ $0 ASC, $1 ASC ], value indices: [ 2 ], distribution key: [ 0 ], read pk prefix len hint: 1 } + + Table 9 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 10 { columns: [ partition_id, backfill_progress, _rw_timestamp ], primary key: [ $0 ASC ], value indices: [ 0, 1 ], distribution key: [], read pk prefix len hint: 0 } + + Table 4294967294 { columns: [ min_final, _rw_timestamp ], primary key: [], value indices: [ 0 ], distribution key: [], read pk prefix len hint: 0 } + diff --git a/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml new file mode 100644 index 0000000000000..b1f9c6d5dc88c --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/tpch_kafka.yaml @@ -0,0 +1,1513 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- id: create_tables + sql: | + CREATE SOURCE supplier ( + s_suppkey INTEGER, + s_name VARCHAR, + s_address VARCHAR, + s_nationkey INTEGER, + s_phone VARCHAR, + s_acctbal NUMERIC, + s_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'supplier', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE part ( + p_partkey INTEGER, + p_name VARCHAR, + p_mfgr VARCHAR, + p_brand VARCHAR, + p_type VARCHAR, + p_size INTEGER, + p_container VARCHAR, + p_retailprice NUMERIC, + p_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'part', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE partsupp ( + ps_partkey INTEGER, + ps_suppkey INTEGER, + ps_availqty INTEGER, + ps_supplycost NUMERIC, + ps_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'partsupp', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE customer ( + c_custkey INTEGER, + c_name VARCHAR, + c_address VARCHAR, + c_nationkey INTEGER, + c_phone VARCHAR, + c_acctbal NUMERIC, + c_mktsegment VARCHAR, + c_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'customer', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE orders ( + o_orderkey BIGINT, + o_custkey INTEGER, + o_orderstatus VARCHAR, + o_totalprice NUMERIC, + o_orderdate DATE, + o_orderpriority VARCHAR, + o_clerk VARCHAR, + o_shippriority INTEGER, + o_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'orders', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE lineitem ( + l_orderkey BIGINT, + l_partkey INTEGER, + l_suppkey INTEGER, + l_linenumber INTEGER, + l_quantity NUMERIC, + l_extendedprice NUMERIC, + l_discount NUMERIC, + l_tax NUMERIC, + l_returnflag VARCHAR, + l_linestatus VARCHAR, + l_shipdate DATE, + l_commitdate DATE, + l_receiptdate DATE, + l_shipinstruct VARCHAR, + l_shipmode VARCHAR, + l_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'lineitem', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE nation ( + n_nationkey INTEGER, + n_name VARCHAR, + n_regionkey INTEGER, + n_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'nation', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; + + CREATE SOURCE region ( + r_regionkey INTEGER, + r_name VARCHAR, + r_comment VARCHAR + ) WITH ( + connector = 'kafka', + topic = 'region', + properties.bootstrap.server = 'fake', + ) FORMAT PLAIN ENCODE JSON; +- id: tpch_q1 + before: + - create_tables + sql: | + select + l_returnflag, + l_linestatus, + sum(l_quantity) as sum_qty, + sum(l_extendedprice) as sum_base_price, + sum(l_extendedprice * (1 - l_discount)) as sum_disc_price, + sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) as sum_charge, + avg(l_quantity) as avg_qty, + avg(l_extendedprice) as avg_price, + avg(l_discount) as avg_disc, + count(*) as count_order + from + lineitem + where + l_shipdate <= date '1998-12-01' - interval '71' day + group by + l_returnflag, + l_linestatus + order by + l_returnflag, + l_linestatus + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [l_returnflag, l_linestatus, sum_qty, sum_base_price, sum_disc_price, sum_charge, avg_qty, avg_price, avg_disc, count_order], stream_key: [], pk_columns: [l_returnflag, l_linestatus], pk_conflict: NoCheck } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), $expr3, $expr4, $expr5, count] } + └─StreamTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_returnflag ASC, l_linestatus ASC], limit: 1, offset: 0, group_key: [$expr6] } + └─StreamProject { exprs: [l_returnflag, l_linestatus, sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), (sum(l_quantity) / count(l_quantity)::Decimal) as $expr3, (sum(l_extendedprice) / count(l_extendedprice)::Decimal) as $expr4, (sum(l_discount) / count(l_discount)::Decimal) as $expr5, count, Vnode(l_returnflag, l_linestatus) as $expr6] } + └─StreamHashAgg [append_only] { group_key: [l_returnflag, l_linestatus], aggs: [sum(l_quantity), sum(l_extendedprice), sum($expr1), sum($expr2), count(l_quantity), count(l_extendedprice), sum(l_discount), count(l_discount), count] } + └─StreamExchange { dist: HashShard(l_returnflag, l_linestatus) } + └─StreamProject { exprs: [l_returnflag, l_linestatus, l_quantity, l_extendedprice, $expr1, ($expr1 * (1:Decimal + l_tax)) as $expr2, l_discount, _row_id] } + └─StreamProject { exprs: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id, (l_extendedprice * (1:Decimal - l_discount)) as $expr1] } + └─StreamFilter { predicate: (l_shipdate <= '1998-09-21 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q2 + before: + - create_tables + sql: | + select + s_acctbal, + s_name, + n_name, + p_partkey, + p_mfgr, + s_address, + s_phone, + s_comment + from + part, + supplier, + partsupp, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and p_size = 4 + and p_type like '%TIN' + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + and ps_supplycost = ( + select + min(ps_supplycost) + from + partsupp, + supplier, + nation, + region + where + p_partkey = ps_partkey + and s_suppkey = ps_suppkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'AFRICA' + ) + order by + s_acctbal desc, + n_name, + s_name, + p_partkey + LIMIT 100; + stream_plan: |- + StreamMaterialize { columns: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id(hidden), min(ps_supplycost)(hidden), ps_partkey(hidden), _row_id#1(hidden), _row_id#2(hidden), r_regionkey(hidden), _row_id#3(hidden), n_nationkey(hidden), _row_id#4(hidden), s_suppkey(hidden)], stream_key: [_row_id, p_partkey, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_columns: [s_acctbal, n_name, s_name, p_partkey, _row_id, _row_id#1, _row_id#2, r_regionkey, _row_id#3, n_nationkey, _row_id#4, s_suppkey, min(ps_supplycost), ps_partkey], pk_conflict: NoCheck } + └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } + └─StreamTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_acctbal DESC, n_name ASC, s_name ASC, p_partkey ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey, Vnode(ps_partkey, min(ps_supplycost)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey AND min(ps_supplycost) = ps_supplycost AND ps_partkey = ps_partkey, output: [s_acctbal, s_name, n_name, p_partkey, p_mfgr, s_address, s_phone, s_comment, _row_id, min(ps_supplycost), ps_partkey, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_partkey, min(ps_supplycost)) } + │ └─StreamHashJoin { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, p_mfgr, ps_partkey, min(ps_supplycost), _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: (p_size = 4:Int32) AND Like(p_type, '%TIN':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamProject { exprs: [ps_partkey, min(ps_supplycost)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [min(ps_supplycost), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_supplycost, ps_partkey, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id, _row_id, r_regionkey] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamShare { id: 6 } + │ │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamShare { id: 10 } + │ │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamShare { id: 17 } + │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamShare { id: 22 } + │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_partkey, ps_supplycost) } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = ps_suppkey, output: [n_name, s_name, s_address, s_phone, s_acctbal, s_comment, ps_partkey, ps_supplycost, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, s_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, s_name, s_address, s_phone, s_acctbal, s_comment, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamShare { id: 17 } + │ │ │ └─StreamProject { exprs: [r_regionkey, r_name, _row_id] } + │ │ │ └─StreamFilter { predicate: (r_name = 'AFRICA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 22 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamShare { id: 10 } + │ └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _row_id] } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamFilter { predicate: IsNotNull(ps_partkey) } + └─StreamShare { id: 6 } + └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_supplycost, _row_id] } + └─StreamRowIdGen { row_id_index: 8 } + └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q3 + before: + - create_tables + sql: | + select + l_orderkey, + sum(l_extendedprice * (1 - l_discount)) as revenue, + o_orderdate, + o_shippriority + from + customer, + orders, + lineitem + where + c_mktsegment = 'FURNITURE' + and c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate < date '1995-03-29' + and l_shipdate > date '1995-03-29' + group by + l_orderkey, + o_orderdate, + o_shippriority + order by + revenue desc, + o_orderdate + LIMIT 10; + stream_plan: |- + StreamMaterialize { columns: [l_orderkey, revenue, o_orderdate, o_shippriority], stream_key: [l_orderkey, o_orderdate, o_shippriority], pk_columns: [revenue, o_orderdate, l_orderkey, o_shippriority], pk_conflict: NoCheck } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority] } + └─StreamTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC, o_orderdate ASC], limit: 10, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [l_orderkey, sum($expr1), o_orderdate, o_shippriority, Vnode(l_orderkey, o_orderdate, o_shippriority) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey, o_orderdate, o_shippriority], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_orderkey, o_orderdate, o_shippriority) } + └─StreamProject { exprs: [l_orderkey, o_orderdate, o_shippriority, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, c_custkey, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, o_shippriority, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, c_custkey, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [o_orderkey, o_orderdate, o_shippriority, _row_id, c_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: (c_mktsegment = 'FURNITURE':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamFilter { predicate: (o_orderdate < '1995-03-29':Date) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: (l_shipdate > '1995-03-29':Date) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q4 + before: + - create_tables + sql: | + select + o_orderpriority, + count(*) as order_count + from + orders + where + o_orderdate >= date '1997-07-01' + and o_orderdate < date '1997-07-01' + interval '3' month + and exists ( + select + * + from + lineitem + where + l_orderkey = o_orderkey + and l_commitdate < l_receiptdate + ) + group by + o_orderpriority + order by + o_orderpriority + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [o_orderpriority, order_count], stream_key: [], pk_columns: [o_orderpriority], pk_conflict: NoCheck } + └─StreamProject { exprs: [o_orderpriority, count] } + └─StreamTopN { order: [o_orderpriority ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_orderpriority ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [o_orderpriority, count, Vnode(o_orderpriority) as _vnode] } + └─StreamHashAgg { group_key: [o_orderpriority], aggs: [count] } + └─StreamExchange { dist: HashShard(o_orderpriority) } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamFilter { predicate: (o_orderdate >= '1997-07-01':Date) AND (o_orderdate < '1997-10-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, _row_id] } + └─StreamFilter { predicate: (l_commitdate < l_receiptdate) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q5 + before: + - create_tables + sql: | + select + n_name, + sum(l_extendedprice * (1 - l_discount)) as revenue + from + customer, + orders, + lineitem, + supplier, + nation, + region + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and l_suppkey = s_suppkey + and c_nationkey = s_nationkey + and s_nationkey = n_nationkey + and n_regionkey = r_regionkey + and r_name = 'MIDDLE EAST' + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '1' year + group by + n_name + order by + revenue desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [n_name, revenue], stream_key: [], pk_columns: [revenue], pk_conflict: NoCheck } + └─StreamProject { exprs: [n_name, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [n_name, sum($expr1), Vnode(n_name) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [n_name], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(n_name) } + └─StreamProject { exprs: [n_name, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, r_regionkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, n_nationkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey AND n_nationkey = c_nationkey, output: [l_extendedprice, l_discount, n_name, _row_id, _row_id, r_regionkey, n_nationkey, _row_id, _row_id, o_custkey, _row_id, _row_id, l_suppkey, o_orderkey, c_nationkey] } + ├─StreamExchange { dist: HashShard(n_nationkey, n_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, n_name, _row_id, r_regionkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ └─StreamFilter { predicate: (r_name = 'MIDDLE EAST':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ └─StreamFilter { predicate: IsNotNull(n_nationkey) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(c_nationkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey AND c_nationkey = s_nationkey, output: [c_nationkey, l_extendedprice, l_discount, s_nationkey, _row_id, _row_id, o_custkey, o_orderkey, _row_id, _row_id, l_suppkey] } + ├─StreamExchange { dist: HashShard(o_orderkey, c_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_custkey = c_custkey, output: [o_orderkey, c_nationkey, _row_id, o_custkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1995-01-01 00:00:00':Timestamp) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey, s_nationkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_suppkey = s_suppkey, output: [l_orderkey, l_extendedprice, l_discount, s_nationkey, _row_id, l_suppkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q6 + before: + - create_tables + sql: | + select + sum(l_extendedprice * l_discount) as revenue + from + lineitem + where + l_shipdate >= date '1994-01-01' + and l_shipdate < date '1994-01-01' + interval '1' year + and l_discount between 0.08 - 0.01 and 0.08 + 0.01 + and l_quantity < 24; + stream_plan: |- + StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [sum(sum($expr1))] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr1)] } + └─StreamProject { exprs: [(l_extendedprice * l_discount) as $expr1, _row_id] } + └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) AND (l_discount >= 0.07:Decimal) AND (l_discount <= 0.09:Decimal) AND (l_quantity < 24:Decimal) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q7 + before: + - create_tables + sql: | + select + supp_nation, + cust_nation, + l_year, + sum(volume) as revenue + from + ( + select + n1.n_name as supp_nation, + n2.n_name as cust_nation, + extract(year from l_shipdate) as l_year, + l_extendedprice * (1 - l_discount) as volume + from + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2 + where + s_suppkey = l_suppkey + and o_orderkey = l_orderkey + and c_custkey = o_custkey + and s_nationkey = n1.n_nationkey + and c_nationkey = n2.n_nationkey + and ( + (n1.n_name = 'ROMANIA' and n2.n_name = 'IRAN') + or (n1.n_name = 'IRAN' and n2.n_name = 'ROMANIA') + ) + and l_shipdate between date '1983-01-01' and date '2000-12-31' + ) as shipping + group by + supp_nation, + cust_nation, + l_year + order by + supp_nation, + cust_nation, + l_year + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [supp_nation, cust_nation, l_year, revenue], stream_key: [], pk_columns: [supp_nation, cust_nation, l_year], pk_conflict: NoCheck } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, n_name ASC, $expr1 ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, n_name, $expr1, sum($expr2), Vnode(n_name, n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, n_name, $expr1) } + └─StreamProject { exprs: [n_name, n_name, Extract('YEAR':Varchar, l_shipdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, n_nationkey, _row_id, s_suppkey, _row_id, _row_id, n_nationkey, _row_id, c_custkey, l_orderkey] } + └─StreamFilter { predicate: (((n_name = 'ROMANIA':Varchar) AND (n_name = 'IRAN':Varchar)) OR ((n_name = 'IRAN':Varchar) AND (n_name = 'ROMANIA':Varchar))) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: all } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, l_shipdate, _row_id, _row_id, n_nationkey, s_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 3 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1983-01-01':Date) AND (l_shipdate <= '2000-12-31':Date) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [n_name, o_orderkey, _row_id, _row_id, n_nationkey, c_custkey, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamShare { id: 3 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q8 + before: + - create_tables + sql: | + select + o_year, + sum(case + when nation = 'IRAN' then volume + else 0 + end) / sum(volume) as mkt_share + from + ( + select + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) as volume, + n2.n_name as nation + from + part, + supplier, + lineitem, + orders, + customer, + nation n1, + nation n2, + region + where + p_partkey = l_partkey + and s_suppkey = l_suppkey + and l_orderkey = o_orderkey + and o_custkey = c_custkey + and c_nationkey = n1.n_nationkey + and n1.n_regionkey = r_regionkey + and r_name = 'ASIA' + and s_nationkey = n2.n_nationkey + and o_orderdate between date '1995-01-01' and date '1996-12-31' + and p_type = 'PROMO ANODIZED STEEL' + ) as all_nations + group by + o_year + order by + o_year + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [o_year, mkt_share], stream_key: [], pk_columns: [o_year], pk_conflict: NoCheck } + └─StreamProject { exprs: [$expr1, $expr4] } + └─StreamTopN { order: [$expr1 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr1 ASC], limit: 1, offset: 0, group_key: [$expr5] } + └─StreamProject { exprs: [$expr1, (sum($expr3) / sum($expr2)) as $expr4, Vnode($expr1) as $expr5] } + └─StreamHashAgg [append_only] { group_key: [$expr1], aggs: [sum($expr3), sum($expr2), count] } + └─StreamExchange { dist: HashShard($expr1) } + └─StreamProject { exprs: [$expr1, Case((n_name = 'IRAN':Varchar), $expr2, 0:Decimal) as $expr3, $expr2, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamProject { exprs: [Extract('YEAR':Varchar, o_orderdate) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey, c_custkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [l_extendedprice, l_discount, o_orderdate, n_name, _row_id, _row_id, r_regionkey, _row_id, n_nationkey, c_custkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [c_custkey, _row_id, _row_id, r_regionkey, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: r_regionkey = n_regionkey, output: [n_nationkey, _row_id, r_regionkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(r_regionkey) } + │ │ │ └─StreamFilter { predicate: (r_name = 'ASIA':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 6 } + │ │ │ └─StreamSourceScan { columns: [r_regionkey, r_name, r_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(n_regionkey) } + │ │ └─StreamShare { id: 6 } + │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [n_name, l_extendedprice, l_discount, o_custkey, o_orderdate, _row_id, _row_id, n_nationkey, _row_id, _row_id, p_partkey, s_suppkey, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, l_orderkey, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, p_partkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ └─StreamShare { id: 6 } + │ │ │ └─StreamProject { exprs: [n_nationkey, n_name, n_regionkey, _row_id] } + │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey, output: [l_orderkey, l_suppkey, l_extendedprice, l_discount, _row_id, p_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: (p_type = 'PROMO ANODIZED STEEL':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1995-01-01':Date) AND (o_orderdate <= '1996-12-31':Date) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q9 + before: + - create_tables + sql: | + select + nation, + o_year, + sum(amount) as sum_profit + from + ( + select + n_name as nation, + extract(year from o_orderdate) as o_year, + l_extendedprice * (1 - l_discount) - ps_supplycost * l_quantity as amount + from + part, + supplier, + lineitem, + partsupp, + orders, + nation + where + s_suppkey = l_suppkey + and ps_suppkey = l_suppkey + and ps_partkey = l_partkey + and p_partkey = l_partkey + and o_orderkey = l_orderkey + and s_nationkey = n_nationkey + and p_name like '%yellow%' + ) as profit + group by + nation, + o_year + order by + nation, + o_year desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [nation, o_year, sum_profit], stream_key: [], pk_columns: [nation, o_year], pk_conflict: NoCheck } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2)] } + └─StreamTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [n_name ASC, $expr1 DESC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [n_name, $expr1, sum($expr2), Vnode(n_name, $expr1) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [n_name, $expr1], aggs: [sum($expr2), count] } + └─StreamExchange { dist: HashShard(n_name, $expr1) } + └─StreamProject { exprs: [n_name, Extract('YEAR':Varchar, o_orderdate) as $expr1, ((l_extendedprice * (1:Decimal - l_discount)) - (ps_supplycost * l_quantity)) as $expr2, _row_id, _row_id, p_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, ps_suppkey, ps_partkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = l_partkey AND ps_suppkey = l_suppkey AND ps_partkey = l_partkey AND ps_suppkey = s_suppkey, output: [l_quantity, l_extendedprice, l_discount, ps_supplycost, o_orderdate, n_name, _row_id, _row_id, p_partkey, ps_suppkey, ps_partkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: p_partkey = ps_partkey, output: [p_partkey, ps_partkey, ps_suppkey, ps_supplycost, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(p_partkey) } + │ │ └─StreamFilter { predicate: Like(p_name, '%yellow%':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamFilter { predicate: IsNotNull(ps_suppkey) } + │ └─StreamRowIdGen { row_id_index: 8 } + │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [n_name, s_suppkey, o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [n_name, s_suppkey, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderdate, l_partkey, l_suppkey, l_quantity, l_extendedprice, l_discount, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: IsNotNull(l_partkey) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q10 + before: + - create_tables + sql: | + select + c_custkey, + c_name, + sum(l_extendedprice * (1.00 - l_discount)) as revenue, + c_acctbal, + n_name, + c_address, + c_phone, + c_comment + from + customer, + orders, + lineitem, + nation + where + c_custkey = o_custkey + and l_orderkey = o_orderkey + and o_orderdate >= date '1994-01-01' + and o_orderdate < date '1994-01-01' + interval '3' month + and l_returnflag = 'R' + and c_nationkey = n_nationkey + group by + c_custkey, + c_name, + c_acctbal, + c_phone, + n_name, + c_address, + c_comment + order by + revenue desc + LIMIT 20; + stream_plan: |- + StreamMaterialize { columns: [c_custkey, c_name, revenue, c_acctbal, n_name, c_address, c_phone, c_comment], stream_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_columns: [revenue, c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], pk_conflict: NoCheck } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 20, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 20, offset: 0, group_key: [$expr2] } + └─StreamProject { exprs: [c_custkey, c_name, sum($expr1), c_acctbal, n_name, c_address, c_phone, c_comment, Vnode(c_custkey) as $expr2] } + └─StreamHashAgg [append_only] { group_key: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment], aggs: [sum($expr1), count] } + └─StreamProject { exprs: [c_custkey, c_name, c_acctbal, c_phone, n_name, c_address, c_comment, (l_extendedprice * (1.00:Decimal - l_discount)) as $expr1, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, l_extendedprice, l_discount, n_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, l_orderkey] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = c_nationkey, output: [n_name, c_custkey, c_name, c_address, c_phone, c_acctbal, c_comment, _row_id, n_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(c_nationkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_orderkey = o_orderkey, output: [l_extendedprice, l_discount, o_custkey, _row_id, l_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamFilter { predicate: (l_returnflag = 'R':Varchar) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_orderkey) } + └─StreamFilter { predicate: (o_orderdate >= '1994-01-01':Date) AND (o_orderdate < '1994-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q11 + before: + - create_tables + sql: | + select + ps_partkey, + sum(ps_supplycost * ps_availqty) as value + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + group by + ps_partkey + having + sum(ps_supplycost * ps_availqty) > ( + select + sum(ps_supplycost * ps_availqty) * 0.0001000000 + from + partsupp, + supplier, + nation + where + ps_suppkey = s_suppkey + and s_nationkey = n_nationkey + and n_name = 'ARGENTINA' + ) + order by + value desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [ps_partkey, value], stream_key: [], pk_columns: [value], pk_conflict: NoCheck } + └─StreamProject { exprs: [ps_partkey, sum($expr1)] } + └─StreamTopN { order: [sum($expr1) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [sum($expr1) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [ps_partkey, sum($expr1), Vnode(ps_partkey) as _vnode] } + └─StreamDynamicFilter { predicate: (sum($expr1) > $expr3), output: [ps_partkey, sum($expr1)] } + ├─StreamProject { exprs: [ps_partkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [ps_partkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, (ps_supplycost * ps_availqty::Decimal) as $expr1, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + │ └─StreamShare { id: 11 } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum($expr2)) * 0.0001000000:Decimal) as $expr3] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr2)] } + └─StreamProject { exprs: [(ps_supplycost * ps_availqty::Decimal) as $expr2, _row_id, _row_id, ps_suppkey, _row_id, s_nationkey] } + └─StreamShare { id: 11 } + └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [ps_partkey, ps_availqty, ps_supplycost, _row_id, _row_id, ps_suppkey, s_nationkey, _row_id] } + ├─StreamExchange { dist: HashShard(s_nationkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_suppkey = s_suppkey, output: [ps_partkey, ps_availqty, ps_supplycost, s_nationkey, _row_id, ps_suppkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamRowIdGen { row_id_index: 10 } + │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(n_nationkey) } + └─StreamFilter { predicate: (n_name = 'ARGENTINA':Varchar) } + └─StreamRowIdGen { row_id_index: 7 } + └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q12 + before: + - create_tables + sql: | + select + l_shipmode, + sum(case + when o_orderpriority = '1-URGENT' + or o_orderpriority = '2-HIGH' + then 1 + else 0 + end) as high_line_count, + sum(case + when o_orderpriority <> '1-URGENT' + and o_orderpriority <> '2-HIGH' + then 1 + else 0 + end) as low_line_count + from + orders, + lineitem + where + o_orderkey = l_orderkey + and l_shipmode in ('FOB', 'SHIP') + and l_commitdate < l_receiptdate + and l_shipdate < l_commitdate + and l_receiptdate >= date '1994-01-01' + and l_receiptdate < date '1994-01-01' + interval '1' year + group by + l_shipmode + order by + l_shipmode + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [l_shipmode, high_line_count, low_line_count], stream_key: [], pk_columns: [l_shipmode], pk_conflict: NoCheck } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2)] } + └─StreamTopN { order: [l_shipmode ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [l_shipmode ASC], limit: 1, offset: 0, group_key: [$expr3] } + └─StreamProject { exprs: [l_shipmode, sum($expr1), sum($expr2), Vnode(l_shipmode) as $expr3] } + └─StreamHashAgg [append_only] { group_key: [l_shipmode], aggs: [sum($expr1), sum($expr2), count] } + └─StreamExchange { dist: HashShard(l_shipmode) } + └─StreamProject { exprs: [l_shipmode, Case(((o_orderpriority = '1-URGENT':Varchar) OR (o_orderpriority = '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr1, Case(((o_orderpriority <> '1-URGENT':Varchar) AND (o_orderpriority <> '2-HIGH':Varchar)), 1:Int32, 0:Int32) as $expr2, _row_id, _row_id, o_orderkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [o_orderpriority, l_shipmode, _row_id, o_orderkey, _row_id] } + ├─StreamExchange { dist: HashShard(o_orderkey) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamFilter { predicate: In(l_shipmode, 'FOB':Varchar, 'SHIP':Varchar) AND (l_commitdate < l_receiptdate) AND (l_shipdate < l_commitdate) AND (l_receiptdate >= '1994-01-01':Date) AND (l_receiptdate < '1995-01-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q13 + before: + - create_tables + sql: | + select + c_count, + count(*) as custdist + from + ( + select + c_custkey, + count(o_orderkey) as c_count + from + customer left outer join orders on + c_custkey = o_custkey + and o_comment not like '%:1%:2%' + group by + c_custkey + ) as c_orders (c_custkey, c_count) + group by + c_count + order by + custdist desc, + c_count desc + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [c_count, custdist], stream_key: [], pk_columns: [custdist, c_count], pk_conflict: NoCheck } + └─StreamProject { exprs: [count(o_orderkey), count] } + └─StreamTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, count(o_orderkey) DESC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [count(o_orderkey), count, Vnode(count(o_orderkey)) as _vnode] } + └─StreamHashAgg { group_key: [count(o_orderkey)], aggs: [count] } + └─StreamExchange { dist: HashShard(count(o_orderkey)) } + └─StreamProject { exprs: [c_custkey, count(o_orderkey)] } + └─StreamHashAgg { group_key: [c_custkey], aggs: [count(o_orderkey), count] } + └─StreamHashJoin { type: LeftOuter, predicate: c_custkey = o_custkey, output: [c_custkey, o_orderkey, _row_id, _row_id] } + ├─StreamExchange { dist: HashShard(c_custkey) } + │ └─StreamRowIdGen { row_id_index: 11 } + │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(o_custkey) } + └─StreamProject { exprs: [o_orderkey, o_custkey, _row_id] } + └─StreamFilter { predicate: Not(Like(o_comment, '%:1%:2%':Varchar)) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q14 + before: + - create_tables + sql: | + select + 100.00 * sum(case + when p_type like 'PROMO%' + then l_extendedprice * (1 - l_discount) + else 0 + end) / sum(l_extendedprice * (1 - l_discount)) as promo_revenue + from + lineitem, + part + where + l_partkey = p_partkey + and l_shipdate >= date '1995-09-01' + and l_shipdate < date '1995-09-01' + interval '1' month; + stream_plan: |- + StreamMaterialize { columns: [promo_revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [((100.00:Decimal * sum(sum($expr1))) / sum(sum($expr2))) as $expr3] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr1)), sum(sum($expr2)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr1), sum($expr2)] } + └─StreamProject { exprs: [Case(Like(p_type, 'PROMO%':Varchar), (l_extendedprice * (1:Decimal - l_discount)), 0:Decimal) as $expr1, (l_extendedprice * (1:Decimal - l_discount)) as $expr2, _row_id, _row_id, l_partkey] } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_partkey = p_partkey, output: [l_extendedprice, l_discount, p_type, _row_id, l_partkey, _row_id] } + ├─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1995-09-01':Date) AND (l_shipdate < '1995-10-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q15 + before: + - create_tables + sql: | + with revenue0 (supplier_no, total_revenue) as ( + select + l_suppkey, + sum(l_extendedprice * (1 - l_discount)) + from + lineitem + where + l_shipdate >= date '1993-01-01' + and l_shipdate < date '1993-01-01' + interval '3' month + group by + l_suppkey + ) + select + s_suppkey, + s_name, + s_address, + s_phone, + total_revenue + from + supplier, + revenue0 + where + s_suppkey = supplier_no + and total_revenue = ( + select + max(total_revenue) + from + revenue0 + ) + order by + s_suppkey + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [s_suppkey, s_name, s_address, s_phone, total_revenue, _row_id(hidden)], stream_key: [], pk_columns: [s_suppkey], pk_conflict: NoCheck } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + └─StreamTopN { order: [s_suppkey ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_suppkey ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, Vnode(sum($expr1)) as _vnode] } + └─StreamHashJoin { type: Inner, predicate: sum($expr1) = max(max(sum($expr1))), output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id] } + ├─StreamExchange { dist: HashShard(sum($expr1)) } + │ └─StreamHashJoin { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_suppkey, s_name, s_address, s_phone, sum($expr1), _row_id, l_suppkey] } + │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + │ └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + │ └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(max(max(sum($expr1)))) } + └─StreamProject { exprs: [max(max(sum($expr1)))] } + └─StreamSimpleAgg { aggs: [max(max(sum($expr1))), count] } + └─StreamExchange { dist: Single } + └─StreamHashAgg { group_key: [_vnode], aggs: [max(sum($expr1)), count] } + └─StreamProject { exprs: [l_suppkey, sum($expr1), Vnode(l_suppkey) as _vnode] } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_suppkey, sum($expr1)] } + └─StreamHashAgg [append_only] { group_key: [l_suppkey], aggs: [sum($expr1), count] } + └─StreamExchange { dist: HashShard(l_suppkey) } + └─StreamProject { exprs: [l_suppkey, (l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id] } + └─StreamFilter { predicate: (l_shipdate >= '1993-01-01':Date) AND (l_shipdate < '1993-04-01 00:00:00':Timestamp) } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q16 + before: + - create_tables + sql: | + select + p_brand, + p_type, + p_size, + count(distinct ps_suppkey) as supplier_cnt + from + partsupp, + part + where + p_partkey = ps_partkey + and p_brand <> 'Brand#45' + and p_type not like 'SMALL PLATED%' + and p_size in (19, 17, 16, 23, 10, 4, 38, 11) + and ps_suppkey not in ( + select + s_suppkey + from + supplier + where + s_comment like '%Customer%Complaints%' + ) + group by + p_brand, + p_type, + p_size + order by + supplier_cnt desc, + p_brand, + p_type, + p_size + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [p_brand, p_type, p_size, supplier_cnt], stream_key: [], pk_columns: [supplier_cnt, p_brand, p_type, p_size], pk_conflict: NoCheck } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey)] } + └─StreamTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count(distinct ps_suppkey) DESC, p_brand ASC, p_type ASC, p_size ASC], limit: 1, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [p_brand, p_type, p_size, count(distinct ps_suppkey), Vnode(p_brand, p_type, p_size) as $expr1] } + └─StreamHashAgg { group_key: [p_brand, p_type, p_size], aggs: [count(distinct ps_suppkey), count] } + └─StreamExchange { dist: HashShard(p_brand, p_type, p_size) } + └─StreamHashJoin { type: LeftAnti, predicate: ps_suppkey = s_suppkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: ps_partkey = p_partkey, output: [ps_suppkey, p_brand, p_type, p_size, _row_id, ps_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(ps_partkey) } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamFilter { predicate: (p_brand <> 'Brand#45':Varchar) AND Not(Like(p_type, 'SMALL PLATED%':Varchar)) AND In(p_size, 19:Int32, 17:Int32, 16:Int32, 23:Int32, 10:Int32, 4:Int32, 38:Int32, 11:Int32) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(s_suppkey) } + └─StreamProject { exprs: [s_suppkey, _row_id] } + └─StreamFilter { predicate: Like(s_comment, '%Customer%Complaints%':Varchar) } + └─StreamRowIdGen { row_id_index: 10 } + └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q17 + before: + - create_tables + sql: | + select + sum(l_extendedprice) / 7.0 as avg_yearly + from + lineitem, + part + where + p_partkey = l_partkey + and p_brand = 'Brand#13' + and p_container = 'JUMBO PKG' + and l_quantity < ( + select + 0.2 * avg(l_quantity) + from + lineitem + where + l_partkey = p_partkey + ); + stream_plan: |- + StreamMaterialize { columns: [avg_yearly], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [(sum(sum(l_extendedprice)) / 7.0:Decimal) as $expr2] } + └─StreamSimpleAgg { aggs: [sum(sum(l_extendedprice)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(l_extendedprice)] } + └─StreamProject { exprs: [l_extendedprice, _row_id, _row_id, l_partkey, p_partkey, l_partkey] } + └─StreamFilter { predicate: (l_quantity < $expr1) } + └─StreamHashJoin { type: Inner, predicate: p_partkey = l_partkey, output: all } + ├─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: l_partkey = p_partkey, output: [l_quantity, l_extendedprice, p_partkey, _row_id, l_partkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(l_partkey) } + │ │ └─StreamShare { id: 3 } + │ │ └─StreamProject { exprs: [l_partkey, l_quantity, l_extendedprice, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 19 } + │ │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(p_partkey) } + │ └─StreamFilter { predicate: (p_brand = 'Brand#13':Varchar) AND (p_container = 'JUMBO PKG':Varchar) } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [(0.2:Decimal * (sum(l_quantity) / count(l_quantity)::Decimal)) as $expr1, l_partkey] } + └─StreamHashAgg [append_only] { group_key: [l_partkey], aggs: [sum(l_quantity), count(l_quantity), count] } + └─StreamExchange { dist: HashShard(l_partkey) } + └─StreamShare { id: 3 } + └─StreamProject { exprs: [l_partkey, l_quantity, l_extendedprice, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q18 + before: + - create_tables + sql: | + select + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice, + sum(l_quantity) quantity + from + customer, + orders, + lineitem + where + o_orderkey in ( + select + l_orderkey + from + lineitem + group by + l_orderkey + having + sum(l_quantity) > 1 + ) + and c_custkey = o_custkey + and o_orderkey = l_orderkey + group by + c_name, + c_custkey, + o_orderkey, + o_orderdate, + o_totalprice + order by + o_totalprice desc, + o_orderdate + LIMIT 100; + stream_plan: |- + StreamMaterialize { columns: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, quantity], stream_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], pk_columns: [o_totalprice, o_orderdate, c_custkey, c_name, o_orderkey], pk_conflict: NoCheck } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity)] } + └─StreamTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [o_totalprice DESC, o_orderdate ASC], limit: 100, offset: 0, group_key: [$expr1] } + └─StreamProject { exprs: [c_name, c_custkey, o_orderkey, o_orderdate, o_totalprice, sum(l_quantity), Vnode(o_orderkey) as $expr1] } + └─StreamHashAgg { group_key: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate], aggs: [sum(l_quantity), count] } + └─StreamHashJoin { type: LeftSemi, predicate: o_orderkey = l_orderkey, output: all } + ├─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, l_quantity, _row_id, _row_id, _row_id] } + │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: c_custkey = o_custkey, output: [c_custkey, c_name, o_orderkey, o_totalprice, o_orderdate, _row_id, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(o_custkey) } + │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamShare { id: 9 } + │ └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamProject { exprs: [l_orderkey] } + └─StreamFilter { predicate: (sum(l_quantity) > 1:Decimal) } + └─StreamProject { exprs: [l_orderkey, sum(l_quantity)] } + └─StreamHashAgg [append_only] { group_key: [l_orderkey], aggs: [sum(l_quantity), count] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamShare { id: 9 } + └─StreamProject { exprs: [l_orderkey, l_quantity, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q19 + before: + - create_tables + sql: | + select + sum(l_extendedprice* (1 - l_discount)) as revenue + from + lineitem, + part + where + ( + p_partkey = l_partkey + and p_brand = 'Brand#52' + and p_container in ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG') + and l_quantity >= 1 and l_quantity <= 11 + and p_size between 1 and 5 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#24' + and p_container in ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK') + and l_quantity >= 30 and l_quantity <= 40 + and p_size between 1 and 10 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ) + or + ( + p_partkey = l_partkey + and p_brand = 'Brand#32' + and p_container in ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG') + and l_quantity >= 10 and l_quantity <= 20 + and p_size between 1 and 15 + and l_shipmode in ('AIR', 'AIR REG') + and l_shipinstruct = 'DELIVER IN PERSON' + ); + stream_plan: |- + StreamMaterialize { columns: [revenue], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamProject { exprs: [sum(sum($expr1))] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum($expr1)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum($expr1)] } + └─StreamProject { exprs: [(l_extendedprice * (1:Decimal - l_discount)) as $expr1, _row_id, _row_id, l_partkey] } + └─StreamFilter { predicate: ((((((p_brand = 'Brand#52':Varchar) AND In(p_container, 'SM CASE':Varchar, 'SM BOX':Varchar, 'SM PACK':Varchar, 'SM PKG':Varchar)) AND ((l_quantity >= 1:Decimal) AND (l_quantity <= 11:Decimal))) AND (p_size <= 5:Int32)) OR ((((p_brand = 'Brand#24':Varchar) AND In(p_container, 'MED BAG':Varchar, 'MED BOX':Varchar, 'MED PKG':Varchar, 'MED PACK':Varchar)) AND ((l_quantity >= 30:Decimal) AND (l_quantity <= 40:Decimal))) AND (p_size <= 10:Int32))) OR ((((p_brand = 'Brand#32':Varchar) AND In(p_container, 'LG CASE':Varchar, 'LG BOX':Varchar, 'LG PACK':Varchar, 'LG PKG':Varchar)) AND ((l_quantity >= 10:Decimal) AND (l_quantity <= 20:Decimal))) AND (p_size <= 15:Int32))) } + └─StreamHashJoin [append_only] { type: Inner, predicate: l_partkey = p_partkey, output: all } + ├─StreamExchange { dist: HashShard(l_partkey) } + │ └─StreamFilter { predicate: In(l_shipmode, 'AIR':Varchar, 'AIR REG':Varchar) AND (l_shipinstruct = 'DELIVER IN PERSON':Varchar) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamFilter { predicate: (p_size >= 1:Int32) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q20 + before: + - create_tables + sql: "select\n\ts_name,\n\ts_address\nfrom\n\tsupplier,\n\tnation\nwhere\n\ts_suppkey in (\n\t\tselect\n\t\t\tps_suppkey\n\t\tfrom\n\t\t\tpartsupp,\n\t\t\t(\n\t\t\t\tselect\n\t\t\t\t\tl_partkey agg_partkey,\n\t\t\t\t\tl_suppkey agg_suppkey,\n\t\t\t\t\t0.5 * sum(l_quantity) AS agg_quantity\n\t\t\t\tfrom\n\t\t\t\t\tlineitem\n\t\t\t\twhere\n\t\t\t\t\tl_shipdate >= date '1994-01-01'\n\t\t\t\t\tand l_shipdate < date '1994-01-01' + interval '1' year\n\t\t\t\tgroup by\n\t\t\t\t\tl_partkey,\n\t\t\t\t\tl_suppkey\n\t\t\t) agg_lineitem\n\t\twhere\n\t\t\tagg_partkey = ps_partkey\n\t\t\tand agg_suppkey = ps_suppkey\n\t\t\tand ps_partkey in (\n\t\t\t\tselect\n\t\t\t\t\tp_partkey\n\t\t\t\tfrom\n\t\t\t\t\tpart\n\t\t\t\twhere\n\t\t\t\t\tp_name like 'forest%'\n\t\t\t)\n\t\t\tand ps_availqty > agg_quantity\n\t)\n\tand s_nationkey = n_nationkey\n\tand n_name = 'KENYA'\norder by\n\ts_name\nLIMIT 1;\n" + stream_plan: |- + StreamMaterialize { columns: [s_name, s_address, _row_id(hidden), _row_id#1(hidden), s_nationkey(hidden), s_suppkey(hidden)], stream_key: [], pk_columns: [s_name], pk_conflict: NoCheck } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + └─StreamTopN { order: [s_name ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [s_name ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey, Vnode(s_suppkey) as _vnode] } + └─StreamHashJoin { type: LeftSemi, predicate: s_suppkey = ps_suppkey, output: [s_name, s_address, _row_id, _row_id, s_nationkey, s_suppkey] } + ├─StreamExchange { dist: HashShard(s_suppkey) } + │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_nationkey = n_nationkey, output: [s_suppkey, s_name, s_address, _row_id, s_nationkey, _row_id] } + │ ├─StreamExchange { dist: HashShard(s_nationkey) } + │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(n_nationkey) } + │ └─StreamFilter { predicate: (n_name = 'KENYA':Varchar) } + │ └─StreamRowIdGen { row_id_index: 7 } + │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(ps_suppkey) } + └─StreamHashJoin { type: LeftSemi, predicate: ps_partkey = p_partkey, output: [ps_suppkey, _row_id, ps_partkey] } + ├─StreamExchange { dist: HashShard(ps_partkey) } + │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, _row_id, l_partkey, l_suppkey] } + │ └─StreamFilter { predicate: ($expr1 > $expr2) } + │ └─StreamHashJoin { type: Inner, predicate: ps_partkey = l_partkey AND ps_suppkey = l_suppkey, output: all } + │ ├─StreamExchange { dist: HashShard(ps_partkey, ps_suppkey) } + │ │ └─StreamProject { exprs: [ps_partkey, ps_suppkey, ps_availqty::Decimal as $expr1, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 8 } + │ │ └─StreamSourceScan { columns: [ps_partkey, ps_suppkey, ps_availqty, ps_supplycost, ps_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamProject { exprs: [l_partkey, l_suppkey, (0.5:Decimal * sum(l_quantity)) as $expr2] } + │ └─StreamHashAgg [append_only] { group_key: [l_partkey, l_suppkey], aggs: [sum(l_quantity), count] } + │ └─StreamExchange { dist: HashShard(l_partkey, l_suppkey) } + │ └─StreamFilter { predicate: (l_shipdate >= '1994-01-01':Date) AND (l_shipdate < '1995-01-01 00:00:00':Timestamp) } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(p_partkey) } + └─StreamProject { exprs: [p_partkey, _row_id] } + └─StreamFilter { predicate: Like(p_name, 'forest%':Varchar) } + └─StreamRowIdGen { row_id_index: 12 } + └─StreamSourceScan { columns: [p_partkey, p_name, p_mfgr, p_brand, p_type, p_size, p_container, p_retailprice, p_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q21 + before: + - create_tables + sql: | + select + s_name, + count(*) as numwait + from + supplier, + lineitem l1, + orders, + nation + where + s_suppkey = l1.l_suppkey + and o_orderkey = l1.l_orderkey + and o_orderstatus = 'F' + and l1.l_receiptdate > l1.l_commitdate + and exists ( + select + * + from + lineitem l2 + where + l2.l_orderkey = l1.l_orderkey + and l2.l_suppkey <> l1.l_suppkey + ) + and not exists ( + select + * + from + lineitem l3 + where + l3.l_orderkey = l1.l_orderkey + and l3.l_suppkey <> l1.l_suppkey + and l3.l_receiptdate > l3.l_commitdate + ) + and s_nationkey = n_nationkey + and n_name = 'GERMANY' + group by + s_name + order by + numwait desc, + s_name + LIMIT 100; + stream_plan: |- + StreamMaterialize { columns: [s_name, numwait], stream_key: [s_name], pk_columns: [numwait, s_name], pk_conflict: NoCheck } + └─StreamProject { exprs: [s_name, count] } + └─StreamTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [count DESC, s_name ASC], limit: 100, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [s_name, count, Vnode(s_name) as _vnode] } + └─StreamHashAgg { group_key: [s_name], aggs: [count] } + └─StreamExchange { dist: HashShard(s_name) } + └─StreamHashJoin { type: LeftAnti, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey, l_orderkey] } + ├─StreamHashJoin { type: LeftSemi, predicate: l_orderkey = l_orderkey AND (l_suppkey <> l_suppkey), output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, _row_id, _row_id, o_orderkey, s_suppkey] } + │ ├─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: s_suppkey = l_suppkey, output: [s_name, l_orderkey, l_suppkey, _row_id, _row_id, n_nationkey, s_suppkey, _row_id, _row_id, o_orderkey] } + │ │ ├─StreamExchange { dist: HashShard(s_suppkey) } + │ │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: n_nationkey = s_nationkey, output: [s_suppkey, s_name, _row_id, n_nationkey, _row_id] } + │ │ │ ├─StreamExchange { dist: HashShard(n_nationkey) } + │ │ │ │ └─StreamFilter { predicate: (n_name = 'GERMANY':Varchar) } + │ │ │ │ └─StreamRowIdGen { row_id_index: 7 } + │ │ │ │ └─StreamSourceScan { columns: [n_nationkey, n_name, n_regionkey, n_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ │ └─StreamExchange { dist: HashShard(s_nationkey) } + │ │ │ └─StreamRowIdGen { row_id_index: 10 } + │ │ │ └─StreamSourceScan { columns: [s_suppkey, s_name, s_address, s_nationkey, s_phone, s_acctbal, s_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_suppkey) } + │ │ └─StreamHashJoin [append_only] { type: Inner, predicate: o_orderkey = l_orderkey, output: [l_orderkey, l_suppkey, _row_id, o_orderkey, _row_id] } + │ │ ├─StreamExchange { dist: HashShard(o_orderkey) } + │ │ │ └─StreamFilter { predicate: (o_orderstatus = 'F':Varchar) } + │ │ │ └─StreamRowIdGen { row_id_index: 12 } + │ │ │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ │ └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + │ │ └─StreamShare { id: 13 } + │ │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ │ └─StreamRowIdGen { row_id_index: 19 } + │ │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(l_orderkey) } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + │ └─StreamShare { id: 13 } + │ └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + │ └─StreamRowIdGen { row_id_index: 19 } + │ └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: HashShard(l_orderkey) } + └─StreamProject { exprs: [l_orderkey, l_suppkey, _row_id] } + └─StreamFilter { predicate: (l_receiptdate > l_commitdate) } + └─StreamShare { id: 13 } + └─StreamProject { exprs: [l_orderkey, l_suppkey, l_commitdate, l_receiptdate, _row_id] } + └─StreamRowIdGen { row_id_index: 19 } + └─StreamSourceScan { columns: [l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } +- id: tpch_q22 + before: + - create_tables + sql: | + select + cntrycode, + count(*) as numcust, + sum(c_acctbal) as totacctbal + from + ( + select + substring(c_phone from 1 for 2) as cntrycode, + c_acctbal + from + customer + where + substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + and c_acctbal > ( + select + avg(c_acctbal) + from + customer + where + c_acctbal > 0.00::numeric + and substring(c_phone from 1 for 2) in + ('30', '24', '31', '38', '25', '34', '37') + ) + and not exists ( + select + * + from + orders + where + o_custkey = c_custkey + ) + ) as custsale + group by + cntrycode + order by + cntrycode + LIMIT 1; + stream_plan: |- + StreamMaterialize { columns: [cntrycode, numcust, totacctbal], stream_key: [], pk_columns: [cntrycode], pk_conflict: NoCheck } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal)] } + └─StreamTopN { order: [$expr2 ASC], limit: 1, offset: 0 } + └─StreamExchange { dist: Single } + └─StreamGroupTopN { order: [$expr2 ASC], limit: 1, offset: 0, group_key: [_vnode] } + └─StreamProject { exprs: [$expr2, count, sum(c_acctbal), Vnode($expr2) as _vnode] } + └─StreamHashAgg { group_key: [$expr2], aggs: [count, sum(c_acctbal)] } + └─StreamExchange { dist: HashShard($expr2) } + └─StreamProject { exprs: [Substr(c_phone, 1:Int32, 2:Int32) as $expr2, c_acctbal, _row_id, c_custkey] } + └─StreamDynamicFilter { predicate: (c_acctbal > $expr1), output: [c_phone, c_acctbal, _row_id, c_custkey] } + ├─StreamHashJoin { type: LeftAnti, predicate: c_custkey = o_custkey, output: [c_phone, c_acctbal, _row_id, c_custkey] } + │ ├─StreamExchange { dist: HashShard(c_custkey) } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamShare { id: 4 } + │ │ └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + │ │ └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + │ │ └─StreamRowIdGen { row_id_index: 11 } + │ │ └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + │ └─StreamExchange { dist: HashShard(o_custkey) } + │ └─StreamProject { exprs: [o_custkey, _row_id] } + │ └─StreamRowIdGen { row_id_index: 12 } + │ └─StreamSourceScan { columns: [o_orderkey, o_custkey, o_orderstatus, o_totalprice, o_orderdate, o_orderpriority, o_clerk, o_shippriority, o_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } + └─StreamExchange { dist: Broadcast } + └─StreamProject { exprs: [(sum(sum(c_acctbal)) / sum0(count(c_acctbal))::Decimal) as $expr1] } + └─StreamSimpleAgg [append_only] { aggs: [sum(sum(c_acctbal)), sum0(count(c_acctbal)), count] } + └─StreamExchange { dist: Single } + └─StreamStatelessSimpleAgg { aggs: [sum(c_acctbal), count(c_acctbal)] } + └─StreamFilter { predicate: (c_acctbal > 0.00:Decimal) AND In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamShare { id: 4 } + └─StreamProject { exprs: [c_custkey, c_phone, c_acctbal, _row_id] } + └─StreamFilter { predicate: In(Substr(c_phone, 1:Int32, 2:Int32), '30':Varchar, '24':Varchar, '31':Varchar, '38':Varchar, '25':Varchar, '34':Varchar, '37':Varchar) } + └─StreamRowIdGen { row_id_index: 11 } + └─StreamSourceScan { columns: [c_custkey, c_name, c_address, c_nationkey, c_phone, c_acctbal, c_mktsegment, c_comment, _rw_kafka_timestamp, _rw_kafka_partition, _rw_kafka_offset, _row_id] } From 15ba09dcb6808fd75a025a05603efc265c86b7a6 Mon Sep 17 00:00:00 2001 From: Noel Kwan <47273164+kwannoel@users.noreply.github.com> Date: Thu, 28 Nov 2024 09:13:21 +0800 Subject: [PATCH 036/163] feat(connector): support postgres_sink in rust (#19328) --- e2e_test/sink/postgres_sink.slt | 461 +++++++++++++ risedev.yml | 1 + src/common/src/types/jsonb.rs | 19 + src/common/src/types/postgres_type.rs | 7 + src/common/src/types/to_sql.rs | 41 ++ .../maybe_tls_connector.rs | 0 src/connector/src/connector_common/mod.rs | 5 + .../src/connector_common/postgres.rs | 450 +++++++++++++ src/connector/src/parser/scalar_adapter.rs | 16 + src/connector/src/sink/mod.rs | 15 + src/connector/src/sink/postgres.rs | 622 ++++++++++++++++++ src/connector/src/source/cdc/external/mod.rs | 67 +- .../src/source/cdc/external/postgres.rs | 306 +-------- .../src/task/task_log_ready_check.rs | 2 +- 14 files changed, 1691 insertions(+), 321 deletions(-) create mode 100644 e2e_test/sink/postgres_sink.slt rename src/connector/src/{source/cdc/external => connector_common}/maybe_tls_connector.rs (100%) create mode 100644 src/connector/src/connector_common/postgres.rs create mode 100644 src/connector/src/sink/postgres.rs diff --git a/e2e_test/sink/postgres_sink.slt b/e2e_test/sink/postgres_sink.slt new file mode 100644 index 0000000000000..c680d3992d68e --- /dev/null +++ b/e2e_test/sink/postgres_sink.slt @@ -0,0 +1,461 @@ +control substitution on + +system ok +PGDATABASE=postgres psql -c "DROP DATABASE IF EXISTS sink_test WITH (FORCE)" + +statement ok +drop table if exists rw_types_table cascade; + +statement ok +drop table if exists rw_types_table_append_only cascade; + +system ok +PGDATABASE=sink_test createdb + +################### test table pk can mismatch + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT, + varchar_column VARCHAR, + text_column TEXT, + smallint_column SMALLINT, + integer_column INTEGER, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + interval_column INTERVAL, + jsonb_column JSONB, + timestamp_column TIMESTAMP +); + +statement ok +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id', +); + +statement ok +drop table rw_types_table cascade; + +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table"; + +################### test table col name cannot mismatch + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT, + varchar_column_mismatch_name VARCHAR, + text_column TEXT, + smallint_column SMALLINT, + integer_column INTEGER, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + interval_column INTERVAL, + jsonb_column JSONB, + timestamp_column TIMESTAMP +); + +statement error +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id', +); + +statement ok +drop table rw_types_table cascade; + +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table"; + +################### test table col can be out of order + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + smallint_column SMALLINT, + integer_column INTEGER, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + interval_column INTERVAL, + jsonb_column JSONB, + timestamp_column TIMESTAMP +); + +################### test sink with_options:primary_key mismatch should fail + +statement error +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id, date_column', +); + +################### test pk match should work + +statement ok +CREATE SINK postgres_rw_types_sink FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id', +); + +################### test insert + +statement ok +INSERT INTO rw_types_table (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column) VALUES + (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'), + (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 'NAN'::decimal, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}'), + (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, '+INF'::decimal, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'); + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} +3 Varcharvalue1 Textvalue1 123 456 789 Infinity 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +################### test upsert (update) + +statement ok +UPDATE rw_types_table SET varchar_column = 'Varcharvalue3', smallint_column = '300' WHERE id = 3; + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} +3 Varcharvalue3 Textvalue1 123 300 789 Infinity 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +################### test delete + +statement ok +DELETE FROM rw_types_table WHERE id = 3; + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} + +################### test upsert (insert) + +system ok +PGDATABASE=sink_test psql -c "DELETE FROM pg_types_table WHERE id = 2" + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +statement ok +UPDATE rw_types_table SET varchar_column = 'Varcharvalue4', smallint_column = '400' WHERE id = 2; + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by id;'); +---- +1 Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +2 Varcharvalue4 Textvalue2 234 400 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} + +################### cleanup sink + +statement ok +DROP SINK postgres_rw_types_sink; + +statement ok +DROP TABLE rw_types_table; + +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table" + +################### test append-only sink + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +)" + +statement ok +CREATE TABLE rw_types_table_append_only ( + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB +) APPEND ONLY; + +statement ok +CREATE SINK postgres_rw_types_sink_append_only FROM rw_types_table_append_only WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='append-only', +); + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table;'); +---- + +statement ok +INSERT INTO rw_types_table_append_only (varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column) VALUES + ('Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'), + ('Varcharvalue2', 'Textvalue2', 234, 567, 890, 'NAN'::decimal, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}'), + ('Varcharvalue3', 'Textvalue1', 333, 456, 789, '+INF'::decimal, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}'); + +statement ok +flush; + +query I +select * from postgres_query('$PGHOST', '$PGPORT', '$PGUSER', '${PGPASSWORD:postgres}', 'sink_test', 'select * from pg_types_table order by integer_column;'); +---- +Varcharvalue1 Textvalue1 123 456 789 12.34 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} +Varcharvalue2 Textvalue2 234 567 890 NaN 67.89 1.23 f 2023-05-23 23:45:01 2023-05-23 23:45:01 2 days {"key": "value2"} +Varcharvalue3 Textvalue1 333 456 789 Infinity 56.78 90.12 t 2023-05-22 12:34:56 2023-05-22 12:34:56 1 day {"key": "value"} + +################### cleanup append-only sink + +statement ok +DROP SINK postgres_rw_types_sink_append_only; + +statement ok +DROP TABLE rw_types_table_append_only; + +system ok +PGDATABASE=sink_test psql -c "DROP TABLE pg_types_table" + +################### Test pg special types + +system ok +PGDATABASE=sink_test psql -c "CREATE TYPE MY_ENUM AS ENUM ('a', 'b', 'c');" + +system ok +PGDATABASE=sink_test psql -c "CREATE TABLE pg_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB, + uuid_column UUID, + numeric_column NUMERIC, + myenum_column MY_ENUM, + numeric_decimal_array_column NUMERIC[], + numeric_utf8_array_column NUMERIC[], + enum_array_column MY_ENUM[], + int_column int[] +)" + +statement ok +CREATE TABLE rw_types_table ( + id BIGINT PRIMARY KEY, + varchar_column VARCHAR, + text_column TEXT, + integer_column INTEGER, + smallint_column SMALLINT, + bigint_column BIGINT, + decimal_column DECIMAL, + real_column REAL, + double_column DOUBLE PRECISION, + boolean_column BOOLEAN, + date_column DATE, + time_column TIME, + timestamp_column TIMESTAMP, + interval_column INTERVAL, + jsonb_column JSONB, + uuid_column VARCHAR, + numeric_column VARCHAR, + myenum_column VARCHAR, + numeric_decimal_array_column DECIMAL[], + numeric_utf8_array_column VARCHAR[], + enum_array_column VARCHAR[], + int_column int[] +) + +statement ok +CREATE SINK rw_types_to_pg_types FROM rw_types_table WITH ( + connector='postgres', + host='$PGHOST', + port='$PGPORT', + user='$PGUSER', + password='$PGPASSWORD', + database='sink_test', + table='pg_types_table', + type='upsert', + primary_key='id', +); + +statement ok +INSERT INTO rw_types_table (id, varchar_column, text_column, integer_column, smallint_column, bigint_column, decimal_column, real_column, double_column, boolean_column, date_column, time_column, timestamp_column, interval_column, jsonb_column, uuid_column, numeric_column, myenum_column, numeric_decimal_array_column, numeric_utf8_array_column, enum_array_column, int_column) VALUES + (1, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, 12.34, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', '448be8d9-297f-4514-85c6-a360e82ae331', '123', 'a', ARRAY[1, 2, 3], ARRAY['1', '2', '3'], ARRAY['a', 'b', 'c'], ARRAY[1, 2, 3]), + (2, 'Varcharvalue2', 'Textvalue2', 234, 567, 890, 'NAN'::decimal, 67.89, 01.23, FALSE, '2023-05-23', '23:45:01', '2023-05-23 23:45:01', '2 days', '{"key": "value2"}', '14b4431e-203a-452b-a331-4d8a0d8f952e', '456', 'b', ARRAY[10, 20, 30], ARRAY['10', '20', '30'], ARRAY['a', 'b', 'c'], ARRAY[10, 20, 30]), + (3, 'Varcharvalue1', 'Textvalue1', 123, 456, 789, '+INF'::decimal, 56.78, 90.12, TRUE, '2023-05-22', '12:34:56', '2023-05-22 12:34:56', '1 day', '{"key": "value"}', '024d2719-ca29-45e1-bc58-4ed38777f4bf', '789', 'c', ARRAY[100, 200, 300], ARRAY['100', '200', '300'], ARRAY['a', 'b', 'c'], ARRAY[100, 200, 300]); + +statement ok +flush; + +# NOTE(kwannoel): postgres_query doesn't support struct etc... yet. +system ok +PGDATABASE=sink_test psql --tuples-only -c "select * from pg_types_table order by id;" +---- + 1 | Varcharvalue1 | Textvalue1 | 123 | 456 | 789 | 12.34 | 56.78 | 90.12 | t | 2023-05-22 | 12:34:56 | 2023-05-22 12:34:56 | 1 day | {"key": "value"} | 448be8d9-297f-4514-85c6-a360e82ae331 | 123 | a | {1,2,3} | {1,2,3} | {a,b,c} | {1,2,3} + 2 | Varcharvalue2 | Textvalue2 | 234 | 567 | 890 | NaN | 67.89 | 1.23 | f | 2023-05-23 | 23:45:01 | 2023-05-23 23:45:01 | 2 days | {"key": "value2"} | 14b4431e-203a-452b-a331-4d8a0d8f952e | 456 | b | {10,20,30} | {10,20,30} | {a,b,c} | {10,20,30} + 3 | Varcharvalue1 | Textvalue1 | 123 | 456 | 789 | Infinity | 56.78 | 90.12 | t | 2023-05-22 | 12:34:56 | 2023-05-22 12:34:56 | 1 day | {"key": "value"} | 024d2719-ca29-45e1-bc58-4ed38777f4bf | 789 | c | {100,200,300} | {100,200,300} | {a,b,c} | {100,200,300} + + +statement ok +DROP SINK rw_types_to_pg_types; + +statement ok +DROP TABLE rw_types_table; + +################### Drop DB + +system ok +PGDATABASE=postgres psql -c "DROP DATABASE sink_test" \ No newline at end of file diff --git a/risedev.yml b/risedev.yml index 8e3668dcb49c2..5f7e43d1b75d1 100644 --- a/risedev.yml +++ b/risedev.yml @@ -117,6 +117,7 @@ profile: - use: postgres port: 8432 user: postgres + password: postgres database: metadata - use: meta-node meta-backend: postgres diff --git a/src/common/src/types/jsonb.rs b/src/common/src/types/jsonb.rs index 4b25741fbe96a..2dbc65ec481a1 100644 --- a/src/common/src/types/jsonb.rs +++ b/src/common/src/types/jsonb.rs @@ -619,3 +619,22 @@ impl<'a> FromSql<'a> for JsonbVal { matches!(*ty, Type::JSONB) } } + +impl ToSql for JsonbRef<'_> { + accepts!(JSONB); + + to_sql_checked!(); + + fn to_sql( + &self, + _ty: &Type, + out: &mut BytesMut, + ) -> Result> + where + Self: Sized, + { + out.put_u8(1); + write!(out, "{}", self.0).unwrap(); + Ok(IsNull::No) + } +} diff --git a/src/common/src/types/postgres_type.rs b/src/common/src/types/postgres_type.rs index c84f3e19f3097..32fd5a3f944d5 100644 --- a/src/common/src/types/postgres_type.rs +++ b/src/common/src/types/postgres_type.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use postgres_types::Type as PgType; + use super::DataType; /// `DataType` information extracted from PostgreSQL `pg_type` @@ -149,4 +151,9 @@ impl DataType { } for_all_base_types! { impl_pg_name } } + + pub fn to_pg_type(&self) -> PgType { + let oid = self.to_oid(); + PgType::from_oid(oid as u32).unwrap() + } } diff --git a/src/common/src/types/to_sql.rs b/src/common/src/types/to_sql.rs index 57aab11daf4d7..de9441d8aaf34 100644 --- a/src/common/src/types/to_sql.rs +++ b/src/common/src/types/to_sql.rs @@ -16,6 +16,7 @@ use std::error::Error; use bytes::BytesMut; use postgres_types::{to_sql_checked, IsNull, ToSql, Type}; +use risingwave_common::types::ScalarRefImpl; use crate::types::ScalarImpl; @@ -58,3 +59,43 @@ impl ToSql for ScalarImpl { true } } + +impl ToSql for ScalarRefImpl<'_> { + to_sql_checked!(); + + fn to_sql(&self, ty: &Type, out: &mut BytesMut) -> Result> + where + Self: Sized, + { + match self { + ScalarRefImpl::Int16(v) => v.to_sql(ty, out), + ScalarRefImpl::Int32(v) => v.to_sql(ty, out), + ScalarRefImpl::Int64(v) => v.to_sql(ty, out), + ScalarRefImpl::Serial(v) => v.to_sql(ty, out), + ScalarRefImpl::Float32(v) => v.to_sql(ty, out), + ScalarRefImpl::Float64(v) => v.to_sql(ty, out), + ScalarRefImpl::Utf8(v) => v.to_sql(ty, out), + ScalarRefImpl::Bool(v) => v.to_sql(ty, out), + ScalarRefImpl::Decimal(v) => v.to_sql(ty, out), + ScalarRefImpl::Interval(v) => v.to_sql(ty, out), + ScalarRefImpl::Date(v) => v.to_sql(ty, out), + ScalarRefImpl::Timestamp(v) => v.to_sql(ty, out), + ScalarRefImpl::Timestamptz(v) => v.to_sql(ty, out), + ScalarRefImpl::Time(v) => v.to_sql(ty, out), + ScalarRefImpl::Bytea(v) => (&**v).to_sql(ty, out), + ScalarRefImpl::Jsonb(v) => v.to_sql(ty, out), + ScalarRefImpl::Int256(_) | ScalarRefImpl::Struct(_) | ScalarRefImpl::List(_) => { + bail_not_implemented!("the postgres encoding for {ty} is unsupported") + } + ScalarRefImpl::Map(_) => todo!(), + } + } + + // return true to accept all types + fn accepts(_ty: &Type) -> bool + where + Self: Sized, + { + true + } +} diff --git a/src/connector/src/source/cdc/external/maybe_tls_connector.rs b/src/connector/src/connector_common/maybe_tls_connector.rs similarity index 100% rename from src/connector/src/source/cdc/external/maybe_tls_connector.rs rename to src/connector/src/connector_common/maybe_tls_connector.rs diff --git a/src/connector/src/connector_common/mod.rs b/src/connector/src/connector_common/mod.rs index 57b614fdf548a..a4a2c5a02e3f5 100644 --- a/src/connector/src/connector_common/mod.rs +++ b/src/connector/src/connector_common/mod.rs @@ -25,4 +25,9 @@ pub use common::{ }; mod iceberg; +#[cfg(not(madsim))] +mod maybe_tls_connector; +pub mod postgres; + pub use iceberg::IcebergCommon; +pub use postgres::{create_pg_client, PostgresExternalTable, SslMode}; diff --git a/src/connector/src/connector_common/postgres.rs b/src/connector/src/connector_common/postgres.rs new file mode 100644 index 0000000000000..fc79bf9b961e6 --- /dev/null +++ b/src/connector/src/connector_common/postgres.rs @@ -0,0 +1,450 @@ +// Copyright 2024 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::collections::HashMap; +use std::fmt; + +use anyhow::anyhow; +use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; +use postgres_openssl::MakeTlsConnector; +use risingwave_common::bail; +use risingwave_common::catalog::{ColumnDesc, ColumnId}; +use risingwave_common::types::{DataType, ScalarImpl, StructType}; +use sea_schema::postgres::def::{ColumnType, TableInfo, Type as SeaType}; +use sea_schema::postgres::discovery::SchemaDiscovery; +use serde_derive::Deserialize; +use sqlx::postgres::{PgConnectOptions, PgSslMode}; +use sqlx::PgPool; +use thiserror_ext::AsReport; +use tokio_postgres::types::Kind as PgKind; +use tokio_postgres::{Client as PgClient, NoTls}; + +#[cfg(not(madsim))] +use super::maybe_tls_connector::MaybeMakeTlsConnector; +use crate::error::ConnectorResult; + +#[derive(Debug, Clone, PartialEq, Deserialize)] +#[serde(rename_all = "lowercase")] +pub enum SslMode { + #[serde(alias = "disable")] + Disabled, + #[serde(alias = "prefer")] + Preferred, + #[serde(alias = "require")] + Required, + /// verify that the server is trustworthy by checking the certificate chain + /// up to the root certificate stored on the client. + #[serde(alias = "verify-ca")] + VerifyCa, + /// Besides verify the certificate, will also verify that the serverhost name + /// matches the name stored in the server certificate. + #[serde(alias = "verify-full")] + VerifyFull, +} + +impl Default for SslMode { + fn default() -> Self { + Self::Preferred + } +} + +pub struct PostgresExternalTable { + column_descs: Vec, + pk_names: Vec, + column_name_to_pg_type: HashMap, +} + +impl PostgresExternalTable { + pub async fn connect( + username: &str, + password: &str, + host: &str, + port: u16, + database: &str, + schema: &str, + table: &str, + ssl_mode: &SslMode, + ssl_root_cert: &Option, + is_append_only: bool, + ) -> ConnectorResult { + tracing::debug!("connect to postgres external table"); + let mut options = PgConnectOptions::new() + .username(username) + .password(password) + .host(host) + .port(port) + .database(database) + .ssl_mode(match ssl_mode { + SslMode::Disabled => PgSslMode::Disable, + SslMode::Preferred => PgSslMode::Prefer, + SslMode::Required => PgSslMode::Require, + SslMode::VerifyCa => PgSslMode::VerifyCa, + SslMode::VerifyFull => PgSslMode::VerifyFull, + }); + + if *ssl_mode == SslMode::VerifyCa || *ssl_mode == SslMode::VerifyFull { + if let Some(ref root_cert) = ssl_root_cert { + options = options.ssl_root_cert(root_cert.as_str()); + } + } + + let connection = PgPool::connect_with(options).await?; + let schema_discovery = SchemaDiscovery::new(connection, schema); + // fetch column schema and primary key + let empty_map = HashMap::new(); + let table_schema = schema_discovery + .discover_table( + TableInfo { + name: table.to_string(), + of_type: None, + }, + &empty_map, + ) + .await?; + + let mut column_name_to_pg_type = HashMap::new(); + let mut column_descs = vec![]; + for col in &table_schema.columns { + let data_type = type_to_rw_type(&col.col_type)?; + let column_desc = if let Some(ref default_expr) = col.default { + // parse the value of "column_default" field in information_schema.columns, + // non number data type will be stored as "'value'::type" + let val_text = default_expr + .0 + .split("::") + .map(|s| s.trim_matches('\'')) + .next() + .expect("default value expression"); + + match ScalarImpl::from_text(val_text, &data_type) { + Ok(scalar) => ColumnDesc::named_with_default_value( + col.name.clone(), + ColumnId::placeholder(), + data_type.clone(), + Some(scalar), + ), + Err(err) => { + tracing::warn!(error=%err.as_report(), "failed to parse postgres default value expression, only constant is supported"); + ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) + } + } + } else { + ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) + }; + { + let pg_type = Self::discovered_type_to_pg_type(&col.col_type)?; + column_name_to_pg_type.insert(col.name.clone(), pg_type); + } + column_descs.push(column_desc); + } + + if !is_append_only && table_schema.primary_key_constraints.is_empty() { + return Err(anyhow!( + "Postgres table should define the primary key for non-append-only tables" + ) + .into()); + } + let mut pk_names = vec![]; + table_schema.primary_key_constraints.iter().for_each(|pk| { + pk_names.extend(pk.columns.clone()); + }); + + Ok(Self { + column_descs, + pk_names, + column_name_to_pg_type, + }) + } + + pub fn column_descs(&self) -> &Vec { + &self.column_descs + } + + pub fn pk_names(&self) -> &Vec { + &self.pk_names + } + + pub fn column_name_to_pg_type(&self) -> &HashMap { + &self.column_name_to_pg_type + } + + // We use `sea-schema` for table schema discovery. + // So we have to map `sea-schema` pg types + // to `tokio-postgres` pg types (which we use for query binding). + fn discovered_type_to_pg_type( + discovered_type: &SeaType, + ) -> anyhow::Result { + use tokio_postgres::types::Type as PgType; + match discovered_type { + SeaType::SmallInt => Ok(PgType::INT2), + SeaType::Integer => Ok(PgType::INT4), + SeaType::BigInt => Ok(PgType::INT8), + SeaType::Decimal(_) => Ok(PgType::NUMERIC), + SeaType::Numeric(_) => Ok(PgType::NUMERIC), + SeaType::Real => Ok(PgType::FLOAT4), + SeaType::DoublePrecision => Ok(PgType::FLOAT8), + SeaType::Varchar(_) => Ok(PgType::VARCHAR), + SeaType::Char(_) => Ok(PgType::CHAR), + SeaType::Text => Ok(PgType::TEXT), + SeaType::Bytea => Ok(PgType::BYTEA), + SeaType::Timestamp(_) => Ok(PgType::TIMESTAMP), + SeaType::TimestampWithTimeZone(_) => Ok(PgType::TIMESTAMPTZ), + SeaType::Date => Ok(PgType::DATE), + SeaType::Time(_) => Ok(PgType::TIME), + SeaType::TimeWithTimeZone(_) => Ok(PgType::TIMETZ), + SeaType::Interval(_) => Ok(PgType::INTERVAL), + SeaType::Boolean => Ok(PgType::BOOL), + SeaType::Point => Ok(PgType::POINT), + SeaType::Uuid => Ok(PgType::UUID), + SeaType::JsonBinary => Ok(PgType::JSONB), + SeaType::Array(t) => { + let Some(t) = t.col_type.as_ref() else { + bail!("missing array type") + }; + match t.as_ref() { + // RW only supports 1 level of nesting. + SeaType::SmallInt => Ok(PgType::INT2_ARRAY), + SeaType::Integer => Ok(PgType::INT4_ARRAY), + SeaType::BigInt => Ok(PgType::INT8_ARRAY), + SeaType::Decimal(_) => Ok(PgType::NUMERIC_ARRAY), + SeaType::Numeric(_) => Ok(PgType::NUMERIC_ARRAY), + SeaType::Real => Ok(PgType::FLOAT4_ARRAY), + SeaType::DoublePrecision => Ok(PgType::FLOAT8_ARRAY), + SeaType::Varchar(_) => Ok(PgType::VARCHAR_ARRAY), + SeaType::Char(_) => Ok(PgType::CHAR_ARRAY), + SeaType::Text => Ok(PgType::TEXT_ARRAY), + SeaType::Bytea => Ok(PgType::BYTEA_ARRAY), + SeaType::Timestamp(_) => Ok(PgType::TIMESTAMP_ARRAY), + SeaType::TimestampWithTimeZone(_) => Ok(PgType::TIMESTAMPTZ_ARRAY), + SeaType::Date => Ok(PgType::DATE_ARRAY), + SeaType::Time(_) => Ok(PgType::TIME_ARRAY), + SeaType::TimeWithTimeZone(_) => Ok(PgType::TIMETZ_ARRAY), + SeaType::Interval(_) => Ok(PgType::INTERVAL_ARRAY), + SeaType::Boolean => Ok(PgType::BOOL_ARRAY), + SeaType::Point => Ok(PgType::POINT_ARRAY), + SeaType::Uuid => Ok(PgType::UUID_ARRAY), + SeaType::JsonBinary => Ok(PgType::JSONB_ARRAY), + SeaType::Array(_) => bail!("nested array type is not supported"), + SeaType::Unknown(name) => { + // Treat as enum type + Ok(PgType::new( + name.clone(), + 0, + PgKind::Array(PgType::new( + name.clone(), + 0, + PgKind::Enum(vec![]), + "".into(), + )), + "".into(), + )) + } + _ => bail!("unsupported array type: {:?}", t), + } + } + SeaType::Unknown(name) => { + // Treat as enum type + Ok(PgType::new( + name.clone(), + 0, + PgKind::Enum(vec![]), + "".into(), + )) + } + _ => bail!("unsupported type: {:?}", discovered_type), + } + } +} + +impl fmt::Display for SslMode { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + f.write_str(match self { + SslMode::Disabled => "disabled", + SslMode::Preferred => "preferred", + SslMode::Required => "required", + SslMode::VerifyCa => "verify-ca", + SslMode::VerifyFull => "verify-full", + }) + } +} + +pub async fn create_pg_client( + user: &str, + password: &str, + host: &str, + port: &str, + database: &str, + ssl_mode: &SslMode, + ssl_root_cert: &Option, +) -> anyhow::Result { + let mut pg_config = tokio_postgres::Config::new(); + pg_config + .user(user) + .password(password) + .host(host) + .port(port.parse::().unwrap()) + .dbname(database); + + let (_verify_ca, verify_hostname) = match ssl_mode { + SslMode::VerifyCa => (true, false), + SslMode::VerifyFull => (true, true), + _ => (false, false), + }; + + #[cfg(not(madsim))] + let connector = match ssl_mode { + SslMode::Disabled => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Disable); + MaybeMakeTlsConnector::NoTls(NoTls) + } + SslMode::Preferred => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Prefer); + match SslConnector::builder(SslMethod::tls()) { + Ok(mut builder) => { + // disable certificate verification for `prefer` + builder.set_verify(SslVerifyMode::NONE); + MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) + } + Err(e) => { + tracing::warn!(error = %e.as_report(), "SSL connector error"); + MaybeMakeTlsConnector::NoTls(NoTls) + } + } + } + SslMode::Required => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); + let mut builder = SslConnector::builder(SslMethod::tls())?; + // disable certificate verification for `require` + builder.set_verify(SslVerifyMode::NONE); + MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) + } + + SslMode::VerifyCa | SslMode::VerifyFull => { + pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); + let mut builder = SslConnector::builder(SslMethod::tls())?; + if let Some(ssl_root_cert) = ssl_root_cert { + builder.set_ca_file(ssl_root_cert).map_err(|e| { + anyhow!(format!("bad ssl root cert error: {}", e.to_report_string())) + })?; + } + let mut connector = MakeTlsConnector::new(builder.build()); + if !verify_hostname { + connector.set_callback(|config, _| { + config.set_verify_hostname(false); + Ok(()) + }); + } + MaybeMakeTlsConnector::Tls(connector) + } + }; + #[cfg(madsim)] + let connector = NoTls; + + let (client, connection) = pg_config.connect(connector).await?; + + tokio::spawn(async move { + if let Err(e) = connection.await { + tracing::error!(error = %e.as_report(), "postgres connection error"); + } + }); + + Ok(client) +} + +pub fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { + let dtype = match col_type { + ColumnType::SmallInt | ColumnType::SmallSerial => DataType::Int16, + ColumnType::Integer | ColumnType::Serial => DataType::Int32, + ColumnType::BigInt | ColumnType::BigSerial => DataType::Int64, + ColumnType::Money | ColumnType::Decimal(_) | ColumnType::Numeric(_) => DataType::Decimal, + ColumnType::Real => DataType::Float32, + ColumnType::DoublePrecision => DataType::Float64, + ColumnType::Varchar(_) | ColumnType::Char(_) | ColumnType::Text => DataType::Varchar, + ColumnType::Bytea => DataType::Bytea, + ColumnType::Timestamp(_) => DataType::Timestamp, + ColumnType::TimestampWithTimeZone(_) => DataType::Timestamptz, + ColumnType::Date => DataType::Date, + ColumnType::Time(_) | ColumnType::TimeWithTimeZone(_) => DataType::Time, + ColumnType::Interval(_) => DataType::Interval, + ColumnType::Boolean => DataType::Boolean, + ColumnType::Point => DataType::Struct(StructType::new(vec![ + ("x", DataType::Float32), + ("y", DataType::Float32), + ])), + ColumnType::Uuid => DataType::Varchar, + ColumnType::Xml => DataType::Varchar, + ColumnType::Json => DataType::Jsonb, + ColumnType::JsonBinary => DataType::Jsonb, + ColumnType::Array(def) => { + let item_type = match def.col_type.as_ref() { + Some(ty) => type_to_rw_type(ty.as_ref())?, + None => { + return Err(anyhow!("ARRAY type missing element type").into()); + } + }; + + DataType::List(Box::new(item_type)) + } + ColumnType::PgLsn => DataType::Int64, + ColumnType::Cidr + | ColumnType::Inet + | ColumnType::MacAddr + | ColumnType::MacAddr8 + | ColumnType::Int4Range + | ColumnType::Int8Range + | ColumnType::NumRange + | ColumnType::TsRange + | ColumnType::TsTzRange + | ColumnType::DateRange + | ColumnType::Enum(_) => DataType::Varchar, + + ColumnType::Line => { + return Err(anyhow!("LINE type not supported").into()); + } + ColumnType::Lseg => { + return Err(anyhow!("LSEG type not supported").into()); + } + ColumnType::Box => { + return Err(anyhow!("BOX type not supported").into()); + } + ColumnType::Path => { + return Err(anyhow!("PATH type not supported").into()); + } + ColumnType::Polygon => { + return Err(anyhow!("POLYGON type not supported").into()); + } + ColumnType::Circle => { + return Err(anyhow!("CIRCLE type not supported").into()); + } + ColumnType::Bit(_) => { + return Err(anyhow!("BIT type not supported").into()); + } + ColumnType::VarBit(_) => { + return Err(anyhow!("VARBIT type not supported").into()); + } + ColumnType::TsVector => { + return Err(anyhow!("TSVECTOR type not supported").into()); + } + ColumnType::TsQuery => { + return Err(anyhow!("TSQUERY type not supported").into()); + } + ColumnType::Unknown(name) => { + // NOTES: user-defined enum type is classified as `Unknown` + tracing::warn!("Unknown Postgres data type: {name}, map to varchar"); + DataType::Varchar + } + }; + + Ok(dtype) +} diff --git a/src/connector/src/parser/scalar_adapter.rs b/src/connector/src/parser/scalar_adapter.rs index 0f5d2d6d6d935..004ebe2cdfcf9 100644 --- a/src/connector/src/parser/scalar_adapter.rs +++ b/src/connector/src/parser/scalar_adapter.rs @@ -315,6 +315,22 @@ impl ScalarAdapter { } } +pub fn validate_pg_type_to_rw_type(pg_type: &DataType, rw_type: &DataType) -> bool { + if pg_type == rw_type { + return true; + } + match rw_type { + DataType::Varchar => matches!(pg_type, DataType::Decimal | DataType::Int256), + DataType::List(box DataType::Varchar) => { + matches!( + pg_type, + DataType::List(box (DataType::Decimal | DataType::Int256)) + ) + } + _ => false, + } +} + fn pg_numeric_is_special(val: &PgNumeric) -> bool { matches!( val, diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index 03b2fc68476ce..76dd4d4a9eac4 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -34,6 +34,7 @@ pub mod mock_coordination_client; pub mod mongodb; pub mod mqtt; pub mod nats; +pub mod postgres; pub mod pulsar; pub mod redis; pub mod remote; @@ -133,6 +134,8 @@ macro_rules! for_all_sinks { { DynamoDb, $crate::sink::dynamodb::DynamoDbSink }, { Mongodb, $crate::sink::mongodb::MongodbSink }, { SqlServer, $crate::sink::sqlserver::SqlServerSink }, + { Postgres, $crate::sink::postgres::PostgresSink }, + { Test, $crate::sink::test_sink::TestSink }, { Table, $crate::sink::trivial::TableSink } } @@ -866,6 +869,12 @@ pub enum SinkError { #[backtrace] anyhow::Error, ), + #[error("Postgres error: {0}")] + Postgres( + #[source] + #[backtrace] + anyhow::Error, + ), #[error(transparent)] Connector( #[from] @@ -951,3 +960,9 @@ impl From<::opensearch::Error> for SinkError { SinkError::ElasticSearchOpenSearch(anyhow!(err)) } } + +impl From for SinkError { + fn from(err: tokio_postgres::Error) -> Self { + SinkError::Postgres(anyhow!(err)) + } +} diff --git a/src/connector/src/sink/postgres.rs b/src/connector/src/sink/postgres.rs new file mode 100644 index 0000000000000..3f722e1016ef5 --- /dev/null +++ b/src/connector/src/sink/postgres.rs @@ -0,0 +1,622 @@ +// Copyright 2024 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::collections::{BTreeMap, HashSet}; +use std::sync::Arc; + +use anyhow::{anyhow, Context}; +use async_trait::async_trait; +use itertools::Itertools; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::bitmap::Bitmap; +use risingwave_common::catalog::Schema; +use risingwave_common::row::{Row, RowExt}; +use serde_derive::Deserialize; +use serde_with::{serde_as, DisplayFromStr}; +use simd_json::prelude::ArrayTrait; +use thiserror_ext::AsReport; +use tokio_postgres::Statement; + +use super::{ + SinkError, SinkWriterMetrics, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, +}; +use crate::connector_common::{create_pg_client, PostgresExternalTable, SslMode}; +use crate::parser::scalar_adapter::{validate_pg_type_to_rw_type, ScalarAdapter}; +use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; +use crate::sink::{DummySinkCommitCoordinator, Result, Sink, SinkParam, SinkWriterParam}; + +pub const POSTGRES_SINK: &str = "postgres"; + +macro_rules! rw_row_to_pg_values { + ($row:expr, $statement:expr) => { + $row.iter().enumerate().map(|(i, d)| { + d.and_then(|d| { + let ty = &$statement.params()[i]; + match ScalarAdapter::from_scalar(d, ty) { + Ok(scalar) => Some(scalar), + Err(e) => { + tracing::error!(error=%e.as_report(), scalar=?d, "Failed to convert scalar to pg value"); + None + } + } + }) + }) + }; +} + +#[serde_as] +#[derive(Clone, Debug, Deserialize)] +pub struct PostgresConfig { + pub host: String, + #[serde_as(as = "DisplayFromStr")] + pub port: u16, + pub user: String, + pub password: String, + pub database: String, + pub table: String, + #[serde(default = "default_schema")] + pub schema: String, + #[serde(default = "Default::default")] + pub ssl_mode: SslMode, + #[serde(rename = "ssl.root.cert")] + pub ssl_root_cert: Option, + #[serde(default = "default_max_batch_rows")] + #[serde_as(as = "DisplayFromStr")] + pub max_batch_rows: usize, + pub r#type: String, // accept "append-only" or "upsert" +} + +fn default_max_batch_rows() -> usize { + 1024 +} + +fn default_schema() -> String { + "public".to_string() +} + +impl PostgresConfig { + pub fn from_btreemap(properties: BTreeMap) -> Result { + let config = + serde_json::from_value::(serde_json::to_value(properties).unwrap()) + .map_err(|e| SinkError::Config(anyhow!(e)))?; + if config.r#type != SINK_TYPE_APPEND_ONLY && config.r#type != SINK_TYPE_UPSERT { + return Err(SinkError::Config(anyhow!( + "`{}` must be {}, or {}", + SINK_TYPE_OPTION, + SINK_TYPE_APPEND_ONLY, + SINK_TYPE_UPSERT + ))); + } + Ok(config) + } +} + +#[derive(Debug)] +pub struct PostgresSink { + pub config: PostgresConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, +} + +impl PostgresSink { + pub fn new( + config: PostgresConfig, + schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + Ok(Self { + config, + schema, + pk_indices, + is_append_only, + }) + } +} + +impl TryFrom for PostgresSink { + type Error = SinkError; + + fn try_from(param: SinkParam) -> std::result::Result { + let schema = param.schema(); + let config = PostgresConfig::from_btreemap(param.properties)?; + PostgresSink::new( + config, + schema, + param.downstream_pk, + param.sink_type.is_append_only(), + ) + } +} + +impl Sink for PostgresSink { + type Coordinator = DummySinkCommitCoordinator; + type LogSinker = LogSinkerOf; + + const SINK_NAME: &'static str = POSTGRES_SINK; + + async fn validate(&self) -> Result<()> { + if !self.is_append_only && self.pk_indices.is_empty() { + return Err(SinkError::Config(anyhow!( + "Primary key not defined for upsert Postgres sink (please define in `primary_key` field)"))); + } + + // Verify our sink schema is compatible with Postgres + { + let pg_table = PostgresExternalTable::connect( + &self.config.user, + &self.config.password, + &self.config.host, + self.config.port, + &self.config.database, + &self.config.schema, + &self.config.table, + &self.config.ssl_mode, + &self.config.ssl_root_cert, + self.is_append_only, + ) + .await?; + + // Check that names and types match, order of columns doesn't matter. + { + let pg_columns = pg_table.column_descs(); + let sink_columns = self.schema.fields(); + if pg_columns.len() != sink_columns.len() { + return Err(SinkError::Config(anyhow!( + "Column count mismatch: Postgres table has {} columns, but sink schema has {} columns", + pg_columns.len(), + sink_columns.len() + ))); + } + + let pg_columns_lookup = pg_columns + .iter() + .map(|c| (c.name.clone(), c.data_type.clone())) + .collect::>(); + for sink_column in sink_columns { + let pg_column = pg_columns_lookup.get(&sink_column.name); + match pg_column { + None => { + return Err(SinkError::Config(anyhow!( + "Column `{}` not found in Postgres table `{}`", + sink_column.name, + self.config.table + ))) + } + Some(pg_column) => { + if !validate_pg_type_to_rw_type(pg_column, &sink_column.data_type()) { + return Err(SinkError::Config(anyhow!( + "Column `{}` in Postgres table `{}` has type `{}`, but sink schema defines it as type `{}`", + sink_column.name, + self.config.table, + pg_column, + sink_column.data_type() + ))); + } + } + } + } + } + + // check that pk matches + { + let pg_pk_names = pg_table.pk_names(); + let sink_pk_names = self + .pk_indices + .iter() + .map(|i| &self.schema.fields()[*i].name) + .collect::>(); + if pg_pk_names.len() != sink_pk_names.len() { + return Err(SinkError::Config(anyhow!( + "Primary key mismatch: Postgres table has primary key on columns {:?}, but sink schema defines primary key on columns {:?}", + pg_pk_names, + sink_pk_names + ))); + } + for name in pg_pk_names { + if !sink_pk_names.contains(name) { + return Err(SinkError::Config(anyhow!( + "Primary key mismatch: Postgres table has primary key on column `{}`, but sink schema does not define it as a primary key", + name + ))); + } + } + } + } + + Ok(()) + } + + async fn new_log_sinker(&self, writer_param: SinkWriterParam) -> Result { + Ok(PostgresSinkWriter::new( + self.config.clone(), + self.schema.clone(), + self.pk_indices.clone(), + self.is_append_only, + ) + .await? + .into_log_sinker(SinkWriterMetrics::new(&writer_param))) + } +} + +struct Buffer { + buffer: Vec, + size: usize, +} + +impl Buffer { + fn new() -> Self { + Self { + buffer: Vec::new(), + size: 0, + } + } + + fn push(&mut self, chunk: StreamChunk) -> usize { + self.size += chunk.cardinality(); + self.buffer.push(chunk); + self.size + } + + fn drain(&mut self) -> Vec { + self.size = 0; + std::mem::take(&mut self.buffer) + } +} + +pub struct PostgresSinkWriter { + config: PostgresConfig, + pk_indices: Vec, + is_append_only: bool, + client: tokio_postgres::Client, + buffer: Buffer, + insert_statement: Statement, + delete_statement: Option, + upsert_statement: Option, +} + +impl PostgresSinkWriter { + async fn new( + config: PostgresConfig, + mut schema: Schema, + pk_indices: Vec, + is_append_only: bool, + ) -> Result { + let client = create_pg_client( + &config.user, + &config.password, + &config.host, + &config.port.to_string(), + &config.database, + &config.ssl_mode, + &config.ssl_root_cert, + ) + .await?; + + // Rewrite schema types for serialization + let schema_types = { + let pg_table = PostgresExternalTable::connect( + &config.user, + &config.password, + &config.host, + config.port, + &config.database, + &config.schema, + &config.table, + &config.ssl_mode, + &config.ssl_root_cert, + is_append_only, + ) + .await?; + let name_to_type = pg_table.column_name_to_pg_type(); + let mut schema_types = Vec::with_capacity(schema.fields.len()); + for field in &mut schema.fields[..] { + let field_name = &field.name; + let actual_data_type = name_to_type.get(field_name).map(|t| (*t).clone()); + let actual_data_type = actual_data_type + .ok_or_else(|| { + SinkError::Config(anyhow!( + "Column `{}` not found in sink schema", + field_name + )) + })? + .clone(); + schema_types.push(actual_data_type); + } + schema_types + }; + + let insert_statement = { + let insert_sql = create_insert_sql(&schema, &config.table); + client + .prepare_typed(&insert_sql, &schema_types) + .await + .context("Failed to prepare insert statement")? + }; + + let delete_statement = if is_append_only { + None + } else { + let delete_types = pk_indices + .iter() + .map(|i| schema_types[*i].clone()) + .collect_vec(); + let delete_sql = create_delete_sql(&schema, &config.table, &pk_indices); + Some( + client + .prepare_typed(&delete_sql, &delete_types) + .await + .context("Failed to prepare delete statement")?, + ) + }; + + let upsert_statement = if is_append_only { + None + } else { + let upsert_sql = create_upsert_sql(&schema, &config.table, &pk_indices); + Some( + client + .prepare_typed(&upsert_sql, &schema_types) + .await + .context("Failed to prepare upsert statement")?, + ) + }; + + let writer = Self { + config, + pk_indices, + is_append_only, + client, + buffer: Buffer::new(), + insert_statement, + delete_statement, + upsert_statement, + }; + Ok(writer) + } + + async fn flush(&mut self) -> Result<()> { + if self.is_append_only { + for chunk in self.buffer.drain() { + for (op, row) in chunk.rows() { + match op { + Op::Insert => { + self.client + .execute_raw( + &self.insert_statement, + rw_row_to_pg_values!(row, self.insert_statement), + ) + .await?; + } + Op::UpdateInsert | Op::Delete | Op::UpdateDelete => { + debug_assert!(!self.is_append_only); + } + } + } + } + } else { + let mut unmatched_update_insert = 0; + for chunk in self.buffer.drain() { + for (op, row) in chunk.rows() { + match op { + Op::Insert => { + self.client + .execute_raw( + &self.insert_statement, + rw_row_to_pg_values!(row, self.insert_statement), + ) + .await?; + } + Op::UpdateInsert => { + unmatched_update_insert += 1; + self.client + .execute_raw( + self.upsert_statement.as_ref().unwrap(), + rw_row_to_pg_values!( + row, + self.upsert_statement.as_ref().unwrap() + ), + ) + .await?; + } + Op::Delete => { + self.client + .execute_raw( + self.delete_statement.as_ref().unwrap(), + rw_row_to_pg_values!( + row.project(&self.pk_indices), + self.delete_statement.as_ref().unwrap() + ), + ) + .await?; + } + Op::UpdateDelete => { + unmatched_update_insert -= 1; + } + } + } + } + assert_eq!(unmatched_update_insert, 0); + } + + Ok(()) + } +} + +#[async_trait] +impl SinkWriter for PostgresSinkWriter { + async fn begin_epoch(&mut self, _epoch: u64) -> Result<()> { + Ok(()) + } + + async fn write_batch(&mut self, chunk: StreamChunk) -> Result<()> { + let cardinality = self.buffer.push(chunk); + if cardinality >= self.config.max_batch_rows { + self.flush().await?; + } + Ok(()) + } + + async fn barrier(&mut self, is_checkpoint: bool) -> Result { + if is_checkpoint { + self.flush().await?; + } + Ok(()) + } + + async fn abort(&mut self) -> Result<()> { + Ok(()) + } + + async fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) -> Result<()> { + Ok(()) + } +} + +fn create_insert_sql(schema: &Schema, table_name: &str) -> String { + let columns: String = schema + .fields() + .iter() + .map(|field| field.name.clone()) + .collect_vec() + .join(", "); + let parameters: String = (0..schema.fields().len()) + .map(|i| format!("${}", i + 1)) + .collect_vec() + .join(", "); + format!("INSERT INTO {table_name} ({columns}) VALUES ({parameters})") +} + +fn create_upsert_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> String { + let columns: String = schema + .fields() + .iter() + .map(|field| field.name.clone()) + .collect_vec() + .join(", "); + let parameters: String = (0..schema.fields().len()) + .map(|i| format!("${}", i + 1)) + .collect_vec() + .join(", "); + let pk_columns = pk_indices + .iter() + .map(|i| schema.fields()[*i].name.clone()) + .collect_vec() + .join(", "); + let update_parameters: String = (0..schema.fields().len()) + .filter(|i| !pk_indices.contains(i)) + .map(|i| { + let column = schema.fields()[i].name.clone(); + let param = format!("${}", i + 1); + format!("{column} = {param}") + }) + .collect_vec() + .join(", "); + format!("INSERT INTO {table_name} ({columns}) VALUES ({parameters}) on conflict ({pk_columns}) do update set {update_parameters}") +} + +fn create_delete_sql(schema: &Schema, table_name: &str, pk_indices: &[usize]) -> String { + let parameters: String = pk_indices + .iter() + .map(|i| format!("{} = ${}", schema.fields()[*i].name, i + 1)) + .collect_vec() + .join(" AND "); + format!("DELETE FROM {table_name} WHERE {parameters}") +} + +#[cfg(test)] +mod tests { + use std::fmt::Display; + + use expect_test::{expect, Expect}; + use risingwave_common::catalog::Field; + use risingwave_common::types::DataType; + + use super::*; + + fn check(actual: impl Display, expect: Expect) { + let actual = actual.to_string(); + expect.assert_eq(&actual); + } + + #[test] + fn test_create_insert_sql() { + let schema = Schema::new(vec![ + Field { + data_type: DataType::Int32, + name: "a".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + Field { + data_type: DataType::Int32, + name: "b".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + ]); + let table_name = "test_table"; + let sql = create_insert_sql(&schema, table_name); + check( + sql, + expect!["INSERT INTO test_table (a, b) VALUES ($1, $2)"], + ); + } + + #[test] + fn test_create_delete_sql() { + let schema = Schema::new(vec![ + Field { + data_type: DataType::Int32, + name: "a".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + Field { + data_type: DataType::Int32, + name: "b".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + ]); + let table_name = "test_table"; + let sql = create_delete_sql(&schema, table_name, &[1]); + check(sql, expect!["DELETE FROM test_table WHERE b = $2"]); + } + + #[test] + fn test_create_upsert_sql() { + let schema = Schema::new(vec![ + Field { + data_type: DataType::Int32, + name: "a".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + Field { + data_type: DataType::Int32, + name: "b".to_string(), + sub_fields: vec![], + type_name: "".to_string(), + }, + ]); + let table_name = "test_table"; + let sql = create_upsert_sql(&schema, table_name, &[1]); + check( + sql, + expect!["INSERT INTO test_table (a, b) VALUES ($1, $2) on conflict (b) do update set a = $1"], + ); + } +} diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index fbfa66ef0e7c6..115b1be99d21b 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -16,12 +16,9 @@ pub mod mock_external_table; pub mod postgres; pub mod sql_server; -#[cfg(not(madsim))] -mod maybe_tls_connector; pub mod mysql; use std::collections::{BTreeMap, HashMap}; -use std::fmt; use anyhow::anyhow; use futures::pin_mut; @@ -34,15 +31,14 @@ use risingwave_common::secret::LocalSecretManager; use risingwave_pb::secret::PbSecretRef; use serde_derive::{Deserialize, Serialize}; +use crate::connector_common::{PostgresExternalTable, SslMode}; use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::mysql_row_to_owned_row; use crate::source::cdc::external::mock_external_table::MockExternalTableReader; use crate::source::cdc::external::mysql::{ MySqlExternalTable, MySqlExternalTableReader, MySqlOffset, }; -use crate::source::cdc::external::postgres::{ - PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, -}; +use crate::source::cdc::external::postgres::{PostgresExternalTableReader, PostgresOffset}; use crate::source::cdc::external::sql_server::{ SqlServerExternalTable, SqlServerExternalTableReader, SqlServerOffset, }; @@ -236,7 +232,7 @@ pub struct ExternalTableConfig { /// `ssl.mode` specifies the SSL/TLS encryption level for secure communication with Postgres. /// Choices include `disabled`, `preferred`, and `required`. /// This field is optional. - #[serde(rename = "ssl.mode", default = "Default::default")] + #[serde(rename = "ssl.mode", default = "postgres_ssl_mode_default")] #[serde(alias = "debezium.database.sslmode")] pub ssl_mode: SslMode, @@ -250,6 +246,11 @@ pub struct ExternalTableConfig { pub encrypt: String, } +fn postgres_ssl_mode_default() -> SslMode { + // NOTE(StrikeW): Default to `disabled` for backward compatibility + SslMode::Disabled +} + impl ExternalTableConfig { pub fn try_from_btreemap( connect_properties: BTreeMap, @@ -263,44 +264,6 @@ impl ExternalTableConfig { } } -#[derive(Debug, Clone, PartialEq, Deserialize)] -#[serde(rename_all = "lowercase")] -pub enum SslMode { - #[serde(alias = "disable")] - Disabled, - #[serde(alias = "prefer")] - Preferred, - #[serde(alias = "require")] - Required, - /// verify that the server is trustworthy by checking the certificate chain - /// up to the root certificate stored on the client. - #[serde(alias = "verify-ca")] - VerifyCa, - /// Besides verify the certificate, will also verify that the serverhost name - /// matches the name stored in the server certificate. - #[serde(alias = "verify-full")] - VerifyFull, -} - -impl Default for SslMode { - fn default() -> Self { - // default to `disabled` for backward compatibility - Self::Disabled - } -} - -impl fmt::Display for SslMode { - fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { - f.write_str(match self { - SslMode::Disabled => "disabled", - SslMode::Preferred => "preferred", - SslMode::Required => "required", - SslMode::VerifyCa => "verify-ca", - SslMode::VerifyFull => "verify-full", - }) - } -} - impl ExternalTableReader for ExternalTableReaderImpl { async fn current_cdc_offset(&self) -> ConnectorResult { match self { @@ -382,7 +345,19 @@ impl ExternalTableImpl { MySqlExternalTable::connect(config).await?, )), CdcSourceType::Postgres => Ok(ExternalTableImpl::Postgres( - PostgresExternalTable::connect(config).await?, + PostgresExternalTable::connect( + &config.username, + &config.password, + &config.host, + config.port.parse::().unwrap(), + &config.database, + &config.schema, + &config.table, + &config.ssl_mode, + &config.ssl_root_cert, + false, + ) + .await?, )), CdcSourceType::SqlServer => Ok(ExternalTableImpl::SqlServer( SqlServerExternalTable::connect(config).await?, diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index 112fd16e6bff5..ecc503934bb69 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -13,36 +13,25 @@ // limitations under the License. use std::cmp::Ordering; -use std::collections::HashMap; -use anyhow::{anyhow, Context}; +use anyhow::Context; use futures::stream::BoxStream; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use openssl::ssl::{SslConnector, SslMethod, SslVerifyMode}; -use postgres_openssl::MakeTlsConnector; -use risingwave_common::catalog::{ColumnDesc, ColumnId, Schema}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row}; -use risingwave_common::types::{DataType, ScalarImpl, StructType}; use risingwave_common::util::iter_util::ZipEqFast; -use sea_schema::postgres::def::{ColumnType, TableInfo}; -use sea_schema::postgres::discovery::SchemaDiscovery; use serde_derive::{Deserialize, Serialize}; -use sqlx::postgres::{PgConnectOptions, PgSslMode}; -use sqlx::PgPool; -use thiserror_ext::AsReport; use tokio_postgres::types::PgLsn; -use tokio_postgres::NoTls; +use crate::connector_common::create_pg_client; use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::postgres_row_to_owned_row; use crate::parser::scalar_adapter::ScalarAdapter; -#[cfg(not(madsim))] -use crate::source::cdc::external::maybe_tls_connector::MaybeMakeTlsConnector; use crate::source::cdc::external::{ CdcOffset, CdcOffsetParseFunc, DebeziumOffset, ExternalTableConfig, ExternalTableReader, - SchemaTableName, SslMode, + SchemaTableName, }; #[derive(Debug, Clone, Default, PartialEq, Serialize, Deserialize)] @@ -78,189 +67,6 @@ impl PostgresOffset { } } -pub struct PostgresExternalTable { - column_descs: Vec, - pk_names: Vec, -} - -impl PostgresExternalTable { - pub async fn connect(config: ExternalTableConfig) -> ConnectorResult { - tracing::debug!("connect to postgres external table"); - let mut options = PgConnectOptions::new() - .username(&config.username) - .password(&config.password) - .host(&config.host) - .port(config.port.parse::().unwrap()) - .database(&config.database) - .ssl_mode(match config.ssl_mode { - SslMode::Disabled => PgSslMode::Disable, - SslMode::Preferred => PgSslMode::Prefer, - SslMode::Required => PgSslMode::Require, - SslMode::VerifyCa => PgSslMode::VerifyCa, - SslMode::VerifyFull => PgSslMode::VerifyFull, - }); - - if config.ssl_mode == SslMode::VerifyCa || config.ssl_mode == SslMode::VerifyFull { - if let Some(ref root_cert) = config.ssl_root_cert { - options = options.ssl_root_cert(root_cert.as_str()); - } - } - - let connection = PgPool::connect_with(options).await?; - let schema_discovery = SchemaDiscovery::new(connection, config.schema.as_str()); - // fetch column schema and primary key - let empty_map = HashMap::new(); - let table_schema = schema_discovery - .discover_table( - TableInfo { - name: config.table.clone(), - of_type: None, - }, - &empty_map, - ) - .await?; - - let mut column_descs = vec![]; - for col in &table_schema.columns { - let data_type = type_to_rw_type(&col.col_type)?; - let column_desc = if let Some(ref default_expr) = col.default { - // parse the value of "column_default" field in information_schema.columns, - // non number data type will be stored as "'value'::type" - let val_text = default_expr - .0 - .split("::") - .map(|s| s.trim_matches('\'')) - .next() - .expect("default value expression"); - - match ScalarImpl::from_text(val_text, &data_type) { - Ok(scalar) => ColumnDesc::named_with_default_value( - col.name.clone(), - ColumnId::placeholder(), - data_type.clone(), - Some(scalar), - ), - Err(err) => { - tracing::warn!(error=%err.as_report(), "failed to parse postgres default value expression, only constant is supported"); - ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) - } - } - } else { - ColumnDesc::named(col.name.clone(), ColumnId::placeholder(), data_type) - }; - column_descs.push(column_desc); - } - - if table_schema.primary_key_constraints.is_empty() { - return Err(anyhow!("Postgres table doesn't define the primary key").into()); - } - let mut pk_names = vec![]; - table_schema.primary_key_constraints.iter().for_each(|pk| { - pk_names.extend(pk.columns.clone()); - }); - - Ok(Self { - column_descs, - pk_names, - }) - } - - pub fn column_descs(&self) -> &Vec { - &self.column_descs - } - - pub fn pk_names(&self) -> &Vec { - &self.pk_names - } -} - -fn type_to_rw_type(col_type: &ColumnType) -> ConnectorResult { - let dtype = match col_type { - ColumnType::SmallInt | ColumnType::SmallSerial => DataType::Int16, - ColumnType::Integer | ColumnType::Serial => DataType::Int32, - ColumnType::BigInt | ColumnType::BigSerial => DataType::Int64, - ColumnType::Money | ColumnType::Decimal(_) | ColumnType::Numeric(_) => DataType::Decimal, - ColumnType::Real => DataType::Float32, - ColumnType::DoublePrecision => DataType::Float64, - ColumnType::Varchar(_) | ColumnType::Char(_) | ColumnType::Text => DataType::Varchar, - ColumnType::Bytea => DataType::Bytea, - ColumnType::Timestamp(_) => DataType::Timestamp, - ColumnType::TimestampWithTimeZone(_) => DataType::Timestamptz, - ColumnType::Date => DataType::Date, - ColumnType::Time(_) | ColumnType::TimeWithTimeZone(_) => DataType::Time, - ColumnType::Interval(_) => DataType::Interval, - ColumnType::Boolean => DataType::Boolean, - ColumnType::Point => DataType::Struct(StructType::new(vec![ - ("x", DataType::Float32), - ("y", DataType::Float32), - ])), - ColumnType::Uuid => DataType::Varchar, - ColumnType::Xml => DataType::Varchar, - ColumnType::Json => DataType::Jsonb, - ColumnType::JsonBinary => DataType::Jsonb, - ColumnType::Array(def) => { - let item_type = match def.col_type.as_ref() { - Some(ty) => type_to_rw_type(ty.as_ref())?, - None => { - return Err(anyhow!("ARRAY type missing element type").into()); - } - }; - - DataType::List(Box::new(item_type)) - } - ColumnType::PgLsn => DataType::Int64, - ColumnType::Cidr - | ColumnType::Inet - | ColumnType::MacAddr - | ColumnType::MacAddr8 - | ColumnType::Int4Range - | ColumnType::Int8Range - | ColumnType::NumRange - | ColumnType::TsRange - | ColumnType::TsTzRange - | ColumnType::DateRange - | ColumnType::Enum(_) => DataType::Varchar, - - ColumnType::Line => { - return Err(anyhow!("LINE type not supported").into()); - } - ColumnType::Lseg => { - return Err(anyhow!("LSEG type not supported").into()); - } - ColumnType::Box => { - return Err(anyhow!("BOX type not supported").into()); - } - ColumnType::Path => { - return Err(anyhow!("PATH type not supported").into()); - } - ColumnType::Polygon => { - return Err(anyhow!("POLYGON type not supported").into()); - } - ColumnType::Circle => { - return Err(anyhow!("CIRCLE type not supported").into()); - } - ColumnType::Bit(_) => { - return Err(anyhow!("BIT type not supported").into()); - } - ColumnType::VarBit(_) => { - return Err(anyhow!("VARBIT type not supported").into()); - } - ColumnType::TsVector => { - return Err(anyhow!("TSVECTOR type not supported").into()); - } - ColumnType::TsQuery => { - return Err(anyhow!("TSQUERY type not supported").into()); - } - ColumnType::Unknown(name) => { - // NOTES: user-defined enum type is classified as `Unknown` - tracing::warn!("Unknown Postgres data type: {name}, map to varchar"); - DataType::Varchar - } - }; - - Ok(dtype) -} - pub struct PostgresExternalTableReader { rw_schema: Schema, field_names: String, @@ -312,76 +118,16 @@ impl PostgresExternalTableReader { "create postgres external table reader" ); - let mut pg_config = tokio_postgres::Config::new(); - pg_config - .user(&config.username) - .password(&config.password) - .host(&config.host) - .port(config.port.parse::().unwrap()) - .dbname(&config.database); - - let (_verify_ca, verify_hostname) = match config.ssl_mode { - SslMode::VerifyCa => (true, false), - SslMode::VerifyFull => (true, true), - _ => (false, false), - }; - - #[cfg(not(madsim))] - let connector = match config.ssl_mode { - SslMode::Disabled => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Disable); - MaybeMakeTlsConnector::NoTls(NoTls) - } - SslMode::Preferred => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Prefer); - match SslConnector::builder(SslMethod::tls()) { - Ok(mut builder) => { - // disable certificate verification for `prefer` - builder.set_verify(SslVerifyMode::NONE); - MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) - } - Err(e) => { - tracing::warn!(error = %e.as_report(), "SSL connector error"); - MaybeMakeTlsConnector::NoTls(NoTls) - } - } - } - SslMode::Required => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); - let mut builder = SslConnector::builder(SslMethod::tls())?; - // disable certificate verification for `require` - builder.set_verify(SslVerifyMode::NONE); - MaybeMakeTlsConnector::Tls(MakeTlsConnector::new(builder.build())) - } - - SslMode::VerifyCa | SslMode::VerifyFull => { - pg_config.ssl_mode(tokio_postgres::config::SslMode::Require); - let mut builder = SslConnector::builder(SslMethod::tls())?; - if let Some(ssl_root_cert) = config.ssl_root_cert { - builder.set_ca_file(ssl_root_cert).map_err(|e| { - anyhow!(format!("bad ssl root cert error: {}", e.to_report_string())) - })?; - } - let mut connector = MakeTlsConnector::new(builder.build()); - if !verify_hostname { - connector.set_callback(|config, _| { - config.set_verify_hostname(false); - Ok(()) - }); - } - MaybeMakeTlsConnector::Tls(connector) - } - }; - #[cfg(madsim)] - let connector = NoTls; - - let (client, connection) = pg_config.connect(connector).await?; - - tokio::spawn(async move { - if let Err(e) = connection.await { - tracing::error!(error = %e.as_report(), "postgres connection error"); - } - }); + let client = create_pg_client( + &config.username, + &config.password, + &config.host, + &config.port, + &config.database, + &config.ssl_mode, + &config.ssl_root_cert, + ) + .await?; let field_names = rw_schema .fields @@ -521,9 +267,8 @@ mod tests { use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; - use crate::source::cdc::external::postgres::{ - PostgresExternalTable, PostgresExternalTableReader, PostgresOffset, - }; + use crate::connector_common::PostgresExternalTable; + use crate::source::cdc::external::postgres::{PostgresExternalTableReader, PostgresOffset}; use crate::source::cdc::external::{ExternalTableConfig, ExternalTableReader, SchemaTableName}; #[ignore] @@ -543,10 +288,23 @@ mod tests { encrypt: "false".to_string(), }; - let table = PostgresExternalTable::connect(config).await.unwrap(); + let table = PostgresExternalTable::connect( + &config.username, + &config.password, + &config.host, + config.port.parse::().unwrap(), + &config.database, + &config.schema, + &config.table, + &config.ssl_mode, + &config.ssl_root_cert, + false, + ) + .await + .unwrap(); - println!("columns: {:?}", &table.column_descs); - println!("primary keys: {:?}", &table.pk_names); + println!("columns: {:?}", &table.column_descs()); + println!("primary keys: {:?}", &table.pk_names()); } #[test] diff --git a/src/risedevtool/src/task/task_log_ready_check.rs b/src/risedevtool/src/task/task_log_ready_check.rs index cc55c6142b47a..ef27169abedb4 100644 --- a/src/risedevtool/src/task/task_log_ready_check.rs +++ b/src/risedevtool/src/task/task_log_ready_check.rs @@ -84,7 +84,7 @@ where &mut self.log, self.status_file.as_ref().unwrap(), self.id.as_ref().unwrap(), - Some(Duration::from_secs(30)), + Some(Duration::from_secs(60)), true, )?; From 32c44babc511f3dd3c3678c9cbe511d8108dceba Mon Sep 17 00:00:00 2001 From: Kexiang Wang Date: Wed, 27 Nov 2024 21:32:56 -0600 Subject: [PATCH 037/163] feat(source): support webhook source table (#19272) Co-authored-by: Eric Fu --- Cargo.lock | 34 ++- ci/scripts/e2e-source-test.sh | 21 +- ci/workflows/main-cron.yml | 2 +- e2e_test/webhook/check_1.slt.part | 24 ++ e2e_test/webhook/check_2.slt.part | 29 +++ e2e_test/webhook/check_3.slt.part | 34 +++ e2e_test/webhook/create_table.slt.part | 55 +++++ e2e_test/webhook/drop_table.slt.part | 14 ++ e2e_test/webhook/sender.py | 145 ++++++++++++ e2e_test/webhook/webhook_source.slt | 28 +++ e2e_test/webhook/webhook_source_recovery.slt | 19 ++ proto/catalog.proto | 8 + proto/expr.proto | 2 + src/common/secret/src/secret_manager.rs | 54 +++-- src/connector/src/source/base.rs | 2 + src/connector/src/source/mod.rs | 2 +- src/expr/impl/Cargo.toml | 2 + src/expr/impl/src/scalar/cmp.rs | 6 + src/expr/impl/src/scalar/hmac.rs | 73 ++++++ src/expr/impl/src/scalar/mod.rs | 1 + src/frontend/Cargo.toml | 7 + src/frontend/planner_test/src/lib.rs | 2 + .../binder/expr/function/builtin_scalar.rs | 2 + src/frontend/src/binder/expr/mod.rs | 18 ++ src/frontend/src/binder/mod.rs | 23 ++ src/frontend/src/catalog/table_catalog.rs | 8 +- src/frontend/src/expr/pure.rs | 2 + src/frontend/src/handler/create_source.rs | 2 +- src/frontend/src/handler/create_table.rs | 90 ++++++- src/frontend/src/handler/create_table_as.rs | 1 + src/frontend/src/handler/explain.rs | 2 + src/frontend/src/handler/mod.rs | 2 + src/frontend/src/lib.rs | 10 + src/frontend/src/optimizer/mod.rs | 4 +- .../src/optimizer/plan_expr_visitor/strong.rs | 2 + .../optimizer/plan_node/stream_materialize.rs | 6 + src/frontend/src/optimizer/plan_node/utils.rs | 1 + .../src/scheduler/distributed/query.rs | 1 + src/frontend/src/utils/with_options.rs | 6 + src/frontend/src/webhook/mod.rs | 219 ++++++++++++++++++ src/frontend/src/webhook/utils.rs | 105 +++++++++ src/meta/model/migration/src/lib.rs | 3 + .../src/m20241001_013810_webhook_source.rs | 37 +++ src/meta/model/src/lib.rs | 4 + src/meta/model/src/table.rs | 4 +- src/meta/src/controller/mod.rs | 1 + src/prost/build.rs | 1 + src/sqlparser/src/ast/ddl.rs | 11 +- src/sqlparser/src/ast/mod.rs | 9 +- src/sqlparser/src/ast/statement.rs | 5 + src/sqlparser/src/keywords.rs | 1 + src/sqlparser/src/parser.rs | 58 ++++- src/sqlparser/tests/testdata/array.yaml | 2 +- src/storage/src/compaction_catalog_manager.rs | 1 + src/tests/sqlsmith/src/lib.rs | 7 + 55 files changed, 1145 insertions(+), 67 deletions(-) create mode 100644 e2e_test/webhook/check_1.slt.part create mode 100644 e2e_test/webhook/check_2.slt.part create mode 100644 e2e_test/webhook/check_3.slt.part create mode 100644 e2e_test/webhook/create_table.slt.part create mode 100644 e2e_test/webhook/drop_table.slt.part create mode 100644 e2e_test/webhook/sender.py create mode 100644 e2e_test/webhook/webhook_source.slt create mode 100644 e2e_test/webhook/webhook_source_recovery.slt create mode 100644 src/expr/impl/src/scalar/hmac.rs create mode 100644 src/frontend/src/webhook/mod.rs create mode 100644 src/frontend/src/webhook/utils.rs create mode 100644 src/meta/model/migration/src/m20241001_013810_webhook_source.rs diff --git a/Cargo.lock b/Cargo.lock index 48298a8f5f08a..a5123cd28225a 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1095,7 +1095,7 @@ dependencies = [ "rand", "regex", "ring 0.17.5", - "rustls-native-certs 0.7.3", + "rustls-native-certs 0.7.0", "rustls-pemfile 2.2.0", "rustls-webpki 0.102.2", "serde", @@ -2340,9 +2340,9 @@ checksum = "1fd0f2584146f6f2ef48085050886acf353beff7305ebd1ae69500e27c67f64b" [[package]] name = "bytes" -version = "1.8.0" +version = "1.7.2" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "9ac0150caa2ae65ca5bd83f25c7de183dea78d4d366469f148435e2acfbad0da" +checksum = "428d9aa8fbc0670b7b8d6030a7fadd0f86151cae55e4dbbece15f3780a3dfaf3" dependencies = [ "serde", ] @@ -5726,9 +5726,9 @@ dependencies = [ [[package]] name = "governor" -version = "0.6.3" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "68a7f542ee6b35af73b06abc0dad1c1bae89964e4e253bc4b587b91c9637867b" +checksum = "821239e5672ff23e2a7060901fa622950bbd80b649cdaadd78d1c1767ed14eb4" dependencies = [ "cfg-if", "dashmap 5.5.3", @@ -5737,10 +5737,8 @@ dependencies = [ "no-std-compat", "nonzero_ext", "parking_lot 0.12.1", - "portable-atomic", "rand", "smallvec", - "spinning_top", ] [[package]] @@ -10238,7 +10236,7 @@ dependencies = [ "percent-encoding", "pin-project-lite", "rustls 0.22.4", - "rustls-native-certs 0.7.3", + "rustls-native-certs 0.7.0", "rustls-pemfile 2.2.0", "rustls-pki-types", "serde", @@ -11238,6 +11236,7 @@ dependencies = [ "bytes", "chrono", "chrono-tz 0.10.0", + "constant_time_eq", "criterion", "educe", "expect-test", @@ -11246,6 +11245,7 @@ dependencies = [ "futures-util", "ginepro", "hex", + "hmac", "icelake", "itertools 0.13.0", "jsonbb", @@ -11298,6 +11298,7 @@ dependencies = [ "async-trait", "auto_enums", "auto_impl", + "axum", "base64 0.22.0", "bk-tree", "bytes", @@ -11368,6 +11369,8 @@ dependencies = [ "thiserror-ext", "tokio-postgres", "tokio-stream 0.1.15", + "tower 0.4.13", + "tower-http", "tracing", "uuid", "workspace-hack", @@ -12305,7 +12308,7 @@ dependencies = [ "flume", "futures-util", "log", - "rustls-native-certs 0.7.3", + "rustls-native-certs 0.7.0", "rustls-pemfile 2.2.0", "rustls-webpki 0.102.2", "thiserror 1.0.63", @@ -12503,9 +12506,9 @@ dependencies = [ [[package]] name = "rustls-native-certs" -version = "0.7.3" +version = "0.7.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "e5bfb394eeed242e909609f56089eecfe5fda225042e8b171791b9c95f5931e5" +checksum = "8f1fb85efa936c42c6d5fc28d2629bb51e4b2f4b8a5211e297d599cc5a093792" dependencies = [ "openssl-probe", "rustls-pemfile 2.2.0", @@ -13579,15 +13582,6 @@ dependencies = [ "lock_api", ] -[[package]] -name = "spinning_top" -version = "0.3.0" -source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "d96d2d1d716fb500937168cc09353ffdc7a012be8475ac7308e1bdf0e3923300" -dependencies = [ - "lock_api", -] - [[package]] name = "spki" version = "0.6.0" diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index b6ea8267a1690..1a9d6ef90ed66 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -32,7 +32,7 @@ mkdir ./connector-node tar xf ./risingwave-connector.tar.gz -C ./connector-node echo "--- Install dependencies" -python3 -m pip install --break-system-packages requests protobuf fastavro confluent_kafka jsonschema +python3 -m pip install --break-system-packages requests protobuf fastavro confluent_kafka jsonschema requests apt-get -y install jq echo "--- e2e, inline test" @@ -154,3 +154,22 @@ risedev slt './e2e_test/source_legacy/basic/old_row_format_syntax/*.slt' echo "--- Run CH-benCHmark" risedev slt './e2e_test/ch_benchmark/batch/ch_benchmark.slt' risedev slt './e2e_test/ch_benchmark/streaming/*.slt' + +risedev ci-kill +echo "--- cluster killed " + +echo "--- starting risingwave cluster for webhook source test" +risedev ci-start ci-1cn-1fe-with-recovery +sleep 5 +# check results +risedev slt "e2e_test/webhook/webhook_source.slt" + +risedev kill + +risedev dev ci-1cn-1fe-with-recovery +echo "--- wait for cluster recovery finish" +sleep 20 +risedev slt "e2e_test/webhook/webhook_source_recovery.slt" + +risedev ci-kill +echo "--- cluster killed " \ No newline at end of file diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 49bb47eb984b6..9a1cf0b3b5b95 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -762,7 +762,7 @@ steps: config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 25 + timeout_in_minutes: 30 retry: *auto-retry - label: "end-to-end test for opendal (parallel)" diff --git a/e2e_test/webhook/check_1.slt.part b/e2e_test/webhook/check_1.slt.part new file mode 100644 index 0000000000000..c207ba53afb00 --- /dev/null +++ b/e2e_test/webhook/check_1.slt.part @@ -0,0 +1,24 @@ +query TT +select data ->> 'source', data->> 'auth_algo' from github_hmac_sha256; +---- +github hmac_sha256 + +query TT +select data ->> 'source', data->> 'auth_algo' from github_hmac_sha1; +---- +github hmac_sha1 + +query TT +select data ->> 'source', data->> 'auth_algo' from rudderstack; +---- +rudderstack plain + +query TT +select data ->> 'source', data->> 'auth_algo' from segment_hmac_sha1; +---- +segment hmac_sha1 + +query TT +select data ->> 'source', data->> 'auth_algo' from hubspot_sha256_v2; +---- +hubspot sha256_v2 \ No newline at end of file diff --git a/e2e_test/webhook/check_2.slt.part b/e2e_test/webhook/check_2.slt.part new file mode 100644 index 0000000000000..0b6305701d882 --- /dev/null +++ b/e2e_test/webhook/check_2.slt.part @@ -0,0 +1,29 @@ +query TT +select data ->> 'source', data->> 'auth_algo' from github_hmac_sha256; +---- +github hmac_sha256 +github hmac_sha256 + +query TT +select data ->> 'source', data->> 'auth_algo' from github_hmac_sha1; +---- +github hmac_sha1 +github hmac_sha1 + +query TT +select data ->> 'source', data->> 'auth_algo' from rudderstack; +---- +rudderstack plain +rudderstack plain + +query TT +select data ->> 'source', data->> 'auth_algo' from segment_hmac_sha1; +---- +segment hmac_sha1 +segment hmac_sha1 + +query TT +select data ->> 'source', data->> 'auth_algo' from hubspot_sha256_v2; +---- +hubspot sha256_v2 +hubspot sha256_v2 \ No newline at end of file diff --git a/e2e_test/webhook/check_3.slt.part b/e2e_test/webhook/check_3.slt.part new file mode 100644 index 0000000000000..0ad97c19f0880 --- /dev/null +++ b/e2e_test/webhook/check_3.slt.part @@ -0,0 +1,34 @@ +query TT +select data ->> 'source', data->> 'auth_algo' from github_hmac_sha256; +---- +github hmac_sha256 +github hmac_sha256 +github hmac_sha256 + +query TT +select data ->> 'source', data->> 'auth_algo' from github_hmac_sha1; +---- +github hmac_sha1 +github hmac_sha1 +github hmac_sha1 + +query TT +select data ->> 'source', data->> 'auth_algo' from rudderstack; +---- +rudderstack plain +rudderstack plain +rudderstack plain + +query TT +select data ->> 'source', data->> 'auth_algo' from segment_hmac_sha1; +---- +segment hmac_sha1 +segment hmac_sha1 +segment hmac_sha1 + +query TT +select data ->> 'source', data->> 'auth_algo' from hubspot_sha256_v2; +---- +hubspot sha256_v2 +hubspot sha256_v2 +hubspot sha256_v2 \ No newline at end of file diff --git a/e2e_test/webhook/create_table.slt.part b/e2e_test/webhook/create_table.slt.part new file mode 100644 index 0000000000000..a7b0eb19a48f5 --- /dev/null +++ b/e2e_test/webhook/create_table.slt.part @@ -0,0 +1,55 @@ +statement ok +create table rudderstack ( + data JSONB +) WITH ( + connector = 'webhook', +) VALIDATE SECRET test_secret AS secure_compare( + headers->>'authorization', + test_secret +); + +statement ok +create table github_hmac_sha1 ( + data JSONB +) WITH ( + connector = 'webhook', +) VALIDATE SECRET test_secret AS secure_compare( + headers->>'x-hub-signature', + 'sha1=' || encode(hmac(test_secret, data, 'sha1'), 'hex') +); + +statement ok +create table github_hmac_sha256 ( + data JSONB +) WITH ( + connector = 'webhook', +) VALIDATE SECRET test_secret AS secure_compare( + headers->>'x-hub-signature-256', + 'sha256=' || encode(hmac(test_secret, data, 'sha256'), 'hex') +); + +statement ok +create table segment_hmac_sha1 ( + data JSONB +) WITH ( + connector = 'webhook', +) VALIDATE SECRET test_secret AS secure_compare( + headers->>'x-signature', + encode(hmac(test_secret, data, 'sha1'), 'hex') +); + +# https://developers.hubspot.com/beta-docs/guides/apps/authentication/validating-requests#validate-requests-using-the-v2-request-signature +statement ok +create table hubspot_sha256_v2 ( + data JSONB +) WITH ( + connector = 'webhook', +) VALIDATE SECRET test_secret AS secure_compare( + headers->>'x-hubspot-signature', + encode( + sha256( + convert_to( + (test_secret || 'POST' || 'http://127.0.0.1:4560/webhook/dev/public/hubspot_sha256_v2' || convert_from(data, 'utf8')) + , 'UTF8') + ), 'hex') +); diff --git a/e2e_test/webhook/drop_table.slt.part b/e2e_test/webhook/drop_table.slt.part new file mode 100644 index 0000000000000..32a2a40800b87 --- /dev/null +++ b/e2e_test/webhook/drop_table.slt.part @@ -0,0 +1,14 @@ +statement ok +DROP TABLE hubspot_sha256_v2; + +statement ok +DROP TABLE segment_hmac_sha1; + +statement ok +DROP TABLE github_hmac_sha256; + +statement ok +DROP TABLE github_hmac_sha1; + +statement ok +DROP TABLE rudderstack; diff --git a/e2e_test/webhook/sender.py b/e2e_test/webhook/sender.py new file mode 100644 index 0000000000000..446f674348444 --- /dev/null +++ b/e2e_test/webhook/sender.py @@ -0,0 +1,145 @@ +import argparse +import requests +import json +import sys +import hmac +import hashlib + +message = { + "event": "order.created", + "source": "placeholder", + "auth_algo": "placeholder", + "data": { + "order_id": 1234, + "customer_name": "Alice", + "amount": 99.99, + "currency": "USD" + }, + "timestamp": 1639581841 +} + +SERVER_URL = "http://127.0.0.1:4560/webhook/dev/public/" + + +def generate_signature_hmac(secret, payload, auth_algo, prefix): + secret_bytes = bytes(secret, 'utf-8') + payload_bytes = bytes(payload, 'utf-8') + signature = "" + if auth_algo == "sha1": + signature = prefix + hmac.new(secret_bytes, payload_bytes, digestmod=hashlib.sha1).hexdigest() + elif auth_algo == "sha256": + signature = prefix + hmac.new(secret_bytes, payload_bytes, digestmod=hashlib.sha256).hexdigest() + else: + print("Unsupported auth type") + sys.exit(1) + return signature + + +def send_webhook(url, headers, payload_json): + response = requests.post(url, headers=headers, data=payload_json) + + # Check response status and exit on failure + if response.status_code == 200: + print("Webhook sent successfully:", response) + else: + print(f"Webhook failed to send, Status Code: {response.status_code}, Response: {response.text}") + sys.exit(1) # Exit the program with an error + + +def send_github_hmac_sha1(secret): + payload = message + payload['source'] = "github" + payload['auth_algo'] = "hmac_sha1" + url = SERVER_URL + "github_hmac_sha1" + + payload_json = json.dumps(payload) + signature = generate_signature_hmac(secret, payload_json, 'sha1', "sha1=") + # Webhook message headers + headers = { + "Content-Type": "application/json", + "X-Hub-Signature": signature # Custom signature header + } + send_webhook(url, headers, payload_json) + + +def send_github_hmac_sha256(secret): + payload = message + payload['source'] = "github" + payload['auth_algo'] = "hmac_sha256" + url = SERVER_URL + "github_hmac_sha256" + + payload_json = json.dumps(payload) + signature = generate_signature_hmac(secret, payload_json, 'sha256', "sha256=") + # Webhook message headers + headers = { + "Content-Type": "application/json", + "X-Hub-Signature-256": signature # Custom signature header + } + send_webhook(url, headers, payload_json) + + +def send_rudderstack(secret): + # apply to both rudderstack, AWS EventBridge and HubSpot with API Key. + payload = message + payload['source'] = "rudderstack" + payload['auth_algo'] = "plain" + url = SERVER_URL + "rudderstack" + + payload_json = json.dumps(payload) + signature = secret + # Webhook message headers + headers = { + "Content-Type": "application/json", + "Authorization": signature # Custom signature header + } + send_webhook(url, headers, payload_json) + + +def send_segment_hmac_sha1(secret): + payload = message + payload['source'] = "segment" + payload['auth_algo'] = "hmac_sha1" + url = SERVER_URL + "segment_hmac_sha1" + + payload_json = json.dumps(payload) + signature = generate_signature_hmac(secret, payload_json, 'sha1', '') + # Webhook message headers + headers = { + "Content-Type": "application/json", + "x-signature": signature # Custom signature header + } + send_webhook(url, headers, payload_json) + + +def send_hubspot_sha256_v2(secret): + payload = message + payload['source'] = "hubspot" + payload['auth_algo'] = "sha256_v2" + url = SERVER_URL + "hubspot_sha256_v2" + + payload_json = json.dumps(payload) + payload = secret + 'POST' + url + str(payload_json) + signature = hashlib.sha256(payload.encode('utf-8')).hexdigest() + # Webhook message headers + headers = { + "Content-Type": "application/json", + "x-hubspot-signature": signature # Custom signature header + } + send_webhook(url, headers, payload_json) + + +if __name__ == "__main__": + parser = argparse.ArgumentParser(description="Simulate sending Webhook messages") + parser.add_argument("--secret", required=True, help="Secret key for generating signature") + args = parser.parse_args() + secret = args.secret + # send data + # github + send_github_hmac_sha1(secret) + send_github_hmac_sha256(secret) + # rudderstack, AWS EventBridge and HubSpot with API Key. + send_rudderstack(secret) + # segment + send_segment_hmac_sha1(secret) + # hubspot + send_hubspot_sha256_v2(secret) diff --git a/e2e_test/webhook/webhook_source.slt b/e2e_test/webhook/webhook_source.slt new file mode 100644 index 0000000000000..3467c99913ec6 --- /dev/null +++ b/e2e_test/webhook/webhook_source.slt @@ -0,0 +1,28 @@ +# Simulation test for table with webhook source + +control substitution on + +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +CREATE SECRET test_secret WITH ( backend = 'meta') AS 'TEST_WEBHOOK'; + +include ./create_table.slt.part + +# insert once +system ok +python3 e2e_test/webhook/sender.py --secret TEST_WEBHOOK + +sleep 3s + +# TODO(kexiang): will use a script to take place of check_1, check_2, check_3 +include ./check_1.slt.part + +# insert again +system ok +python3 e2e_test/webhook/sender.py --secret TEST_WEBHOOK + +sleep 3s + +include ./check_2.slt.part diff --git a/e2e_test/webhook/webhook_source_recovery.slt b/e2e_test/webhook/webhook_source_recovery.slt new file mode 100644 index 0000000000000..5755c7dfe4f6c --- /dev/null +++ b/e2e_test/webhook/webhook_source_recovery.slt @@ -0,0 +1,19 @@ +# Simulation test for table with webhook source after recovery + +control substitution on + +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +# insert once +system ok +python3 e2e_test/webhook/sender.py --secret TEST_WEBHOOK + +sleep 3s + +include ./check_3.slt.part + +include ./drop_table.slt.part + +statement ok +DROP SECRET test_secret diff --git a/proto/catalog.proto b/proto/catalog.proto index 0448af750273c..699804d8fe188 100644 --- a/proto/catalog.proto +++ b/proto/catalog.proto @@ -94,6 +94,11 @@ message StreamSourceInfo { map format_encode_secret_refs = 16; } +message WebhookSourceInfo { + secret.SecretRef secret_ref = 1; + expr.ExprNode signature_expr = 2; +} + message Source { // For shared source, this is the same as the job id. // For non-shared source and table with connector, this is a different oid. @@ -438,6 +443,9 @@ message Table { // for a streaming job. Instead, refer to `stream_plan.StreamFragmentGraph.max_parallelism`. optional uint32 maybe_vnode_count = 40; + // The information used by webhook source to validate the incoming data. + optional WebhookSourceInfo webhook_info = 41; + // Per-table catalog version, used by schema change. `None` for internal // tables and tests. Not to be confused with the global catalog version for // notification service. diff --git a/proto/expr.proto b/proto/expr.proto index e56668c990447..ed7a790ab63e4 100644 --- a/proto/expr.proto +++ b/proto/expr.proto @@ -197,6 +197,8 @@ message ExprNode { INET_NTOA = 329; QUOTE_LITERAL = 330; QUOTE_NULLABLE = 331; + HMAC = 332; + SECURE_COMPARE = 333; // Unary operators NEG = 401; diff --git a/src/common/secret/src/secret_manager.rs b/src/common/secret/src/secret_manager.rs index 723c0f5791d4d..41cae5070810f 100644 --- a/src/common/secret/src/secret_manager.rs +++ b/src/common/secret/src/secret_manager.rs @@ -18,6 +18,7 @@ use std::io::Write; use std::path::PathBuf; use anyhow::{anyhow, Context}; +use parking_lot::lock_api::RwLockReadGuard; use parking_lot::RwLock; use prost::Message; use risingwave_pb::catalog::PbSecret; @@ -134,30 +135,43 @@ impl LocalSecretManager { ) -> SecretResult> { let secret_guard = self.secrets.read(); for (option_key, secret_ref) in secret_refs { - let secret_id = secret_ref.secret_id; - let pb_secret_bytes = secret_guard - .get(&secret_id) - .ok_or(SecretError::ItemNotFound(secret_id))?; - let secret_value_bytes = Self::get_secret_value(pb_secret_bytes)?; - match secret_ref.ref_as() { - RefAsType::Text => { - // We converted the secret string from sql to bytes using `as_bytes` in frontend. - // So use `from_utf8` here to convert it back to string. - options.insert(option_key, String::from_utf8(secret_value_bytes.clone())?); - } - RefAsType::File => { - let path_str = - self.get_or_init_secret_file(secret_id, secret_value_bytes.clone())?; - options.insert(option_key, path_str); - } - RefAsType::Unspecified => { - return Err(SecretError::UnspecifiedRefType(secret_id)); - } - } + let path_str = self.fill_secret_inner(secret_ref, &secret_guard)?; + options.insert(option_key, path_str); } Ok(options) } + pub fn fill_secret(&self, secret_ref: PbSecretRef) -> SecretResult { + let secret_guard: RwLockReadGuard<'_, parking_lot::RawRwLock, HashMap>> = + self.secrets.read(); + self.fill_secret_inner(secret_ref, &secret_guard) + } + + fn fill_secret_inner( + &self, + secret_ref: PbSecretRef, + secret_guard: &RwLockReadGuard<'_, parking_lot::RawRwLock, HashMap>>, + ) -> SecretResult { + let secret_id = secret_ref.secret_id; + let pb_secret_bytes = secret_guard + .get(&secret_id) + .ok_or(SecretError::ItemNotFound(secret_id))?; + let secret_value_bytes = Self::get_secret_value(pb_secret_bytes)?; + match secret_ref.ref_as() { + RefAsType::Text => { + // We converted the secret string from sql to bytes using `as_bytes` in frontend. + // So use `from_utf8` here to convert it back to string. + Ok(String::from_utf8(secret_value_bytes.clone())?) + } + RefAsType::File => { + let path_str = + self.get_or_init_secret_file(secret_id, secret_value_bytes.clone())?; + Ok(path_str) + } + RefAsType::Unspecified => Err(SecretError::UnspecifiedRefType(secret_id)), + } + } + /// Get the secret file for the given secret id and return the path string. If the file does not exist, create it. /// WARNING: This method should be called only when the secret manager is locked. fn get_or_init_secret_file( diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index e031a85a34d62..fc804ff4acdb6 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -58,6 +58,8 @@ const SPLIT_TYPE_FIELD: &str = "split_type"; const SPLIT_INFO_FIELD: &str = "split_info"; pub const UPSTREAM_SOURCE_KEY: &str = "connector"; +pub const WEBHOOK_CONNECTOR: &str = "webhook"; + pub trait TryFromBTreeMap: Sized + UnknownFields { /// Used to initialize the source properties from the raw untyped `WITH` options. fn try_from_btreemap( diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 899fc2a2379f5..21d0c7c3937d6 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -28,7 +28,7 @@ pub mod pulsar; use std::future::IntoFuture; -pub use base::{UPSTREAM_SOURCE_KEY, *}; +pub use base::{UPSTREAM_SOURCE_KEY, WEBHOOK_CONNECTOR, *}; pub(crate) use common::*; use google_cloud_pubsub::subscription::Subscription; pub use google_pubsub::GOOGLE_PUBSUB_CONNECTOR; diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index 257cf19b77e7d..b8e2448a76397 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -37,12 +37,14 @@ chrono = { version = "0.4", default-features = false, features = [ "std", ] } chrono-tz = { version = "0.10", features = ["case-insensitive"] } +constant_time_eq = "0.3" educe = "0.6" fancy-regex = "0.14" futures-async-stream = { workspace = true } futures-util = "0.3" ginepro = "0.8" hex = "0.4" +hmac = "0.12" icelake = { workspace = true } itertools = { workspace = true } jsonbb = { workspace = true } diff --git a/src/expr/impl/src/scalar/cmp.rs b/src/expr/impl/src/scalar/cmp.rs index 20c0afc307e74..ed0e2bbb832e7 100644 --- a/src/expr/impl/src/scalar/cmp.rs +++ b/src/expr/impl/src/scalar/cmp.rs @@ -14,6 +14,7 @@ use std::fmt::Debug; +use constant_time_eq::constant_time_eq; use risingwave_common::array::{Array, BoolArray}; use risingwave_common::bitmap::Bitmap; use risingwave_common::row::Row; @@ -425,6 +426,11 @@ fn batch_is_not_null(a: &impl Array) -> BoolArray { BoolArray::new(a.null_bitmap().clone(), Bitmap::ones(a.len())) } +#[function("secure_compare(varchar, varchar) -> boolean")] +pub fn secure_compare(left: &str, right: &str) -> bool { + constant_time_eq(left.as_bytes(), right.as_bytes()) +} + #[cfg(test)] mod tests { use std::str::FromStr; diff --git a/src/expr/impl/src/scalar/hmac.rs b/src/expr/impl/src/scalar/hmac.rs new file mode 100644 index 0000000000000..f2184915e6a0f --- /dev/null +++ b/src/expr/impl/src/scalar/hmac.rs @@ -0,0 +1,73 @@ +// Copyright 2024 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 hmac::{Hmac, Mac}; +use risingwave_expr::{function, ExprError, Result}; +use sha1::Sha1; +use sha2::Sha256; + +#[function("hmac(varchar, bytea, varchar) -> bytea")] +pub fn hmac(secret: &str, payload: &[u8], sha_type: &str) -> Result> { + if sha_type == "sha1" { + Ok(hmac_sha1(secret, payload)) + } else if sha_type == "sha256" { + Ok(hmac_sha256(secret, payload)) + } else { + return Err(ExprError::InvalidParam { + name: "sha_type", + reason: format!("Unsupported SHA type: {}", sha_type).into(), + }); + } +} + +fn hmac_sha256(secret: &str, payload: &[u8]) -> Box<[u8]> { + let mut mac = + Hmac::::new_from_slice(secret.as_bytes()).expect("HMAC can take key of any size"); + mac.update(payload); + + let code_bytes = mac.finalize().into_bytes(); + code_bytes.as_slice().into() +} + +fn hmac_sha1(secret: &str, payload: &[u8]) -> Box<[u8]> { + let mut mac = + Hmac::::new_from_slice(secret.as_bytes()).expect("HMAC can take key of any size"); + mac.update(payload); + + let code_bytes = mac.finalize().into_bytes(); + code_bytes.as_slice().into() +} + +#[cfg(test)] +mod tests { + use hex::encode; + + use super::*; + + #[test] + fn test_verify_signature_hmac_sha256() { + let secret = "your_secret_key"; + let payload = b"your_webhook_payload"; + let signature = "cef8b98a91902c492b85d97f049aa4bfc5e7e3f9b8b7bf7cb49c5f829d2dac85"; + assert!(encode(hmac_sha256(secret, payload)) == signature); + } + + #[test] + fn test_verify_signature_hmac_sha1() { + let secret = "your_secret_key"; + let payload = b"your_webhook_payload"; + let signature = "65cb920a4b8c6ab8e2eab861a096a7bc2c05d8ba"; + assert!(encode(hmac_sha1(secret, payload)) == signature); + } +} diff --git a/src/expr/impl/src/scalar/mod.rs b/src/expr/impl/src/scalar/mod.rs index fbf9b512ea86d..ea5ce353fac48 100644 --- a/src/expr/impl/src/scalar/mod.rs +++ b/src/expr/impl/src/scalar/mod.rs @@ -47,6 +47,7 @@ mod extract; mod field; mod format; mod format_type; +mod hmac; mod in_; mod int256; mod jsonb_access; diff --git a/src/frontend/Cargo.toml b/src/frontend/Cargo.toml index 2db5ef4bb3727..51a78099f4e61 100644 --- a/src/frontend/Cargo.toml +++ b/src/frontend/Cargo.toml @@ -21,6 +21,7 @@ async-recursion = "1.1.0" async-trait = "0.1" auto_enums = { workspace = true } auto_impl = "1" +axum = { workspace = true } base64 = "0.22" bk-tree = "0.5.0" bytes = "1" @@ -99,6 +100,12 @@ tokio = { version = "0.2", package = "madsim-tokio", features = [ tokio-postgres = "0.7" tokio-stream = { workspace = true } tonic = { workspace = true } +tower = { version = "0.4", features = ["util", "load-shed"] } +tower-http = { version = "0.6", features = [ + "add-extension", + "cors", + "compression-gzip", +] } tracing = "0.1" uuid = "1" zstd = { version = "0.13", default-features = false } diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index 0efa5f66865e8..ddeac9b8b9161 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -435,6 +435,7 @@ impl TestCase { cdc_table_info, include_column_options, wildcard_idx, + webhook_info, .. } => { let format_encode = format_encode.map(|schema| schema.into_v2_with_warning()); @@ -453,6 +454,7 @@ impl TestCase { with_version_column, cdc_table_info, include_column_options, + webhook_info, ) .await?; } diff --git a/src/frontend/src/binder/expr/function/builtin_scalar.rs b/src/frontend/src/binder/expr/function/builtin_scalar.rs index 7570f72c5d095..5859d54c26d96 100644 --- a/src/frontend/src/binder/expr/function/builtin_scalar.rs +++ b/src/frontend/src/binder/expr/function/builtin_scalar.rs @@ -302,6 +302,8 @@ impl Binder { ("sha512", raw_call(ExprType::Sha512)), ("encrypt", raw_call(ExprType::Encrypt)), ("decrypt", raw_call(ExprType::Decrypt)), + ("hmac", raw_call(ExprType::Hmac)), + ("secure_compare",raw_call(ExprType::SecureCompare)), ("left", raw_call(ExprType::Left)), ("right", raw_call(ExprType::Right)), ("inet_aton", raw_call(ExprType::InetAton)), diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 59aa42be0e79b..c5c89c1d0e1a9 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -89,6 +89,24 @@ impl Binder { .into(), ) } + } else if let Some(ctx) = self.secure_compare_context.as_ref() { + // Currently, the generated columns are not supported yet. So the ident here should only be one of the following + // - `headers` + // - secret name + // - the name of the payload column + // TODO(Kexiang): Generated columns or INCLUDE clause should be supported. + if ident.real_value() == *"headers" { + Ok(InputRef::new(0, DataType::Jsonb).into()) + } else if ident.real_value() == ctx.secret_name { + Ok(InputRef::new(1, DataType::Varchar).into()) + } else if ident.real_value() == ctx.column_name { + Ok(InputRef::new(2, DataType::Bytea).into()) + } else { + Err( + ErrorCode::ItemNotFound(format!("Unknown arg: {}", ident.real_value())) + .into(), + ) + } } else { self.bind_column(&[ident]) } diff --git a/src/frontend/src/binder/mod.rs b/src/frontend/src/binder/mod.rs index f0d42fd321f3f..7e14482fbf0af 100644 --- a/src/frontend/src/binder/mod.rs +++ b/src/frontend/src/binder/mod.rs @@ -132,6 +132,19 @@ pub struct Binder { /// The temporary sources that will be used during binding phase temporary_source_manager: TemporarySourceManager, + + /// Information for `secure_compare` function. It's ONLY available when binding the + /// `VALIDATE` clause of Webhook source i.e. `VALIDATE SECRET ... AS SECURE_COMPARE(...)`. + secure_compare_context: Option, +} + +// There's one more hidden name, `HEADERS`, which is a reserved identifier for HTTP headers. Its type is `JSONB`. +#[derive(Default, Clone, Debug)] +pub struct SecureCompareContext { + /// The column name to store the whole payload in `JSONB`, but during validation it will be used as `bytea` + pub column_name: String, + /// The secret (usually a token provided by the webhook source user) to validate the calls + pub secret_name: String, } #[derive(Clone, Debug, Default)] @@ -332,6 +345,7 @@ impl Binder { param_types: ParameterTypes::new(param_types), udf_context: UdfContext::new(), temporary_source_manager: session.temporary_source_manager(), + secure_compare_context: None, } } @@ -354,6 +368,15 @@ impl Binder { Self::new_inner(session, BindFor::Ddl, vec![]) } + pub fn new_for_ddl_with_secure_compare( + session: &SessionImpl, + ctx: SecureCompareContext, + ) -> Binder { + let mut binder = Self::new_inner(session, BindFor::Ddl, vec![]); + binder.secure_compare_context = Some(ctx); + binder + } + pub fn new_for_system(session: &SessionImpl) -> Binder { Self::new_inner(session, BindFor::System, vec![]) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 8835b655e06fa..7b745f436ff34 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -25,7 +25,7 @@ use risingwave_common::hash::{VnodeCount, VnodeCountCompat}; use risingwave_common::util::epoch::Epoch; use risingwave_common::util::sort_util::ColumnOrder; use risingwave_pb::catalog::table::{OptionalAssociatedSourceId, PbTableType, PbTableVersion}; -use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus, PbTable}; +use risingwave_pb::catalog::{PbCreateType, PbStreamJobStatus, PbTable, PbWebhookSourceInfo}; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::DefaultColumnDesc; @@ -180,6 +180,8 @@ pub struct TableCatalog { /// [`StreamMaterialize::derive_table_catalog`]: crate::optimizer::plan_node::StreamMaterialize::derive_table_catalog /// [`TableCatalogBuilder::build`]: crate::optimizer::plan_node::utils::TableCatalogBuilder::build pub vnode_count: VnodeCount, + + pub webhook_info: Option, } #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] @@ -464,6 +466,7 @@ impl TableCatalog { retention_seconds: self.retention_seconds, cdc_table_id: self.cdc_table_id.clone(), maybe_vnode_count: self.vnode_count.to_protobuf(), + webhook_info: self.webhook_info.clone(), } } @@ -656,6 +659,7 @@ impl From for TableCatalog { .collect_vec(), cdc_table_id: tb.cdc_table_id, vnode_count, + webhook_info: tb.webhook_info, } } } @@ -747,6 +751,7 @@ mod tests { version_column_index: None, cdc_table_id: None, maybe_vnode_count: VnodeCount::set(233).to_protobuf(), + webhook_info: None, } .into(); @@ -814,6 +819,7 @@ mod tests { version_column_index: None, cdc_table_id: None, vnode_count: VnodeCount::set(233), + webhook_info: None, } ); assert_eq!(table, TableCatalog::from(table.to_prost(0, 0))); diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 83a8cfa537bae..9c3adc901b464 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -232,6 +232,8 @@ impl ExprVisitor for ImpureAnalyzer { | Type::Sha256 | Type::Sha384 | Type::Sha512 + | Type::Hmac + | Type::SecureCompare | Type::Decrypt | Type::Encrypt | Type::Tand diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d9f4fcce5ff85..f605687b3a731 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -54,12 +54,12 @@ use risingwave_connector::source::datagen::DATAGEN_CONNECTOR; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType}; use risingwave_connector::source::test_source::TEST_CONNECTOR; -pub use risingwave_connector::source::UPSTREAM_SOURCE_KEY; use risingwave_connector::source::{ ConnectorProperties, AZBLOB_CONNECTOR, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, }; +pub use risingwave_connector::source::{UPSTREAM_SOURCE_KEY, WEBHOOK_CONNECTOR}; use risingwave_connector::WithPropertiesExt; use risingwave_pb::catalog::{PbSchemaRegistryNameStrategy, StreamSourceInfo, WatermarkDesc}; use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index 961be381b5f8c..77975b1dec952 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -36,22 +36,25 @@ use risingwave_connector::source::cdc::external::{ }; use risingwave_connector::{source, WithOptionsSecResolved}; use risingwave_pb::catalog::source::OptionalAssociatedTableId; -use risingwave_pb::catalog::{PbSource, PbTable, Table, WatermarkDesc}; +use risingwave_pb::catalog::{PbSource, PbTable, PbWebhookSourceInfo, Table, WatermarkDesc}; use risingwave_pb::ddl_service::TableJobType; use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn; use risingwave_pb::plan_common::{ AdditionalColumn, ColumnDescVersion, DefaultColumnDesc, GeneratedColumnDesc, }; +use risingwave_pb::secret::secret_ref::PbRefAsType; +use risingwave_pb::secret::PbSecretRef; use risingwave_pb::stream_plan::StreamFragmentGraph; use risingwave_sqlparser::ast::{ - CdcTableInfo, ColumnDef, ColumnOption, DataType as AstDataType, ExplainOptions, Format, - FormatEncodeOptions, ObjectName, OnConflict, SourceWatermark, TableConstraint, + CdcTableInfo, ColumnDef, ColumnOption, DataType, DataType as AstDataType, ExplainOptions, + Format, FormatEncodeOptions, ObjectName, OnConflict, SecretRefAsType, SourceWatermark, + TableConstraint, WebhookSourceInfo, }; use risingwave_sqlparser::parser::IncludeOption; use thiserror_ext::AsReport; use super::RwPgResponse; -use crate::binder::{bind_data_type, bind_struct_field, Clause}; +use crate::binder::{bind_data_type, bind_struct_field, Clause, SecureCompareContext}; use crate::catalog::root_catalog::SchemaPath; use crate::catalog::source_catalog::SourceCatalog; use crate::catalog::table_catalog::TableVersion; @@ -541,6 +544,7 @@ pub(crate) fn gen_create_table_plan( append_only: bool, on_conflict: Option, with_version_column: Option, + webhook_info: Option, ) -> Result<(PlanRef, PbTable)> { let definition = context.normalized_sql().to_owned(); let mut columns = bind_sql_columns(&column_defs)?; @@ -565,6 +569,7 @@ pub(crate) fn gen_create_table_plan( on_conflict, with_version_column, Some(col_id_gen.into_version()), + webhook_info, ) } @@ -581,6 +586,7 @@ pub(crate) fn gen_create_table_plan_without_source( on_conflict: Option, with_version_column: Option, version: Option, + webhook_info: Option, ) -> Result<(PlanRef, PbTable)> { let pk_names = bind_sql_pk_names(&column_defs, bind_table_constraints(&constraints)?)?; let (mut columns, pk_column_ids, row_id_index) = @@ -622,6 +628,7 @@ pub(crate) fn gen_create_table_plan_without_source( None, database_id, schema_id, + webhook_info, ) } @@ -652,6 +659,7 @@ fn gen_table_plan_with_source( Some(cloned_source_catalog), database_id, schema_id, + None, ) } @@ -672,6 +680,7 @@ fn gen_table_plan_inner( source_catalog: Option, database_id: DatabaseId, schema_id: SchemaId, + webhook_info: Option, ) -> Result<(PlanRef, PbTable)> { let session = context.session_ctx().clone(); let retention_seconds = context.with_options().retention_seconds(); @@ -742,6 +751,7 @@ fn gen_table_plan_inner( is_external_source, retention_seconds, None, + webhook_info, )?; let mut table = materialize.table().to_prost(schema_id, database_id); @@ -867,6 +877,7 @@ pub(crate) fn gen_create_table_plan_for_cdc_table( true, None, Some(cdc_table_id), + None, )?; let mut table = materialize.table().to_prost(schema_id, database_id); @@ -948,6 +959,7 @@ pub(super) async fn handle_create_table_plan( on_conflict: Option, with_version_column: Option, include_column_options: IncludeOption, + webhook_info: Option, ) -> Result<(PlanRef, Option, PbTable, TableJobType)> { let col_id_gen = ColumnIdGenerator::new_initial(); let format_encode = check_create_table_with_source( @@ -978,6 +990,10 @@ pub(super) async fn handle_create_table_plan( TableJobType::General, ), (None, None) => { + let webhook_info = webhook_info + .map(|info| bind_webhook_info(&handler_args.session, &column_defs, info)) + .transpose()?; + let context = OptimizerContext::new(handler_args, explain_options); let (plan, table) = gen_create_table_plan( context, @@ -989,6 +1005,7 @@ pub(super) async fn handle_create_table_plan( append_only, on_conflict, with_version_column, + webhook_info, )?; ((plan, None, table), TableJobType::General) @@ -1255,6 +1272,7 @@ pub async fn handle_create_table( with_version_column: Option, cdc_table_info: Option, include_column_options: IncludeOption, + webhook_info: Option, ) -> Result { let session = handler_args.session.clone(); @@ -1287,6 +1305,7 @@ pub async fn handle_create_table( on_conflict, with_version_column, include_column_options, + webhook_info, ) .await?; @@ -1319,7 +1338,8 @@ pub fn check_create_table_with_source( if cdc_table_info.is_some() { return Ok(format_encode); } - let defined_source = with_options.contains_key(UPSTREAM_SOURCE_KEY); + let defined_source = with_options.is_source_connector(); + if !include_column_options.is_empty() && !defined_source { return Err(ErrorCode::InvalidInputSyntax( "INCLUDE should be used with a connector".to_owned(), @@ -1387,6 +1407,7 @@ pub async fn generate_stream_graph_for_replace_table( append_only, on_conflict, with_version_column, + original_catalog.webhook_info.clone(), )?; ((plan, None, table), TableJobType::General) } @@ -1490,6 +1511,65 @@ fn get_source_and_resolved_table_name( Ok((source, resolved_table_name, database_id, schema_id)) } +// validate the webhook_info and also bind the webhook_info to protobuf +fn bind_webhook_info( + session: &Arc, + columns_defs: &[ColumnDef], + webhook_info: WebhookSourceInfo, +) -> Result { + // validate columns + if columns_defs.len() != 1 || columns_defs[0].data_type.as_ref().unwrap() != &DataType::Jsonb { + return Err(ErrorCode::InvalidInputSyntax( + "Table with webhook source should have exactly one JSONB column".to_owned(), + ) + .into()); + } + + let WebhookSourceInfo { + secret_ref, + signature_expr, + } = webhook_info; + + // validate secret_ref + let db_name = session.database(); + let (schema_name, secret_name) = + Binder::resolve_schema_qualified_name(db_name, secret_ref.secret_name.clone())?; + let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?; + let pb_secret_ref = PbSecretRef { + secret_id: secret_catalog.id.secret_id(), + ref_as: match secret_ref.ref_as { + SecretRefAsType::Text => PbRefAsType::Text, + SecretRefAsType::File => PbRefAsType::File, + } + .into(), + }; + + let secure_compare_context = SecureCompareContext { + column_name: columns_defs[0].name.real_value(), + secret_name, + }; + let mut binder = Binder::new_for_ddl_with_secure_compare(session, secure_compare_context); + let expr = binder.bind_expr(signature_expr.clone())?; + + // validate expr, ensuring it is SECURE_COMPARE() + if expr.as_function_call().is_none() + || expr.as_function_call().unwrap().func_type() + != crate::optimizer::plan_node::generic::ExprType::SecureCompare + { + return Err(ErrorCode::InvalidInputSyntax( + "The signature verification function must be SECURE_COMPARE()".to_owned(), + ) + .into()); + } + + let pb_webhook_info = PbWebhookSourceInfo { + secret_ref: Some(pb_secret_ref), + signature_expr: Some(expr.to_expr_proto()), + }; + + Ok(pb_webhook_info) +} + #[cfg(test)] mod tests { use risingwave_common::catalog::{ diff --git a/src/frontend/src/handler/create_table_as.rs b/src/frontend/src/handler/create_table_as.rs index 27c527969f9b2..5bffd99a08747 100644 --- a/src/frontend/src/handler/create_table_as.rs +++ b/src/frontend/src/handler/create_table_as.rs @@ -108,6 +108,7 @@ pub async fn handle_create_as( on_conflict, with_version_column, Some(col_id_gen.into_version()), + None, )?; let graph = build_graph(plan)?; diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 20a7036dc1487..66003051a70b9 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -65,6 +65,7 @@ async fn do_handle_explain( cdc_table_info, include_column_options, wildcard_idx, + webhook_info, .. } => { let format_encode = format_encode.map(|s| s.into_v2_with_warning()); @@ -83,6 +84,7 @@ async fn do_handle_explain( on_conflict, with_version_column, include_column_options, + webhook_info, ) .await?; let context = plan.ctx(); diff --git a/src/frontend/src/handler/mod.rs b/src/frontend/src/handler/mod.rs index cfa6dc10277e8..535b7c8cf6062 100644 --- a/src/frontend/src/handler/mod.rs +++ b/src/frontend/src/handler/mod.rs @@ -345,6 +345,7 @@ pub async fn handle( with_version_column, cdc_table_info, include_column_options, + webhook_info, } => { if or_replace { bail_not_implemented!("CREATE OR REPLACE TABLE"); @@ -380,6 +381,7 @@ pub async fn handle( with_version_column, cdc_table_info, include_column_options, + webhook_info, ) .await } diff --git a/src/frontend/src/lib.rs b/src/frontend/src/lib.rs index 986d4cfb35660..4d788bbd7f014 100644 --- a/src/frontend/src/lib.rs +++ b/src/frontend/src/lib.rs @@ -71,6 +71,7 @@ pub(crate) mod error; mod meta_client; pub mod test_utils; mod user; +pub mod webhook; pub mod health_service; mod monitor; @@ -170,6 +171,11 @@ pub struct FrontendOpts { /// Total available memory for the frontend node in bytes. Used for batch computing. #[clap(long, env = "RW_FRONTEND_TOTAL_MEMORY_BYTES", default_value_t = default_frontend_total_memory_bytes())] pub frontend_total_memory_bytes: usize, + + /// The address that the webhook service listens to. + /// Usually the localhost + desired port. + #[clap(long, env = "RW_WEBHOOK_LISTEN_ADDR", default_value = "0.0.0.0:4560")] + pub webhook_listen_addr: String, } impl risingwave_common::opts::Opts for FrontendOpts { @@ -204,6 +210,7 @@ pub fn start( // slow compile in release mode. Box::pin(async move { let listen_addr = opts.listen_addr.clone(); + let webhook_listen_addr = opts.webhook_listen_addr.parse().unwrap(); let tcp_keepalive = TcpKeepalive::new().with_time(Duration::from_secs(opts.tcp_keepalive_idle_secs as _)); @@ -219,6 +226,9 @@ pub fn start( .collect::>(), ); + let webhook_service = crate::webhook::WebhookService::new(webhook_listen_addr); + let _task = tokio::spawn(webhook_service.serve()); + pg_serve( &listen_addr, tcp_keepalive, diff --git a/src/frontend/src/optimizer/mod.rs b/src/frontend/src/optimizer/mod.rs index 9572a46d90959..e5e9c2b8c0bd3 100644 --- a/src/frontend/src/optimizer/mod.rs +++ b/src/frontend/src/optimizer/mod.rs @@ -58,7 +58,7 @@ use risingwave_common::types::DataType; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_connector::sink::catalog::SinkFormatDesc; -use risingwave_pb::catalog::WatermarkDesc; +use risingwave_pb::catalog::{PbWebhookSourceInfo, WatermarkDesc}; use risingwave_pb::stream_plan::StreamScanType; use self::heuristic_optimizer::ApplyOrder; @@ -656,6 +656,7 @@ impl PlanRoot { with_external_source: bool, retention_seconds: Option, cdc_table_id: Option, + webhook_info: Option, ) -> Result { assert_eq!(self.phase, PlanPhase::Logical); assert_eq!(self.plan.convention(), Convention::Logical); @@ -890,6 +891,7 @@ impl PlanRoot { version, retention_seconds, cdc_table_id, + webhook_info, ) } diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index c53bde642ad3e..4fa604cfb101a 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -221,6 +221,8 @@ impl Strong { | ExprType::Sha256 | ExprType::Sha384 | ExprType::Sha512 + | ExprType::Hmac + | ExprType::SecureCompare | ExprType::Left | ExprType::Right | ExprType::Format diff --git a/src/frontend/src/optimizer/plan_node/stream_materialize.rs b/src/frontend/src/optimizer/plan_node/stream_materialize.rs index 342bfbedd1825..c8f673f11b52a 100644 --- a/src/frontend/src/optimizer/plan_node/stream_materialize.rs +++ b/src/frontend/src/optimizer/plan_node/stream_materialize.rs @@ -24,6 +24,7 @@ use risingwave_common::catalog::{ use risingwave_common::hash::VnodeCount; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::sort_util::{ColumnOrder, OrderType}; +use risingwave_pb::catalog::PbWebhookSourceInfo; use risingwave_pb::stream_plan::stream_node::PbNodeBody; use super::derive::derive_columns; @@ -110,6 +111,7 @@ impl StreamMaterialize { cardinality, retention_seconds, create_type, + None, )?; Ok(Self::new(input, table)) @@ -135,6 +137,7 @@ impl StreamMaterialize { version: Option, retention_seconds: Option, cdc_table_id: Option, + webhook_info: Option, ) -> Result { let input = Self::rewrite_input(input, user_distributed_by, TableType::Table)?; @@ -153,6 +156,7 @@ impl StreamMaterialize { Cardinality::unknown(), // unknown cardinality for tables retention_seconds, CreateType::Foreground, + webhook_info, )?; table.cdc_table_id = cdc_table_id; @@ -227,6 +231,7 @@ impl StreamMaterialize { cardinality: Cardinality, retention_seconds: Option, create_type: CreateType, + webhook_info: Option, ) -> Result { let input = rewritten_input; @@ -285,6 +290,7 @@ impl StreamMaterialize { retention_seconds: retention_seconds.map(|i| i.into()), cdc_table_id: None, vnode_count: VnodeCount::Placeholder, // will be filled in by the meta service later + webhook_info, }) } diff --git a/src/frontend/src/optimizer/plan_node/utils.rs b/src/frontend/src/optimizer/plan_node/utils.rs index 2433a659bad0a..327380088f29c 100644 --- a/src/frontend/src/optimizer/plan_node/utils.rs +++ b/src/frontend/src/optimizer/plan_node/utils.rs @@ -196,6 +196,7 @@ impl TableCatalogBuilder { retention_seconds: None, cdc_table_id: None, vnode_count: VnodeCount::Placeholder, // will be filled in by the meta service later + webhook_info: None, } } diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 1018d3c41f3b8..d6d63720af04f 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -590,6 +590,7 @@ pub(crate) mod tests { created_at_cluster_version: None, cdc_table_id: None, vnode_count: VnodeCount::set(vnode_count), + webhook_info: None, }; let batch_plan_node: PlanRef = LogicalScan::create( "".to_string(), diff --git a/src/frontend/src/utils/with_options.rs b/src/frontend/src/utils/with_options.rs index 9d61021dab4fe..cc8d6747aedd0 100644 --- a/src/frontend/src/utils/with_options.rs +++ b/src/frontend/src/utils/with_options.rs @@ -30,6 +30,7 @@ use risingwave_sqlparser::ast::{ use super::OverwriteOptions; use crate::catalog::ConnectionId; use crate::error::{ErrorCode, Result as RwResult, RwError}; +use crate::handler::create_source::{UPSTREAM_SOURCE_KEY, WEBHOOK_CONNECTOR}; use crate::session::SessionImpl; use crate::Binder; @@ -156,6 +157,11 @@ impl WithOptions { ))) } } + + pub fn is_source_connector(&self) -> bool { + self.inner.contains_key(UPSTREAM_SOURCE_KEY) + && self.inner.get(UPSTREAM_SOURCE_KEY).unwrap() != WEBHOOK_CONNECTOR + } } /// Get the secret id from the name. diff --git a/src/frontend/src/webhook/mod.rs b/src/frontend/src/webhook/mod.rs new file mode 100644 index 0000000000000..9938258a98979 --- /dev/null +++ b/src/frontend/src/webhook/mod.rs @@ -0,0 +1,219 @@ +// Copyright 2024 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::net::{IpAddr, SocketAddr}; +use std::sync::Arc; + +use anyhow::{anyhow, Context}; +use axum::body::Bytes; +use axum::extract::{Extension, Path}; +use axum::http::{HeaderMap, Method, StatusCode}; +use axum::routing::post; +use axum::Router; +use pgwire::net::Address; +use pgwire::pg_server::SessionManager; +use risingwave_common::secret::LocalSecretManager; +use risingwave_sqlparser::ast::{Expr, ObjectName}; +use tokio::net::TcpListener; +use tower::ServiceBuilder; +use tower_http::add_extension::AddExtensionLayer; +use tower_http::compression::CompressionLayer; +use tower_http::cors::{self, CorsLayer}; + +use crate::handler::handle; +use crate::webhook::utils::{err, Result}; +mod utils; + +pub type Service = Arc; + +// We always use the `root` user to connect to the database to allow the webhook service to access all tables. +const USER: &str = "root"; + +#[derive(Clone)] +pub struct WebhookService { + webhook_addr: SocketAddr, +} + +pub(super) mod handlers { + use std::net::Ipv4Addr; + + use risingwave_pb::catalog::WebhookSourceInfo; + use risingwave_sqlparser::ast::{Query, SetExpr, Statement, Value, Values}; + use utils::{header_map_to_json, verify_signature}; + + use super::*; + use crate::catalog::root_catalog::SchemaPath; + use crate::session::SESSION_MANAGER; + + pub async fn handle_post_request( + Extension(_srv): Extension, + headers: HeaderMap, + Path((database, schema, table)): Path<(String, String, String)>, + body: Bytes, + ) -> Result<()> { + let session_mgr = SESSION_MANAGER + .get() + .expect("session manager has been initialized"); + + // Can be any address, we use the port of meta to indicate that it's a internal request. + let dummy_addr = Address::Tcp(SocketAddr::new(IpAddr::V4(Ipv4Addr::new(0, 0, 0, 0)), 5691)); + + // TODO(kexiang): optimize this + // get a session object for the corresponding database + let session = session_mgr + .connect(database.as_str(), USER, Arc::new(dummy_addr)) + .map_err(|e| { + err( + anyhow!(e).context(format!( + "Failed to create session for database `{}` with user `{}`", + database, USER + )), + StatusCode::UNAUTHORIZED, + ) + })?; + + let WebhookSourceInfo { + secret_ref, + signature_expr, + } = { + let search_path = session.config().search_path(); + let schema_path = SchemaPath::new(Some(schema.as_str()), &search_path, USER); + + let reader = session.env().catalog_reader().read_guard(); + let (table_catalog, _schema) = reader + .get_any_table_by_name(database.as_str(), schema_path, &table) + .map_err(|e| err(e, StatusCode::NOT_FOUND))?; + + table_catalog + .webhook_info + .as_ref() + .ok_or_else(|| { + err( + anyhow!("Table `{}` is not with webhook source", table), + StatusCode::FORBIDDEN, + ) + })? + .clone() + }; + + let secret_string = LocalSecretManager::global() + .fill_secret(secret_ref.unwrap()) + .map_err(|e| err(e, StatusCode::NOT_FOUND))?; + + // Once limitation here is that the key is no longer case-insensitive, users must user the lowercase key when defining the webhook source table. + let headers_jsonb = header_map_to_json(&headers); + + let is_valid = verify_signature( + headers_jsonb, + secret_string.as_str(), + body.as_ref(), + signature_expr.unwrap(), + ) + .await?; + + if !is_valid { + return Err(err( + anyhow!("Signature verification failed"), + StatusCode::UNAUTHORIZED, + )); + } + + let payload = String::from_utf8(body.to_vec()).map_err(|e| { + err( + anyhow!(e).context("Failed to parse body"), + StatusCode::UNPROCESSABLE_ENTITY, + ) + })?; + + let insert_stmt = Statement::Insert { + table_name: ObjectName::from(vec![table.as_str().into()]), + columns: vec![], + source: Box::new(Query { + with: None, + body: SetExpr::Values(Values(vec![vec![Expr::Value(Value::SingleQuotedString( + payload, + ))]])), + order_by: vec![], + limit: None, + offset: None, + fetch: None, + }), + returning: vec![], + }; + + let _rsp = handle(session, insert_stmt, Arc::from(""), vec![]) + .await + .map_err(|e| { + err( + anyhow!(e).context("Failed to insert into target table"), + StatusCode::INTERNAL_SERVER_ERROR, + ) + })?; + + Ok(()) + } +} + +impl WebhookService { + pub fn new(webhook_addr: SocketAddr) -> Self { + Self { webhook_addr } + } + + pub async fn serve(self) -> anyhow::Result<()> { + use handlers::*; + let srv = Arc::new(self); + + let cors_layer = CorsLayer::new() + .allow_origin(cors::Any) + .allow_methods(vec![Method::POST]); + + let api_router: Router = Router::new() + .route("/:database/:schema/:table", post(handle_post_request)) + .layer( + ServiceBuilder::new() + .layer(AddExtensionLayer::new(srv.clone())) + .into_inner(), + ) + .layer(cors_layer); + + let app: Router = Router::new() + .nest("/webhook", api_router) + .layer(CompressionLayer::new()); + + let listener = TcpListener::bind(&srv.webhook_addr) + .await + .context("Failed to bind dashboard address")?; + + #[cfg(not(madsim))] + axum::serve(listener, app) + .await + .context("Failed to serve dashboard service")?; + + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use std::net::SocketAddr; + + #[tokio::test] + #[ignore] + async fn test_webhook_server() -> anyhow::Result<()> { + let addr = SocketAddr::from(([127, 0, 0, 1], 4560)); + let service = crate::webhook::WebhookService::new(addr); + service.serve().await?; + Ok(()) + } +} diff --git a/src/frontend/src/webhook/utils.rs b/src/frontend/src/webhook/utils.rs new file mode 100644 index 0000000000000..c368d534127d9 --- /dev/null +++ b/src/frontend/src/webhook/utils.rs @@ -0,0 +1,105 @@ +// Copyright 2024 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::collections::HashMap; + +use anyhow::anyhow; +use axum::http::{HeaderMap, StatusCode}; +use axum::response::IntoResponse; +use axum::Json; +use risingwave_common::row::OwnedRow; +use risingwave_common::types::JsonbVal; +use risingwave_pb::expr::ExprNode; +use serde_json::json; +use thiserror_ext::AsReport; + +use crate::expr::ExprImpl; + +pub struct WebhookError { + err: anyhow::Error, + code: StatusCode, +} + +pub(crate) type Result = std::result::Result; + +pub(crate) fn err(err: impl Into, code: StatusCode) -> WebhookError { + WebhookError { + err: err.into(), + code, + } +} + +impl From for WebhookError { + fn from(value: anyhow::Error) -> Self { + WebhookError { + err: value, + code: StatusCode::INTERNAL_SERVER_ERROR, + } + } +} + +impl IntoResponse for WebhookError { + fn into_response(self) -> axum::response::Response { + let mut resp = Json(json!({ + "error": self.err.to_report_string(), + })) + .into_response(); + *resp.status_mut() = self.code; + resp + } +} + +pub(crate) fn header_map_to_json(headers: &HeaderMap) -> JsonbVal { + let mut header_map = HashMap::new(); + + for (key, value) in headers { + let key = key.as_str().to_string(); + let value = value.to_str().unwrap_or("").to_string(); + header_map.insert(key, value); + } + + let json_value = json!(header_map); + JsonbVal::from(json_value) +} + +pub(crate) async fn verify_signature( + headers_jsonb: JsonbVal, + secret: &str, + payload: &[u8], + signature_expr: ExprNode, +) -> Result { + let row = OwnedRow::new(vec![ + Some(headers_jsonb.into()), + Some(secret.into()), + Some(payload.into()), + ]); + + let signature_expr_impl = ExprImpl::from_expr_proto(&signature_expr) + .map_err(|e| err(e, StatusCode::INTERNAL_SERVER_ERROR))?; + + let result = signature_expr_impl + .eval_row(&row) + .await + .map_err(|e| { + tracing::error!(error = %e.as_report(), "Fail to validate for webhook events."); + err(e, StatusCode::INTERNAL_SERVER_ERROR) + })? + .ok_or_else(|| { + err( + anyhow!("`SECURE_COMPARE()` failed"), + StatusCode::BAD_REQUEST, + ) + })?; + Ok(*result.as_bool()) +} diff --git a/src/meta/model/migration/src/lib.rs b/src/meta/model/migration/src/lib.rs index ce7c6d82d4d4c..baf4373343e12 100644 --- a/src/meta/model/migration/src/lib.rs +++ b/src/meta/model/migration/src/lib.rs @@ -22,6 +22,8 @@ mod m20240726_063833_auto_schema_change; mod m20240806_143329_add_rate_limit_to_source_catalog; mod m20240820_081248_add_time_travel_per_table_epoch; mod m20240911_083152_variable_vnode_count; +mod m20241001_013810_webhook_source; + mod m20241016_065621_hummock_gc_history; mod m20241022_072553_node_label; mod m20241025_062548_singleton_vnode_count; @@ -92,6 +94,7 @@ impl MigratorTrait for Migrator { Box::new(m20241115_085007_remove_function_type::Migration), Box::new(m20241120_182555_hummock_add_time_travel_sst_index::Migration), Box::new(m20241022_072553_node_label::Migration), + Box::new(m20241001_013810_webhook_source::Migration), ] } } diff --git a/src/meta/model/migration/src/m20241001_013810_webhook_source.rs b/src/meta/model/migration/src/m20241001_013810_webhook_source.rs new file mode 100644 index 0000000000000..b42f90e426a9c --- /dev/null +++ b/src/meta/model/migration/src/m20241001_013810_webhook_source.rs @@ -0,0 +1,37 @@ +use sea_orm_migration::prelude::{Table as MigrationTable, *}; + +use crate::utils::ColumnDefExt; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + MigrationTable::alter() + .table(Table::Table) + .add_column(ColumnDef::new(Table::WebhookInfo).rw_binary(manager)) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + MigrationTable::alter() + .table(Table::Table) + .drop_column(Table::WebhookInfo) + .to_owned(), + ) + .await + } +} + +#[derive(DeriveIden)] +enum Table { + Table, + WebhookInfo, +} diff --git a/src/meta/model/src/lib.rs b/src/meta/model/src/lib.rs index 6610484a89185..9995d8482b8fb 100644 --- a/src/meta/model/src/lib.rs +++ b/src/meta/model/src/lib.rs @@ -369,6 +369,10 @@ derive_array_from_blob!( PbColumnCatalogArray ); derive_from_blob!(StreamSourceInfo, risingwave_pb::catalog::PbStreamSourceInfo); +derive_from_blob!( + WebhookSourceInfo, + risingwave_pb::catalog::PbWebhookSourceInfo +); derive_from_blob!(WatermarkDesc, risingwave_pb::catalog::PbWatermarkDesc); derive_array_from_blob!( WatermarkDescArray, diff --git a/src/meta/model/src/table.rs b/src/meta/model/src/table.rs index b8ba38d438b7c..20fae0c926b1c 100644 --- a/src/meta/model/src/table.rs +++ b/src/meta/model/src/table.rs @@ -23,7 +23,7 @@ use serde::{Deserialize, Serialize}; use crate::{ Cardinality, ColumnCatalogArray, ColumnOrderArray, FragmentId, I32Array, ObjectId, SourceId, - TableId, TableVersion, + TableId, TableVersion, WebhookSourceInfo, }; #[derive( @@ -135,6 +135,7 @@ pub struct Model { pub incoming_sinks: I32Array, pub cdc_table_id: Option, pub vnode_count: i32, + pub webhook_info: Option, } #[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] @@ -267,6 +268,7 @@ impl From for ActiveModel { incoming_sinks: Set(pb_table.incoming_sinks.into()), cdc_table_id: Set(pb_table.cdc_table_id), vnode_count, + webhook_info: Set(pb_table.webhook_info.as_ref().map(WebhookSourceInfo::from)), } } } diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index a8397b4f03526..831a639eac416 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -202,6 +202,7 @@ impl From> for PbTable { retention_seconds: value.0.retention_seconds.map(|id| id as u32), cdc_table_id: value.0.cdc_table_id, maybe_vnode_count: VnodeCount::set(value.0.vnode_count).to_protobuf(), + webhook_info: value.0.webhook_info.map(|info| info.to_protobuf()), } } } diff --git a/src/prost/build.rs b/src/prost/build.rs index c4744e14c1b60..1dc7cd66c3456 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -116,6 +116,7 @@ fn main() -> Result<(), Box> { // The requirement is from Source node -> SourceCatalog -> WatermarkDesc -> expr .type_attribute("catalog.WatermarkDesc", "#[derive(Eq, Hash)]") .type_attribute("catalog.StreamSourceInfo", "#[derive(Eq, Hash)]") + .type_attribute("catalog.WebhookSourceInfo", "#[derive(Eq, Hash)]") .type_attribute("secret.SecretRef", "#[derive(Eq, Hash)]") .type_attribute("catalog.IndexColumnProperties", "#[derive(Eq, Hash)]") .type_attribute("expr.ExprNode", "#[derive(Eq, Hash)]") diff --git a/src/sqlparser/src/ast/ddl.rs b/src/sqlparser/src/ast/ddl.rs index 35eac31cefa72..7d3d07a9f8d5e 100644 --- a/src/sqlparser/src/ast/ddl.rs +++ b/src/sqlparser/src/ast/ddl.rs @@ -22,7 +22,8 @@ use serde::{Deserialize, Serialize}; use super::{FormatEncodeOptions, Value}; use crate::ast::{ - display_comma_separated, display_separated, DataType, Expr, Ident, ObjectName, SetVariableValue, + display_comma_separated, display_separated, DataType, Expr, Ident, ObjectName, SecretRef, + SetVariableValue, }; use crate::tokenizer::Token; @@ -818,3 +819,11 @@ impl fmt::Display for ReferentialAction { }) } } + +/// secure secret definition for webhook source +#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))] +pub struct WebhookSourceInfo { + pub secret_ref: SecretRef, + pub signature_expr: Expr, +} diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 9007613f30cee..510df40865f6f 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -39,7 +39,7 @@ pub use self::data_type::{DataType, StructField}; pub use self::ddl::{ AlterColumnOperation, AlterConnectionOperation, AlterDatabaseOperation, AlterFunctionOperation, AlterSchemaOperation, AlterSecretOperation, AlterTableOperation, ColumnDef, ColumnOption, - ColumnOptionDef, ReferentialAction, SourceWatermark, TableConstraint, + ColumnOptionDef, ReferentialAction, SourceWatermark, TableConstraint, WebhookSourceInfo, }; pub use self::legacy_source::{ get_delimiter, AvroSchema, CompatibleFormatEncode, DebeziumAvroSchema, ProtobufSchema, @@ -1308,6 +1308,8 @@ pub enum Statement { cdc_table_info: Option, /// `INCLUDE a AS b INCLUDE c` include_column_options: IncludeOption, + /// `VALIDATE SECRET secure_secret_name AS secure_compare ()` + webhook_info: Option, }, /// CREATE INDEX CreateIndex { @@ -1848,6 +1850,7 @@ impl fmt::Display for Statement { query, cdc_table_info, include_column_options, + webhook_info, } => { // We want to allow the following options // Empty column list, allowed by PostgreSQL: @@ -1897,6 +1900,10 @@ impl fmt::Display for Statement { write!(f, " FROM {}", info.source_name)?; write!(f, " TABLE '{}'", info.external_table_name)?; } + if let Some(info)= webhook_info { + write!(f, " VALIDATE SECRET {}", info.secret_ref.secret_name)?; + write!(f, " AS {}", info.signature_expr)?; + } Ok(()) } Statement::CreateIndex { diff --git a/src/sqlparser/src/ast/statement.rs b/src/sqlparser/src/ast/statement.rs index 72680161defea..d817c8c2759ae 100644 --- a/src/sqlparser/src/ast/statement.rs +++ b/src/sqlparser/src/ast/statement.rs @@ -297,6 +297,11 @@ impl Parser<'_> { } } Ok(expected.into()) + } else if connector.contains("webhook") { + parser_err!( + "Source with webhook connector is not supported. \ + Please use the `CREATE TABLE ... WITH ...` statement instead.", + ); } else { Ok(parse_format_encode(self)?) } diff --git a/src/sqlparser/src/keywords.rs b/src/sqlparser/src/keywords.rs index ccc0ef23502bd..83e7d31debf07 100644 --- a/src/sqlparser/src/keywords.rs +++ b/src/sqlparser/src/keywords.rs @@ -553,6 +553,7 @@ define_keywords!( USER, USING, UUID, + VALIDATE, VALUE, VALUES, VALUE_OF, diff --git a/src/sqlparser/src/parser.rs b/src/sqlparser/src/parser.rs index 81dbc54e84c80..09f6393eff19d 100644 --- a/src/sqlparser/src/parser.rs +++ b/src/sqlparser/src/parser.rs @@ -22,6 +22,7 @@ use alloc::{ }; use core::fmt; +use ddl::WebhookSourceInfo; use itertools::Itertools; use tracing::{debug, instrument}; use winnow::combinator::{alt, cut_err, dispatch, fail, opt, peek, preceded, repeat, separated}; @@ -37,6 +38,7 @@ use crate::tokenizer::*; use crate::{impl_parse_to, parser_v2}; pub(crate) const UPSTREAM_SOURCE_KEY: &str = "connector"; +pub(crate) const WEBHOOK_CONNECTOR: &str = "webhook"; #[derive(Debug, Clone, PartialEq)] pub enum ParserError { @@ -2556,8 +2558,13 @@ impl Parser<'_> { .iter() .find(|&opt| opt.name.real_value() == UPSTREAM_SOURCE_KEY); let connector = option.map(|opt| opt.value.to_string()); + let contain_webhook = + connector.is_some() && connector.as_ref().unwrap().contains(WEBHOOK_CONNECTOR); - let format_encode = if let Some(connector) = connector { + // webhook connector does not require row format + let format_encode = if let Some(connector) = connector + && !contain_webhook + { Some(self.parse_format_encode_with_connector(&connector, false)?) } else { None // Table is NOT created with an external connector. @@ -2584,6 +2591,28 @@ impl Parser<'_> { None }; + let webhook_info = if self.parse_keyword(Keyword::VALIDATE) { + if !contain_webhook { + parser_err!("VALIDATE is only supported for tables created with webhook source"); + } + + self.expect_keyword(Keyword::SECRET)?; + let secret_ref = self.parse_secret_ref()?; + if secret_ref.ref_as == SecretRefAsType::File { + parser_err!("Secret for SECURE_COMPARE() does not support AS FILE"); + }; + + self.expect_keyword(Keyword::AS)?; + let signature_expr = self.parse_function()?; + + Some(WebhookSourceInfo { + secret_ref, + signature_expr, + }) + } else { + None + }; + Ok(Statement::CreateTable { name: table_name, temporary, @@ -2601,6 +2630,7 @@ impl Parser<'_> { query, cdc_table_info, include_column_options: include_options, + webhook_info, }) } @@ -3622,16 +3652,8 @@ impl Parser<'_> { _ => self.expected_at(checkpoint, "A value")?, }, Keyword::SECRET => { - let secret_name = self.parse_object_name()?; - let ref_as = if self.parse_keywords(&[Keyword::AS, Keyword::FILE]) { - SecretRefAsType::File - } else { - SecretRefAsType::Text - }; - Ok(Value::Ref(SecretRef { - secret_name, - ref_as, - })) + let secret = self.parse_secret_ref()?; + Ok(Value::Ref(secret)) } _ => self.expected_at(checkpoint, "a concrete value"), }, @@ -3645,6 +3667,19 @@ impl Parser<'_> { } } + fn parse_secret_ref(&mut self) -> PResult { + let secret_name = self.parse_object_name()?; + let ref_as = if self.parse_keywords(&[Keyword::AS, Keyword::FILE]) { + SecretRefAsType::File + } else { + SecretRefAsType::Text + }; + Ok(SecretRef { + secret_name, + ref_as, + }) + } + fn parse_set_variable(&mut self) -> PResult { alt(( Keyword::DEFAULT.value(SetVariableValue::Default), @@ -5079,7 +5114,6 @@ impl Parser<'_> { let source = Box::new(self.parse_query()?); let returning = self.parse_returning(Optional)?; - Ok(Statement::Insert { table_name, columns, diff --git a/src/sqlparser/tests/testdata/array.yaml b/src/sqlparser/tests/testdata/array.yaml index b2cf8e2070b11..7262b7b0e645e 100644 --- a/src/sqlparser/tests/testdata/array.yaml +++ b/src/sqlparser/tests/testdata/array.yaml @@ -76,4 +76,4 @@ ^ - input: CREATE TABLE t (params STRUCT>[]) formatted_sql: CREATE TABLE t (params STRUCT>[]) - formatted_ast: 'CreateTable { or_replace: false, temporary: false, if_not_exists: false, name: ObjectName([Ident { value: "t", quote_style: None }]), columns: [ColumnDef { name: Ident { value: "params", quote_style: None }, data_type: Some(Array(Struct([StructField { name: Ident { value: "a", quote_style: None }, data_type: Struct([StructField { name: Ident { value: "b", quote_style: None }, data_type: Int }]) }]))), collation: None, options: [] }], wildcard_idx: None, constraints: [], with_options: [], format_encode: None, source_watermarks: [], append_only: false, on_conflict: None, with_version_column: None, query: None, cdc_table_info: None, include_column_options: [] }' + formatted_ast: 'CreateTable { or_replace: false, temporary: false, if_not_exists: false, name: ObjectName([Ident { value: "t", quote_style: None }]), columns: [ColumnDef { name: Ident { value: "params", quote_style: None }, data_type: Some(Array(Struct([StructField { name: Ident { value: "a", quote_style: None }, data_type: Struct([StructField { name: Ident { value: "b", quote_style: None }, data_type: Int }]) }]))), collation: None, options: [] }], wildcard_idx: None, constraints: [], with_options: [], format_encode: None, source_watermarks: [], append_only: false, on_conflict: None, with_version_column: None, query: None, cdc_table_info: None, include_column_options: [], webhook_info: None }' diff --git a/src/storage/src/compaction_catalog_manager.rs b/src/storage/src/compaction_catalog_manager.rs index 3133cae023300..22ed5cf4fd0b5 100644 --- a/src/storage/src/compaction_catalog_manager.rs +++ b/src/storage/src/compaction_catalog_manager.rs @@ -568,6 +568,7 @@ mod tests { created_at_cluster_version: None, cdc_table_id: None, maybe_vnode_count: None, + webhook_info: None, } } diff --git a/src/tests/sqlsmith/src/lib.rs b/src/tests/sqlsmith/src/lib.rs index cc16414f73cef..1f0f11815aa8d 100644 --- a/src/tests/sqlsmith/src/lib.rs +++ b/src/tests/sqlsmith/src/lib.rs @@ -284,6 +284,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); query: None, cdc_table_info: None, include_column_options: [], + webhook_info: None, }, CreateTable { or_replace: false, @@ -332,6 +333,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); query: None, cdc_table_info: None, include_column_options: [], + webhook_info: None, }, CreateTable { or_replace: false, @@ -391,6 +393,7 @@ CREATE TABLE t3(v1 int, v2 bool, v3 smallint); query: None, cdc_table_info: None, include_column_options: [], + webhook_info: None, }, ], )"#]], @@ -526,6 +529,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY query: None, cdc_table_info: None, include_column_options: [], + webhook_info: None, }, CreateTable { or_replace: false, @@ -581,6 +585,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY query: None, cdc_table_info: None, include_column_options: [], + webhook_info: None, }, CreateTable { or_replace: false, @@ -643,6 +648,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY query: None, cdc_table_info: None, include_column_options: [], + webhook_info: None, }, CreateTable { or_replace: false, @@ -723,6 +729,7 @@ CREATE TABLE t4(v1 int PRIMARY KEY, v2 smallint PRIMARY KEY, v3 bool PRIMARY KEY query: None, cdc_table_info: None, include_column_options: [], + webhook_info: None, }, ], )"#]], From 521f67493e192c6c11e6eb1d241ac7b3b140a862 Mon Sep 17 00:00:00 2001 From: Bugen Zhao Date: Thu, 28 Nov 2024 13:55:52 +0800 Subject: [PATCH 038/163] refactor(ci): introduce meta backend agnostic e2e tests (#19568) Signed-off-by: Bugen Zhao --- ci/scripts/run-e2e-test.sh | 7 -- ci/workflows/main-cron.yml | 83 ++++++------ ci/workflows/pull-request.yml | 53 ++++---- risedev.yml | 119 +----------------- src/risedevtool/src/bin/risedev-dev.rs | 2 +- src/risedevtool/src/task/meta_node_service.rs | 4 + 6 files changed, 75 insertions(+), 193 deletions(-) diff --git a/ci/scripts/run-e2e-test.sh b/ci/scripts/run-e2e-test.sh index e84ead4a81df3..5a450f20b6edc 100755 --- a/ci/scripts/run-e2e-test.sh +++ b/ci/scripts/run-e2e-test.sh @@ -48,13 +48,6 @@ cluster_start() { # Give it a while to make sure the single-node is ready. sleep 10 else - # Initialize backends. - if [[ $mode == *"mysql-backend" ]]; then - mysql -h mysql -P 3306 -u root -p123456 -e "DROP DATABASE IF EXISTS metadata; CREATE DATABASE metadata;" - elif [[ $mode == *"pg-backend" ]]; then - PGPASSWORD=postgres psql -h db -p 5432 -U postgres -c "DROP DATABASE IF EXISTS metadata;" -c "CREATE DATABASE metadata;" - fi - risedev ci-start "$mode" fi } diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 9a1cf0b3b5b95..3a371afa4641a 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -4,6 +4,37 @@ auto-retry: &auto-retry - signal_reason: agent_stop limit: 3 +sql-backend: &sql-backend + matrix: + setup: + backend: [""] + endpoint: [""] + adjustments: + - with: + backend: "" + endpoint: "" + skip: true # hack + - with: + backend: "sqlite" + # sqlite3 /tmp/rwmeta.db + endpoint: "sqlite:///tmp/rwmeta.db?mode=rwc" + - with: + backend: "postgres" + # PGPASSWORD=postgres psql -h db -p 5432 -U postgres -d rwmeta + endpoint: "postgres://postgres:postgres@db:5432/rwmeta" + - with: + backend: "mysql" + # mysql -h mysql -P 3306 -u root -p123456 -D rwmeta + endpoint: "mysql://root:123456@mysql:3306/rwmeta" + env: + RISEDEV_SQL_ENDPOINT: "{{matrix.endpoint}}" + +docker-compose-common: &docker-compose-common + config: ci/docker-compose.yml + mount-buildkite-agent: true + propagate-environment: true + run: rw-build-env + steps: - label: "build" command: "ci/scripts/build.sh -p ci-release" @@ -62,8 +93,9 @@ steps: timeout_in_minutes: 10 retry: *auto-retry - - label: "end-to-end test (release)" + - label: "end-to-end test (release, {{matrix.backend}} backend)" key: "e2e-test-release" + <<: *sql-backend command: "ci/scripts/cron-e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe" if: | !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null @@ -75,11 +107,10 @@ steps: - "docslt" plugins: - docker-compose#v5.1.0: - run: rw-build-env - config: ci/docker-compose.yml - mount-buildkite-agent: true + <<: *docker-compose-common + run: pg-mysql-backend-test-env - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 25 + timeout_in_minutes: 30 retry: *auto-retry - label: "slow end-to-end test (release)" @@ -716,55 +747,15 @@ steps: - "build" - "build-other" - "docslt" - plugins: - - docker-compose#v4.9.0: - run: rw-build-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 25 - retry: *auto-retry - - - label: "end-to-end test (postgres backend)" - key: e2e-test-postgres-backend - command: "ci/scripts/e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe-pg-backend" - if: | - !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-e2e-test-other-backends" - || build.env("CI_STEPS") =~ /(^|,)e2e-test-other-backends?(,|$$)/ - depends_on: - - "build" - - "build-other" - - "docslt" plugins: - docker-compose#v5.1.0: - run: pg-mysql-backend-test-env + run: rw-build-env config: ci/docker-compose.yml mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs timeout_in_minutes: 25 retry: *auto-retry - - label: "end-to-end test (mysql backend)" - key: e2e-test-mysql-backend - command: "ci/scripts/e2e-test.sh -p ci-release -m ci-3streaming-2serving-3fe-mysql-backend" - if: | - !(build.pull_request.labels includes "ci/main-cron/run-selected") && build.env("CI_STEPS") == null - || build.pull_request.labels includes "ci/run-e2e-test-other-backends" - || build.env("CI_STEPS") =~ /(^|,)e2e-test-other-backends?(,|$$)/ - depends_on: - - "build" - - "build-other" - - "docslt" - plugins: - - docker-compose#v5.1.0: - run: pg-mysql-backend-test-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 30 - retry: *auto-retry - - label: "end-to-end test for opendal (parallel)" key: "e2e-test-opendal-parallel" command: "ci/scripts/e2e-test-parallel-for-opendal.sh -p ci-release" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 45502ab3037c0..e1c7afa7986e4 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -13,12 +13,39 @@ cargo-cache: &cargo-cache backend: s3 s3: bucket: rw-ci-cache-bucket - args: '--no-progress' + args: "--no-progress" paths: - ".cargo/registry/index" - ".cargo/registry/cache" - ".cargo/git" +other-sql-backend: &other-sql-backend + matrix: + setup: + label: [""] + endpoint: [""] + adjustments: + - with: + label: "" + endpoint: "" + skip: true # hack + - with: + label: "postgres" + # PGPASSWORD=postgres psql -h db -p 5432 -U postgres -d rwmeta + endpoint: "postgres://postgres:postgres@db:5432/rwmeta" + - with: + label: "mysql" + # mysql -h mysql -P 3306 -u root -p123456 -D rwmeta + endpoint: "mysql://root:123456@mysql:3306/rwmeta" + env: + RISEDEV_SQL_ENDPOINT: "{{matrix.endpoint}}" + +docker-compose-common: &docker-compose-common + config: ci/docker-compose.yml + mount-buildkite-agent: true + propagate-environment: true + run: rw-build-env + steps: - label: "check ci image rebuild" plugins: @@ -789,24 +816,9 @@ steps: timeout_in_minutes: 30 retry: *auto-retry - - label: "end-to-end test (postgres backend)" - command: "ci/scripts/e2e-test.sh -p ci-dev -m ci-3streaming-2serving-3fe-pg-backend" - if: build.pull_request.labels includes "ci/run-e2e-test-other-backends" || build.env("CI_STEPS") =~ /(^|,)e2e-test-other-backends?(,|$$)/ - depends_on: - - "build" - - "build-other" - - "docslt" - plugins: - - docker-compose#v5.1.0: - run: pg-mysql-backend-test-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 30 - retry: *auto-retry - - - label: "end-to-end test (mysql backend)" - command: "ci/scripts/e2e-test.sh -p ci-dev -m ci-3streaming-2serving-3fe-mysql-backend" + - label: "end-to-end test ({{matrix.label}} backend)" + <<: *other-sql-backend + command: "ci/scripts/e2e-test.sh -p ci-dev -m ci-3streaming-2serving-3fe" if: build.pull_request.labels includes "ci/run-e2e-test-other-backends" || build.env("CI_STEPS") =~ /(^|,)e2e-test-other-backends?(,|$$)/ depends_on: - "build" @@ -814,9 +826,8 @@ steps: - "docslt" plugins: - docker-compose#v5.1.0: + <<: *docker-compose-common run: pg-mysql-backend-test-env - config: ci/docker-compose.yml - mount-buildkite-agent: true - ./ci/plugins/upload-failure-logs timeout_in_minutes: 32 retry: *auto-retry diff --git a/risedev.yml b/risedev.yml index 5f7e43d1b75d1..b51b0c768d333 100644 --- a/risedev.yml +++ b/risedev.yml @@ -777,125 +777,8 @@ profile: config-path: src/config/ci.toml steps: - use: minio - - use: sqlite - - use: meta-node - meta-backend: sqlite - - use: compute-node - port: 5687 - exporter-port: 1222 - enable-tiered-cache: true - role: streaming - parallelism: 4 - - use: compute-node - port: 5688 - exporter-port: 1223 - enable-tiered-cache: true - role: streaming - parallelism: 4 - - use: compute-node - port: 5689 - exporter-port: 1224 - enable-tiered-cache: true - role: streaming - parallelism: 4 - - use: compute-node - port: 5685 - exporter-port: 1225 - enable-tiered-cache: true - role: serving - parallelism: 4 - - use: compute-node - port: 5686 - exporter-port: 1226 - enable-tiered-cache: true - role: serving - parallelism: 8 - - use: frontend - port: 4565 - exporter-port: 2222 - health-check-port: 6786 - - use: frontend - port: 4566 - exporter-port: 2223 - health-check-port: 6787 - - use: frontend - port: 4567 - exporter-port: 2224 - health-check-port: 6788 - - use: compactor - - ci-3streaming-2serving-3fe-pg-backend: - config-path: src/config/ci.toml - steps: - - use: minio - - use: postgres - port: 5432 - address: db - database: metadata - user: postgres - password: postgres - user-managed: true - application: metastore - use: meta-node - meta-backend: postgres - - use: compute-node - port: 5687 - exporter-port: 1222 - enable-tiered-cache: true - role: streaming - parallelism: 4 - - use: compute-node - port: 5688 - exporter-port: 1223 - enable-tiered-cache: true - role: streaming - parallelism: 4 - - use: compute-node - port: 5689 - exporter-port: 1224 - enable-tiered-cache: true - role: streaming - parallelism: 4 - - use: compute-node - port: 5685 - exporter-port: 1225 - enable-tiered-cache: true - role: serving - parallelism: 4 - - use: compute-node - port: 5686 - exporter-port: 1226 - enable-tiered-cache: true - role: serving - parallelism: 8 - - use: frontend - port: 4565 - exporter-port: 2222 - health-check-port: 6786 - - use: frontend - port: 4566 - exporter-port: 2223 - health-check-port: 6787 - - use: frontend - port: 4567 - exporter-port: 2224 - health-check-port: 6788 - - use: compactor - - ci-3streaming-2serving-3fe-mysql-backend: - config-path: src/config/ci.toml - steps: - - use: minio - - use: mysql - port: 3306 - address: mysql - database: metadata - user: root - password: 123456 - user-managed: true - application: metastore - - use: meta-node - meta-backend: mysql + meta-backend: env - use: compute-node port: 5687 exporter-port: 1222 diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index 8bf9aa52fe8fc..41e3c3811125d 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -384,7 +384,7 @@ fn main() -> Result<()> { tracing_subscriber::fmt() .with_env_filter( EnvFilter::builder() - .with_default_directive(LevelFilter::WARN.into()) + .with_default_directive(LevelFilter::INFO.into()) .with_env_var("RISEDEV_RUST_LOG") .from_env_lossy(), ) diff --git a/src/risedevtool/src/task/meta_node_service.rs b/src/risedevtool/src/task/meta_node_service.rs index 752dd87cd0cda..721f33573fa7d 100644 --- a/src/risedevtool/src/task/meta_node_service.rs +++ b/src/risedevtool/src/task/meta_node_service.rs @@ -42,6 +42,10 @@ use crate::{ fn sql_endpoint_from_env() -> String { static SQL_ENDPOINT: LazyLock = LazyLock::new(|| { if let Ok(endpoint) = env::var("RISEDEV_SQL_ENDPOINT") { + tracing::info!( + "sql endpoint from env RISEDEV_SQL_ENDPOINT resolved to `{}`", + endpoint + ); endpoint } else { let temp_path = NamedTempFile::with_suffix(".db").unwrap().into_temp_path(); From b8c70c156bb3f224293edd273a03849bb6468834 Mon Sep 17 00:00:00 2001 From: Lanqing Yang Date: Thu, 28 Nov 2024 02:01:30 -0800 Subject: [PATCH 039/163] feat(frontend): support dot output for distsql (#19552) Co-authored-by: Noel Kwan <47273164+kwannoel@users.noreply.github.com> --- e2e_test/batch/explain.slt | 12 +++ .../testdata/input/explain_dot_format.yaml | 6 ++ .../testdata/output/explain_dot_format.yaml | 29 +++++++ src/frontend/src/handler/explain.rs | 26 ++++++- src/frontend/src/optimizer/plan_node/mod.rs | 43 +---------- src/frontend/src/scheduler/plan_fragmenter.rs | 29 +++++++ .../src/utils/stream_graph_formatter.rs | 77 +++++++++++++++++++ 7 files changed, 177 insertions(+), 45 deletions(-) diff --git a/e2e_test/batch/explain.slt b/e2e_test/batch/explain.slt index 1f9cebdf1b68b..f050c6cea5133 100644 --- a/e2e_test/batch/explain.slt +++ b/e2e_test/batch/explain.slt @@ -7,6 +7,18 @@ explain create index i on t(v); statement ok explain create sink sink_t from t with ( connector = 'kafka', type = 'append-only' ) +# statement ok +# set batch_parallelism=1; + +# query I +# explain (distsql, format dot) SELECT approx_percentile(0.5) WITHIN GROUP (order by v) from t; +# ---- +# digraph { +# 0 [ label = "Stage 0: QueryStage { id: 0, parallelism: Some(1), exchange_info: Some(ExchangeInfo { mode: Single, distribution: None }), has_table_scan: false }" ] +# 1 [ label = "Stage 1: QueryStage { id: 1, parallelism: Some(4), exchange_info: Some(ExchangeInfo { mode: Single, distribution: None }), has_table_scan: true }" ] +# 0 -> 1 [ label = "" ] +# } + statement ok drop table t; diff --git a/src/frontend/planner_test/tests/testdata/input/explain_dot_format.yaml b/src/frontend/planner_test/tests/testdata/input/explain_dot_format.yaml index 377f395b73b8d..ce0134d364d70 100644 --- a/src/frontend/planner_test/tests/testdata/input/explain_dot_format.yaml +++ b/src/frontend/planner_test/tests/testdata/input/explain_dot_format.yaml @@ -40,5 +40,11 @@ COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col17, COALESCE((SELECT b FROM t2 WHERE t1.a = t2.c), 0) col18 from t1; + expected_outputs: + - explain_output +- name: test dot output format (distsql, stream) + sql: | + CREATE TABLE t (v1 int); + explain (distsql, format dot) create materialized view m1 as SELECT approx_percentile(0.5) WITHIN GROUP (order by v1) from t; expected_outputs: - explain_output \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/explain_dot_format.yaml b/src/frontend/planner_test/tests/testdata/output/explain_dot_format.yaml index 2fce651976e82..6d53415a6d2d2 100644 --- a/src/frontend/planner_test/tests/testdata/output/explain_dot_format.yaml +++ b/src/frontend/planner_test/tests/testdata/output/explain_dot_format.yaml @@ -437,3 +437,32 @@ 176 -> 182 [ ] 182 -> 183 [ ] } +- name: test dot output format (distsql, stream) + sql: | + CREATE TABLE t (v1 int); + explain (distsql, format dot) create materialized view m1 as SELECT approx_percentile(0.5) WITHIN GROUP (order by v1) from t; + explain_output: | + digraph { + 0 [ label = "Fragment 0\l" ] + 1 [ label = "StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck }\ltables: [\"Materialize: 4294967294\"]\l" ] + 2 [ label = "StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 }\ltables: [\"GlobalApproxPercentileBucketState: 0\",\"GlobalApproxPercentileCountState: 1\"]\l" ] + 3 [ label = "StreamExchange Single from 1" ] + 4 [ label = "Fragment 1\l" ] + 5 [ label = "StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 }" ] + 6 [ label = "StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] }" ] + 7 [ label = "StreamTableScan { table: t, columns: [v1, _row_id] }\ltables: [\"StreamScan: 2\"]\l" ] + 8 [ label = "Upstream" ] + 9 [ label = "BatchPlanNode" ] + 10 [ label = "Table 0\lcolumns: [\"sign\",\"bucket_id\",\"count\",\"_rw_timestamp\"]\lprimary key: [\"$0 ASC\",\"$1 ASC\"]\lvalue indices: [\"0\",\"1\",\"2\"]\ldistribution key: []\lread pk prefix len hint: \"0\"\l" ] + 11 [ label = "Table 1\lcolumns: [\"total_count\",\"_rw_timestamp\"]\lprimary key: []\lvalue indices: [\"0\"]\ldistribution key: []\lread pk prefix len hint: \"0\"\l" ] + 12 [ label = "Table 2\lcolumns: [\"vnode\",\"_row_id\",\"backfill_finished\",\"row_count\",\"_rw_timestamp\"]\lprimary key: [\"$0 ASC\"]\lvalue indices: [\"1\",\"2\",\"3\"]\ldistribution key: [\"0\"]\lread pk prefix len hint: \"1\"\lvnode column idx: \"0\"\l" ] + 13 [ label = "Table 4294967294\lcolumns: [\"approx_percentile\",\"_rw_timestamp\"]\lprimary key: []\lvalue indices: [\"0\"]\ldistribution key: []\lread pk prefix len hint: \"0\"\l" ] + 0 -> 1 [ label = "" ] + 1 -> 2 [ label = "" ] + 2 -> 3 [ label = "" ] + 4 -> 5 [ label = "" ] + 5 -> 6 [ label = "" ] + 6 -> 7 [ label = "" ] + 7 -> 8 [ label = "" ] + 7 -> 9 [ label = "" ] + } diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index 66003051a70b9..ffcb170373dd7 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use petgraph::dot::Dot; use pgwire::pg_response::{PgResponse, StatementType}; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; use risingwave_common::bail_not_implemented; @@ -35,7 +36,7 @@ use crate::optimizer::plan_node::{Convention, Explain}; use crate::optimizer::OptimizerContext; use crate::scheduler::BatchPlanFragmenter; use crate::stream_fragmenter::build_graph; -use crate::utils::explain_stream_graph; +use crate::utils::{explain_stream_graph, explain_stream_graph_as_dot}; use crate::OptimizerContextRef; async fn do_handle_explain( @@ -46,6 +47,8 @@ async fn do_handle_explain( ) -> Result<()> { // Workaround to avoid `Rc` across `await` point. let mut batch_plan_fragmenter = None; + let mut batch_plan_fragmenter_fmt = ExplainFormat::Json; + let session = handler_args.session.clone(); { @@ -218,10 +221,19 @@ async fn do_handle_explain( session.config().batch_parallelism().0, plan.clone(), )?); + batch_plan_fragmenter_fmt = if explain_format == ExplainFormat::Dot { + ExplainFormat::Dot + } else { + ExplainFormat::Json + } } Convention::Stream => { let graph = build_graph(plan.clone())?; - blocks.push(explain_stream_graph(&graph, explain_verbose)); + if explain_format == ExplainFormat::Dot { + blocks.push(explain_stream_graph_as_dot(&graph, explain_verbose)) + } else { + blocks.push(explain_stream_graph(&graph, explain_verbose)); + } } } } @@ -255,8 +267,14 @@ async fn do_handle_explain( if let Some(fragmenter) = batch_plan_fragmenter { let query = fragmenter.generate_complete_query().await?; - let stage_graph_json = serde_json::to_string_pretty(&query.stage_graph).unwrap(); - blocks.push(stage_graph_json); + let stage_graph = if batch_plan_fragmenter_fmt == ExplainFormat::Dot { + let graph = query.stage_graph.to_petgraph(); + let dot = Dot::new(&graph); + dot.to_string() + } else { + serde_json::to_string_pretty(&query.stage_graph).unwrap() + }; + blocks.push(stage_graph); } Ok(()) diff --git a/src/frontend/src/optimizer/plan_node/mod.rs b/src/frontend/src/optimizer/plan_node/mod.rs index 165f867b3c76e..5e7e76500e9a0 100644 --- a/src/frontend/src/optimizer/plan_node/mod.rs +++ b/src/frontend/src/optimizer/plan_node/mod.rs @@ -39,7 +39,7 @@ use fixedbitset::FixedBitSet; use itertools::Itertools; use paste::paste; use petgraph::dot::{Config, Dot}; -use petgraph::graph::{Graph, NodeIndex}; +use petgraph::graph::Graph; use pretty_xmlish::{Pretty, PrettyConfig}; use risingwave_common::catalog::Schema; use risingwave_common::util::recursive::{self, Recurse}; @@ -56,7 +56,7 @@ use super::property::{Distribution, FunctionalDependencySet, MonotonicityMap, Or use crate::error::{ErrorCode, Result}; use crate::optimizer::ExpressionSimplifyRewriter; use crate::session::current::notice_to_user; -use crate::utils::PrettySerde; +use crate::utils::{build_graph_from_pretty, PrettySerde}; /// A marker trait for different conventions, used for enforcing type safety. /// @@ -736,45 +736,6 @@ impl Explain for PlanRef { } } -fn build_graph_from_pretty( - pretty: &Pretty<'_>, - graph: &mut Graph, - nodes: &mut HashMap, - parent_label: Option<&str>, -) { - if let Pretty::Record(r) = pretty { - let mut label = String::new(); - label.push_str(&r.name); - for (k, v) in &r.fields { - label.push('\n'); - label.push_str(k); - label.push_str(": "); - label.push_str( - &serde_json::to_string(&PrettySerde(v.clone(), false)) - .expect("failed to serialize plan to dot"), - ); - } - // output alignment. - if !r.fields.is_empty() { - label.push('\n'); - } - - let current_node = *nodes - .entry(label.clone()) - .or_insert_with(|| graph.add_node(label.clone())); - - if let Some(parent_label) = parent_label { - if let Some(&parent_node) = nodes.get(parent_label) { - graph.add_edge(parent_node, current_node, "contains".to_string()); - } - } - - for child in &r.children { - build_graph_from_pretty(child, graph, nodes, Some(&label)); - } - } -} - pub(crate) fn pretty_config() -> PrettyConfig { PrettyConfig { indent: 3, diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 9cec27601a246..6a4682367ff0b 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -24,6 +24,7 @@ use enum_as_inner::EnumAsInner; use futures::TryStreamExt; use iceberg::expr::Predicate as IcebergPredicate; use itertools::Itertools; +use petgraph::{Directed, Graph}; use pgwire::pg_server::SessionId; use risingwave_batch::error::BatchError; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeSelector; @@ -839,6 +840,34 @@ impl StageGraph { Ok(()) } + + /// Converts the `StageGraph` into a `petgraph::graph::Graph`. + pub fn to_petgraph(&self) -> Graph { + let mut graph = Graph::::new(); + + let mut node_indices = HashMap::new(); + + // Add all stages as nodes + for (&stage_id, stage_ref) in self.stages.iter().sorted_by_key(|(id, _)| **id) { + let node_label = format!("Stage {}: {:?}", stage_id, stage_ref); + let node_index = graph.add_node(node_label); + node_indices.insert(stage_id, node_index); + } + + // Add edges between stages based on child_edges + for (&parent_id, children) in &self.child_edges { + if let Some(&parent_index) = node_indices.get(&parent_id) { + for &child_id in children { + if let Some(&child_index) = node_indices.get(&child_id) { + // Add an edge from parent to child + graph.add_edge(parent_index, child_index, "".to_string()); + } + } + } + } + + graph + } } struct StageGraphBuilder { diff --git a/src/frontend/src/utils/stream_graph_formatter.rs b/src/frontend/src/utils/stream_graph_formatter.rs index effe7e69a3c05..888f66d7889c9 100644 --- a/src/frontend/src/utils/stream_graph_formatter.rs +++ b/src/frontend/src/utils/stream_graph_formatter.rs @@ -16,12 +16,16 @@ use std::cmp::max; use std::collections::{BTreeMap, HashMap}; use itertools::Itertools; +use petgraph::dot::Dot; +use petgraph::graph::NodeIndex; +use petgraph::Graph; use pretty_xmlish::{Pretty, PrettyConfig}; use risingwave_common::util::stream_graph_visitor; use risingwave_pb::catalog::Table; use risingwave_pb::stream_plan::stream_fragment_graph::StreamFragmentEdge; use risingwave_pb::stream_plan::{stream_node, DispatcherType, StreamFragmentGraph, StreamNode}; +use super::PrettySerde; use crate::TableCatalog; /// ice: in the future, we may allow configurable width, boundaries, etc. @@ -36,6 +40,12 @@ pub fn explain_stream_graph(graph: &StreamFragmentGraph, is_verbose: bool) -> St output } +pub fn explain_stream_graph_as_dot(sg: &StreamFragmentGraph, is_verbose: bool) -> String { + let graph = StreamGraphFormatter::new(is_verbose).explain_graph_as_dot(sg); + let dot = Dot::new(&graph); + dot.to_string() +} + /// A formatter to display the final stream plan graph, used for `explain (distsql) create /// materialized view ...` struct StreamGraphFormatter { @@ -88,6 +98,34 @@ impl StreamGraphFormatter { } } + fn explain_graph_as_dot(&mut self, graph: &StreamFragmentGraph) -> Graph { + self.edges.clear(); + for edge in &graph.edges { + self.edges.insert(edge.link_id, edge.clone()); + } + + let mut g = Graph::::new(); + let mut nodes = HashMap::new(); + for (_, fragment) in graph.fragments.iter().sorted_by_key(|(id, _)| **id) { + let mut label = String::new(); + label.push_str("Fragment "); + label.push_str(&fragment.get_fragment_id().to_string()); + label.push('\n'); + nodes.insert(label.clone(), g.add_node(label.clone())); + + build_graph_from_pretty( + &self.explain_node(fragment.node.as_ref().unwrap()), + &mut g, + &mut nodes, + Some(&label), + ); + } + for tb in self.tables.values() { + build_graph_from_pretty(&self.explain_table(tb), &mut g, &mut nodes, None); + } + g + } + fn explain_table<'a>(&self, tb: &Table) -> Pretty<'a> { let tb = TableCatalog::from(tb.clone()); let columns = tb @@ -201,3 +239,42 @@ impl StreamGraphFormatter { Pretty::simple_record(one_line_explain, fields, children) } } + +pub fn build_graph_from_pretty( + pretty: &Pretty<'_>, + graph: &mut Graph, + nodes: &mut HashMap, + parent_label: Option<&str>, +) { + if let Pretty::Record(r) = pretty { + let mut label = String::new(); + label.push_str(&r.name); + for (k, v) in &r.fields { + label.push('\n'); + label.push_str(k); + label.push_str(": "); + label.push_str( + &serde_json::to_string(&PrettySerde(v.clone(), false)) + .expect("failed to serialize plan to dot"), + ); + } + // output alignment. + if !r.fields.is_empty() { + label.push('\n'); + } + + let current_node = *nodes + .entry(label.clone()) + .or_insert_with(|| graph.add_node(label.clone())); + + if let Some(parent_label) = parent_label { + if let Some(&parent_node) = nodes.get(parent_label) { + graph.add_edge(parent_node, current_node, "".to_string()); + } + } + + for child in &r.children { + build_graph_from_pretty(child, graph, nodes, Some(&label)); + } + } +} From c00fe35fbf1515d243c5d7c233c68b4b1638b7bd Mon Sep 17 00:00:00 2001 From: xiangjinwu <17769960+xiangjinwu@users.noreply.github.com> Date: Thu, 28 Nov 2024 18:10:09 +0800 Subject: [PATCH 040/163] fix(source): resolve avro `Ref` during `avro_schema_to_column_descs` without hack (#19601) --- e2e_test/source_inline/kafka/avro/ref.slt | 128 ++++++++++++++++++ .../codec/src/decoder/avro/schema.rs | 98 +++++++++++--- .../codec/tests/integration_tests/avro.rs | 2 +- src/connector/src/parser/avro/parser.rs | 2 +- 4 files changed, 209 insertions(+), 21 deletions(-) create mode 100644 e2e_test/source_inline/kafka/avro/ref.slt diff --git a/e2e_test/source_inline/kafka/avro/ref.slt b/e2e_test/source_inline/kafka/avro/ref.slt new file mode 100644 index 0000000000000..15f36cf4c0d5d --- /dev/null +++ b/e2e_test/source_inline/kafka/avro/ref.slt @@ -0,0 +1,128 @@ +control substitution on + + +system ok +rpk topic create avro-ref + + +system ok +sr_register avro-ref-value AVRO < Node + + +system ok +curl -X DELETE "${RISEDEV_SCHEMA_REGISTRY_URL}/subjects/avro-ref-value" + + +system ok +curl -X DELETE "${RISEDEV_SCHEMA_REGISTRY_URL}/subjects/avro-ref-value?permanent=true" + + +system ok +sr_register avro-ref-value AVRO <, ) -> anyhow::Result> { + let resolved = ResolvedSchema::try_from(schema)?; if let Schema::Record(RecordSchema { fields, .. }) = schema { let mut index = 0; + let mut ancestor_records: Vec = vec![]; let fields = fields .iter() .map(|field| { - avro_field_to_column_desc(&field.name, &field.schema, &mut index, map_handling) + avro_field_to_column_desc( + &field.name, + &field.schema, + &mut index, + &mut ancestor_records, + resolved.get_names(), + map_handling, + ) }) .collect::>()?; Ok(fields) @@ -105,10 +113,22 @@ fn avro_field_to_column_desc( name: &str, schema: &Schema, index: &mut i32, + ancestor_records: &mut Vec, + refs: &NamesRef<'_>, map_handling: Option, ) -> anyhow::Result { - let data_type = avro_type_mapping(schema, map_handling)?; + let data_type = avro_type_mapping(schema, ancestor_records, refs, map_handling)?; match schema { + Schema::Ref { name: ref_name } => { + avro_field_to_column_desc( + name, + refs[ref_name], // `ResolvedSchema::try_from` already handles lookup failure + index, + ancestor_records, + refs, + map_handling, + ) + } Schema::Record(RecordSchema { name: schema_name, fields, @@ -116,7 +136,16 @@ fn avro_field_to_column_desc( }) => { let vec_column = fields .iter() - .map(|f| avro_field_to_column_desc(&f.name, &f.schema, index, map_handling)) + .map(|f| { + avro_field_to_column_desc( + &f.name, + &f.schema, + index, + ancestor_records, + refs, + map_handling, + ) + }) .collect::>()?; *index += 1; Ok(ColumnDesc { @@ -146,9 +175,11 @@ fn avro_field_to_column_desc( } } -/// This function expects resolved schema (no `Ref`). +/// This function expects original schema (with `Ref`). fn avro_type_mapping( schema: &Schema, + ancestor_records: &mut Vec, + refs: &NamesRef<'_>, map_handling: Option, ) -> anyhow::Result { let data_type = match schema { @@ -190,16 +221,34 @@ fn avro_type_mapping( return Ok(DataType::Decimal); } - StructType::new( + let unique_name = name.fullname(None); + if ancestor_records.contains(&unique_name) { + bail!( + "circular reference detected in Avro schema: {} -> {}", + ancestor_records.join(" -> "), + unique_name + ); + } + + ancestor_records.push(unique_name); + let ty = StructType::new( fields .iter() - .map(|f| Ok((&f.name, avro_type_mapping(&f.schema, map_handling)?))) + .map(|f| { + Ok(( + &f.name, + avro_type_mapping(&f.schema, ancestor_records, refs, map_handling)?, + )) + }) .collect::>>()?, ) - .into() + .into(); + ancestor_records.pop(); + ty } Schema::Array(item_schema) => { - let item_type = avro_type_mapping(item_schema.as_ref(), map_handling)?; + let item_type = + avro_type_mapping(item_schema.as_ref(), ancestor_records, refs, map_handling)?; DataType::List(Box::new(item_type)) } Schema::Union(union_schema) => { @@ -219,7 +268,7 @@ fn avro_type_mapping( "Union contains duplicate types: {union_schema:?}", ); match get_nullable_union_inner(union_schema) { - Some(inner) => avro_type_mapping(inner, map_handling)?, + Some(inner) => avro_type_mapping(inner, ancestor_records, refs, map_handling)?, None => { // Convert the union to a struct, each field of the struct represents a variant of the union. // Refer to https://github.com/risingwavelabs/risingwave/issues/16273#issuecomment-2179761345 to see why it's not perfect. @@ -232,10 +281,11 @@ fn avro_type_mapping( // null will mean the whole struct is null .filter(|variant| !matches!(variant, &&Schema::Null)) .map(|variant| { - avro_type_mapping(variant, map_handling).and_then(|t| { - let name = avro_schema_to_struct_field_name(variant)?; - Ok((name, t)) - }) + avro_type_mapping(variant, ancestor_records, refs, map_handling) + .and_then(|t| { + let name = avro_schema_to_struct_field_name(variant)?; + Ok((name, t)) + }) }) .try_collect::<_, Vec<_>, _>() .context("failed to convert Avro union to struct")?; @@ -250,7 +300,12 @@ fn avro_type_mapping( { DataType::Decimal } else { - bail_not_implemented!("Avro type: {:?}", schema); + avro_type_mapping( + refs[name], // `ResolvedSchema::try_from` already handles lookup failure + ancestor_records, + refs, + map_handling, + )? } } Schema::Map(value_schema) => { @@ -268,8 +323,13 @@ fn avro_type_mapping( } } Some(MapHandling::Map) | None => { - let value = avro_type_mapping(value_schema.as_ref(), map_handling) - .context("failed to convert Avro map type")?; + let value = avro_type_mapping( + value_schema.as_ref(), + ancestor_records, + refs, + map_handling, + ) + .context("failed to convert Avro map type")?; DataType::Map(MapType::from_kv(DataType::Varchar, value)) } } diff --git a/src/connector/codec/tests/integration_tests/avro.rs b/src/connector/codec/tests/integration_tests/avro.rs index d916dc1aba426..2221917cc2b05 100644 --- a/src/connector/codec/tests/integration_tests/avro.rs +++ b/src/connector/codec/tests/integration_tests/avro.rs @@ -56,7 +56,7 @@ fn avro_schema_str_to_risingwave_schema( ResolvedAvroSchema::create(avro_schema.into()).context("failed to resolve Avro schema")?; let rw_schema = - avro_schema_to_column_descs(&resolved_schema.resolved_schema, config.map_handling) + avro_schema_to_column_descs(&resolved_schema.original_schema, config.map_handling) .context("failed to convert Avro schema to RisingWave schema")? .iter() .map(ColumnDesc::from) diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index ac93ab3e69807..e817267778a50 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -229,7 +229,7 @@ impl AvroParserConfig { } pub fn map_to_columns(&self) -> ConnectorResult> { - avro_schema_to_column_descs(&self.schema.resolved_schema, self.map_handling) + avro_schema_to_column_descs(&self.schema.original_schema, self.map_handling) .map_err(Into::into) } } From 4b1cb92e5ce37c72139f933b525ed9c41aec575f Mon Sep 17 00:00:00 2001 From: zwang28 <70626450+zwang28@users.noreply.github.com> Date: Thu, 28 Nov 2024 18:34:37 +0800 Subject: [PATCH 041/163] refactor(storage): use Arc in HummockVersion (#19535) --- src/ctl/src/cmd_impl/hummock/sst_dump.rs | 7 +- .../rw_catalog/rw_hummock_version.rs | 2 +- .../picker/manual_compaction_picker.rs | 35 +++++--- .../picker/space_reclaim_compaction_picker.rs | 11 ++- .../tombstone_reclaim_compaction_picker.rs | 12 +-- .../picker/trivial_move_compaction_picker.rs | 7 +- .../picker/ttl_reclaim_compaction_picker.rs | 10 ++- .../picker/vnode_watermark_picker.rs | 28 +++--- .../src/hummock/compaction/selector/mod.rs | 21 +++-- src/meta/src/hummock/manager/commit_epoch.rs | 5 +- src/meta/src/hummock/manager/tests.rs | 88 +++++++++++++------ src/meta/src/hummock/manager/versioning.rs | 12 +-- src/meta/src/hummock/test_utils.rs | 88 ++++++++++--------- .../compaction_group/hummock_version_ext.rs | 45 +++++++--- .../hummock_sdk/src/compaction_group/mod.rs | 18 ++-- src/storage/hummock_sdk/src/sstable_info.rs | 81 +++++++++++++++-- .../src/hummock_read_version_tests.rs | 88 +++++++++++-------- .../hummock_test/src/hummock_storage_tests.rs | 42 +++++---- src/storage/src/hummock/compactor/iterator.rs | 27 ++++-- .../event_handler/uploader/test_utils.rs | 27 +++--- src/storage/src/hummock/sstable/builder.rs | 7 +- src/storage/src/hummock/test_utils.rs | 10 ++- 22 files changed, 445 insertions(+), 226 deletions(-) diff --git a/src/ctl/src/cmd_impl/hummock/sst_dump.rs b/src/ctl/src/cmd_impl/hummock/sst_dump.rs index 6a36b8bd86833..3594858ac2111 100644 --- a/src/ctl/src/cmd_impl/hummock/sst_dump.rs +++ b/src/ctl/src/cmd_impl/hummock/sst_dump.rs @@ -29,7 +29,7 @@ use risingwave_common::util::value_encoding::{BasicSerde, EitherSerde, ValueRowD use risingwave_frontend::TableCatalog; use risingwave_hummock_sdk::key::FullKey; use risingwave_hummock_sdk::level::Level; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_object_store::object::{ObjectMetadata, ObjectStoreImpl}; use risingwave_rpc_client::MetaClient; @@ -200,12 +200,13 @@ pub async fn sst_dump_via_sstable_store( table_data: &TableData, args: &SstDumpArgs, ) -> anyhow::Result<()> { - let sstable_info = SstableInfo { + let sstable_info = SstableInfoInner { object_id, file_size, meta_offset, ..Default::default() - }; + } + .into(); let sstable_cache = sstable_store .sstable(&sstable_info, &mut StoreLocalStatistic::default()) .await?; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs index 6fb22ebf65090..77101b88fbeaf 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_hummock_version.rs @@ -110,7 +110,7 @@ fn version_to_sstable_rows(version: HummockVersion) -> Vec { for cg in version.levels.into_values() { for level in cg.levels.into_iter().chain(cg.l0.sub_levels) { for sst in level.table_infos { - let key_range = sst.key_range; + let key_range = sst.key_range.clone(); sstables.push(RwHummockSstable { sstable_id: sst.sst_id as _, object_id: sst.object_id as _, diff --git a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs index 597d18cce4fd0..5fc4d19242ae5 100644 --- a/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/manual_compaction_picker.rs @@ -17,7 +17,7 @@ use std::sync::Arc; use itertools::Itertools; use risingwave_hummock_sdk::level::{InputLevel, Level, Levels, OverlappingLevel}; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_pb::hummock::LevelType; use super::{CompactionInput, CompactionPicker, LocalPickerStatistic}; @@ -180,10 +180,11 @@ impl ManualCompactionPicker { fn filter_level_by_option(&self, level: &Level) -> bool { let mut hint_sst_ids: HashSet = HashSet::new(); hint_sst_ids.extend(self.option.sst_ids.iter()); - let tmp_sst_info = SstableInfo { + let tmp_sst_info = SstableInfoInner { key_range: self.option.key_range.clone(), ..Default::default() - }; + } + .into(); if self .overlap_strategy .check_overlap_with_tables(&[tmp_sst_info], &level.table_infos) @@ -231,10 +232,10 @@ impl CompactionPicker for ManualCompactionPicker { } let mut hint_sst_ids: HashSet = HashSet::new(); hint_sst_ids.extend(self.option.sst_ids.iter()); - let mut tmp_sst_info = SstableInfo::default(); + let mut tmp_sst_info = SstableInfoInner::default(); let mut range_overlap_info = RangeOverlapInfo::default(); tmp_sst_info.key_range = self.option.key_range.clone(); - range_overlap_info.update(&tmp_sst_info); + range_overlap_info.update(&tmp_sst_info.into()); let level = self.option.level; let target_level = self.target_level; assert!( @@ -465,9 +466,11 @@ pub mod tests { let level_table_info = &mut levels.levels[0].table_infos; let table_info_1 = &mut level_table_info[1]; - table_info_1.table_ids.resize(2, 0); - table_info_1.table_ids[0] = 1; - table_info_1.table_ids[1] = 2; + let mut t_inner = table_info_1.get_inner(); + t_inner.table_ids.resize(2, 0); + t_inner.table_ids[0] = 1; + t_inner.table_ids[1] = 2; + *table_info_1 = t_inner.into(); // test internal_table_id let option = ManualCompactionOption { @@ -499,9 +502,11 @@ pub mod tests { // include all table_info let level_table_info = &mut levels.levels[0].table_infos; for table_info in level_table_info { - table_info.table_ids.resize(2, 0); - table_info.table_ids[0] = 1; - table_info.table_ids[1] = 2; + let mut t_inner = table_info.get_inner(); + t_inner.table_ids.resize(2, 0); + t_inner.table_ids[0] = 1; + t_inner.table_ids[1] = 2; + *table_info = t_inner.into(); } // test key range filter first @@ -575,12 +580,14 @@ pub mod tests { for iter in [l0.sub_levels.iter_mut(), levels.iter_mut()] { for (idx, l) in iter.enumerate() { for t in &mut l.table_infos { - t.table_ids.clear(); + let mut t_inner = t.get_inner(); + t_inner.table_ids.clear(); if idx == 0 { - t.table_ids.push(((t.sst_id % 2) + 1) as _); + t_inner.table_ids.push(((t.sst_id % 2) + 1) as _); } else { - t.table_ids.push(3); + t_inner.table_ids.push(3); } + *t = t_inner.into(); } } } diff --git a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs index b3e9eb75321a0..8d7f08ca7f5c1 100644 --- a/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/space_reclaim_compaction_picker.rs @@ -173,7 +173,9 @@ mod test { use itertools::Itertools; use risingwave_common::catalog::TableId; + use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::level::Level; + use risingwave_hummock_sdk::sstable_info::SstableInfoInner; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_pb::hummock::compact_task; pub use risingwave_pb::hummock::LevelType; @@ -234,7 +236,14 @@ mod test { { let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); assert_eq!(10, sst_10.sst_id); - sst_10.key_range.right_exclusive = true; + *sst_10 = SstableInfoInner { + key_range: KeyRange { + right_exclusive: true, + ..sst_10.get_inner().key_range.clone() + }, + ..sst_10.get_inner() + } + .into(); } assert_eq!(levels.len(), 4); diff --git a/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs index b4ab933931bd7..a1dc751606f6f 100644 --- a/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/tombstone_reclaim_compaction_picker.rs @@ -135,7 +135,9 @@ pub mod tests { use super::*; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction::create_overlap_strategy; - use crate::hummock::compaction::selector::tests::{generate_level, generate_table}; + use crate::hummock::compaction::selector::tests::{ + generate_level, generate_table, generate_table_impl, + }; #[test] fn test_basic() { @@ -165,21 +167,21 @@ pub mod tests { let picker = TombstoneReclaimCompactionPicker::new(strategy.clone(), 40, 20); let ret = picker.pick_compaction(&levels, &levels_handler, &mut state); assert!(ret.is_none()); - let mut sst = generate_table(3, 1, 201, 300, 1); + let mut sst = generate_table_impl(3, 1, 201, 300, 1); sst.stale_key_count = 40; sst.total_key_count = 100; - levels.levels[1].table_infos.push(sst); + levels.levels[1].table_infos.push(sst.into()); let ret = picker .pick_compaction(&levels, &levels_handler, &mut state) .unwrap(); assert_eq!(2, ret.input_levels.len()); assert_eq!(3, ret.input_levels[0].table_infos[0].sst_id); - let mut sst = generate_table(4, 1, 1, 100, 1); + let mut sst = generate_table_impl(4, 1, 1, 100, 1); sst.stale_key_count = 30; sst.range_tombstone_count = 30; sst.total_key_count = 100; - levels.levels[0].table_infos.push(sst); + levels.levels[0].table_infos.push(sst.into()); let picker = TombstoneReclaimCompactionPicker::new(strategy, 50, 10); let mut state = TombstoneReclaimPickerState::default(); let ret = picker diff --git a/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs index 458919cd8b717..5d4b8ad9eb8d7 100644 --- a/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/trivial_move_compaction_picker.rs @@ -117,7 +117,7 @@ impl TrivialMovePicker { pub mod tests { use std::sync::Arc; - use risingwave_hummock_sdk::sstable_info::SstableInfo; + use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use crate::hummock::compaction::compaction_config::CompactionConfigBuilder; use crate::hummock::compaction::create_overlap_strategy; @@ -125,12 +125,13 @@ pub mod tests { #[test] fn test_allowed_trivial_move_min_size() { - let sst = SstableInfo { + let sst: SstableInfo = SstableInfoInner { sst_id: 1, file_size: 100, sst_size: 100, ..Default::default() - }; + } + .into(); let config = Arc::new( CompactionConfigBuilder::new() diff --git a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs index 36ee06a3bb795..97215574b8761 100644 --- a/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/ttl_reclaim_compaction_picker.rs @@ -203,6 +203,7 @@ mod test { use itertools::Itertools; use risingwave_hummock_sdk::level::Level; + use risingwave_hummock_sdk::sstable_info::SstableInfoInner; use risingwave_hummock_sdk::version::HummockVersionStateTableInfo; use risingwave_pb::hummock::compact_task; pub use risingwave_pb::hummock::LevelType; @@ -349,7 +350,14 @@ mod test { { let sst_10 = levels[3].table_infos.get_mut(8).unwrap(); assert_eq!(10, sst_10.sst_id); - sst_10.key_range.right_exclusive = true; + *sst_10 = SstableInfoInner { + key_range: KeyRange { + right_exclusive: true, + ..sst_10.key_range.clone() + }, + ..sst_10.get_inner() + } + .into(); } assert_eq!(levels.len(), 4); diff --git a/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs b/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs index 5171c48ad9c34..26626e48a2423 100644 --- a/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs +++ b/src/meta/src/hummock/compaction/picker/vnode_watermark_picker.rs @@ -22,6 +22,7 @@ use risingwave_hummock_sdk::table_watermark::ReadTableWatermark; use crate::hummock::compaction::picker::CompactionInput; use crate::hummock::level_handler::LevelHandler; + pub struct VnodeWatermarkCompactionPicker {} impl VnodeWatermarkCompactionPicker { @@ -109,7 +110,7 @@ mod tests { use risingwave_common::hash::VirtualNode; use risingwave_hummock_sdk::key::{FullKey, TableKey}; use risingwave_hummock_sdk::key_range::KeyRange; - use risingwave_hummock_sdk::sstable_info::SstableInfo; + use risingwave_hummock_sdk::sstable_info::SstableInfoInner; use risingwave_hummock_sdk::table_watermark::{ReadTableWatermark, WatermarkDirection}; use crate::hummock::compaction::picker::vnode_watermark_picker::should_delete_sst_by_watermark; @@ -132,7 +133,7 @@ mod tests { TableKey(builder.freeze()) }; - let sst_info = SstableInfo { + let sst_info = SstableInfoInner { object_id: 1, sst_id: 1, key_range: KeyRange { @@ -146,13 +147,14 @@ mod tests { }, table_ids: vec![2], ..Default::default() - }; + } + .into(); assert!( !should_delete_sst_by_watermark(&sst_info, &table_watermarks), "should fail because no matching watermark found" ); - let sst_info = SstableInfo { + let sst_info = SstableInfoInner { object_id: 1, sst_id: 1, key_range: KeyRange { @@ -166,13 +168,14 @@ mod tests { }, table_ids: vec![1], ..Default::default() - }; + } + .into(); assert!( !should_delete_sst_by_watermark(&sst_info, &table_watermarks), "should fail because no matching vnode found" ); - let sst_info = SstableInfo { + let sst_info = SstableInfoInner { object_id: 1, sst_id: 1, key_range: KeyRange { @@ -186,13 +189,14 @@ mod tests { }, table_ids: vec![1], ..Default::default() - }; + } + .into(); assert!( !should_delete_sst_by_watermark(&sst_info, &table_watermarks), "should fail because different vnodes found" ); - let sst_info = SstableInfo { + let sst_info = SstableInfoInner { object_id: 1, sst_id: 1, key_range: KeyRange { @@ -206,13 +210,14 @@ mod tests { }, table_ids: vec![1], ..Default::default() - }; + } + .into(); assert!( !should_delete_sst_by_watermark(&sst_info, &table_watermarks), "should fail because right key is greater than watermark" ); - let sst_info = SstableInfo { + let sst_info = SstableInfoInner { object_id: 1, sst_id: 1, key_range: KeyRange { @@ -226,7 +231,8 @@ mod tests { }, table_ids: vec![1], ..Default::default() - }; + } + .into(); assert!(should_delete_sst_by_watermark(&sst_info, &table_watermarks)); } } diff --git a/src/meta/src/hummock/compaction/selector/mod.rs b/src/meta/src/hummock/compaction/selector/mod.rs index 02b50ff2ff671..07309ff72ee8f 100644 --- a/src/meta/src/hummock/compaction/selector/mod.rs +++ b/src/meta/src/hummock/compaction/selector/mod.rs @@ -129,7 +129,7 @@ pub mod tests { use itertools::Itertools; use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::level::{Level, OverlappingLevel}; - use risingwave_hummock_sdk::sstable_info::SstableInfo; + use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_pb::hummock::LevelType; use super::*; @@ -179,8 +179,18 @@ pub mod tests { right: usize, epoch: u64, ) -> SstableInfo { + generate_table_impl(id, table_prefix, left, right, epoch).into() + } + + pub fn generate_table_impl( + id: u64, + table_prefix: u64, + left: usize, + right: usize, + epoch: u64, + ) -> SstableInfoInner { let object_size = (right - left + 1) as u64; - SstableInfo { + SstableInfoInner { object_id: id, sst_id: id, key_range: KeyRange { @@ -209,7 +219,7 @@ pub mod tests { max_epoch: u64, ) -> SstableInfo { let object_size = (right - left + 1) as u64; - SstableInfo { + SstableInfoInner { object_id: id, sst_id: id, key_range: KeyRange { @@ -225,6 +235,7 @@ pub mod tests { sst_size: object_size, ..Default::default() } + .into() } pub fn generate_tables( @@ -237,10 +248,10 @@ pub mod tests { let mut start = keys.start; let mut tables = vec![]; for id in ids { - let mut table = generate_table(id, 1, start, start + step - 1, epoch); + let mut table = generate_table_impl(id, 1, start, start + step - 1, epoch); table.file_size = file_size; table.sst_size = file_size; - tables.push(table); + tables.push(table.into()); start += step; } tables diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 37ff58b25a4d4..4dbb1c71b35d8 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -396,13 +396,14 @@ impl HummockManager { .sum(); // TODO(li0k): replace with `split_sst` - let branch_sst = split_sst_with_table_ids( - &mut sst.sst_info, + let (modified_sst_info, branch_sst) = split_sst_with_table_ids( + &sst.sst_info, &mut new_sst_id, origin_sst_size - new_sst_size, new_sst_size, match_ids, ); + sst.sst_info = modified_sst_info; commit_sstables .entry(group_id) diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index c96911aea163d..2380a3ade966f 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -29,7 +29,7 @@ use risingwave_hummock_sdk::compaction_group::hummock_version_ext::get_compactio use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{gen_key_from_str, FullKey}; use risingwave_hummock_sdk::key_range::KeyRange; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_hummock_sdk::table_stats::{to_prost_table_stats_map, TableStats, TableStatsMap}; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ @@ -76,6 +76,10 @@ fn pin_versions_sum(pin_versions: &[HummockPinnedVersion]) -> usize { } fn gen_sstable_info(sst_id: u64, table_ids: Vec, epoch: u64) -> SstableInfo { + gen_sstable_info_impl(sst_id, table_ids, epoch).into() +} + +fn gen_sstable_info_impl(sst_id: u64, table_ids: Vec, epoch: u64) -> SstableInfoInner { let table_key_l = gen_key_from_str(VirtualNode::ZERO, "1"); let table_key_r = gen_key_from_str(VirtualNode::MAX_FOR_TEST, "1"); let full_key_l = FullKey::for_test( @@ -87,7 +91,7 @@ fn gen_sstable_info(sst_id: u64, table_ids: Vec, epoch: u64) -> SstableInfo let full_key_r = FullKey::for_test(TableId::new(*table_ids.last().unwrap()), table_key_r, epoch).encode(); - SstableInfo { + SstableInfoInner { sst_id, key_range: KeyRange { left: full_key_l.into(), @@ -1287,7 +1291,7 @@ async fn test_version_stats() { .into_iter() .enumerate() .map(|(idx, table_ids)| LocalSstableInfo { - sst_info: SstableInfo { + sst_info: SstableInfoInner { object_id: sst_ids[idx], sst_id: sst_ids[idx], key_range: KeyRange { @@ -1299,7 +1303,8 @@ async fn test_version_stats() { table_ids: table_ids.clone(), sst_size: 1024 * 1024 * 1024, ..Default::default() - }, + } + .into(), table_stats: table_ids .iter() .map(|table_id| (*table_id, table_stats_change.clone())) @@ -1673,12 +1678,24 @@ async fn test_move_state_tables_to_dedicated_compaction_group_trivial_expired() table_stats: Default::default(), created_at: u64::MAX, }; - let mut sst_3 = sst_2.clone(); - let mut sst_4 = sst_1.clone(); - sst_3.sst_info.sst_id = 8; - sst_3.sst_info.object_id = 8; - sst_4.sst_info.sst_id = 9; - sst_4.sst_info.object_id = 9; + let sst_3 = LocalSstableInfo { + sst_info: SstableInfoInner { + sst_id: 8, + object_id: 8, + ..sst_2.sst_info.get_inner() + } + .into(), + ..sst_2.clone() + }; + let sst_4 = LocalSstableInfo { + sst_info: SstableInfoInner { + sst_id: 9, + object_id: 9, + ..sst_1.sst_info.get_inner() + } + .into(), + ..sst_1.clone() + }; hummock_meta_client .commit_epoch( 30, @@ -1745,7 +1762,7 @@ async fn test_move_state_tables_to_dedicated_compaction_group_trivial_expired() .report_compact_task( task2.task_id, TaskStatus::Success, - vec![SstableInfo { + vec![SstableInfoInner { object_id: 12, sst_id: 12, key_range: KeyRange::default(), @@ -1755,7 +1772,8 @@ async fn test_move_state_tables_to_dedicated_compaction_group_trivial_expired() file_size: 100, sst_size: 100, ..Default::default() - }], + } + .into()], None, HashMap::default(), ) @@ -2222,10 +2240,15 @@ async fn test_partition_level() { const MB: u64 = 1024 * 1024; let mut selector = default_compaction_selector(); for epoch in 31..100 { - let mut sst = gen_local_sstable_info(global_sst_id, vec![100], test_epoch(epoch)); - sst.sst_info.file_size = 10 * MB; - sst.sst_info.sst_size = 10 * MB; - sst.sst_info.uncompressed_file_size = 10 * MB; + let mut sst = gen_sstable_info_impl(global_sst_id, vec![100], test_epoch(epoch)); + sst.file_size = 10 * MB; + sst.sst_size = 10 * MB; + sst.uncompressed_file_size = 10 * MB; + let sst = LocalSstableInfo { + sst_info: sst.into(), + table_stats: Default::default(), + created_at: u64::MAX, + }; hummock_meta_client .commit_epoch( @@ -2245,7 +2268,7 @@ async fn test_partition_level() { .await .unwrap() { - let mut sst = gen_sstable_info(global_sst_id, vec![100], test_epoch(epoch)); + let mut sst = gen_sstable_info_impl(global_sst_id, vec![100], test_epoch(epoch)); sst.file_size = task .input_ssts .iter() @@ -2263,7 +2286,7 @@ async fn test_partition_level() { .report_compact_task( task.task_id, TaskStatus::Success, - vec![sst], + vec![sst.into()], None, HashMap::default(), ) @@ -2426,12 +2449,24 @@ async fn test_merge_compaction_group_task_expired() { table_stats: Default::default(), created_at: u64::MAX, }; - let mut sst_3 = sst_2.clone(); - let mut sst_4 = sst_1.clone(); - sst_3.sst_info.sst_id = 3; - sst_3.sst_info.object_id = 3; - sst_4.sst_info.sst_id = 4; - sst_4.sst_info.object_id = 4; + let sst_3 = LocalSstableInfo { + sst_info: SstableInfoInner { + sst_id: 3, + object_id: 3, + ..sst_2.sst_info.get_inner() + } + .into(), + ..sst_2.clone() + }; + let sst_4 = LocalSstableInfo { + sst_info: SstableInfoInner { + sst_id: 4, + object_id: 4, + ..sst_1.sst_info.get_inner() + } + .into(), + ..sst_1.clone() + }; hummock_meta_client .commit_epoch( 30, @@ -2521,7 +2556,7 @@ async fn test_merge_compaction_group_task_expired() { .report_compact_task( task2.task_id, TaskStatus::Success, - vec![SstableInfo { + vec![SstableInfoInner { object_id: report_sst_id, sst_id: report_sst_id, key_range: KeyRange::default(), @@ -2531,7 +2566,8 @@ async fn test_merge_compaction_group_task_expired() { file_size: 100, sst_size: 100, ..Default::default() - }], + } + .into()], None, HashMap::default(), ) diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index fd516a2ba3e1e..78d84b829d93d 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -333,7 +333,7 @@ mod tests { use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::level::{Level, Levels}; - use risingwave_hummock_sdk::sstable_info::SstableInfo; + use risingwave_hummock_sdk::sstable_info::SstableInfoInner; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{CompactionGroupId, HummockVersionId}; use risingwave_pb::hummock::write_limits::WriteLimit; @@ -458,7 +458,7 @@ mod tests { #[test] fn test_estimate_table_stats() { - let sst = SstableInfo { + let sst = SstableInfoInner { key_range: KeyRange { left: vec![1; 10].into(), right: vec![1; 20].into(), @@ -468,7 +468,8 @@ mod tests { total_key_count: 6000, uncompressed_file_size: 6_000_000, ..Default::default() - }; + } + .into(); let changes = estimate_table_stats(&sst); assert_eq!(changes.len(), 3); for stats in changes.values() { @@ -519,7 +520,7 @@ mod tests { #[test] fn test_estimate_table_stats_large_key_range() { - let sst = SstableInfo { + let sst = SstableInfoInner { key_range: KeyRange { left: vec![1; 1000].into(), right: vec![1; 2000].into(), @@ -529,7 +530,8 @@ mod tests { total_key_count: 6000, uncompressed_file_size: 60_000, ..Default::default() - }; + } + .into(); let changes = estimate_table_stats(&sst); assert_eq!(changes.len(), 3); for t in &sst.table_ids { diff --git a/src/meta/src/hummock/test_utils.rs b/src/meta/src/hummock/test_utils.rs index 2e750a06aa222..5c5e20f94126e 100644 --- a/src/meta/src/hummock/test_utils.rs +++ b/src/meta/src/hummock/test_utils.rs @@ -25,7 +25,7 @@ use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::key::key_with_epoch; use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::level::Levels; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionStateTableInfo}; use risingwave_hummock_sdk::{ @@ -153,31 +153,34 @@ pub fn generate_test_sstables_with_table_id( let mut sst_info = vec![]; for (i, sst_id) in sst_ids.into_iter().enumerate() { let object_size = 2; - sst_info.push(SstableInfo { - object_id: sst_id, - sst_id, - key_range: KeyRange { - left: Bytes::from(key_with_epoch( - format!("{:03}\0\0_key_test_{:05}", table_id, i + 1) - .as_bytes() - .to_vec(), - epoch, - )), - right: Bytes::from(key_with_epoch( - format!("{:03}\0\0_key_test_{:05}", table_id, (i + 1) * 10) - .as_bytes() - .to_vec(), - epoch, - )), - right_exclusive: false, - }, - file_size: object_size, - table_ids: vec![table_id], - uncompressed_file_size: object_size, - max_epoch: epoch, - sst_size: object_size, - ..Default::default() - }); + sst_info.push( + SstableInfoInner { + object_id: sst_id, + sst_id, + key_range: KeyRange { + left: Bytes::from(key_with_epoch( + format!("{:03}\0\0_key_test_{:05}", table_id, i + 1) + .as_bytes() + .to_vec(), + epoch, + )), + right: Bytes::from(key_with_epoch( + format!("{:03}\0\0_key_test_{:05}", table_id, (i + 1) * 10) + .as_bytes() + .to_vec(), + epoch, + )), + right_exclusive: false, + }, + file_size: object_size, + table_ids: vec![table_id], + uncompressed_file_size: object_size, + max_epoch: epoch, + sst_size: object_size, + ..Default::default() + } + .into(), + ); } sst_info } @@ -186,21 +189,24 @@ pub fn generate_test_tables(epoch: u64, sst_ids: Vec) -> let mut sst_info = vec![]; for (i, sst_id) in sst_ids.into_iter().enumerate() { let object_size = 2; - sst_info.push(SstableInfo { - object_id: sst_id, - sst_id, - key_range: KeyRange { - left: Bytes::from(iterator_test_key_of_epoch(sst_id, i + 1, epoch)), - right: Bytes::from(iterator_test_key_of_epoch(sst_id, (i + 1) * 10, epoch)), - right_exclusive: false, - }, - file_size: object_size, - table_ids: vec![sst_id as u32, sst_id as u32 * 10000], - uncompressed_file_size: object_size, - max_epoch: epoch, - sst_size: object_size, - ..Default::default() - }); + sst_info.push( + SstableInfoInner { + object_id: sst_id, + sst_id, + key_range: KeyRange { + left: Bytes::from(iterator_test_key_of_epoch(sst_id, i + 1, epoch)), + right: Bytes::from(iterator_test_key_of_epoch(sst_id, (i + 1) * 10, epoch)), + right_exclusive: false, + }, + file_size: object_size, + table_ids: vec![sst_id as u32, sst_id as u32 * 10000], + uncompressed_file_size: object_size, + max_epoch: epoch, + sst_size: object_size, + ..Default::default() + } + .into(), + ); } sst_info } diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index ac9521b0f5272..bcf99e18bfe7a 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -1175,13 +1175,14 @@ fn split_sst_info_for_level( .cloned() .collect_vec(); if !removed_table_ids.is_empty() { - let branch_sst = split_sst_with_table_ids( + let (modified_sst, branch_sst) = split_sst_with_table_ids( sst_info, new_sst_id, sst_info.sst_size / 2, sst_info.sst_size / 2, member_table_ids.iter().cloned().collect_vec(), ); + *sst_info = modified_sst; insert_table_infos.push(branch_sst); } } @@ -1511,13 +1512,17 @@ mod tests { use crate::key::{gen_key_from_str, FullKey}; use crate::key_range::KeyRange; use crate::level::{Level, Levels, OverlappingLevel}; - use crate::sstable_info::SstableInfo; + use crate::sstable_info::{SstableInfo, SstableInfoInner}; use crate::version::{ GroupDelta, GroupDeltas, HummockVersion, HummockVersionDelta, IntraLevelDelta, }; use crate::HummockVersionId; fn gen_sstable_info(sst_id: u64, table_ids: Vec, epoch: u64) -> SstableInfo { + gen_sstable_info_impl(sst_id, table_ids, epoch).into() + } + + fn gen_sstable_info_impl(sst_id: u64, table_ids: Vec, epoch: u64) -> SstableInfoInner { let table_key_l = gen_key_from_str(VirtualNode::ZERO, "1"); let table_key_r = gen_key_from_str(VirtualNode::MAX_FOR_TEST, "1"); let full_key_l = FullKey::for_test( @@ -1530,7 +1535,7 @@ mod tests { FullKey::for_test(TableId::new(*table_ids.last().unwrap()), table_key_r, epoch) .encode(); - SstableInfo { + SstableInfoInner { sst_id, key_range: KeyRange { left: full_key_l.into(), @@ -1575,22 +1580,24 @@ mod tests { .l0 .sub_levels .push(Level { - table_infos: vec![SstableInfo { + table_infos: vec![SstableInfoInner { object_id: 11, sst_id: 11, ..Default::default() - }], + } + .into()], ..Default::default() }); assert_eq!(version.get_object_ids().len(), 1); // Add to non sub level version.levels.get_mut(&0).unwrap().levels.push(Level { - table_infos: vec![SstableInfo { + table_infos: vec![SstableInfoInner { object_id: 22, sst_id: 22, ..Default::default() - }], + } + .into()], ..Default::default() }); assert_eq!(version.get_object_ids().len(), 2); @@ -1652,11 +1659,12 @@ mod tests { 1, 0, HashSet::new(), - vec![SstableInfo { + vec![SstableInfoInner { object_id: 1, sst_id: 1, ..Default::default() - }], + } + .into()], 0, ))], }, @@ -1677,11 +1685,12 @@ mod tests { cg1.levels[0] = Level { level_idx: 1, level_type: LevelType::Nonoverlapping, - table_infos: vec![SstableInfo { + table_infos: vec![SstableInfoInner { object_id: 1, sst_id: 1, ..Default::default() - }], + } + .into()], ..Default::default() }; assert_eq!( @@ -1707,7 +1716,16 @@ mod tests { } fn gen_sst_info(object_id: u64, table_ids: Vec, left: Bytes, right: Bytes) -> SstableInfo { - SstableInfo { + gen_sst_info_impl(object_id, table_ids, left, right).into() + } + + fn gen_sst_info_impl( + object_id: u64, + table_ids: Vec, + left: Bytes, + right: Bytes, + ) -> SstableInfoInner { + SstableInfoInner { object_id, sst_id: object_id, key_range: KeyRange { @@ -2214,8 +2232,9 @@ mod tests { { // test key_range left = right - let mut sst = gen_sstable_info(1, vec![1], epoch); + let mut sst = gen_sstable_info_impl(1, vec![1], epoch); sst.key_range.right = sst.key_range.left.clone(); + let sst: SstableInfo = sst.into(); let split_key = group_split::build_split_key(1, VirtualNode::ZERO); let origin_sst = sst.clone(); let sst_size = origin_sst.sst_size; diff --git a/src/storage/hummock_sdk/src/compaction_group/mod.rs b/src/storage/hummock_sdk/src/compaction_group/mod.rs index dbe58018f15e3..62ced96501723 100644 --- a/src/storage/hummock_sdk/src/compaction_group/mod.rs +++ b/src/storage/hummock_sdk/src/compaction_group/mod.rs @@ -135,12 +135,13 @@ pub mod group_split { /// `sst_size`: `right_size`, /// } pub fn split_sst( - mut origin_sst_info: SstableInfo, + origin_sst_info: SstableInfo, new_sst_id: &mut u64, split_key: Bytes, left_size: u64, right_size: u64, ) -> (Option, Option) { + let mut origin_sst_info = origin_sst_info.get_inner(); let mut branch_table_info = origin_sst_info.clone(); branch_table_info.sst_id = *new_sst_id; *new_sst_id += 1; @@ -183,9 +184,9 @@ pub mod group_split { // This function does not make any assumptions about the incoming sst, so add some judgement to ensure that the generated sst meets the restrictions. if origin_sst_info.table_ids.is_empty() { - (None, Some(branch_table_info)) + (None, Some(branch_table_info.into())) } else if branch_table_info.table_ids.is_empty() { - (Some(origin_sst_info), None) + (Some(origin_sst_info.into()), None) } else if KeyComparator::compare_encoded_full_key( &origin_sst_info.key_range.left, &origin_sst_info.key_range.right, @@ -193,9 +194,9 @@ pub mod group_split { .is_eq() { // avoid empty key_range of origin_sst - (None, Some(branch_table_info)) + (None, Some(branch_table_info.into())) } else { - (Some(origin_sst_info), Some(branch_table_info)) + (Some(origin_sst_info.into()), Some(branch_table_info.into())) } } @@ -203,12 +204,13 @@ pub mod group_split { /// This function is used to split the sst into two parts based on the `table_ids`. /// In contrast to `split_sst`, this function does not modify the `key_range` and does not guarantee that the split ssts can be merged, which needs to be guaranteed by the caller. pub fn split_sst_with_table_ids( - sst_info: &mut SstableInfo, + origin_sst_info: &SstableInfo, new_sst_id: &mut u64, old_sst_size: u64, new_sst_size: u64, new_table_ids: Vec, - ) -> SstableInfo { + ) -> (SstableInfo, SstableInfo) { + let mut sst_info = origin_sst_info.get_inner(); let mut branch_table_info = sst_info.clone(); branch_table_info.sst_id = *new_sst_id; branch_table_info.sst_size = new_sst_size; @@ -235,7 +237,7 @@ pub mod group_split { .retain(|table_id| !branch_table_info.table_ids.contains(table_id)); } - branch_table_info + (sst_info.into(), branch_table_info.into()) } // Should avoid split same table_id into two groups diff --git a/src/storage/hummock_sdk/src/sstable_info.rs b/src/storage/hummock_sdk/src/sstable_info.rs index 10afe52ab9ad9..eb43b1c61b750 100644 --- a/src/storage/hummock_sdk/src/sstable_info.rs +++ b/src/storage/hummock_sdk/src/sstable_info.rs @@ -13,6 +13,8 @@ // limitations under the License. use std::mem::size_of; +use std::ops::Deref; +use std::sync::Arc; use risingwave_pb::hummock::{PbBloomFilterType, PbKeyRange, PbSstableInfo}; @@ -21,7 +23,7 @@ use crate::version::{ObjectIdReader, SstableIdReader}; use crate::{HummockSstableId, HummockSstableObjectId}; #[derive(Debug, PartialEq, Clone, Default)] -pub struct SstableInfo { +pub struct SstableInfoInner { pub object_id: u64, pub sst_id: u64, pub key_range: KeyRange, @@ -38,7 +40,7 @@ pub struct SstableInfo { pub sst_size: u64, } -impl SstableInfo { +impl SstableInfoInner { pub fn estimated_encode_len(&self) -> usize { let mut basic = size_of::() // object_id + size_of::() // sstable_id @@ -63,7 +65,7 @@ impl SstableInfo { } } -impl From for SstableInfo { +impl From for SstableInfoInner { fn from(pb_sstable_info: PbSstableInfo) -> Self { assert!(pb_sstable_info.table_ids.is_sorted()); Self { @@ -101,7 +103,7 @@ impl From for SstableInfo { } } -impl From<&PbSstableInfo> for SstableInfo { +impl From<&PbSstableInfo> for SstableInfoInner { fn from(pb_sstable_info: &PbSstableInfo) -> Self { assert!(pb_sstable_info.table_ids.is_sorted()); Self { @@ -138,8 +140,8 @@ impl From<&PbSstableInfo> for SstableInfo { } } -impl From for PbSstableInfo { - fn from(sstable_info: SstableInfo) -> Self { +impl From for PbSstableInfo { + fn from(sstable_info: SstableInfoInner) -> Self { assert!(sstable_info.sst_size > 0); assert!(sstable_info.table_ids.is_sorted()); PbSstableInfo { @@ -177,8 +179,8 @@ impl From for PbSstableInfo { } } -impl From<&SstableInfo> for PbSstableInfo { - fn from(sstable_info: &SstableInfo) -> Self { +impl From<&SstableInfoInner> for PbSstableInfo { + fn from(sstable_info: &SstableInfoInner) -> Self { assert!(sstable_info.sst_size > 0); assert!(sstable_info.table_ids.is_sorted()); PbSstableInfo { @@ -215,7 +217,68 @@ impl From<&SstableInfo> for PbSstableInfo { impl SstableInfo { pub fn remove_key_range(&mut self) { - self.key_range = KeyRange::default(); + let mut sst = self.get_inner(); + sst.key_range = KeyRange::default(); + *self = sst.into() + } +} + +impl SstableIdReader for SstableInfoInner { + fn sst_id(&self) -> HummockSstableId { + self.sst_id + } +} + +impl ObjectIdReader for SstableInfoInner { + fn object_id(&self) -> HummockSstableObjectId { + self.object_id + } +} + +#[derive(Debug, PartialEq, Clone, Default)] +pub struct SstableInfo(Arc); + +impl From<&PbSstableInfo> for SstableInfo { + fn from(s: &PbSstableInfo) -> Self { + SstableInfo(SstableInfoInner::from(s).into()) + } +} + +impl From for SstableInfo { + fn from(s: PbSstableInfo) -> Self { + SstableInfo(SstableInfoInner::from(s).into()) + } +} + +impl From for PbSstableInfo { + fn from(s: SstableInfo) -> Self { + (&s).into() + } +} + +impl From for SstableInfo { + fn from(s: SstableInfoInner) -> Self { + Self(s.into()) + } +} + +impl From<&SstableInfo> for PbSstableInfo { + fn from(s: &SstableInfo) -> Self { + s.0.as_ref().into() + } +} + +impl Deref for SstableInfo { + type Target = SstableInfoInner; + + fn deref(&self) -> &Self::Target { + &self.0 + } +} + +impl SstableInfo { + pub fn get_inner(&self) -> SstableInfoInner { + (*self.0).clone() } } diff --git a/src/storage/hummock_test/src/hummock_read_version_tests.rs b/src/storage/hummock_test/src/hummock_read_version_tests.rs index e0fdf9011aca6..2e763dfd3954b 100644 --- a/src/storage/hummock_test/src/hummock_read_version_tests.rs +++ b/src/storage/hummock_test/src/hummock_read_version_tests.rs @@ -25,7 +25,7 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_hummock_sdk::key::{key_with_epoch, map_table_key_range}; use risingwave_hummock_sdk::key_range::KeyRange; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::SstableInfoInner; use risingwave_hummock_sdk::LocalSstableInfo; use risingwave_meta::hummock::test_utils::setup_compute_env; use risingwave_storage::hummock::event_handler::TEST_LOCAL_INSTANCE_ID; @@ -150,44 +150,62 @@ async fn test_read_version_basic() { let dummy_sst = Arc::new(StagingSstableInfo::new( vec![ - LocalSstableInfo::for_test(SstableInfo { - object_id: 1, - sst_id: 1, - key_range: KeyRange { - left: key_with_epoch(iterator_test_user_key_of(1).encode(), test_epoch(1)) + LocalSstableInfo::for_test( + SstableInfoInner { + object_id: 1, + sst_id: 1, + key_range: KeyRange { + left: key_with_epoch( + iterator_test_user_key_of(1).encode(), + test_epoch(1), + ) .into(), - right: key_with_epoch(iterator_test_user_key_of(2).encode(), test_epoch(2)) + right: key_with_epoch( + iterator_test_user_key_of(2).encode(), + test_epoch(2), + ) .into(), - right_exclusive: false, - }, - file_size: 1, - table_ids: vec![0], - meta_offset: 1, - stale_key_count: 1, - total_key_count: 1, - uncompressed_file_size: 1, - sst_size: 1, - ..Default::default() - }), - LocalSstableInfo::for_test(SstableInfo { - object_id: 2, - sst_id: 2, - key_range: KeyRange { - left: key_with_epoch(iterator_test_user_key_of(3).encode(), test_epoch(3)) + right_exclusive: false, + }, + file_size: 1, + table_ids: vec![0], + meta_offset: 1, + stale_key_count: 1, + total_key_count: 1, + uncompressed_file_size: 1, + sst_size: 1, + ..Default::default() + } + .into(), + ), + LocalSstableInfo::for_test( + SstableInfoInner { + object_id: 2, + sst_id: 2, + key_range: KeyRange { + left: key_with_epoch( + iterator_test_user_key_of(3).encode(), + test_epoch(3), + ) .into(), - right: key_with_epoch(iterator_test_user_key_of(3).encode(), test_epoch(3)) + right: key_with_epoch( + iterator_test_user_key_of(3).encode(), + test_epoch(3), + ) .into(), - right_exclusive: false, - }, - file_size: 1, - table_ids: vec![0], - meta_offset: 1, - stale_key_count: 1, - total_key_count: 1, - uncompressed_file_size: 1, - sst_size: 1, - ..Default::default() - }), + right_exclusive: false, + }, + file_size: 1, + table_ids: vec![0], + meta_offset: 1, + stale_key_count: 1, + total_key_count: 1, + uncompressed_file_size: 1, + sst_size: 1, + ..Default::default() + } + .into(), + ), ], vec![], epoch_id_vec_for_clear, diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index d85354636a850..fa95238e49d20 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -29,7 +29,7 @@ use risingwave_common::util::epoch::{test_epoch, EpochExt, INVALID_EPOCH}; use risingwave_hummock_sdk::key::{ gen_key_from_bytes, prefixed_range_with_vnode, FullKey, TableKey, UserKey, TABLE_PREFIX_LEN, }; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_hummock_sdk::table_stats::TableStats; use risingwave_hummock_sdk::table_watermark::{ TableWatermarksIndex, VnodeWatermark, WatermarkDirection, @@ -2663,14 +2663,15 @@ async fn test_commit_multi_epoch() { }; let epoch1 = initial_epoch.next_epoch(); - let sst1_epoch1 = SstableInfo { + let sst1_epoch1: SstableInfo = SstableInfoInner { sst_id: 11, object_id: 1, table_ids: vec![existing_table_id.table_id], file_size: 100, sst_size: 100, ..Default::default() - }; + } + .into(); commit_epoch( epoch1, @@ -2704,14 +2705,15 @@ async fn test_commit_multi_epoch() { assert_eq!(cg_id, info.compaction_group_id); } - let sst1_epoch2 = SstableInfo { + let sst1_epoch2: SstableInfo = SstableInfoInner { sst_id: 22, object_id: 2, table_ids: vec![existing_table_id.table_id], file_size: 100, sst_size: 100, ..Default::default() - }; + } + .into(); let epoch2 = epoch1.next_epoch(); @@ -2740,14 +2742,15 @@ async fn test_commit_multi_epoch() { let new_table_id = TableId::new(2); - let sst2_epoch1 = SstableInfo { + let sst2_epoch1: SstableInfo = SstableInfoInner { sst_id: 33, object_id: 3, table_ids: vec![new_table_id.table_id], file_size: 100, sst_size: 100, ..Default::default() - }; + } + .into(); commit_epoch( epoch1, @@ -2779,14 +2782,15 @@ async fn test_commit_multi_epoch() { new_cg_id }; - let sst2_epoch2 = SstableInfo { + let sst2_epoch2: SstableInfo = SstableInfoInner { sst_id: 44, object_id: 4, table_ids: vec![new_table_id.table_id], file_size: 100, sst_size: 100, ..Default::default() - }; + } + .into(); commit_epoch(epoch2, sst2_epoch2.clone(), vec![], &[new_table_id]).await; @@ -2810,14 +2814,15 @@ async fn test_commit_multi_epoch() { let epoch3 = epoch2.next_epoch(); - let sst_epoch3 = SstableInfo { + let sst_epoch3: SstableInfo = SstableInfoInner { sst_id: 55, object_id: 5, table_ids: vec![existing_table_id.table_id, new_table_id.table_id], file_size: 100, sst_size: 100, ..Default::default() - }; + } + .into(); commit_epoch( epoch3, @@ -2920,32 +2925,35 @@ async fn test_commit_with_large_size() { }; let epoch1 = initial_epoch.next_epoch(); - let sst1_epoch1 = SstableInfo { + let sst1_epoch1: SstableInfo = SstableInfoInner { sst_id: 11, object_id: 1, table_ids: vec![existing_table_id.table_id], file_size: 512 << 20, sst_size: 512 << 20, ..Default::default() - }; + } + .into(); - let sst1_epoch2 = SstableInfo { + let sst1_epoch2: SstableInfo = SstableInfoInner { sst_id: 12, object_id: 2, table_ids: vec![existing_table_id.table_id], file_size: 512 << 20, sst_size: 512 << 20, ..Default::default() - }; + } + .into(); - let sst1_epoch3 = SstableInfo { + let sst1_epoch3: SstableInfo = SstableInfoInner { sst_id: 13, object_id: 3, table_ids: vec![existing_table_id.table_id], file_size: 512 << 20, sst_size: 512 << 20, ..Default::default() - }; + } + .into(); commit_epoch( epoch1, diff --git a/src/storage/src/hummock/compactor/iterator.rs b/src/storage/src/hummock/compactor/iterator.rs index c53f945af2c50..b5137ebf4d63f 100644 --- a/src/storage/src/hummock/compactor/iterator.rs +++ b/src/storage/src/hummock/compactor/iterator.rs @@ -726,6 +726,7 @@ mod tests { use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::key::{next_full_key, prev_full_key, FullKey, FullKeyTracker}; use risingwave_hummock_sdk::key_range::KeyRange; + use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use crate::hummock::compactor::ConcatSstableIterator; use crate::hummock::iterator::test_utils::mock_sstable_store; @@ -1198,15 +1199,27 @@ mod tests { .await; let split_key = test_key_of(5000).encode(); - - let mut sst_1 = table_info.clone(); - sst_1.key_range.right = split_key.clone().into(); - sst_1.key_range.right_exclusive = true; + let sst_1: SstableInfo = SstableInfoInner { + key_range: KeyRange { + left: table_info.key_range.left.clone(), + right: split_key.clone().into(), + right_exclusive: true, + }, + ..table_info.get_inner() + } + .into(); let total_key_count = sst_1.total_key_count; - let mut sst_2 = table_info.clone(); - sst_2.sst_id = sst_1.sst_id + 1; - sst_2.key_range.left = split_key.clone().into(); + let sst_2: SstableInfo = SstableInfoInner { + sst_id: sst_1.sst_id + 1, + key_range: KeyRange { + left: split_key.clone().into(), + right: table_info.key_range.right.clone(), + right_exclusive: table_info.key_range.right_exclusive, + }, + ..table_info.get_inner() + } + .into(); { // test concate diff --git a/src/storage/src/hummock/event_handler/uploader/test_utils.rs b/src/storage/src/hummock/event_handler/uploader/test_utils.rs index 3e7b92624109a..26fe3dc19220a 100644 --- a/src/storage/src/hummock/event_handler/uploader/test_utils.rs +++ b/src/storage/src/hummock/event_handler/uploader/test_utils.rs @@ -32,7 +32,7 @@ use risingwave_common::util::epoch::{test_epoch, EpochExt}; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::key::{FullKey, TableKey}; use risingwave_hummock_sdk::key_range::KeyRange; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::SstableInfoInner; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; use risingwave_pb::hummock::{PbHummockVersion, StateTableInfoDelta}; @@ -159,17 +159,20 @@ pub(super) fn gen_sstable_info( let start_full_key = FullKey::new(TEST_TABLE_ID, TableKey(dummy_table_key()), start_epoch); let end_full_key = FullKey::new(TEST_TABLE_ID, TableKey(dummy_table_key()), end_epoch); let gen_sst_object_id = (start_epoch << 8) + end_epoch; - vec![LocalSstableInfo::for_test(SstableInfo { - object_id: gen_sst_object_id, - sst_id: gen_sst_object_id, - key_range: KeyRange { - left: start_full_key.encode().into(), - right: end_full_key.encode().into(), - right_exclusive: true, - }, - table_ids: vec![TEST_TABLE_ID.table_id], - ..Default::default() - })] + vec![LocalSstableInfo::for_test( + SstableInfoInner { + object_id: gen_sst_object_id, + sst_id: gen_sst_object_id, + key_range: KeyRange { + left: start_full_key.encode().into(), + right: end_full_key.encode().into(), + right_exclusive: true, + }, + table_ids: vec![TEST_TABLE_ID.table_id], + ..Default::default() + } + .into(), + )] } pub(super) fn test_uploader_context(upload_fn: F) -> UploaderContext diff --git a/src/storage/src/hummock/sstable/builder.rs b/src/storage/src/hummock/sstable/builder.rs index 0ad46e5e000a2..123e6d2f8df1f 100644 --- a/src/storage/src/hummock/sstable/builder.rs +++ b/src/storage/src/hummock/sstable/builder.rs @@ -20,7 +20,7 @@ use bytes::{Bytes, BytesMut}; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::key::{user_key, FullKey, MAX_KEY_LEN}; use risingwave_hummock_sdk::key_range::KeyRange; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_hummock_sdk::table_stats::{TableStats, TableStatsMap}; use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; use risingwave_pb::hummock::BloomFilterType; @@ -507,7 +507,7 @@ impl SstableBuilder { } }; - let sst_info = SstableInfo { + let sst_info: SstableInfo = SstableInfoInner { object_id: self.sstable_id, sst_id: self.sstable_id, bloom_filter_kind, @@ -526,7 +526,8 @@ impl SstableBuilder { max_epoch, range_tombstone_count: 0, sst_size: meta.estimated_size as u64, - }; + } + .into(); tracing::trace!( "meta_size {} bloom_filter_size {} add_key_counts {} stale_key_count {} min_epoch {} max_epoch {} epoch_count {}", meta.encoded_size(), diff --git a/src/storage/src/hummock/test_utils.rs b/src/storage/src/hummock/test_utils.rs index 148810f9b016d..98e081e72af15 100644 --- a/src/storage/src/hummock/test_utils.rs +++ b/src/storage/src/hummock/test_utils.rs @@ -27,7 +27,7 @@ use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::key::{FullKey, TableKey, UserKey}; use risingwave_hummock_sdk::key_range::KeyRange; -use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::sstable_info::{SstableInfo, SstableInfoInner}; use risingwave_hummock_sdk::{EpochWithGap, HummockEpoch, HummockSstableObjectId}; use super::iterator::test_utils::iterator_test_table_key_of; @@ -108,7 +108,7 @@ pub fn gen_dummy_sst_info( } file_size += batch.size() as u64; } - SstableInfo { + SstableInfoInner { object_id: id, sst_id: id, key_range: KeyRange { @@ -124,6 +124,7 @@ pub fn gen_dummy_sst_info( sst_size: file_size, ..Default::default() } + .into() } /// Number of keys in table generated in `generate_table`. @@ -203,7 +204,7 @@ pub async fn put_sst( meta.meta_offset = writer.data_len() as u64; meta.bloom_filter = bloom_filter; - let sst = SstableInfo { + let sst = SstableInfoInner { object_id: sst_object_id, sst_id: sst_object_id, key_range: KeyRange { @@ -216,7 +217,8 @@ pub async fn put_sst( uncompressed_file_size: meta.estimated_size as u64, table_ids, ..Default::default() - }; + } + .into(); let writer_output = writer.finish(meta).await?; writer_output.await.unwrap()?; Ok(sst) From 5bc4d0c844a7d4a0e6b00159ce8fd39339f0eaad Mon Sep 17 00:00:00 2001 From: Lanqing Yang Date: Thu, 28 Nov 2024 03:26:18 -0800 Subject: [PATCH 042/163] feat(frontend): support explain with graphviz visualization in dashboard (#19558) --- dashboard/package-lock.json | 236 +++++++++++++++++++++++++++ dashboard/package.json | 1 + dashboard/pages/explain_distsql.tsx | 238 +++------------------------- 3 files changed, 255 insertions(+), 220 deletions(-) diff --git a/dashboard/package-lock.json b/dashboard/package-lock.json index 13322f53cd404..7ba669b11ef37 100644 --- a/dashboard/package-lock.json +++ b/dashboard/package-lock.json @@ -27,6 +27,7 @@ "d3-selection": "^3.0.0", "fabric": "^5.2.1", "framer-motion": "^6.5.1", + "graphviz-react": "^1.2.5", "lodash": "^4.17.21", "next": "^14.2.12", "nuqs": "^1.14.1", @@ -4619,6 +4620,111 @@ "node": ">=12" } }, + "node_modules/d3-graphviz": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/d3-graphviz/-/d3-graphviz-2.6.1.tgz", + "integrity": "sha512-878AFSagQyr5tTOrM7YiVYeUC2/NoFcOB3/oew+LAML0xekyJSw9j3WOCUMBsc95KYe9XBYZ+SKKuObVya1tJQ==", + "license": "BSD-3-Clause", + "dependencies": { + "d3-dispatch": "^1.0.3", + "d3-format": "^1.2.0", + "d3-interpolate": "^1.1.5", + "d3-path": "^1.0.5", + "d3-selection": "^1.1.0", + "d3-timer": "^1.0.6", + "d3-transition": "^1.1.1", + "d3-zoom": "^1.5.0", + "viz.js": "^1.8.2" + } + }, + "node_modules/d3-graphviz/node_modules/d3-color": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/d3-color/-/d3-color-1.4.1.tgz", + "integrity": "sha512-p2sTHSLCJI2QKunbGb7ocOh7DgTAn8IrLx21QRc/BSnodXM4sv6aLQlnfpvehFMLZEfBc6g9pH9SWQccFYfJ9Q==", + "license": "BSD-3-Clause" + }, + "node_modules/d3-graphviz/node_modules/d3-dispatch": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/d3-dispatch/-/d3-dispatch-1.0.6.tgz", + "integrity": "sha512-fVjoElzjhCEy+Hbn8KygnmMS7Or0a9sI2UzGwoB7cCtvI1XpVN9GpoYlnb3xt2YV66oXYb1fLJ8GMvP4hdU1RA==", + "license": "BSD-3-Clause" + }, + "node_modules/d3-graphviz/node_modules/d3-drag": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/d3-drag/-/d3-drag-1.2.5.tgz", + "integrity": "sha512-rD1ohlkKQwMZYkQlYVCrSFxsWPzI97+W+PaEIBNTMxRuxz9RF0Hi5nJWHGVJ3Om9d2fRTe1yOBINJyy/ahV95w==", + "license": "BSD-3-Clause", + "dependencies": { + "d3-dispatch": "1", + "d3-selection": "1" + } + }, + "node_modules/d3-graphviz/node_modules/d3-ease": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/d3-ease/-/d3-ease-1.0.7.tgz", + "integrity": "sha512-lx14ZPYkhNx0s/2HX5sLFUI3mbasHjSSpwO/KaaNACweVwxUruKyWVcb293wMv1RqTPZyZ8kSZ2NogUZNcLOFQ==", + "license": "BSD-3-Clause" + }, + "node_modules/d3-graphviz/node_modules/d3-format": { + "version": "1.4.5", + "resolved": "https://registry.npmjs.org/d3-format/-/d3-format-1.4.5.tgz", + "integrity": "sha512-J0piedu6Z8iB6TbIGfZgDzfXxUFN3qQRMofy2oPdXzQibYGqPB/9iMcxr/TGalU+2RsyDO+U4f33id8tbnSRMQ==", + "license": "BSD-3-Clause" + }, + "node_modules/d3-graphviz/node_modules/d3-interpolate": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/d3-interpolate/-/d3-interpolate-1.4.0.tgz", + "integrity": "sha512-V9znK0zc3jOPV4VD2zZn0sDhZU3WAE2bmlxdIwwQPPzPjvyLkd8B3JUVdS1IDUFDkWZ72c9qnv1GK2ZagTZ8EA==", + "license": "BSD-3-Clause", + "dependencies": { + "d3-color": "1" + } + }, + "node_modules/d3-graphviz/node_modules/d3-path": { + "version": "1.0.9", + "resolved": "https://registry.npmjs.org/d3-path/-/d3-path-1.0.9.tgz", + "integrity": "sha512-VLaYcn81dtHVTjEHd8B+pbe9yHWpXKZUC87PzoFmsFrJqgFwDe/qxfp5MlfsfM1V5E/iVt0MmEbWQ7FVIXh/bg==", + "license": "BSD-3-Clause" + }, + "node_modules/d3-graphviz/node_modules/d3-selection": { + "version": "1.4.2", + "resolved": "https://registry.npmjs.org/d3-selection/-/d3-selection-1.4.2.tgz", + "integrity": "sha512-SJ0BqYihzOjDnnlfyeHT0e30k0K1+5sR3d5fNueCNeuhZTnGw4M4o8mqJchSwgKMXCNFo+e2VTChiSJ0vYtXkg==", + "license": "BSD-3-Clause" + }, + "node_modules/d3-graphviz/node_modules/d3-timer": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/d3-timer/-/d3-timer-1.0.10.tgz", + "integrity": "sha512-B1JDm0XDaQC+uvo4DT79H0XmBskgS3l6Ve+1SBCfxgmtIb1AVrPIoqd+nPSv+loMX8szQ0sVUhGngL7D5QPiXw==", + "license": "BSD-3-Clause" + }, + "node_modules/d3-graphviz/node_modules/d3-transition": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/d3-transition/-/d3-transition-1.3.2.tgz", + "integrity": "sha512-sc0gRU4PFqZ47lPVHloMn9tlPcv8jxgOQg+0zjhfZXMQuvppjG6YuwdMBE0TuqCZjeJkLecku/l9R0JPcRhaDA==", + "license": "BSD-3-Clause", + "dependencies": { + "d3-color": "1", + "d3-dispatch": "1", + "d3-ease": "1", + "d3-interpolate": "1", + "d3-selection": "^1.1.0", + "d3-timer": "1" + } + }, + "node_modules/d3-graphviz/node_modules/d3-zoom": { + "version": "1.8.3", + "resolved": "https://registry.npmjs.org/d3-zoom/-/d3-zoom-1.8.3.tgz", + "integrity": "sha512-VoLXTK4wvy1a0JpH2Il+F2CiOhVu7VRXWF5M/LroMIh3/zBAC3WAt7QoIvPibOavVo20hN6/37vwAsdBejLyKQ==", + "license": "BSD-3-Clause", + "dependencies": { + "d3-dispatch": "1", + "d3-drag": "1", + "d3-interpolate": "1", + "d3-selection": "1", + "d3-transition": "1" + } + }, "node_modules/d3-hierarchy": { "version": "3.1.2", "resolved": "https://registry.npmjs.org/d3-hierarchy/-/d3-hierarchy-3.1.2.tgz", @@ -6867,6 +6973,21 @@ "integrity": "sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag==", "dev": true }, + "node_modules/graphviz-react": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/graphviz-react/-/graphviz-react-1.2.5.tgz", + "integrity": "sha512-IRFDzEt09hRzfqrrvAW1PAPBqG4t8hykArcoxq7UhEqO5RUKCBN6126D6rjiL2QAwAznbUSg0Fba2RnSH2V4sA==", + "license": "MIT", + "dependencies": { + "d3-graphviz": "^2.6.1" + }, + "engines": { + "npm": ">= 8.3" + }, + "peerDependencies": { + "react": ">= 16.13.1" + } + }, "node_modules/has": { "version": "1.0.3", "resolved": "https://registry.npmjs.org/has/-/has-1.0.3.tgz", @@ -11296,6 +11417,13 @@ "d3-timer": "^3.0.1" } }, + "node_modules/viz.js": { + "version": "1.8.2", + "resolved": "https://registry.npmjs.org/viz.js/-/viz.js-1.8.2.tgz", + "integrity": "sha512-W+1+N/hdzLpQZEcvz79n2IgUE9pfx6JLdHh3Kh8RGvLL8P1LdJVQmi2OsDcLdY4QVID4OUy+FPelyerX0nJxIQ==", + "deprecated": "2.x is no longer supported, 3.x published as @viz-js/viz", + "license": "MIT" + }, "node_modules/w3c-hr-time": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/w3c-hr-time/-/w3c-hr-time-1.0.2.tgz", @@ -15163,6 +15291,101 @@ "d3-array": "2.5.0 - 3" } }, + "d3-graphviz": { + "version": "2.6.1", + "resolved": "https://registry.npmjs.org/d3-graphviz/-/d3-graphviz-2.6.1.tgz", + "integrity": "sha512-878AFSagQyr5tTOrM7YiVYeUC2/NoFcOB3/oew+LAML0xekyJSw9j3WOCUMBsc95KYe9XBYZ+SKKuObVya1tJQ==", + "requires": { + "d3-dispatch": "^1.0.3", + "d3-format": "^1.2.0", + "d3-interpolate": "^1.1.5", + "d3-path": "^1.0.5", + "d3-selection": "^1.1.0", + "d3-timer": "^1.0.6", + "d3-transition": "^1.1.1", + "d3-zoom": "^1.5.0", + "viz.js": "^1.8.2" + }, + "dependencies": { + "d3-color": { + "version": "1.4.1", + "resolved": "https://registry.npmjs.org/d3-color/-/d3-color-1.4.1.tgz", + "integrity": "sha512-p2sTHSLCJI2QKunbGb7ocOh7DgTAn8IrLx21QRc/BSnodXM4sv6aLQlnfpvehFMLZEfBc6g9pH9SWQccFYfJ9Q==" + }, + "d3-dispatch": { + "version": "1.0.6", + "resolved": "https://registry.npmjs.org/d3-dispatch/-/d3-dispatch-1.0.6.tgz", + "integrity": "sha512-fVjoElzjhCEy+Hbn8KygnmMS7Or0a9sI2UzGwoB7cCtvI1XpVN9GpoYlnb3xt2YV66oXYb1fLJ8GMvP4hdU1RA==" + }, + "d3-drag": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/d3-drag/-/d3-drag-1.2.5.tgz", + "integrity": "sha512-rD1ohlkKQwMZYkQlYVCrSFxsWPzI97+W+PaEIBNTMxRuxz9RF0Hi5nJWHGVJ3Om9d2fRTe1yOBINJyy/ahV95w==", + "requires": { + "d3-dispatch": "1", + "d3-selection": "1" + } + }, + "d3-ease": { + "version": "1.0.7", + "resolved": "https://registry.npmjs.org/d3-ease/-/d3-ease-1.0.7.tgz", + "integrity": "sha512-lx14ZPYkhNx0s/2HX5sLFUI3mbasHjSSpwO/KaaNACweVwxUruKyWVcb293wMv1RqTPZyZ8kSZ2NogUZNcLOFQ==" + }, + "d3-format": { + "version": "1.4.5", + "resolved": "https://registry.npmjs.org/d3-format/-/d3-format-1.4.5.tgz", + "integrity": "sha512-J0piedu6Z8iB6TbIGfZgDzfXxUFN3qQRMofy2oPdXzQibYGqPB/9iMcxr/TGalU+2RsyDO+U4f33id8tbnSRMQ==" + }, + "d3-interpolate": { + "version": "1.4.0", + "resolved": "https://registry.npmjs.org/d3-interpolate/-/d3-interpolate-1.4.0.tgz", + "integrity": "sha512-V9znK0zc3jOPV4VD2zZn0sDhZU3WAE2bmlxdIwwQPPzPjvyLkd8B3JUVdS1IDUFDkWZ72c9qnv1GK2ZagTZ8EA==", + "requires": { + "d3-color": "1" + } + }, + "d3-path": { + "version": "1.0.9", + "resolved": "https://registry.npmjs.org/d3-path/-/d3-path-1.0.9.tgz", + "integrity": "sha512-VLaYcn81dtHVTjEHd8B+pbe9yHWpXKZUC87PzoFmsFrJqgFwDe/qxfp5MlfsfM1V5E/iVt0MmEbWQ7FVIXh/bg==" + }, + "d3-selection": { + "version": "1.4.2", + "resolved": "https://registry.npmjs.org/d3-selection/-/d3-selection-1.4.2.tgz", + "integrity": "sha512-SJ0BqYihzOjDnnlfyeHT0e30k0K1+5sR3d5fNueCNeuhZTnGw4M4o8mqJchSwgKMXCNFo+e2VTChiSJ0vYtXkg==" + }, + "d3-timer": { + "version": "1.0.10", + "resolved": "https://registry.npmjs.org/d3-timer/-/d3-timer-1.0.10.tgz", + "integrity": "sha512-B1JDm0XDaQC+uvo4DT79H0XmBskgS3l6Ve+1SBCfxgmtIb1AVrPIoqd+nPSv+loMX8szQ0sVUhGngL7D5QPiXw==" + }, + "d3-transition": { + "version": "1.3.2", + "resolved": "https://registry.npmjs.org/d3-transition/-/d3-transition-1.3.2.tgz", + "integrity": "sha512-sc0gRU4PFqZ47lPVHloMn9tlPcv8jxgOQg+0zjhfZXMQuvppjG6YuwdMBE0TuqCZjeJkLecku/l9R0JPcRhaDA==", + "requires": { + "d3-color": "1", + "d3-dispatch": "1", + "d3-ease": "1", + "d3-interpolate": "1", + "d3-selection": "^1.1.0", + "d3-timer": "1" + } + }, + "d3-zoom": { + "version": "1.8.3", + "resolved": "https://registry.npmjs.org/d3-zoom/-/d3-zoom-1.8.3.tgz", + "integrity": "sha512-VoLXTK4wvy1a0JpH2Il+F2CiOhVu7VRXWF5M/LroMIh3/zBAC3WAt7QoIvPibOavVo20hN6/37vwAsdBejLyKQ==", + "requires": { + "d3-dispatch": "1", + "d3-drag": "1", + "d3-interpolate": "1", + "d3-selection": "1", + "d3-transition": "1" + } + } + } + }, "d3-hierarchy": { "version": "3.1.2", "resolved": "https://registry.npmjs.org/d3-hierarchy/-/d3-hierarchy-3.1.2.tgz", @@ -16848,6 +17071,14 @@ "integrity": "sha512-EtKwoO6kxCL9WO5xipiHTZlSzBm7WLT627TqC/uVRd0HKmq8NXyebnNYxDoBi7wt8eTWrUrKXCOVaFq9x1kgag==", "dev": true }, + "graphviz-react": { + "version": "1.2.5", + "resolved": "https://registry.npmjs.org/graphviz-react/-/graphviz-react-1.2.5.tgz", + "integrity": "sha512-IRFDzEt09hRzfqrrvAW1PAPBqG4t8hykArcoxq7UhEqO5RUKCBN6126D6rjiL2QAwAznbUSg0Fba2RnSH2V4sA==", + "requires": { + "d3-graphviz": "^2.6.1" + } + }, "has": { "version": "1.0.3", "resolved": "https://registry.npmjs.org/has/-/has-1.0.3.tgz", @@ -20003,6 +20234,11 @@ "d3-timer": "^3.0.1" } }, + "viz.js": { + "version": "1.8.2", + "resolved": "https://registry.npmjs.org/viz.js/-/viz.js-1.8.2.tgz", + "integrity": "sha512-W+1+N/hdzLpQZEcvz79n2IgUE9pfx6JLdHh3Kh8RGvLL8P1LdJVQmi2OsDcLdY4QVID4OUy+FPelyerX0nJxIQ==" + }, "w3c-hr-time": { "version": "1.0.2", "resolved": "https://registry.npmjs.org/w3c-hr-time/-/w3c-hr-time-1.0.2.tgz", diff --git a/dashboard/package.json b/dashboard/package.json index 66cdc9477fad6..e31a5d0875cbe 100644 --- a/dashboard/package.json +++ b/dashboard/package.json @@ -34,6 +34,7 @@ "d3-selection": "^3.0.0", "fabric": "^5.2.1", "framer-motion": "^6.5.1", + "graphviz-react": "^1.2.5", "lodash": "^4.17.21", "next": "^14.2.12", "nuqs": "^1.14.1", diff --git a/dashboard/pages/explain_distsql.tsx b/dashboard/pages/explain_distsql.tsx index 4cd21b1c91584..d7bec3be03508 100644 --- a/dashboard/pages/explain_distsql.tsx +++ b/dashboard/pages/explain_distsql.tsx @@ -1,38 +1,10 @@ -/* - * Copyright 2024 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. - * - */ - import { Box, Button, Stack, Textarea } from "@chakra-ui/react" -import { Fragment } from "react" -import Title from "../components/Title" - -import React, { useState } from "react" - -import ReactFlow, { - Background, - Controls, - Edge, - MiniMap, - Node, -} from "react-flow-renderer" +import { Graphviz } from "graphviz-react" +import { Fragment, SetStateAction, useState } from "react" import styled from "styled-components" +import Title from "../components/Title" import NodeType from "./node" -import * as d3 from "d3" - const ContainerDiv = styled(Box)` font-family: sans-serif; text-align: left; @@ -50,226 +22,52 @@ const position = { const nodeTypes = { node: NodeType } -function getColor() { - return ( - "hsl(" + - 360 * Math.random() + - "," + - (25 + 70 * Math.random()) + - "%," + - (85 + 10 * Math.random()) + - "%)" - ) -} - -function getStyle() { - return { - background: `linear-gradient(${getColor()}, white, white)`, - height: 50, - width: 150, - border: "0.5px solid black", - padding: "5px", - "border-radius": "5px", - } -} - -function layoutElements( - nodeList: any, - edgeList: any, - stageToNode: { [key: string]: number }, - rootStageId: string -) { - const idToNode = new Map() - nodeList.forEach((node: { id: any }) => { - idToNode.set(node.id, [{ id: node.id, children: [] }, node]) - }) - - edgeList.forEach((edge: any) => { - const sourceNode = idToNode.get(edge.source)[0] - const targetNode = idToNode.get(edge.target)[0] - sourceNode.children.push(targetNode) - }) - - var rootNode = idToNode.get(stageToNode[rootStageId].toString())[0] - var root = d3.hierarchy(rootNode) - var tree = d3.tree().nodeSize([60, 180]) - const treeRoot = tree(root) - - treeRoot.each((treeNode: { x: number; y: number; data: any }) => { - const node = idToNode.get(treeNode.data.id)[1] - if (node == undefined) return - node.position = { - x: treeNode.y, - y: treeNode.x, - } - }) -} - -function parseSubElements( - root: any, - stage: string, - style: any, - nodeList: any, - edgeList: any, - visited: Set, - nodeStagePairs: number[][] -) { - if (root.children.length == 0) return - for (var i = 0; i < root.children.length; i++) { - const child = root.children[i] - var edge = { - id: `e${root.plan_node_id}-${child.plan_node_id}`, - source: root.plan_node_id.toString(), - target: child.plan_node_id.toString(), - type: "smoothstep", - } - edgeList.push(edge) - if (visited.has(child.plan_node_id)) continue - var node = { - id: child.plan_node_id.toString(), - data: { - label: `#${child.plan_node_id} ${child.plan_node_type}`, - stage: stage, - content: Object.values(child.schema), - }, - position: position, - type: "node", - style: style, - } - if (child.source_stage_id != null) { - nodeStagePairs.push([child.plan_node_id, child.source_stage_id]) - } - parseSubElements( - child, - stage, - style, - nodeList, - edgeList, - visited, - nodeStagePairs - ) - nodeList.push(node) - } -} - -type PlanNode = { - plan_node_id: number - plan_node_type: string - schema: [any] - children: [PlanNode] - source_stage_id: number -} - -type Stage = { - root: PlanNode - children: [number] - source_stage_id: number -} - -function parseElements(input: any) { - var nodeList: Node[] = [] - var edgeList: Edge[] = [] - var stages: { [key: number]: Stage } = input.stages - var visited: Set = new Set() - var stageToNode: { [key: string]: number } = {} - var nodeStagePairs: number[][] = [] - - const rootStageId = input.root_stage_id.toString() - for (const [key, value] of Object.entries(stages)) { - const root: PlanNode = value.root - stageToNode[key] = root.plan_node_id - var style = getStyle() - var node = { - id: root.plan_node_id.toString(), - data: { - label: `#${root.plan_node_id} ${root.plan_node_type}`, - stage: key, - content: Object.values(root.schema), - }, - position: position, - type: "node", - style: style, - } - if (root.source_stage_id != null) { - nodeStagePairs.push([root.plan_node_id, root.source_stage_id]) - } - visited.add(node.id) - parseSubElements( - root, - key, - style, - nodeList, - edgeList, - visited, - nodeStagePairs - ) - nodeList.push(node) - } - for (var i = 0; i < nodeStagePairs.length; i++) { - var source = nodeStagePairs[i][0] - var target = stageToNode[nodeStagePairs[i][1].toString()] - var edge = { - id: `e${target}-${source}`, - source: source.toString(), - target: target.toString(), - type: "smoothstep", - } - edgeList.push(edge) - } - - layoutElements(nodeList, edgeList, stageToNode, rootStageId) - return { node: nodeList, edge: edgeList } -} - export default function Explain() { const [input, setInput] = useState("") const [isUpdate, setIsUpdate] = useState(false) - const [nodes, setNodes] = useState([]) - const [edges, setEdges] = useState([]) + const [isDotParsed, setIsDotParsed] = useState(false) const handleChange = (event: { - target: { value: React.SetStateAction } + target: { value: SetStateAction } }) => { setInput(event.target.value) setIsUpdate(true) } + const handleClick = () => { if (!isUpdate) return - const jsonInput = JSON.parse(input) - var elements = parseElements(jsonInput) - setEdges(elements.edge) - setNodes(elements.node) - setIsUpdate(false) + setIsDotParsed(true) } return ( - Distributed Plan Explain + Render Graphviz Dot format