diff --git a/.typos.toml b/.typos.toml index 4b6af71b5ddf6..001a8f9d276ec 100644 --- a/.typos.toml +++ b/.typos.toml @@ -33,7 +33,7 @@ extend-exclude = [ "src/sqlparser/tests/testdata/", "src/frontend/planner_test/tests/testdata", "src/tests/sqlsmith/tests/freeze", - "src/license/src/manager.rs", + "src/license/**/*.rs", # JWT license key "Cargo.lock", "**/Cargo.toml", "**/go.mod", diff --git a/Cargo.lock b/Cargo.lock index 73178ca6ba387..6f18a4607ad36 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -1904,7 +1904,7 @@ dependencies = [ "bitflags 2.8.0", "cexpr", "clang-sys", - "itertools 0.10.5", + "itertools 0.12.1", "lazy_static", "lazycell", "log", @@ -1927,7 +1927,7 @@ dependencies = [ "bitflags 2.8.0", "cexpr", "clang-sys", - "itertools 0.10.5", + "itertools 0.13.0", "proc-macro2", "quote", "regex", @@ -2724,9 +2724,9 @@ dependencies = [ [[package]] name = "const-str" -version = "0.5.6" +version = "0.6.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "aca749d3d3f5b87a0d6100509879f9cf486ab510803a4a4e1001da1ff61c2bd6" +checksum = "671927c085eb5827d30b95df08f6c6a2301eafe2274c368bb2c16f42e03547eb" [[package]] name = "constant_time_eq" @@ -5090,9 +5090,9 @@ dependencies = [ [[package]] name = "fs-err" -version = "3.0.0" +version = "3.1.0" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "8bb60e7409f34ef959985bc9d9c5ee8f5db24ee46ed9775850548021710f807f" +checksum = "1f89bda4c2a21204059a977ed3bfe746677dfd137b83c339e702b0ac91d482aa" dependencies = [ "autocfg", ] @@ -9466,7 +9466,7 @@ source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "81bddcdb20abf9501610992b6759a4c888aef7d1a7247ef75e2404275ac24af1" dependencies = [ "anyhow", - "itertools 0.10.5", + "itertools 0.12.1", "proc-macro2", "quote", "syn 2.0.87", diff --git a/README.md b/README.md index 96bc0f35c7ddf..139feeb6f0381 100644 --- a/README.md +++ b/README.md @@ -9,25 +9,18 @@
-### 🌊 Reimagine real-time data engineering. +### 🌊 Ride the Wave of Real-Time Data.
- -

   ðŸ“š  - Documentation   ðŸš€  - - Slack Community - +

+ Docs | Benchmarks | Demos

+ +

+

Slack @@ -46,11 +39,11 @@
-RisingWave is a Postgres-compatible SQL database engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. +RisingWave is the world's most advanced event stream processing platform engineered to provide the simplest and most cost-efficient approach for processing, analyzing, and managing real-time event streaming data. It provides both a Postgres-compatible [SQL interface](https://docs.risingwave.com/sql/overview) and a DataFrame-style [Python interface](https://docs.risingwave.com/python-sdk/intro). -RisingWave can ingest millions of events per second, continuously join and analyze live data streams with historical tables, serve ad-hoc queries in real-time, and deliver fresh, consistent results wherever needed. +RisingWave can ingest millions of events per second, continuously join and analyze live data streams with historical tables, serve ad-hoc queries at low latency, and deliver fresh, consistent results wherever needed. -![RisingWave](./docs/dev/src/images/architecture_20240908.png) +![RisingWave](./docs/dev/src/images/architecture_20250127.png) ## Try it out in 60 seconds @@ -61,14 +54,28 @@ curl -L https://risingwave.com/sh | sh To learn about other installation options, such as using a Docker image, see [Quick Start](https://docs.risingwave.com/docs/current/get-started/). -## When is RisingWave the perfect fit? -RisingWave is the ideal solution for: +## What Is RisingWave Optimized For? +RisingWave simplifies the development of real-time data pipelines and applications. It is purpose-built to: + +* **Ingestion**: Ingest millions of events per second from both streaming and batch data sources. +* **Stream processing**: Perform real-time incremental data processing to join and analyze live data streams with historical tables. +* **Serving**: Persist data and serve ad-hoc queries with single-digit millisecond latency. +* **Delivery**: Deliver fresh, consistent results to data lakes (e.g., Apache Iceberg) or any destination. + + +## Why Is RisingWave Special? +RisingWave stands apart from traditional stream processing systems due to its: + +### PostgreSQL Compatibility +* **Seamless integration**: Works with a wide range of tools in the PostgreSQL ecosystem. +* **Expressive SQL**: Supports structured, semi-structured, and unstructured data using a rich, familiar SQL dialect. +* **No manual state tuning**: Eliminates the need for complex state management configurations. + +### Decoupled Compute and Storage +* **Optimized for complex queries**: Ensures high performance, especially for complex operations like joins and time windowing. +* **Fast failure recovery**: Recovers from system crashes within seconds. +* **Dynamic scaling**: Adjusts resources instantly to handle workload spikes. -* Managing real-time data sources like Kafka streams, database CDC, and more. -* Executing complex, on-the-fly queries, including joins, aggregations, and time windowing. -* Interactively and concurrently exploring consistent, up-to-the-moment results. -* Seamlessly delivering results to downstream systems. -* Processing both streaming and batch data with a unified codebase. ## In what use cases does RisingWave excel? diff --git a/ci/workflows/main-cron.yml b/ci/workflows/main-cron.yml index 8fcc035500398..f977c65a056ad 100644 --- a/ci/workflows/main-cron.yml +++ b/ci/workflows/main-cron.yml @@ -144,7 +144,7 @@ steps: files: "*-junit.xml" format: "junit" - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 13 + timeout_in_minutes: 18 retry: *auto-retry - label: "end-to-end test (parallel, in-memory) (release)" @@ -160,7 +160,7 @@ steps: plugins: - docker-compose#v5.5.0: *docker-compose - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 15 + timeout_in_minutes: 20 retry: *auto-retry - group: "end-to-end connector test (release)" @@ -342,7 +342,7 @@ steps: - label: "end-to-end test (madsim)" key: "e2e-test-deterministic" - command: "TEST_NUM=32 timeout 130m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=32 timeout 135m 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" @@ -357,12 +357,12 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 135 + timeout_in_minutes: 140 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 130m ci/scripts/deterministic-e2e-test.sh" + command: "TEST_NUM=32 RW_SIM_RANDOM_VNODE_COUNT=true timeout 135m 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" @@ -377,12 +377,12 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 135 + timeout_in_minutes: 140 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 70m ci/scripts/deterministic-recovery-test.sh" + command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.0 timeout 75m 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" @@ -392,13 +392,13 @@ steps: - docker-compose#v5.5.0: *docker-compose # Only upload zipped files, otherwise the logs is too much. - ./ci/plugins/upload-failure-logs-zipped - timeout_in_minutes: 75 + timeout_in_minutes: 80 retry: *auto-retry # Ddl statements will randomly run with background_ddl. - label: "background_ddl, arrangement_backfill recovery test (madsim)" key: "background-ddl-arrangement-backfill-recovery-test-deterministic" - command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.8 USE_ARRANGEMENT_BACKFILL=true timeout 70m ci/scripts/deterministic-recovery-test.sh" + command: "TEST_NUM=12 KILL_RATE=1.0 BACKGROUND_DDL_RATE=0.8 USE_ARRANGEMENT_BACKFILL=true timeout 75m 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" @@ -408,7 +408,7 @@ steps: - docker-compose#v5.5.0: *docker-compose # Only upload zipped files, otherwise the logs is too much. - ./ci/plugins/upload-failure-logs-zipped - timeout_in_minutes: 70 + timeout_in_minutes: 80 retry: *auto-retry - label: "end-to-end iceberg sink test (release)" diff --git a/ci/workflows/pull-request.yml b/ci/workflows/pull-request.yml index 19fda2f2d99d5..1b2adcbac28b7 100644 --- a/ci/workflows/pull-request.yml +++ b/ci/workflows/pull-request.yml @@ -160,7 +160,7 @@ steps: plugins: - docker-compose#v5.5.0: *docker-compose - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 25 + timeout_in_minutes: 30 retry: *auto-retry - label: "end-to-end test for opendal (parallel)" @@ -562,7 +562,7 @@ steps: environment: - GITHUB_TOKEN - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 30 + timeout_in_minutes: 35 cancel_on_build_failing: true retry: *auto-retry @@ -583,7 +583,7 @@ steps: # - test-collector#v1.0.0: # files: "*-junit.xml" # format: "junit" - timeout_in_minutes: 35 + timeout_in_minutes: 40 cancel_on_build_failing: true retry: *auto-retry @@ -748,7 +748,7 @@ steps: run: ci-standard-env propagate-environment: true - ./ci/plugins/upload-failure-logs - timeout_in_minutes: 32 + timeout_in_minutes: 37 retry: *auto-retry # FIXME(kwannoel): Let the github PR labeller label it, if sqlsmith source files has changes. diff --git a/docs/dev/src/images/architecture_20250127.png b/docs/dev/src/images/architecture_20250127.png new file mode 100644 index 0000000000000..703a38e83f356 Binary files /dev/null and b/docs/dev/src/images/architecture_20250127.png differ diff --git a/e2e_test/batch/join/asof_join.slt b/e2e_test/batch/join/asof_join.slt new file mode 100644 index 0000000000000..bf905b661e107 --- /dev/null +++ b/e2e_test/batch/join/asof_join.slt @@ -0,0 +1,43 @@ +statement ok +SET RW_IMPLICIT_FLUSH TO true; + +statement ok +create table t1 (v1 int, v2 int, v3 int primary key); + +statement ok +create table t2 (v1 int, v2 int, v3 int primary key); + +statement ok +insert into t1 values (1, 2, 3), (2, 3, 4), (1, 2, 9); + +statement ok +insert into t2 values (1, NULL, 8), (1, 3, 4), (1, 2, 5), (1, 2, 6); + +# asof inner join +query IIIIII +SELECT t1.v1 t1_v1, t1.v2 t1_v2, t1.v3 t1_v3, t2.v1 t2_v1, t2.v2 t2_v2, t2.v3 t2_v3 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 order by t1.v1, t1.v3; +---- +1 2 3 1 3 4 +1 2 9 1 3 4 + +# asof left join +query IIIIII +SELECT t1.v1 t1_v1, t1.v2 t1_v2, t1.v3 t1_v3, t2.v1 t2_v1, t2.v2 t2_v2, t2.v3 t2_v3 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 order by t1.v1, t1.v3; +---- +1 2 3 1 3 4 +1 2 9 1 3 4 +2 3 4 NULL NULL NULL + +# asof left join +query IIIIII +SELECT t1.v1 t1_v1, t1.v2 t1_v2, t1.v3 t1_v3, t2.v1 t2_v1, t2.v2 t2_v2, t2.v3 t2_v3 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 > t2.v2 order by t1.v1, t1.v3; +---- +1 2 3 NULL NULL NULL +1 2 9 NULL NULL NULL +2 3 4 NULL NULL NULL + +statement ok +drop table t1; + +statement ok +drop table t2; \ No newline at end of file diff --git a/e2e_test/ddl/alter_table_column_issue_17121.slt b/e2e_test/ddl/alter_table_column_issue_17121.slt new file mode 100644 index 0000000000000..d96c030d0a20c --- /dev/null +++ b/e2e_test/ddl/alter_table_column_issue_17121.slt @@ -0,0 +1,41 @@ +# https://github.com/risingwavelabs/risingwave/issues/17121 + +statement ok +create table t(v int); + +statement ok +insert into t values (1); + +statement ok +alter table t add column now1 timestamptz default now(); + +# Epoch (then `now()`) will advance after `FLUSH`. +statement ok +flush; + +statement ok +insert into t values (2); + +statement ok +flush; + +query I +select v from t order by now1; +---- +1 +2 + +# Add a new column again, causing the table to be replaced again. +statement ok +alter table t add column v2 varchar; + +# We show that the "snapshot value" of `now1` does not get refreshed upon the above `ALTER TABLE`. +# Otherwise, the `now1` column of `v=1` would be greater than that of `v=2`. +query I +select v from t order by now1; +---- +1 +2 + +statement ok +drop table t; diff --git a/e2e_test/s3/file_sink.py b/e2e_test/s3/file_sink.py index 6eca0e2b9194f..1979b7f6606ab 100644 --- a/e2e_test/s3/file_sink.py +++ b/e2e_test/s3/file_sink.py @@ -17,6 +17,12 @@ def gen_data(file_num, item_num_per_file): assert item_num_per_file % 2 == 0, \ f'item_num_per_file should be even to ensure sum(mark) == 0: {item_num_per_file}' + + struct_type = pa.struct([ + ('field1', pa.int32()), + ('field2', pa.string()) + ]) + return [ [{ 'id': file_id * item_num_per_file + item_id, @@ -44,6 +50,7 @@ def gen_data(file_num, item_num_per_file): 'test_timestamptz_ms': pa.scalar(datetime.now().timestamp() * 1000, type=pa.timestamp('ms', tz='+00:00')), 'test_timestamptz_us': pa.scalar(datetime.now().timestamp() * 1000000, type=pa.timestamp('us', tz='+00:00')), 'test_timestamptz_ns': pa.scalar(datetime.now().timestamp() * 1000000000, type=pa.timestamp('ns', tz='+00:00')), + 'nested_struct': pa.scalar((item_id, f'struct_value_{item_id}'), type=struct_type), } for item_id in range(item_num_per_file)] for file_id in range(file_num) ] @@ -65,7 +72,7 @@ def _table(): print("test table function file scan") cur.execute(f''' SELECT - id, + id, name, sex, mark, @@ -89,7 +96,8 @@ def _table(): test_timestamptz_s, test_timestamptz_ms, test_timestamptz_us, - test_timestamptz_ns + test_timestamptz_ns, + nested_struct FROM file_scan( 'parquet', 's3', @@ -104,7 +112,6 @@ def _table(): except ValueError as e: print(f"cur.fetchone() got ValueError: {e}") - print("file scan test pass") # Execute a SELECT statement cur.execute(f'''CREATE TABLE {_table()}( @@ -132,8 +139,8 @@ def _table(): test_timestamptz_s timestamptz, test_timestamptz_ms timestamptz, test_timestamptz_us timestamptz, - test_timestamptz_ns timestamptz - + test_timestamptz_ns timestamptz, + nested_struct STRUCT<"field1" int, "field2" varchar> ) WITH ( connector = 's3', match_pattern = '*.parquet', @@ -213,7 +220,8 @@ def _table(): test_timestamptz_s, test_timestamptz_ms, test_timestamptz_us, - test_timestamptz_ns + test_timestamptz_ns, + nested_struct from {_table()} WITH ( connector = 's3', match_pattern = '*.parquet', @@ -230,7 +238,7 @@ def _table(): print('Sink into s3 in parquet encode...') # Execute a SELECT statement cur.execute(f'''CREATE TABLE test_parquet_sink_table( - id bigint primary key,\ + id bigint primary key, name TEXT, sex bigint, mark bigint, @@ -254,7 +262,8 @@ def _table(): test_timestamptz_s timestamptz, test_timestamptz_ms timestamptz, test_timestamptz_us timestamptz, - test_timestamptz_ns timestamptz + test_timestamptz_ns timestamptz, + nested_struct STRUCT<"field1" int, "field2" varchar>, ) WITH ( connector = 's3', match_pattern = 'test_parquet_sink/*.parquet', @@ -263,8 +272,8 @@ def _table(): s3.credentials.access = 'hummockadmin', s3.credentials.secret = 'hummockadmin', s3.endpoint_url = 'http://hummock001.127.0.0.1:9301', + refresh.interval.sec = 1, ) FORMAT PLAIN ENCODE PARQUET;''') - total_rows = file_num * item_num_per_file MAX_RETRIES = 40 for retry_no in range(MAX_RETRIES): @@ -305,7 +314,8 @@ def _table(): test_timestamptz_s, test_timestamptz_ms, test_timestamptz_us, - test_timestamptz_ns + test_timestamptz_ns, + nested_struct from {_table()} WITH ( connector = 'snowflake', match_pattern = '*.parquet', @@ -316,7 +326,8 @@ def _table(): s3.endpoint_url = 'http://hummock001.127.0.0.1:9301', s3.path = 'test_json_sink/', type = 'append-only', - force_append_only='true' + force_append_only='true', + refresh.interval.sec = 1, ) FORMAT PLAIN ENCODE JSON(force_append_only='true');''') print('Sink into s3 in json encode...') @@ -346,7 +357,8 @@ def _table(): test_timestamptz_s timestamptz, test_timestamptz_ms timestamptz, test_timestamptz_us timestamptz, - test_timestamptz_ns timestamptz + test_timestamptz_ns timestamptz, + nested_struct STRUCT<"field1" int, "field2" varchar> ) WITH ( connector = 's3', match_pattern = 'test_json_sink/*.json', diff --git a/e2e_test/source_inline/kafka/shared_source.slt.serial b/e2e_test/source_inline/kafka/shared_source.slt.serial index 42d4cf86725ef..1559962f8b878 100644 --- a/e2e_test/source_inline/kafka/shared_source.slt.serial +++ b/e2e_test/source_inline/kafka/shared_source.slt.serial @@ -99,6 +99,10 @@ SET streaming_use_shared_source TO false; statement ok create materialized view mv_2 as select * from s0; +statement ok +SET streaming_use_shared_source TO true; + + sleep 2s query ?? rowsort @@ -370,3 +374,32 @@ drop source s0 cascade; statement ok drop source s_before_produce cascade; + +# test: scan.startup.mode=latest should not be blocked when there's no data to backfill +# https://github.com/risingwavelabs/risingwave/issues/20083#issuecomment-2609422824 +statement ok +create source s_latest (v1 int, v2 varchar) with ( + ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, + topic = 'shared_source', + scan.startup.mode = 'latest' +) FORMAT PLAIN ENCODE JSON; + +# Note: batch kafka scan ignores scan.startup.mode +query ? rowsort +select count(*) from s_latest; +---- +55 + +statement ok +create materialized view mv_latest as select * from s_latest; + +query ? rowsort +select count(*) from mv_latest; +---- +0 + +statement ok +drop source s_latest cascade; + +system ok +rpk topic delete shared_source; diff --git a/e2e_test/streaming/bug_fixes/issue_20342.slt b/e2e_test/streaming/bug_fixes/issue_20342.slt new file mode 100644 index 0000000000000..f36740eb3068a --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_20342.slt @@ -0,0 +1,9 @@ +statement ok +set streaming_use_arrangement_backfill=false; + +include ./issue_20342.slt.part + +statement ok +set streaming_use_arrangement_backfill=true; + +include ./issue_20342.slt.part diff --git a/e2e_test/streaming/bug_fixes/issue_20342.slt.part b/e2e_test/streaming/bug_fixes/issue_20342.slt.part new file mode 100644 index 0000000000000..5c93e0a0434c9 --- /dev/null +++ b/e2e_test/streaming/bug_fixes/issue_20342.slt.part @@ -0,0 +1,99 @@ +statement ok +set streaming_parallelism=1; + +# pk: [v1] +# stream key: [v1] +statement ok +create table t(v1 int primary key, v2 int); + +# pk: [v2, v1] +# stream key: [v1] +statement ok +create materialized view m1 as select * from t order by v2; + +statement ok +insert into t select x as v1, x as v2 from generate_series(1, 10000) t(x); + +statement ok +flush; + +skipif madsim +statement ok +set backfill_rate_limit=1; + +skipif madsim +statement ok +set background_ddl=true; + +statement ok +create materialized view m2 as select count(*) from m1; + +skipif madsim +sleep 2s + +statement ok +update t set v2 = 100000 where v1 = 1; + +statement ok +flush; + +statement ok +update t set v2 = 100001 where v1 = 2; + +statement ok +flush; + +statement ok +update t set v2 = 100002 where v1 = 3; + +statement ok +flush; + +statement ok +update t set v2 = 100003 where v1 = 4; + +statement ok +flush; + +statement ok +update t set v2 = 100004 where v1 = 5; + +statement ok +flush; + +statement ok +update t set v2 = 100005 where v1 = 6; + +statement ok +flush; + +statement ok +update t set v2 = 100006 where v1 = 7; + +statement ok +flush; + +statement ok +update t set v2 = 100007 where v1 = 8; + +statement ok +flush; + +statement ok +set backfill_rate_limit=default; + +statement ok +set background_ddl=false; + +statement ok +set streaming_use_arrangement_backfill=true; + +statement ok +alter materialized view m2 set backfill_rate_limit=default; + +skipif madsim +statement ok +wait; + +statement ok +drop table t cascade; \ No newline at end of file diff --git a/proto/batch_plan.proto b/proto/batch_plan.proto index 7ffdf94e3c30a..6e07ceae4d5d4 100644 --- a/proto/batch_plan.proto +++ b/proto/batch_plan.proto @@ -289,6 +289,7 @@ message HashJoinNode { // Null safe means it treats `null = null` as true. // Each key pair can be null safe independently. (left_key, right_key, null_safe) repeated bool null_safe = 6; + optional plan_common.AsOfJoinDesc asof_desc = 7; } message SortMergeJoinNode { diff --git a/proto/meta.proto b/proto/meta.proto index 975798de90da9..d8351060d72a5 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -471,6 +471,7 @@ message MetaSnapshot { reserved "parallel_unit_mappings"; GetSessionParamsResponse session_params = 20; repeated catalog.Secret secrets = 23; + uint64 compute_node_total_cpu_count = 24; repeated common.WorkerNode nodes = 10; hummock.HummockVersion hummock_version = 12; backup_service.MetaBackupManifestId meta_backup_manifest_id = 14; @@ -540,6 +541,7 @@ message SubscribeResponse { FragmentWorkerSlotMapping streaming_worker_slot_mapping = 27; FragmentWorkerSlotMappings serving_worker_slot_mappings = 28; catalog.Secret secret = 29; + uint64 compute_node_total_cpu_count = 30; } reserved 12; reserved "parallel_unit_mapping"; diff --git a/proto/plan_common.proto b/proto/plan_common.proto index b8b4989e78b8a..5fb77a2aa0659 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -21,9 +21,6 @@ enum ColumnDescVersion { // Introduced in https://github.com/risingwavelabs/risingwave/pull/13707#discussion_r1429947537, // in case DEFAULT_KEY_COLUMN_NAME changes COLUMN_DESC_VERSION_PR_13707 = 1; - - // for test only - COLUMN_DESC_VERSION_MAX = 2147483647; } message ColumnDesc { diff --git a/proto/stream_plan.proto b/proto/stream_plan.proto index 3963e3d03e4c3..9b886e3cef1b3 100644 --- a/proto/stream_plan.proto +++ b/proto/stream_plan.proto @@ -356,9 +356,6 @@ enum AggNodeVersion { // https://github.com/risingwavelabs/risingwave/issues/13465#issuecomment-1821016508 AGG_NODE_VERSION_ISSUE_13465 = 2; - - // Used for test only. - AGG_NODE_VERSION_MAX = 2147483647; } message SimpleAggNode { diff --git a/src/batch/executors/benches/hash_join.rs b/src/batch/executors/benches/hash_join.rs index 330fc299594d0..6d64461dd1c2d 100644 --- a/src/batch/executors/benches/hash_join.rs +++ b/src/batch/executors/benches/hash_join.rs @@ -76,6 +76,7 @@ fn create_hash_join_executor( "HashJoinExecutor".into(), CHUNK_SIZE, None, + None, BatchSpillMetrics::for_test(), ShutdownToken::empty(), MemoryContext::none(), diff --git a/src/batch/executors/src/executor/join/hash_join.rs b/src/batch/executors/src/executor/join/hash_join.rs index 9c31e945723f5..44518e5155496 100644 --- a/src/batch/executors/src/executor/join/hash_join.rs +++ b/src/batch/executors/src/executor/join/hash_join.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::Ordering; use std::iter; use std::iter::empty; use std::marker::PhantomData; @@ -25,8 +26,8 @@ use risingwave_common::bitmap::{Bitmap, BitmapBuilder, FilterByBitmap}; use risingwave_common::catalog::Schema; use risingwave_common::hash::{HashKey, HashKeyDispatcher, PrecomputedBuildHasher}; use risingwave_common::memory::{MemoryContext, MonitoredGlobalAlloc}; -use risingwave_common::row::{repeat_n, RowExt}; -use risingwave_common::types::{DataType, Datum}; +use risingwave_common::row::{repeat_n, Row, RowExt}; +use risingwave_common::types::{DataType, Datum, DefaultOrd}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common_estimate_size::EstimateSize; @@ -35,7 +36,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::data::DataChunk as PbDataChunk; use risingwave_pb::Message; -use super::{ChunkedData, JoinType, RowId}; +use super::{AsOfDesc, AsOfInequalityType, ChunkedData, JoinType, RowId}; use crate::error::{BatchError, Result}; use crate::executor::{ BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder, @@ -83,6 +84,8 @@ pub struct HashJoinExecutor { null_matched: Vec, identity: String, chunk_size: usize, + /// Whether the join is an as-of join + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, @@ -179,6 +182,7 @@ pub struct EquiJoinParams { next_build_row_with_same_key: ChunkedData>, chunk_size: usize, shutdown_rx: ShutdownToken, + asof_desc: Option, } impl EquiJoinParams { @@ -194,6 +198,7 @@ impl EquiJoinParams { next_build_row_with_same_key: ChunkedData>, chunk_size: usize, shutdown_rx: ShutdownToken, + asof_desc: Option, ) -> Self { Self { probe_side, @@ -206,6 +211,7 @@ impl EquiJoinParams { next_build_row_with_same_key, chunk_size, shutdown_rx, + asof_desc, } } } @@ -648,6 +654,7 @@ impl HashJoinExecutor { self.cond.clone(), format!("{}-sub{}", self.identity.clone(), i), self.chunk_size, + self.asof_desc.clone(), self.spill_backend.clone(), self.spill_metrics.clone(), Some(partition_size), @@ -683,9 +690,12 @@ impl HashJoinExecutor { next_build_row_with_same_key, self.chunk_size, self.shutdown_rx.clone(), + self.asof_desc, ); - if let Some(cond) = self.cond.as_ref() { + if let Some(cond) = self.cond.as_ref() + && params.asof_desc.is_none() + { let stream = match self.join_type { JoinType::Inner => Self::do_inner_join_with_non_equi_condition(params, cond), JoinType::LeftOuter => { @@ -709,6 +719,9 @@ impl HashJoinExecutor { JoinType::FullOuter => { Self::do_full_outer_join_with_non_equi_condition(params, cond) } + JoinType::AsOfInner | JoinType::AsOfLeftOuter => { + unreachable!("AsOf join should not reach here") + } }; // For non-equi join, we need an output chunk builder to align the output chunks. let mut output_chunk_builder = @@ -726,8 +739,10 @@ impl HashJoinExecutor { } } else { let stream = match self.join_type { - JoinType::Inner => Self::do_inner_join(params), - JoinType::LeftOuter => Self::do_left_outer_join(params), + JoinType::Inner | JoinType::AsOfInner => Self::do_inner_join(params), + JoinType::LeftOuter | JoinType::AsOfLeftOuter => { + Self::do_left_outer_join(params) + } JoinType::LeftSemi => Self::do_left_semi_anti_join::(params), JoinType::LeftAnti => Self::do_left_semi_anti_join::(params), JoinType::RightOuter => Self::do_right_outer_join(params), @@ -754,6 +769,7 @@ impl HashJoinExecutor { next_build_row_with_same_key, chunk_size, shutdown_rx, + asof_desc, .. }: EquiJoinParams, ) { @@ -767,19 +783,39 @@ impl HashJoinExecutor { .enumerate() .filter_by_bitmap(probe_chunk.visibility()) { - for build_row_id in - next_build_row_with_same_key.row_id_iter(hash_map.get(probe_key).copied()) - { - shutdown_rx.check()?; - let build_chunk = &build_side[build_row_id.chunk_id()]; - if let Some(spilled) = Self::append_one_row( - &mut chunk_builder, - &probe_chunk, - probe_row_id, - build_chunk, - build_row_id.row_id(), + let build_side_row_iter = + next_build_row_with_same_key.row_id_iter(hash_map.get(probe_key).copied()); + if let Some(asof_desc) = &asof_desc { + if let Some(build_row_id) = Self::find_asof_matched_rows( + probe_chunk.row_at_unchecked_vis(probe_row_id), + &build_side, + build_side_row_iter, + asof_desc, ) { - yield spilled + shutdown_rx.check()?; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + &build_side[build_row_id.chunk_id()], + build_row_id.row_id(), + ) { + yield spilled + } + } + } else { + for build_row_id in build_side_row_iter { + shutdown_rx.check()?; + let build_chunk = &build_side[build_row_id.chunk_id()]; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + build_chunk, + build_row_id.row_id(), + ) { + yield spilled + } } } } @@ -814,6 +850,7 @@ impl HashJoinExecutor { next_build_row_with_same_key, chunk_size, shutdown_rx, + asof_desc, .. }: EquiJoinParams, ) { @@ -828,19 +865,49 @@ impl HashJoinExecutor { .filter_by_bitmap(probe_chunk.visibility()) { if let Some(first_matched_build_row_id) = hash_map.get(probe_key) { - for build_row_id in - next_build_row_with_same_key.row_id_iter(Some(*first_matched_build_row_id)) - { - shutdown_rx.check()?; - let build_chunk = &build_side[build_row_id.chunk_id()]; - if let Some(spilled) = Self::append_one_row( - &mut chunk_builder, - &probe_chunk, - probe_row_id, - build_chunk, - build_row_id.row_id(), + let build_side_row_iter = + next_build_row_with_same_key.row_id_iter(Some(*first_matched_build_row_id)); + if let Some(asof_desc) = &asof_desc { + if let Some(build_row_id) = Self::find_asof_matched_rows( + probe_chunk.row_at_unchecked_vis(probe_row_id), + &build_side, + build_side_row_iter, + asof_desc, ) { - yield spilled + shutdown_rx.check()?; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + &build_side[build_row_id.chunk_id()], + build_row_id.row_id(), + ) { + yield spilled + } + } else { + shutdown_rx.check()?; + let probe_row = probe_chunk.row_at_unchecked_vis(probe_row_id); + if let Some(spilled) = Self::append_one_row_with_null_build_side( + &mut chunk_builder, + probe_row, + build_data_types.len(), + ) { + yield spilled + } + } + } else { + for build_row_id in build_side_row_iter { + shutdown_rx.check()?; + let build_chunk = &build_side[build_row_id.chunk_id()]; + if let Some(spilled) = Self::append_one_row( + &mut chunk_builder, + &probe_chunk, + probe_row_id, + build_chunk, + build_row_id.row_id(), + ) { + yield spilled + } } } } else { @@ -1106,7 +1173,6 @@ impl HashJoinExecutor { .enumerate() .filter_by_bitmap(probe_chunk.visibility()) { - non_equi_state.found_matched = false; if let Some(first_matched_build_row_id) = hash_map.get(probe_key) { non_equi_state .first_output_row_id @@ -1917,6 +1983,64 @@ impl HashJoinExecutor { ) -> Option { chunk_builder.append_one_row(repeat_n(Datum::None, probe_column_count).chain(build_row_ref)) } + + fn find_asof_matched_rows( + probe_row_ref: RowRef<'_>, + build_side: &[DataChunk], + build_side_row_iter: RowIdIter<'_>, + asof_join_condition: &AsOfDesc, + ) -> Option { + let probe_inequality_value = probe_row_ref.datum_at(asof_join_condition.left_idx); + if let Some(probe_inequality_scalar) = probe_inequality_value { + let mut result_row_id: Option = None; + let mut build_row_ref; + + for build_row_id in build_side_row_iter { + build_row_ref = + build_side[build_row_id.chunk_id()].row_at_unchecked_vis(build_row_id.row_id()); + let build_inequality_value = build_row_ref.datum_at(asof_join_condition.right_idx); + if let Some(build_inequality_scalar) = build_inequality_value { + let mut pick_result = |compare: fn(Ordering) -> bool| { + if let Some(result_row_id_inner) = result_row_id { + let result_row_ref = build_side[result_row_id_inner.chunk_id()] + .row_at_unchecked_vis(result_row_id_inner.row_id()); + let result_inequality_scalar = result_row_ref + .datum_at(asof_join_condition.right_idx) + .unwrap(); + if compare( + probe_inequality_scalar.default_cmp(&build_inequality_scalar), + ) && compare( + probe_inequality_scalar.default_cmp(&result_inequality_scalar), + ) { + result_row_id = Some(build_row_id); + } + } else if compare( + probe_inequality_scalar.default_cmp(&build_inequality_scalar), + ) { + result_row_id = Some(build_row_id); + } + }; + match asof_join_condition.inequality_type { + AsOfInequalityType::Lt => { + pick_result(Ordering::is_lt); + } + AsOfInequalityType::Le => { + pick_result(Ordering::is_le); + } + AsOfInequalityType::Gt => { + pick_result(Ordering::is_gt); + } + AsOfInequalityType::Ge => { + pick_result(Ordering::is_ge); + } + } + } + } + result_row_id + } else { + None + } + } } /// `DataChunkMutator` transforms the given data chunk for non-equi join. @@ -2030,13 +2154,17 @@ impl DataChunkMutator { break; } } - if ANTI_JOIN && !has_more_output_rows && !*found_matched { - new_visibility.set(start_row_id, true); + if !has_more_output_rows && ANTI_JOIN { + if !*found_matched { + new_visibility.set(start_row_id, true); + } + *found_matched = false; } first_output_row_ids.clear(); - self.0.set_visibility(new_visibility.finish()); + self.0 + .set_visibility(new_visibility.finish() & self.0.visibility()); self } @@ -2058,7 +2186,8 @@ impl DataChunkMutator { build_row_ids.clear(); - self.0.set_visibility(new_visibility.finish()); + self.0 + .set_visibility(new_visibility.finish() & self.0.visibility()); self } @@ -2135,7 +2264,8 @@ impl DataChunkMutator { build_row_ids.clear(); - self.0.set_visibility(new_visibility.finish()); + self.0 + .set_visibility(new_visibility.finish() & self.0.visibility()); self } @@ -2190,6 +2320,11 @@ impl BoxedExecutorBuilder for HashJoinExecutor<()> { let identity = context.plan_node().get_identity().clone(); + let asof_desc = hash_join_node + .asof_desc + .map(|desc| AsOfDesc::from_protobuf(&desc)) + .transpose()?; + Ok(HashJoinExecutorArgs { join_type, output_indices, @@ -2202,6 +2337,7 @@ impl BoxedExecutorBuilder for HashJoinExecutor<()> { identity: identity.clone(), right_key_types, chunk_size: context.context().get_config().developer.chunk_size, + asof_desc, spill_backend: if context.context().get_config().enable_spill { Some(Disk) } else { @@ -2227,6 +2363,7 @@ struct HashJoinExecutorArgs { identity: String, right_key_types: Vec, chunk_size: usize, + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, shutdown_rx: ShutdownToken, @@ -2248,6 +2385,7 @@ impl HashKeyDispatcher for HashJoinExecutorArgs { self.cond.map(Arc::new), self.identity, self.chunk_size, + self.asof_desc, self.spill_backend, self.spill_metrics, self.shutdown_rx, @@ -2273,6 +2411,7 @@ impl HashJoinExecutor { cond: Option>, identity: String, chunk_size: usize, + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, shutdown_rx: ShutdownToken, @@ -2289,6 +2428,7 @@ impl HashJoinExecutor { cond, identity, chunk_size, + asof_desc, spill_backend, spill_metrics, None, @@ -2309,6 +2449,7 @@ impl HashJoinExecutor { cond: Option>, identity: String, chunk_size: usize, + asof_desc: Option, spill_backend: Option, spill_metrics: Arc, memory_upper_bound: Option, @@ -2347,6 +2488,7 @@ impl HashJoinExecutor { cond, identity, chunk_size, + asof_desc, shutdown_rx, spill_backend, spill_metrics, @@ -2627,6 +2769,7 @@ mod tests { cond, "HashJoinExecutor".to_owned(), chunk_size, + None, if test_spill { Some(SpillBackend::Memory) } else { @@ -3458,7 +3601,6 @@ mod tests { &expect )); assert_eq!(state.first_output_row_id, Vec::::new()); - assert!(state.found_matched); } #[tokio::test] @@ -3558,7 +3700,6 @@ mod tests { &expect )); assert_eq!(state.first_output_row_id, Vec::::new()); - assert!(state.found_matched); } #[tokio::test] diff --git a/src/batch/executors/src/executor/join/lookup_join_base.rs b/src/batch/executors/src/executor/join/lookup_join_base.rs index d3f806e01c547..3a8466cbad26e 100644 --- a/src/batch/executors/src/executor/join/lookup_join_base.rs +++ b/src/batch/executors/src/executor/join/lookup_join_base.rs @@ -178,6 +178,7 @@ impl LookupJoinBase { next_build_row_with_same_key, self.chunk_size, self.shutdown_rx.clone(), + None, ); if let Some(cond) = self.condition.as_ref() { @@ -197,7 +198,9 @@ impl LookupJoinBase { JoinType::RightOuter | JoinType::RightSemi | JoinType::RightAnti - | JoinType::FullOuter => unimplemented!(), + | JoinType::FullOuter + | JoinType::AsOfInner + | JoinType::AsOfLeftOuter => unimplemented!(), }; // For non-equi join, we need an output chunk builder to align the output chunks. let mut output_chunk_builder = @@ -222,7 +225,9 @@ impl LookupJoinBase { JoinType::RightOuter | JoinType::RightSemi | JoinType::RightAnti - | JoinType::FullOuter => unimplemented!(), + | JoinType::FullOuter + | JoinType::AsOfInner + | JoinType::AsOfLeftOuter => unimplemented!(), }; #[for_await] for chunk in stream { diff --git a/src/batch/executors/src/executor/join/mod.rs b/src/batch/executors/src/executor/join/mod.rs index 6c404a7cf5b6b..571babc4ca1fb 100644 --- a/src/batch/executors/src/executor/join/mod.rs +++ b/src/batch/executors/src/executor/join/mod.rs @@ -30,7 +30,7 @@ use risingwave_common::array::{DataChunk, RowRef}; use risingwave_common::row::Row; use risingwave_common::types::{DataType, DatumRef}; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_pb::plan_common::JoinType as PbJoinType; +use risingwave_pb::plan_common::{AsOfJoinDesc, AsOfJoinInequalityType, JoinType as PbJoinType}; use crate::error::Result; @@ -49,6 +49,8 @@ pub enum JoinType { /// Anti join when build side should output when matched RightAnti, FullOuter, + AsOfInner, + AsOfLeftOuter, } impl JoinType { @@ -62,7 +64,9 @@ impl JoinType { PbJoinType::RightSemi => JoinType::RightSemi, PbJoinType::RightAnti => JoinType::RightAnti, PbJoinType::FullOuter => JoinType::FullOuter, - PbJoinType::AsofInner | PbJoinType::AsofLeftOuter | PbJoinType::Unspecified => { + PbJoinType::AsofInner => JoinType::AsOfInner, + PbJoinType::AsofLeftOuter => JoinType::AsOfLeftOuter, + PbJoinType::Unspecified => { unreachable!() } } @@ -73,28 +77,6 @@ impl JoinType { impl JoinType { #![allow(dead_code)] - #[inline(always)] - pub(super) fn need_join_remaining(self) -> bool { - matches!( - self, - JoinType::RightOuter | JoinType::RightAnti | JoinType::FullOuter - ) - } - - fn need_build(self) -> bool { - match self { - JoinType::RightSemi => true, - other => other.need_join_remaining(), - } - } - - fn need_probe(self) -> bool { - matches!( - self, - JoinType::FullOuter | JoinType::LeftOuter | JoinType::LeftAnti | JoinType::LeftSemi - ) - } - fn keep_all(self) -> bool { matches!( self, @@ -111,6 +93,40 @@ impl JoinType { } } +#[derive(Clone, Debug)] +pub enum AsOfInequalityType { + Le, + Lt, + Ge, + Gt, +} + +#[derive(Clone, Debug)] +pub struct AsOfDesc { + pub left_idx: usize, + pub right_idx: usize, + pub inequality_type: AsOfInequalityType, +} + +impl AsOfDesc { + pub fn from_protobuf(desc_proto: &AsOfJoinDesc) -> crate::error::Result { + let typ = match desc_proto.inequality_type() { + AsOfJoinInequalityType::AsOfInequalityTypeLt => AsOfInequalityType::Lt, + AsOfJoinInequalityType::AsOfInequalityTypeLe => AsOfInequalityType::Le, + AsOfJoinInequalityType::AsOfInequalityTypeGt => AsOfInequalityType::Gt, + AsOfJoinInequalityType::AsOfInequalityTypeGe => AsOfInequalityType::Ge, + AsOfJoinInequalityType::AsOfInequalityTypeUnspecified => { + bail!("unspecified AsOf join inequality type") + } + }; + Ok(Self { + left_idx: desc_proto.left_idx as usize, + right_idx: desc_proto.right_idx as usize, + inequality_type: typ, + }) + } +} + /// The layout be like: /// /// [ `left` chunk | `right` chunk ] diff --git a/src/batch/executors/src/executor/join/nested_loop_join.rs b/src/batch/executors/src/executor/join/nested_loop_join.rs index 14917646e84c1..19bc46baccf3f 100644 --- a/src/batch/executors/src/executor/join/nested_loop_join.rs +++ b/src/batch/executors/src/executor/join/nested_loop_join.rs @@ -116,6 +116,9 @@ impl NestedLoopJoinExecutor { JoinType::RightSemi => Self::do_right_semi_anti_join::, JoinType::RightAnti => Self::do_right_semi_anti_join::, JoinType::FullOuter => Self::do_full_outer_join, + JoinType::AsOfInner | JoinType::AsOfLeftOuter => { + unimplemented!("AsOf join is not supported in NestedLoopJoinExecutor") + } }; #[for_await] diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index 698ee898109bd..8a0080f59f612 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -27,7 +27,7 @@ ignored = ["workspace-hack", "workspace-config", "task_stats_alloc", "tikv-jemal [dependencies] clap = { workspace = true } console = "0.15" -const-str = "0.5" +const-str = "0.6" home = "0.5" risingwave_batch_executors = { workspace = true } risingwave_cmd = { workspace = true } diff --git a/src/cmd_all/src/bin/risingwave.rs b/src/cmd_all/src/bin/risingwave.rs index a8cd77bf6e21d..ebdeab941a82e 100644 --- a/src/cmd_all/src/bin/risingwave.rs +++ b/src/cmd_all/src/bin/risingwave.rs @@ -122,7 +122,7 @@ impl Component { Self::Frontend => frontend(parse_opts(matches)), Self::Compactor => compactor(parse_opts(matches)), Self::Ctl => ctl(parse_opts(matches)), - Self::Playground => single_node(SingleNodeOpts::new_for_playground()), + Self::Playground => playground(), Self::Standalone => standalone(parse_opts(matches)), Self::SingleNode => single_node(parse_opts(matches)), } @@ -248,6 +248,10 @@ fn single_node(opts: SingleNodeOpts) -> ! { risingwave_rt::main_okk(|shutdown| risingwave_cmd_all::standalone(opts, shutdown)); } +fn playground() -> ! { + single_node(SingleNodeOpts::new_for_playground()); +} + #[cfg(test)] mod tests { use std::assert_matches::assert_matches; diff --git a/src/cmd_all/src/single_node.rs b/src/cmd_all/src/single_node.rs index 0f6f574ad5a8c..5ecc60bff743c 100644 --- a/src/cmd_all/src/single_node.rs +++ b/src/cmd_all/src/single_node.rs @@ -194,14 +194,11 @@ pub fn map_single_node_opts_to_standalone_opts(opts: SingleNodeOpts) -> ParsedSt } // Set listen addresses (force to override) - meta_opts.listen_addr = "0.0.0.0:5690".to_owned(); + meta_opts.listen_addr = "127.0.0.1:5690".to_owned(); meta_opts.advertise_addr = "127.0.0.1:5690".to_owned(); meta_opts.dashboard_host = Some("0.0.0.0:5691".to_owned()); - compute_opts.listen_addr = "0.0.0.0:5688".to_owned(); - compactor_opts.listen_addr = "0.0.0.0:6660".to_owned(); - if let Some(frontend_addr) = &opts.node_opts.listen_addr { - frontend_opts.listen_addr.clone_from(frontend_addr); - } + compute_opts.listen_addr = "127.0.0.1:5688".to_owned(); + compactor_opts.listen_addr = "127.0.0.1:6660".to_owned(); // Set Meta addresses for all nodes (force to override) let meta_addr = "http://127.0.0.1:5690".to_owned(); diff --git a/src/common/common_service/src/observer_manager.rs b/src/common/common_service/src/observer_manager.rs index c1c62abd33443..b56215fe163b8 100644 --- a/src/common/common_service/src/observer_manager.rs +++ b/src/common/common_service/src/observer_manager.rs @@ -122,6 +122,7 @@ where Info::Snapshot(_) | Info::HummockWriteLimits(_) => unreachable!(), Info::HummockStats(_) => true, Info::Recovery(_) => true, + Info::ComputeNodeTotalCpuCount(_) => true, Info::StreamingWorkerSlotMapping(_) => { notification.version > info diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index 146ef76d2ec93..8a1a4a5072d6b 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -137,7 +137,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -152,7 +152,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -193,7 +193,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: additional_column_type, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -213,7 +213,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: Some(system_column), } } @@ -263,7 +263,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -285,7 +285,7 @@ impl ColumnDesc { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } @@ -304,7 +304,7 @@ impl ColumnDesc { description: None, generated_or_default_column: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index ab4afeddea566..15383bd3c3e7c 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -36,7 +36,7 @@ impl ColumnDescTestExt for ColumnDesc { column_id, name: name.to_owned(), additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, ..Default::default() } } @@ -62,7 +62,7 @@ impl ColumnDescTestExt for ColumnDesc { description: None, additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, } } } diff --git a/src/common/src/util/stream_graph_visitor.rs b/src/common/src/util/stream_graph_visitor.rs index 5e579565b7e9f..f14ceadc09c42 100644 --- a/src/common/src/util/stream_graph_visitor.rs +++ b/src/common/src/util/stream_graph_visitor.rs @@ -19,21 +19,11 @@ use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{agg_call_state, StreamNode}; /// A utility for visiting and mutating the [`NodeBody`] of the [`StreamNode`]s recursively. -pub fn visit_stream_node(stream_node: &mut StreamNode, mut f: F) -where - F: FnMut(&mut NodeBody), -{ - fn visit_inner(stream_node: &mut StreamNode, f: &mut F) - where - F: FnMut(&mut NodeBody), - { +pub fn visit_stream_node_mut(stream_node: &mut StreamNode, mut f: impl FnMut(&mut NodeBody)) { + visit_stream_node_cont_mut(stream_node, |stream_node| { f(stream_node.node_body.as_mut().unwrap()); - for input in &mut stream_node.input { - visit_inner(input, f); - } - } - - visit_inner(stream_node, &mut f) + true + }) } /// A utility for to accessing the [`StreamNode`] mutably. The returned bool is used to determine whether the access needs to continue. @@ -56,6 +46,14 @@ where visit_inner(stream_node, &mut f) } +/// A utility for visiting the [`NodeBody`] of the [`StreamNode`]s recursively. +pub fn visit_stream_node(stream_node: &StreamNode, mut f: impl FnMut(&NodeBody)) { + visit_stream_node_cont(stream_node, |stream_node| { + f(stream_node.node_body.as_ref().unwrap()); + true + }) +} + /// A utility for to accessing the [`StreamNode`] immutably. The returned bool is used to determine whether the access needs to continue. pub fn visit_stream_node_cont(stream_node: &StreamNode, mut f: F) where @@ -78,11 +76,14 @@ where /// A utility for visiting and mutating the [`NodeBody`] of the [`StreamNode`]s in a /// [`StreamFragment`] recursively. -pub fn visit_fragment(fragment: &mut StreamFragment, f: F) -where - F: FnMut(&mut NodeBody), -{ - visit_stream_node(fragment.node.as_mut().unwrap(), f) +pub fn visit_fragment_mut(fragment: &mut StreamFragment, f: impl FnMut(&mut NodeBody)) { + visit_stream_node_mut(fragment.node.as_mut().unwrap(), f) +} + +/// A utility for visiting the [`NodeBody`] of the [`StreamNode`]s in a +/// [`StreamFragment`] recursively. +pub fn visit_fragment(fragment: &StreamFragment, f: impl FnMut(&NodeBody)) { + visit_stream_node(fragment.node.as_ref().unwrap(), f) } /// Visit the tables of a [`StreamNode`]. @@ -279,7 +280,7 @@ pub fn visit_stream_node_tables_inner( } }; if visit_child_recursively { - visit_stream_node(stream_node, visit_body) + visit_stream_node_mut(stream_node, visit_body) } else { visit_body(stream_node.node_body.as_mut().unwrap()) } diff --git a/src/compute/src/observer/observer_manager.rs b/src/compute/src/observer/observer_manager.rs index b21e01bf3c3f2..19c273f9bd4f2 100644 --- a/src/compute/src/observer/observer_manager.rs +++ b/src/compute/src/observer/observer_manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::license::LicenseManager; use risingwave_common::secret::LocalSecretManager; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::ObserverState; @@ -45,6 +46,9 @@ impl ObserverState for ComputeObserverNode { panic!("error type notification"); } }, + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } _ => { panic!("error type notification"); } @@ -57,6 +61,7 @@ impl ObserverState for ComputeObserverNode { unreachable!(); }; LocalSecretManager::global().init_secrets(snapshot.secrets); + LicenseManager::get().update_cpu_core_count(snapshot.compute_node_total_cpu_count as _); } } diff --git a/src/connector/codec/src/decoder/avro/schema.rs b/src/connector/codec/src/decoder/avro/schema.rs index 20218fccbb026..8a4cfaa56999f 100644 --- a/src/connector/codec/src/decoder/avro/schema.rs +++ b/src/connector/codec/src/decoder/avro/schema.rs @@ -152,7 +152,7 @@ fn avro_field_to_column_desc( description: None, additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, }) } _ => { @@ -162,7 +162,7 @@ fn avro_field_to_column_desc( column_id: *index, name: name.to_owned(), additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, ..Default::default() }) } diff --git a/src/connector/codec/src/decoder/protobuf/parser.rs b/src/connector/codec/src/decoder/protobuf/parser.rs index 6701d4aaa7a31..b1916a8086f96 100644 --- a/src/connector/codec/src/decoder/protobuf/parser.rs +++ b/src/connector/codec/src/decoder/protobuf/parser.rs @@ -79,7 +79,7 @@ fn pb_field_to_col_desc( description: None, additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, }) } else { *index += 1; @@ -88,7 +88,7 @@ fn pb_field_to_col_desc( name: field_descriptor.name().to_owned(), column_type: Some(field_type.to_protobuf()), additional_column: Some(AdditionalColumn { column_type: None }), - version: ColumnDescVersion::Pr13707 as i32, + version: ColumnDescVersion::LATEST as _, ..Default::default() }) } diff --git a/src/connector/src/connector_common/connection.rs b/src/connector/src/connector_common/connection.rs index 3604b1bd2f99c..89e7711929ce5 100644 --- a/src/connector/src/connector_common/connection.rs +++ b/src/connector/src/connector_common/connection.rs @@ -135,6 +135,10 @@ pub struct IcebergConnection { /// Path of iceberg warehouse, only applicable in storage catalog. #[serde(rename = "warehouse.path")] pub warehouse_path: Option, + /// Catalog id, can be omitted for storage catalog or when + /// caller's AWS account ID matches glue id + #[serde(rename = "glue.id")] + pub glue_id: Option, /// Catalog name, can be omitted for storage catalog, but /// must be set for other catalogs. #[serde(rename = "catalog.name")] @@ -255,6 +259,7 @@ impl Connection for IcebergConnection { secret_key: self.secret_key.clone(), gcs_credential: self.gcs_credential.clone(), warehouse_path: self.warehouse_path.clone(), + glue_id: self.glue_id.clone(), catalog_name: self.catalog_name.clone(), catalog_uri: self.catalog_uri.clone(), credential: self.credential.clone(), diff --git a/src/connector/src/connector_common/iceberg/mod.rs b/src/connector/src/connector_common/iceberg/mod.rs index c04e20e53c202..936177668d765 100644 --- a/src/connector/src/connector_common/iceberg/mod.rs +++ b/src/connector/src/connector_common/iceberg/mod.rs @@ -57,6 +57,10 @@ pub struct IcebergCommon { /// Path of iceberg warehouse, only applicable in storage catalog. #[serde(rename = "warehouse.path")] pub warehouse_path: Option, + /// AWS Client id, can be omitted for storage catalog or when + /// caller's AWS account ID matches glue id + #[serde(rename = "glue.id")] + pub glue_id: Option, /// Catalog name, can be omitted for storage catalog, but /// must be set for other catalogs. #[serde(rename = "catalog.name")] @@ -274,6 +278,10 @@ impl IcebergCommon { format!("https://glue.{}.amazonaws.com", region), ); } + + if let Some(glue_id) = self.glue_id.as_deref() { + java_catalog_configs.insert("glue.id".to_owned(), glue_id.to_owned()); + } } _ => {} } diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index ac498839ed67c..186ecd314c5fc 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -35,7 +35,7 @@ macro_rules! for_all_classified_sources { { GooglePubsub, $crate::source::google_pubsub::PubsubProperties, $crate::source::google_pubsub::PubsubSplit }, { Mqtt, $crate::source::mqtt::MqttProperties, $crate::source::mqtt::split::MqttSplit }, { Nats, $crate::source::nats::NatsProperties, $crate::source::nats::split::NatsSplit }, - { S3, $crate::source::filesystem::S3Properties, $crate::source::filesystem::FsSplit }, + { S3, $crate::source::filesystem::LegacyS3Properties, $crate::source::filesystem::LegacyFsSplit }, { Gcs, $crate::source::filesystem::opendal_source::GcsProperties , $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalGcs> }, { OpendalS3, $crate::source::filesystem::opendal_source::OpendalS3Properties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalS3> }, { PosixFs, $crate::source::filesystem::opendal_source::PosixFsProperties, $crate::source::filesystem::OpendalFsSplit<$crate::source::filesystem::opendal_source::OpendalPosixFs> }, diff --git a/src/connector/src/parser/parquet_parser.rs b/src/connector/src/parser/parquet_parser.rs index 5bef3b6310981..29148c9305ddc 100644 --- a/src/connector/src/parser/parquet_parser.rs +++ b/src/connector/src/parser/parquet_parser.rs @@ -26,6 +26,7 @@ use risingwave_common::util::tokio_util::compat::FuturesAsyncReadCompatExt; use crate::parser::ConnectorResult; use crate::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use crate::source::filesystem::opendal_source::{OpendalGcs, OpendalPosixFs, OpendalS3}; +use crate::source::iceberg::is_parquet_schema_match_source_schema; use crate::source::reader::desc::SourceDesc; use crate::source::{ConnectorProperties, SourceColumnDesc}; /// `ParquetParser` is responsible for converting the incoming `record_batch_stream` @@ -109,6 +110,10 @@ impl ParquetParser { if let Some(parquet_column) = record_batch.column_by_name(rw_column_name) + && is_parquet_schema_match_source_schema( + parquet_column.data_type(), + rw_data_type, + ) { let arrow_field = IcebergArrowConvert .to_arrow_field(rw_column_name, rw_data_type)?; diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index f84d0aa2cb339..a18287062bd47 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -1432,6 +1432,7 @@ mod test { secret_key: Some("hummockadmin".to_owned()), gcs_credential: None, catalog_type: Some("jdbc".to_owned()), + glue_id: None, catalog_name: Some("demo".to_owned()), database_name: Some("demo_db".to_owned()), table_name: "demo_table".to_owned(), diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index 406d678a65ee6..3b7c01c185c18 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -17,7 +17,6 @@ use std::sync::Arc; use anyhow::anyhow; use async_trait::async_trait; -use aws_sdk_s3::types::Object; use bytes::Bytes; use enum_as_inner::EnumAsInner; use futures::future::try_join_all; @@ -49,7 +48,6 @@ use super::{AZBLOB_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONN use crate::error::ConnectorResult as Result; use crate::parser::schema_change::SchemaChangeEnvelope; use crate::parser::ParserConfig; -use crate::source::filesystem::FsPageItem; use crate::source::monitor::EnumeratorMetrics; use crate::source::SplitImpl::{CitusCdc, MongodbCdc, MysqlCdc, PostgresCdc, SqlServerCdc}; use crate::with_options::WithOptions; @@ -824,17 +822,6 @@ pub trait SplitMetaData: Sized { /// [`None`] and the created source stream will be a pending stream. pub type ConnectorState = Option>; -#[derive(Debug, Clone, Default)] -pub struct FsFilterCtrlCtx; -pub type FsFilterCtrlCtxRef = Arc; - -#[async_trait] -pub trait FsListInner: Sized { - // fixme: better to implement as an Iterator, but the last page still have some contents - async fn get_next_page From<&'a Object>>(&mut self) -> Result<(Vec, bool)>; - fn filter_policy(&self, ctx: &FsFilterCtrlCtx, page_num: usize, item: &FsPageItem) -> bool; -} - #[cfg(test)] mod tests { use maplit::*; diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index f89758a0ef51c..50f56ea1f53fe 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -24,16 +24,16 @@ use super::opendal_source::OpendalSource; use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; -/// [`FsSplit`] Describes a file or a split of a file. A file is a generic concept, +/// [`LegacyFsSplit`] Describes a file or a split of a file. A file is a generic concept, /// and can be a local file, a distributed file system, or am object in S3 bucket. #[derive(Debug, Clone, Eq, PartialEq, Hash, Serialize, Deserialize)] -pub struct FsSplit { +pub struct LegacyFsSplit { pub name: String, pub offset: usize, pub size: usize, } -impl From<&Object> for FsSplit { +impl From<&Object> for LegacyFsSplit { fn from(value: &Object) -> Self { Self { name: value.key().unwrap().to_owned(), @@ -43,7 +43,7 @@ impl From<&Object> for FsSplit { } } -impl SplitMetaData for FsSplit { +impl SplitMetaData for LegacyFsSplit { fn id(&self) -> SplitId { self.name.as_str().into() } @@ -63,7 +63,7 @@ impl SplitMetaData for FsSplit { } } -impl FsSplit { +impl LegacyFsSplit { pub fn new(name: String, start: usize, size: usize) -> Self { Self { name, diff --git a/src/connector/src/source/filesystem/mod.rs b/src/connector/src/source/filesystem/mod.rs index dd6f5c82ef715..b2f4e091cef2b 100644 --- a/src/connector/src/source/filesystem/mod.rs +++ b/src/connector/src/source/filesystem/mod.rs @@ -13,10 +13,11 @@ // limitations under the License. pub use opendal_source::GcsProperties; -pub use s3::{S3FileReader, S3Properties, S3SplitEnumerator, S3_CONNECTOR}; +pub use s3::{ + LegacyS3FileReader, LegacyS3Properties, LegacyS3SplitEnumerator, LEGACY_S3_CONNECTOR, +}; pub mod file_common; pub mod nd_streaming; -pub use file_common::{FsPage, FsPageItem, FsSplit, OpendalFsSplit}; +pub use file_common::{FsPage, FsPageItem, LegacyFsSplit, OpendalFsSplit}; pub mod opendal_source; mod s3; -pub mod s3_v2; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index a770d600282a2..ff1cf47b9b364 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -18,7 +18,7 @@ use anyhow::anyhow; use async_trait::async_trait; use chrono::{DateTime, Utc}; use futures::stream::{self, BoxStream}; -use futures::StreamExt; +use futures::{StreamExt, TryStreamExt}; use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamptz; @@ -54,7 +54,9 @@ impl SplitEnumerator for OpendalEnumerator { let empty_split: OpendalFsSplit = OpendalFsSplit::empty_split(); let prefix = self.prefix.as_deref().unwrap_or("/"); - match self.op.list(prefix).await { + let mut lister = self.op.lister(prefix).await?; + // fetch one item as validation, no need to get all + match lister.try_next().await { Ok(_) => return Ok(vec![empty_split]), Err(e) => { return Err(anyhow!(e) diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index ff15b1c00dc37..79c55a6a0282d 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -15,12 +15,14 @@ use anyhow::Context; use async_trait::async_trait; use aws_sdk_s3::client::Client; +use itertools::Itertools; use crate::aws_utils::{default_conn_config, s3_client}; use crate::connector_common::AwsAuthProps; -use crate::source::filesystem::file_common::FsSplit; -use crate::source::filesystem::s3::S3Properties; -use crate::source::{FsListInner, SourceEnumeratorContextRef, SplitEnumerator}; +use crate::error::ConnectorResult; +use crate::source::filesystem::file_common::LegacyFsSplit; +use crate::source::filesystem::s3::LegacyS3Properties; +use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; /// Get the prefix from a glob pub fn get_prefix(glob: &str) -> String { @@ -56,7 +58,7 @@ pub fn get_prefix(glob: &str) -> String { } #[derive(Debug, Clone)] -pub struct S3SplitEnumerator { +pub struct LegacyS3SplitEnumerator { pub(crate) bucket_name: String, // prefix is used to reduce the number of objects to be listed pub(crate) prefix: Option, @@ -68,9 +70,9 @@ pub struct S3SplitEnumerator { } #[async_trait] -impl SplitEnumerator for S3SplitEnumerator { - type Properties = S3Properties; - type Split = FsSplit; +impl SplitEnumerator for LegacyS3SplitEnumerator { + type Properties = LegacyS3Properties; + type Split = LegacyFsSplit; async fn new( properties: Self::Properties, @@ -89,7 +91,7 @@ impl SplitEnumerator for S3SplitEnumerator { (None, None) }; - Ok(S3SplitEnumerator { + Ok(LegacyS3SplitEnumerator { bucket_name: properties.bucket_name, matcher, prefix, @@ -99,15 +101,63 @@ impl SplitEnumerator for S3SplitEnumerator { } async fn list_splits(&mut self) -> crate::error::ConnectorResult> { - let mut objects = Vec::new(); - loop { - let (files, has_finished) = self.get_next_page::().await?; - objects.extend(files); - if has_finished { - break; - } + // fetch one page as validation, no need to get all pages + let (_, _) = self.get_next_page::().await?; + + Ok(vec![LegacyFsSplit { + name: "empty_split".to_owned(), + offset: 0, + size: 0, + }]) + } +} + +#[async_trait] +pub trait FsListInner: Sized { + // fixme: better to implement as an Iterator, but the last page still have some contents + async fn get_next_page From<&'a aws_sdk_s3::types::Object>>( + &mut self, + ) -> ConnectorResult<(Vec, bool)>; +} + +#[async_trait] +impl FsListInner for LegacyS3SplitEnumerator { + async fn get_next_page From<&'a aws_sdk_s3::types::Object>>( + &mut self, + ) -> ConnectorResult<(Vec, bool)> { + let mut has_finished = false; + let mut req = self + .client + .list_objects_v2() + .bucket(&self.bucket_name) + .set_prefix(self.prefix.clone()); + if let Some(continuation_token) = self.next_continuation_token.take() { + req = req.continuation_token(continuation_token); + } + let mut res = req + .send() + .await + .with_context(|| format!("failed to list objects in bucket `{}`", self.bucket_name))?; + if res.is_truncated().unwrap_or_default() { + self.next_continuation_token + .clone_from(&res.next_continuation_token); + } else { + has_finished = true; + self.next_continuation_token = None; } - Ok(objects) + let objects = res.contents.take().unwrap_or_default(); + let matched_objs: Vec = objects + .iter() + .filter(|obj| obj.key().is_some()) + .filter(|obj| { + self.matcher + .as_ref() + .map(|m| m.matches(obj.key().unwrap())) + .unwrap_or(true) + }) + .map(T::from) + .collect_vec(); + Ok((matched_objs, has_finished)) } } @@ -142,7 +192,7 @@ mod tests { compression_format: CompressionFormat::None, }; let mut enumerator = - S3SplitEnumerator::new(props.into(), SourceEnumeratorContext::dummy().into()) + LegacyS3SplitEnumerator::new(props.into(), SourceEnumeratorContext::dummy().into()) .await .unwrap(); let splits = enumerator.list_splits().await.unwrap(); diff --git a/src/connector/src/source/filesystem/s3/mod.rs b/src/connector/src/source/filesystem/s3/mod.rs index 76b7e9f1f0ca1..0e7960789b2c3 100644 --- a/src/connector/src/source/filesystem/s3/mod.rs +++ b/src/connector/src/source/filesystem/s3/mod.rs @@ -14,18 +14,18 @@ pub mod enumerator; use std::collections::HashMap; -pub use enumerator::S3SplitEnumerator; +pub use enumerator::LegacyS3SplitEnumerator; use crate::source::filesystem::file_common::CompressionFormat; mod source; use serde::Deserialize; -pub use source::S3FileReader; +pub use source::LegacyS3FileReader; use crate::connector_common::AwsAuthProps; -use crate::source::filesystem::FsSplit; +use crate::source::filesystem::LegacyFsSplit; use crate::source::{SourceProperties, UnknownFields}; -pub const S3_CONNECTOR: &str = "s3"; +pub const LEGACY_S3_CONNECTOR: &str = "s3"; /// These are supported by both `s3` and `s3_v2` (opendal) sources. #[derive(Clone, Debug, Deserialize, PartialEq, with_options::WithOptions)] @@ -47,7 +47,7 @@ pub struct S3PropertiesCommon { } #[derive(Clone, Debug, Deserialize, PartialEq, with_options::WithOptions)] -pub struct S3Properties { +pub struct LegacyS3Properties { #[serde(flatten)] pub common: S3PropertiesCommon, @@ -55,7 +55,7 @@ pub struct S3Properties { pub unknown_fields: HashMap, } -impl From for S3Properties { +impl From for LegacyS3Properties { fn from(common: S3PropertiesCommon) -> Self { Self { common, @@ -64,22 +64,22 @@ impl From for S3Properties { } } -impl SourceProperties for S3Properties { - type Split = FsSplit; - type SplitEnumerator = S3SplitEnumerator; - type SplitReader = S3FileReader; +impl SourceProperties for LegacyS3Properties { + type Split = LegacyFsSplit; + type SplitEnumerator = LegacyS3SplitEnumerator; + type SplitReader = LegacyS3FileReader; - const SOURCE_NAME: &'static str = S3_CONNECTOR; + const SOURCE_NAME: &'static str = LEGACY_S3_CONNECTOR; } -impl UnknownFields for S3Properties { +impl UnknownFields for LegacyS3Properties { fn unknown_fields(&self) -> HashMap { self.unknown_fields.clone() } } -impl From<&S3Properties> for AwsAuthProps { - fn from(props: &S3Properties) -> Self { +impl From<&LegacyS3Properties> for AwsAuthProps { + fn from(props: &LegacyS3Properties) -> Self { let props = &props.common; Self { region: Some(props.region_name.clone()), diff --git a/src/connector/src/source/filesystem/s3/source/mod.rs b/src/connector/src/source/filesystem/s3/source/mod.rs index e7e6c5db0daed..87e9067537ad7 100644 --- a/src/connector/src/source/filesystem/s3/source/mod.rs +++ b/src/connector/src/source/filesystem/s3/source/mod.rs @@ -14,4 +14,4 @@ mod reader; mod split_stream; -pub use reader::S3FileReader; +pub use reader::LegacyS3FileReader; diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 5f2631b913ab4..eeb613f52c48d 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -36,9 +36,9 @@ use crate::connector_common::AwsAuthProps; use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::base::{SplitMetaData, SplitReader}; -use crate::source::filesystem::file_common::FsSplit; +use crate::source::filesystem::file_common::LegacyFsSplit; use crate::source::filesystem::nd_streaming::need_nd_streaming; -use crate::source::filesystem::s3::S3Properties; +use crate::source::filesystem::s3::LegacyS3Properties; use crate::source::{ into_chunk_stream, BoxSourceChunkStream, Column, SourceContextRef, SourceMessage, SourceMeta, }; @@ -46,22 +46,22 @@ use crate::source::{ const STREAM_READER_CAPACITY: usize = 4096; #[derive(Debug)] -pub struct S3FileReader { +pub struct LegacyS3FileReader { #[expect(dead_code)] split_offset: HashMap, bucket_name: String, s3_client: s3_client::Client, - splits: Vec, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, } -impl S3FileReader { +impl LegacyS3FileReader { #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn stream_read_object( client_for_s3: s3_client::Client, bucket_name: String, - split: FsSplit, + split: LegacyFsSplit, source_ctx: SourceContextRef, ) { let actor_id = source_ctx.actor_id.to_string(); @@ -73,7 +73,7 @@ impl S3FileReader { let object_name = split.name.clone(); - let byte_stream = match S3FileReader::get_object( + let byte_stream = match LegacyS3FileReader::get_object( &client_for_s3, &bucket_name, &object_name, @@ -171,13 +171,13 @@ impl S3FileReader { } #[async_trait] -impl SplitReader for S3FileReader { - type Properties = S3Properties; - type Split = FsSplit; +impl SplitReader for LegacyS3FileReader { + type Properties = LegacyS3Properties; + type Split = LegacyFsSplit; async fn new( - props: S3Properties, - splits: Vec, + props: LegacyS3Properties, + splits: Vec, parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, @@ -189,7 +189,7 @@ impl SplitReader for S3FileReader { let bucket_name = props.common.bucket_name; let s3_client = s3_client(&sdk_config, Some(default_conn_config())); - let s3_file_reader = S3FileReader { + let s3_file_reader = LegacyS3FileReader { split_offset: HashMap::new(), bucket_name, s3_client, @@ -206,7 +206,7 @@ impl SplitReader for S3FileReader { } } -impl S3FileReader { +impl LegacyS3FileReader { #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_stream_inner(self) { for split in self.splits { @@ -248,7 +248,7 @@ mod tests { }; use crate::source::filesystem::file_common::CompressionFormat; use crate::source::filesystem::s3::S3PropertiesCommon; - use crate::source::filesystem::S3SplitEnumerator; + use crate::source::filesystem::LegacyS3SplitEnumerator; use crate::source::{ SourceColumnDesc, SourceContext, SourceEnumeratorContext, SplitEnumerator, }; @@ -256,7 +256,7 @@ mod tests { #[tokio::test] #[ignore] async fn test_s3_split_reader() { - let props: S3Properties = S3PropertiesCommon { + let props: LegacyS3Properties = S3PropertiesCommon { region_name: "ap-southeast-1".to_owned(), bucket_name: "mingchao-s3-source".to_owned(), match_pattern: None, @@ -267,7 +267,7 @@ mod tests { } .into(); let mut enumerator = - S3SplitEnumerator::new(props.clone(), SourceEnumeratorContext::dummy().into()) + LegacyS3SplitEnumerator::new(props.clone(), SourceEnumeratorContext::dummy().into()) .await .unwrap(); let splits = enumerator.list_splits().await.unwrap(); @@ -292,9 +292,10 @@ mod tests { }, }; - let reader = S3FileReader::new(props, splits, config, SourceContext::dummy().into(), None) - .await - .unwrap(); + let reader = + LegacyS3FileReader::new(props, splits, config, SourceContext::dummy().into(), None) + .await + .unwrap(); let msg_stream = reader.into_stream_inner(); #[for_await] diff --git a/src/connector/src/source/filesystem/s3_v2/lister.rs b/src/connector/src/source/filesystem/s3_v2/lister.rs deleted file mode 100644 index ee0132742a40d..0000000000000 --- a/src/connector/src/source/filesystem/s3_v2/lister.rs +++ /dev/null @@ -1,67 +0,0 @@ -// Copyright 2025 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::Context; -use async_trait::async_trait; -use aws_sdk_s3::types::Object; -use itertools::Itertools; - -use crate::error::ConnectorResult; -use crate::source::filesystem::{FsPageItem, S3SplitEnumerator}; -use crate::source::{FsFilterCtrlCtx, FsListInner}; - -#[async_trait] -impl FsListInner for S3SplitEnumerator { - async fn get_next_page From<&'a Object>>( - &mut self, - ) -> ConnectorResult<(Vec, bool)> { - let mut has_finished = false; - let mut req = self - .client - .list_objects_v2() - .bucket(&self.bucket_name) - .set_prefix(self.prefix.clone()); - if let Some(continuation_token) = self.next_continuation_token.take() { - req = req.continuation_token(continuation_token); - } - let mut res = req - .send() - .await - .with_context(|| format!("failed to list objects in bucket `{}`", self.bucket_name))?; - if res.is_truncated().unwrap_or_default() { - self.next_continuation_token - .clone_from(&res.next_continuation_token); - } else { - has_finished = true; - self.next_continuation_token = None; - } - let objects = res.contents.take().unwrap_or_default(); - let matched_objs: Vec = objects - .iter() - .filter(|obj| obj.key().is_some()) - .filter(|obj| { - self.matcher - .as_ref() - .map(|m| m.matches(obj.key().unwrap())) - .unwrap_or(true) - }) - .map(T::from) - .collect_vec(); - Ok((matched_objs, has_finished)) - } - - fn filter_policy(&self, _ctx: &FsFilterCtrlCtx, _page_num: usize, _item: &FsPageItem) -> bool { - true - } -} diff --git a/src/connector/src/source/filesystem/s3_v2/mod.rs b/src/connector/src/source/filesystem/s3_v2/mod.rs deleted file mode 100644 index ae95f6fedb186..0000000000000 --- a/src/connector/src/source/filesystem/s3_v2/mod.rs +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright 2025 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -pub mod lister; diff --git a/src/connector/src/source/iceberg/parquet_file_handler.rs b/src/connector/src/source/iceberg/parquet_file_handler.rs index 187140855c29b..6baf30a5795e8 100644 --- a/src/connector/src/source/iceberg/parquet_file_handler.rs +++ b/src/connector/src/source/iceberg/parquet_file_handler.rs @@ -36,7 +36,7 @@ use parquet::file::metadata::{FileMetaData, ParquetMetaData, ParquetMetaDataRead 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::catalog::{ColumnDesc, ColumnId}; use risingwave_common::types::DataType as RwDataType; use risingwave_common::util::tokio_util::compat::FuturesAsyncReadCompatExt; use url::Url; @@ -217,55 +217,64 @@ pub async fn list_data_directory( } } -/// Extracts valid column indices from a Parquet file schema based on the user's requested schema. +/// Extracts a suitable `ProjectionMask` from a Parquet file schema based on the user's requested schema. /// -/// This function is used for column pruning of Parquet files. It calculates the intersection -/// between the columns in the currently read Parquet file and the schema provided by the user. -/// This is useful for reading a `RecordBatch` with the appropriate `ProjectionMask`, ensuring that -/// only the necessary columns are read. +/// This function is utilized for column pruning of Parquet files. It checks the user's requested schema +/// against the schema of the currently read Parquet file. If the provided `columns` are `None` +/// or if the Parquet file contains nested data types, it returns `ProjectionMask::all()`. Otherwise, +/// it returns only the columns where both the data type and column name match the requested schema, +/// facilitating efficient reading of the `RecordBatch`. /// /// # Parameters -/// - `columns`: A vector of `Column` representing the user's requested schema. +/// - `columns`: An optional vector of `Column` representing the user's requested schema. /// - `metadata`: A reference to `FileMetaData` containing the schema and metadata of the Parquet file. /// /// # Returns -/// - A `ConnectorResult>`, which contains the indices of the valid columns in the -/// Parquet file schema that match the requested schema. If an error occurs during processing, -/// it returns an appropriate error. -pub fn extract_valid_column_indices( - rw_columns: Vec, +/// - A `ConnectorResult`, which represents the valid columns in the Parquet file schema +/// that correspond to the requested schema. If an error occurs during processing, it returns an +/// appropriate error. +pub fn get_project_mask( + columns: Option>, metadata: &FileMetaData, -) -> ConnectorResult> { - let parquet_column_names = metadata - .schema_descr() - .columns() - .iter() - .map(|c| c.name()) - .collect_vec(); +) -> ConnectorResult { + match columns { + Some(rw_columns) => { + let root_column_names = metadata + .schema_descr() + .root_schema() + .get_fields() + .iter() + .map(|field| field.name()) + .collect_vec(); - let converted_arrow_schema = - parquet_to_arrow_schema(metadata.schema_descr(), metadata.key_value_metadata()) - .map_err(anyhow::Error::from)?; + let converted_arrow_schema = + parquet_to_arrow_schema(metadata.schema_descr(), metadata.key_value_metadata()) + .map_err(anyhow::Error::from)?; + let valid_column_indices: Vec = rw_columns + .iter() + .filter_map(|column| { + root_column_names + .iter() + .position(|&name| name == column.name) + .and_then(|pos| { + let arrow_data_type: &risingwave_common::array::arrow::arrow_schema_udf::DataType = converted_arrow_schema.field_with_name(&column.name).ok()?.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 + } + }) + }) + .collect(); - let valid_column_indices: Vec = rw_columns - .iter() - .filter_map(|column| { - parquet_column_names - .iter() - .position(|&name| name == column.name) - .and_then(|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 - } - }) - }) - .collect(); - Ok(valid_column_indices) + Ok(ProjectionMask::roots( + metadata.schema_descr(), + valid_column_indices, + )) + } + None => Ok(ProjectionMask::all()), + } } /// Reads a specified Parquet file and converts its content into a stream of chunks. @@ -289,13 +298,7 @@ pub async fn read_parquet_file( let parquet_metadata = reader.get_metadata().await.map_err(anyhow::Error::from)?; let file_metadata = parquet_metadata.file_metadata(); - let projection_mask = match rw_columns { - Some(columns) => { - let column_indices = extract_valid_column_indices(columns, file_metadata)?; - ProjectionMask::leaves(file_metadata.schema_descr(), column_indices) - } - None => ProjectionMask::all(), - }; + let projection_mask = get_project_mask(rw_columns, file_metadata)?; // For the Parquet format, we directly convert from a record batch to a stream chunk. // Therefore, the offset of the Parquet file represents the current position in terms of the number of rows read from the file. @@ -318,11 +321,12 @@ pub async fn read_parquet_file( .enumerate() .map(|(index, field_ref)| { let data_type = IcebergArrowConvert.type_from_field(field_ref).unwrap(); - SourceColumnDesc::simple( + let column_desc = ColumnDesc::named( field_ref.name().clone(), - data_type, ColumnId::new(index as i32), - ) + data_type, + ); + SourceColumnDesc::from(&column_desc) }) .collect(), }; @@ -367,7 +371,7 @@ pub async fn get_parquet_fields( /// - 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( +pub fn is_parquet_schema_match_source_schema( arrow_data_type: &ArrowDateType, rw_data_type: &RwDataType, ) -> bool { diff --git a/src/connector/src/source/kafka/private_link.rs b/src/connector/src/source/kafka/private_link.rs index 06d4b8e97155e..73c15d351ac33 100644 --- a/src/connector/src/source/kafka/private_link.rs +++ b/src/connector/src/source/kafka/private_link.rs @@ -22,6 +22,7 @@ use risingwave_common::bail; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::connection::PrivateLinkService; +use serde_derive::Deserialize; use crate::connector_common::{ AwsPrivateLinkItem, PRIVATE_LINK_BROKER_REWRITE_MAP_KEY, PRIVATE_LINK_TARGETS_KEY, @@ -38,6 +39,11 @@ pub(super) enum PrivateLinkContextRole { Producer, } +#[derive(Debug, Deserialize)] +struct PrivateLinkEndpointItem { + host: String, +} + impl std::fmt::Display for PrivateLinkContextRole { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { match self { @@ -135,10 +141,18 @@ pub fn insert_privatelink_broker_rewrite_map( } if let Some(endpoint) = privatelink_endpoint { - for (link, broker) in link_targets.iter().zip_eq_fast(broker_addrs.into_iter()) { - // rewrite the broker address to endpoint:port - broker_rewrite_map.insert(broker.to_owned(), format!("{}:{}", &endpoint, link.port)); - } + // new syntax: endpoint can either be a string or a json array of strings + // if it is a string, rewrite all broker addresses to the same endpoint + // eg. privatelink.endpoint='some_url' ==> broker1:9092 -> some_url:9092, broker2:9093 -> some_url:9093 + // if it is a json array, rewrite each broker address to the corresponding endpoint + // eg. privatelink.endpoint = '[{"host": "aaaa"}, {"host": "bbbb"}, {"host": "cccc"}]' + // ==> broker1:9092 -> aaaa:9092, broker2:9093 -> bbbb:9093, broker3:9094 -> cccc:9094 + handle_privatelink_endpoint( + &endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + )?; } else { if svc.is_none() { bail!("Privatelink endpoint not found."); @@ -166,3 +180,149 @@ pub fn insert_privatelink_broker_rewrite_map( with_options.insert(PRIVATE_LINK_BROKER_REWRITE_MAP_KEY.to_owned(), json); Ok(()) } + +fn handle_privatelink_endpoint( + endpoint: &str, + broker_rewrite_map: &mut HashMap, + link_targets: &[AwsPrivateLinkItem], + broker_addrs: &[&str], +) -> ConnectorResult<()> { + let endpoint = if let Ok(json) = serde_json::from_str::(endpoint) { + json + } else { + serde_json::Value::String(endpoint.to_owned()) + }; + if matches!(endpoint, serde_json::Value::String(_)) { + let endpoint = endpoint.as_str().unwrap(); + for (link, broker) in link_targets.iter().zip_eq_fast(broker_addrs.iter()) { + // rewrite the broker address to endpoint:port + broker_rewrite_map.insert(broker.to_string(), format!("{}:{}", endpoint, link.port)); + } + } else if matches!(endpoint, serde_json::Value::Array(_)) { + let endpoint_list: Vec = endpoint + .as_array() + .unwrap() + .iter() + .map(|v| { + serde_json::from_value(v.clone()).map_err(|_| { + anyhow!( + "expect json schema {{\"host\": \"endpoint url\"}} but got {}", + v + ) + }) + }) + .collect::, _>>()?; + for ((link, broker), endpoint) in link_targets + .iter() + .zip_eq_fast(broker_addrs.iter()) + .zip_eq_fast(endpoint_list.iter()) + { + // rewrite the broker address to endpoint:port + broker_rewrite_map.insert( + broker.to_string(), + format!("{}:{}", endpoint.host, link.port), + ); + } + } else { + bail!( + "expect a string or a json array for privatelink.endpoint, but got {:?}", + endpoint + ) + } + + Ok(()) +} + +#[cfg(test)] +mod tests { + use super::*; + + #[test] + fn test_handle_privatelink_endpoint() { + let endpoint = "some_url"; // raw string + let link_targets = vec![ + AwsPrivateLinkItem { + az_id: None, + port: 9092, + }, + AwsPrivateLinkItem { + az_id: None, + port: 9093, + }, + ]; + let broker_addrs = vec!["broker1:9092", "broker2:9093"]; + let mut broker_rewrite_map = HashMap::new(); + handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap(); + + assert_eq!(broker_rewrite_map.len(), 2); + assert_eq!(broker_rewrite_map["broker1:9092"], "some_url:9092"); + assert_eq!(broker_rewrite_map["broker2:9093"], "some_url:9093"); + + // example 2: json array + let endpoint = r#"[{"host": "aaaa"}, {"host": "bbbb"}, {"host": "cccc"}]"#; + let broker_addrs = vec!["broker1:9092", "broker2:9093", "broker3:9094"]; + let link_targets = vec![ + AwsPrivateLinkItem { + az_id: None, + port: 9092, + }, + AwsPrivateLinkItem { + az_id: None, + port: 9093, + }, + AwsPrivateLinkItem { + az_id: None, + port: 9094, + }, + ]; + let mut broker_rewrite_map = HashMap::new(); + handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap(); + + assert_eq!(broker_rewrite_map.len(), 3); + assert_eq!(broker_rewrite_map["broker1:9092"], "aaaa:9092"); + assert_eq!(broker_rewrite_map["broker2:9093"], "bbbb:9093"); + assert_eq!(broker_rewrite_map["broker3:9094"], "cccc:9094"); + + // no `host` in the json array + let endpoint = r#"[{"somekey_1": "aaaa"}, {"somekey_2": "bbbb"}, {"somekey_3": "cccc"}]"#; + let mut broker_rewrite_map = HashMap::new(); + let err = handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap_err(); + assert_eq!( + err.to_string(), + "expect json schema {\"host\": \"endpoint url\"} but got {\"somekey_1\":\"aaaa\"}" + ); + + // illegal json + let endpoint = r#"{}"#; + let mut broker_rewrite_map = HashMap::new(); + let err = handle_privatelink_endpoint( + endpoint, + &mut broker_rewrite_map, + &link_targets, + &broker_addrs, + ) + .unwrap_err(); + assert_eq!( + err.to_string(), + "expect a string or a json array for privatelink.endpoint, but got Object {}" + ); + } +} diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index e0eddb0dee507..502f06ae31a6c 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -124,9 +124,9 @@ impl SplitReader for KafkaSplitReader { properties.common.sync_call_timeout, ) .await?; - tracing::debug!("fetch kafka watermarks: low: {low}, high: {high}, split: {split:?}"); - // note: low is inclusive, high is exclusive - if low == high { + tracing::info!("fetch kafka watermarks: low: {low}, high: {high}, split: {split:?}"); + // note: low is inclusive, high is exclusive, start_offset is exclusive + if low == high || split.start_offset.is_some_and(|offset| offset + 1 >= high) { backfill_info.insert(split.id(), BackfillInfo::NoDataToBackfill); } else { debug_assert!(high > 0); @@ -138,7 +138,15 @@ impl SplitReader for KafkaSplitReader { ); } } - tracing::debug!("backfill_info: {:?}", backfill_info); + tracing::info!( + topic = properties.common.topic, + source_name = source_ctx.source_name, + fragment_id = source_ctx.fragment_id, + source_id = source_ctx.source_id.table_id, + actor_id = source_ctx.actor_id, + "backfill_info: {:?}", + backfill_info + ); consumer.assign(&tpl)?; diff --git a/src/connector/src/source/manager.rs b/src/connector/src/source/manager.rs index e6618d92c185b..5bd45579f500f 100644 --- a/src/connector/src/source/manager.rs +++ b/src/connector/src/source/manager.rs @@ -185,7 +185,7 @@ impl From<&SourceColumnDesc> for ColumnDesc { type_name: "".to_owned(), generated_or_default_column: None, description: None, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, } } diff --git a/src/connector/src/source/mod.rs b/src/connector/src/source/mod.rs index 12b843dcf749f..3ea03f7b4df19 100644 --- a/src/connector/src/source/mod.rs +++ b/src/connector/src/source/mod.rs @@ -16,7 +16,7 @@ pub mod prelude { // import all split enumerators pub use crate::source::datagen::DatagenSplitEnumerator; pub use crate::source::filesystem::opendal_source::OpendalEnumerator; - pub use crate::source::filesystem::S3SplitEnumerator; + pub use crate::source::filesystem::LegacyS3SplitEnumerator; pub use crate::source::google_pubsub::PubsubSplitEnumerator as GooglePubsubSplitEnumerator; pub use crate::source::iceberg::IcebergSplitEnumerator; pub use crate::source::kafka::KafkaSplitEnumerator; @@ -83,7 +83,7 @@ pub use util::fill_adaptive_split; pub use crate::source::filesystem::opendal_source::{ AZBLOB_CONNECTOR, GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, }; -pub use crate::source::filesystem::S3_CONNECTOR; +pub use crate::source::filesystem::LEGACY_S3_CONNECTOR; pub use crate::source::nexmark::NEXMARK_CONNECTOR; pub use crate::source::pulsar::PULSAR_CONNECTOR; diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index 95fdf24b111aa..ed4692a2db2a5 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -21,7 +21,7 @@ use risingwave_pb::catalog::PbStreamSourceInfo; use risingwave_pb::plan_common::PbColumnCatalog; #[expect(deprecated)] -use super::fs_reader::FsSourceReader; +use super::fs_reader::LegacyFsSourceReader; use super::reader::SourceReader; use crate::error::ConnectorResult; use crate::parser::additional_columns::source_add_partition_offset_cols; @@ -44,8 +44,8 @@ pub struct SourceDesc { #[deprecated = "will be replaced by new fs source (list + fetch)"] #[expect(deprecated)] #[derive(Debug)] -pub struct FsSourceDesc { - pub source: FsSourceReader, +pub struct LegacyFsSourceDesc { + pub source: LegacyFsSourceReader, pub columns: Vec, pub metrics: Arc, } @@ -139,7 +139,7 @@ impl SourceDescBuilder { #[deprecated = "will be replaced by new fs source (list + fetch)"] #[expect(deprecated)] - pub fn build_fs_source_desc(&self) -> ConnectorResult { + pub fn build_fs_source_desc(&self) -> ConnectorResult { let parser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; match ( @@ -161,10 +161,13 @@ impl SourceDescBuilder { let columns = self.column_catalogs_to_source_column_descs(); - let source = - FsSourceReader::new(self.with_properties.clone(), columns.clone(), parser_config)?; + let source = LegacyFsSourceReader::new( + self.with_properties.clone(), + columns.clone(), + parser_config, + )?; - Ok(FsSourceDesc { + Ok(LegacyFsSourceDesc { source, columns, metrics: self.metrics.clone(), diff --git a/src/connector/src/source/reader/fs_reader.rs b/src/connector/src/source/reader/fs_reader.rs index 41817c3a85a97..6a2ea3223e0a5 100644 --- a/src/connector/src/source/reader/fs_reader.rs +++ b/src/connector/src/source/reader/fs_reader.rs @@ -29,15 +29,14 @@ use crate::source::{ use crate::WithOptionsSecResolved; #[derive(Clone, Debug)] -pub struct FsSourceReader { +pub struct LegacyFsSourceReader { pub config: ConnectorProperties, pub columns: Vec, pub properties: WithOptionsSecResolved, pub parser_config: SpecificParserConfig, } -impl FsSourceReader { - #[allow(clippy::too_many_arguments)] +impl LegacyFsSourceReader { pub fn new( properties: WithOptionsSecResolved, columns: Vec, diff --git a/src/connector/src/with_options.rs b/src/connector/src/with_options.rs index 1e84c3012184c..ad72d370254b0 100644 --- a/src/connector/src/with_options.rs +++ b/src/connector/src/with_options.rs @@ -21,8 +21,8 @@ use crate::source::cdc::external::CdcTableType; use crate::source::cdc::MYSQL_CDC_CONNECTOR; use crate::source::iceberg::ICEBERG_CONNECTOR; use crate::source::{ - AZBLOB_CONNECTOR, GCS_CONNECTOR, KAFKA_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, - UPSTREAM_SOURCE_KEY, + AZBLOB_CONNECTOR, GCS_CONNECTOR, KAFKA_CONNECTOR, LEGACY_S3_CONNECTOR, OPENDAL_S3_CONNECTOR, + POSIX_FS_CONNECTOR, UPSTREAM_SOURCE_KEY, }; /// Marker trait for `WITH` options. Only for `#[derive(WithOptions)]`, should not be used manually. @@ -153,6 +153,12 @@ pub trait WithPropertiesExt: Get + Sized { !self.is_iceberg_connector() } + fn is_legacy_fs_connector(&self) -> bool { + self.get(UPSTREAM_SOURCE_KEY) + .map(|s| s.eq_ignore_ascii_case(LEGACY_S3_CONNECTOR)) + .unwrap_or(false) + } + fn is_new_fs_connector(&self) -> bool { self.get(UPSTREAM_SOURCE_KEY) .map(|s| { diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index ead783cec997e..a307d1a143f29 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -448,6 +448,12 @@ IcebergConfig: field_type: String comments: Path of iceberg warehouse, only applicable in storage catalog. required: false + - name: glue.id + field_type: String + comments: |- + AWS Client id, can be omitted for storage catalog or when + caller's AWS account ID matches glue id + required: false - name: catalog.name field_type: String comments: |- diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 4cb45b983f0fa..cb9329190b903 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -103,6 +103,12 @@ IcebergProperties: field_type: String comments: Path of iceberg warehouse, only applicable in storage catalog. required: false + - name: glue.id + field_type: String + comments: |- + AWS Client id, can be omitted for storage catalog or when + caller's AWS account ID matches glue id + required: false - name: catalog.name field_type: String comments: |- @@ -473,6 +479,33 @@ KinesisProperties: required: false alias: - kinesis.assumerole.external_id +LegacyS3Properties: + fields: + - name: s3.region_name + field_type: String + required: true + - name: s3.bucket_name + field_type: String + required: true + - name: match_pattern + field_type: String + required: false + default: Default::default + - name: s3.credentials.access + field_type: String + required: false + default: Default::default + - name: s3.credentials.secret + field_type: String + required: false + default: Default::default + - name: s3.endpoint_url + field_type: String + required: false + - name: compression_format + field_type: CompressionFormat + required: false + default: Default::default MongodbCommon: fields: - name: mongodb.url @@ -1065,33 +1098,6 @@ PulsarProperties: contains a generated suffix in the subscription name. The subscription name will be `{subscription_name_prefix}-{fragment_id}-{actor_id}`. required: false -S3Properties: - fields: - - name: s3.region_name - field_type: String - required: true - - name: s3.bucket_name - field_type: String - required: true - - name: match_pattern - field_type: String - required: false - default: Default::default - - name: s3.credentials.access - field_type: String - required: false - default: Default::default - - name: s3.credentials.secret - field_type: String - required: false - default: Default::default - - name: s3.endpoint_url - field_type: String - required: false - - name: compression_format - field_type: CompressionFormat - required: false - default: Default::default TestSourceProperties: fields: - name: properties diff --git a/src/frontend/planner_test/tests/testdata/input/asof_join.yaml b/src/frontend/planner_test/tests/testdata/input/asof_join.yaml index f6ca65716c2ea..5697a8f2c111d 100644 --- a/src/frontend/planner_test/tests/testdata/input/asof_join.yaml +++ b/src/frontend/planner_test/tests/testdata/input/asof_join.yaml @@ -4,14 +4,15 @@ SELECT * FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1; expected_outputs: - stream_error + - batch_error - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 || 'a' and t1.v2 > t2.v2; expected_outputs: - - batch_error - stream_plan + - batch_plan - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); @@ -19,6 +20,7 @@ SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 *2 < t2.v2; expected_outputs: - stream_plan + - batch_plan - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); @@ -26,6 +28,7 @@ SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 and t1.v3 < t2.v3; expected_outputs: - stream_error + - batch_error - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); @@ -33,3 +36,4 @@ SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v2 < t2.v2; expected_outputs: - stream_error + - batch_error diff --git a/src/frontend/planner_test/tests/testdata/output/asof_join.yaml b/src/frontend/planner_test/tests/testdata/output/asof_join.yaml index 508c9de04f18d..74f8464a74bd7 100644 --- a/src/frontend/planner_test/tests/testdata/output/asof_join.yaml +++ b/src/frontend/planner_test/tests/testdata/output/asof_join.yaml @@ -1,7 +1,16 @@ # This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT * FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1; + batch_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' stream_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 || 'a' and t1.v2 > t2.v2; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: AsofInner, predicate: t1.v1 = $expr1 AND (t1.v2 > t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2] } + ├─BatchExchange { order: [], dist: HashShard(t1.v1) } + │ └─BatchScan { table: t1, columns: [t1.v1, t1.v2], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard($expr1) } + └─BatchProject { exprs: [t2.v1, t2.v2, ConcatOp(t2.v1, 'a':Varchar) as $expr1] } + └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck } └─StreamAsOfJoin { type: AsofInner, predicate: t1.v1 = $expr1 AND (t1.v2 > t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } @@ -10,10 +19,15 @@ └─StreamExchange { dist: HashShard($expr1) } └─StreamProject { exprs: [t2.v1, t2.v2, ConcatOp(t2.v1, 'a':Varchar) as $expr1, t2._row_id] } └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } - batch_error: |- - Not supported: AsOf join in batch query - HINT: AsOf join is only supported in streaming query - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF LEFT JOIN t2 ON t1.v1 = t2.v1 and t1.v2 *2 < t2.v2; + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchHashJoin { type: AsofLeftOuter, predicate: t1.v1 = t2.v1 AND ($expr1 < t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2] } + ├─BatchExchange { order: [], dist: HashShard(t1.v1) } + │ └─BatchProject { exprs: [t1.v1, t1.v2, (t1.v2 * 2:Int32) as $expr1] } + │ └─BatchScan { table: t1, columns: [t1.v1, t1.v2], distribution: SomeShard } + └─BatchExchange { order: [], dist: HashShard(t2.v1) } + └─BatchScan { table: t2, columns: [t2.v1, t2.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [t1_v1, t1_v2, t2_v1, t2_v2, t1._row_id(hidden), t2._row_id(hidden)], stream_key: [t1._row_id, t2._row_id, t1_v1], pk_columns: [t1._row_id, t2._row_id, t1_v1], pk_conflict: NoCheck } └─StreamAsOfJoin { type: AsofLeftOuter, predicate: t1.v1 = t2.v1 AND ($expr1 < t2.v2), output: [t1.v1, t1.v2, t2.v1, t2.v2, t1._row_id, t2._row_id] } @@ -23,6 +37,8 @@ └─StreamExchange { dist: HashShard(t2.v1) } └─StreamTableScan { table: t2, columns: [t2.v1, t2.v2, t2._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t2._row_id], pk: [_row_id], dist: UpstreamHashShard(t2._row_id) } - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v1 = t2.v1 and t1.v2 < t2.v2 and t1.v3 < t2.v3; + batch_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' stream_error: 'Invalid input syntax: AsOf join requires exactly 1 ineuquality condition' - sql: CREATE TABLE t1(v1 varchar, v2 int, v3 int); CREATE TABLE t2(v1 varchar, v2 int, v3 int); SELECT t1.v1 t1_v1, t1.v2 t1_v2, t2.v1 t2_v1, t2.v2 t2_v2 FROM t1 ASOF JOIN t2 ON t1.v2 < t2.v2; + batch_error: 'Invalid input syntax: AsOf join requires at least 1 equal condition' stream_error: 'Invalid input syntax: AsOf join requires at least 1 equal condition' diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 2002555c857be..6373023090803 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -1000,7 +1000,7 @@ pub fn bind_struct_field(column_def: &StructField) -> Result { generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, }) } diff --git a/src/frontend/src/catalog/table_catalog.rs b/src/frontend/src/catalog/table_catalog.rs index 53e72d5fb77c9..fcae2de055e71 100644 --- a/src/frontend/src/catalog/table_catalog.rs +++ b/src/frontend/src/catalog/table_catalog.rs @@ -920,7 +920,7 @@ mod tests { description: None, generated_or_default_column: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, }, is_hidden: false diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index 731db3f9f3363..f8cec19c9313a 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -60,8 +60,8 @@ use risingwave_connector::source::nexmark::source::{get_event_data_types_with_na use risingwave_connector::source::test_source::TEST_CONNECTOR; 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, + KINESIS_CONNECTOR, LEGACY_S3_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, + OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, }; pub use risingwave_connector::source::{UPSTREAM_SOURCE_KEY, WEBHOOK_CONNECTOR}; use risingwave_connector::WithPropertiesExt; diff --git a/src/frontend/src/handler/create_source/validate.rs b/src/frontend/src/handler/create_source/validate.rs index 605c57f1f3736..d20f6656ebead 100644 --- a/src/frontend/src/handler/create_source/validate.rs +++ b/src/frontend/src/handler/create_source/validate.rs @@ -71,7 +71,7 @@ static CONNECTORS_COMPATIBLE_FORMATS: LazyLock vec![Encode::Native], Format::Plain => vec![Encode::Bytes, Encode::Json], ), - S3_CONNECTOR => hashmap!( + LEGACY_S3_CONNECTOR => hashmap!( Format::Plain => vec![Encode::Csv, Encode::Json], ), OPENDAL_S3_CONNECTOR => hashmap!( @@ -143,10 +143,10 @@ pub fn validate_compatibility( // reject s3_v2 creation return Err(RwError::from(Deprecated( OPENDAL_S3_CONNECTOR.to_owned(), - S3_CONNECTOR.to_owned(), + LEGACY_S3_CONNECTOR.to_owned(), ))); } - if connector == S3_CONNECTOR { + if connector == LEGACY_S3_CONNECTOR { // S3 connector is deprecated, use OPENDAL_S3_CONNECTOR instead // do s3 -> s3_v2 migration let entry = props.get_mut(UPSTREAM_SOURCE_KEY).unwrap(); diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index a8590d3480947..e68d34e85928d 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -253,7 +253,7 @@ pub fn bind_sql_columns(column_defs: &[ColumnDef]) -> Result> generated_or_default_column: None, description: None, additional_column: AdditionalColumn { column_type: None }, - version: ColumnDescVersion::Pr13707, + version: ColumnDescVersion::LATEST, system_column: None, }, is_hidden: false, diff --git a/src/frontend/src/observer/observer_manager.rs b/src/frontend/src/observer/observer_manager.rs index faff51aec1a82..299120bbf17ee 100644 --- a/src/frontend/src/observer/observer_manager.rs +++ b/src/frontend/src/observer/observer_manager.rs @@ -20,6 +20,7 @@ use parking_lot::RwLock; use risingwave_batch::worker_manager::worker_node_manager::WorkerNodeManagerRef; use risingwave_common::catalog::CatalogVersion; use risingwave_common::hash::WorkerSlotMapping; +use risingwave_common::license::LicenseManager; use risingwave_common::secret::LocalSecretManager; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; @@ -114,6 +115,9 @@ impl ObserverState for FrontendObserverNode { Info::Recovery(_) => { self.compute_client_pool.invalidate_all(); } + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } } } @@ -147,6 +151,7 @@ impl ObserverState for FrontendObserverNode { session_params, version, secrets, + compute_node_total_cpu_count, } = snapshot; for db in databases { @@ -208,6 +213,7 @@ impl ObserverState for FrontendObserverNode { *self.session_params.write() = serde_json::from_str(&session_params.unwrap().params).unwrap(); LocalSecretManager::global().init_secrets(secrets); + LicenseManager::get().update_cpu_core_count(compute_node_total_cpu_count as _); } } diff --git a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs index 9256131f569ad..a902160066844 100644 --- a/src/frontend/src/optimizer/plan_node/batch_hash_join.rs +++ b/src/frontend/src/optimizer/plan_node/batch_hash_join.rs @@ -15,13 +15,13 @@ use pretty_xmlish::{Pretty, XmlNode}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::HashJoinNode; -use risingwave_pb::plan_common::JoinType; +use risingwave_pb::plan_common::{AsOfJoinDesc, JoinType}; use super::batch::prelude::*; use super::utils::{childless_record, Distill}; use super::{ - generic, EqJoinPredicate, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, ToBatchPb, - ToDistributedBatch, + generic, EqJoinPredicate, ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, + ToBatchPb, ToDistributedBatch, }; use crate::error::Result; use crate::expr::{Expr, ExprRewriter, ExprVisitor}; @@ -38,14 +38,19 @@ use crate::utils::ColIndexMappingRewriteExt; pub struct BatchHashJoin { pub base: PlanBase, core: generic::Join, - /// The join condition must be equivalent to `logical.on`, but separated into equal and /// non-equal parts to facilitate execution later eq_join_predicate: EqJoinPredicate, + /// `AsOf` desc + asof_desc: Option, } impl BatchHashJoin { - pub fn new(core: generic::Join, eq_join_predicate: EqJoinPredicate) -> Self { + pub fn new( + core: generic::Join, + eq_join_predicate: EqJoinPredicate, + asof_desc: Option, + ) -> Self { let dist = Self::derive_dist(core.left.distribution(), core.right.distribution(), &core); let base = PlanBase::new_batch_with_core(&core, dist, Order::any()); @@ -53,6 +58,7 @@ impl BatchHashJoin { base, core, eq_join_predicate, + asof_desc, } } @@ -66,11 +72,16 @@ impl BatchHashJoin { // we can not derive the hash distribution from the side where outer join can generate a // NULL row (Distribution::HashShard(_), Distribution::HashShard(_)) => match join.join_type { - JoinType::AsofInner | JoinType::AsofLeftOuter | JoinType::Unspecified => { + JoinType::Unspecified => { unreachable!() } JoinType::FullOuter => Distribution::SomeShard, - JoinType::Inner | JoinType::LeftOuter | JoinType::LeftSemi | JoinType::LeftAnti => { + JoinType::Inner + | JoinType::LeftOuter + | JoinType::LeftSemi + | JoinType::LeftAnti + | JoinType::AsofInner + | JoinType::AsofLeftOuter => { let l2o = join.l2i_col_mapping().composite(&join.i2o_col_mapping()); l2o.rewrite_provided_distribution(left) } @@ -127,7 +138,7 @@ impl PlanTreeNodeBinary for BatchHashJoin { let mut core = self.core.clone(); core.left = left; core.right = right; - Self::new(core, self.eq_join_predicate.clone()) + Self::new(core, self.eq_join_predicate.clone(), self.asof_desc) } } @@ -215,6 +226,7 @@ impl ToBatchPb for BatchHashJoin { .as_expr_unless_true() .map(|x| x.to_expr_proto()), output_indices: self.core.output_indices.iter().map(|&x| x as u32).collect(), + asof_desc: self.asof_desc, }) } } @@ -238,7 +250,15 @@ impl ExprRewritable for BatchHashJoin { fn rewrite_exprs(&self, r: &mut dyn ExprRewriter) -> PlanRef { let mut core = self.core.clone(); core.rewrite_exprs(r); - Self::new(core, self.eq_join_predicate.rewrite_exprs(r)).into() + let eq_join_predicate = self.eq_join_predicate.rewrite_exprs(r); + let desc = self.asof_desc.map(|_| { + LogicalJoin::get_inequality_desc_from_predicate( + eq_join_predicate.other_cond().clone(), + core.left.schema().len(), + ) + .unwrap() + }); + Self::new(core, eq_join_predicate, desc).into() } } diff --git a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs index 449e26434d34b..071e63dfc6281 100644 --- a/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs +++ b/src/frontend/src/optimizer/plan_node/eq_join_predicate.rs @@ -162,6 +162,16 @@ impl EqJoinPredicate { &mut self.other_cond } + /// Get the equal predicate + pub fn eq_predicate(&self) -> Self { + Self { + other_cond: Condition::true_cond(), + eq_keys: self.eq_keys.clone(), + left_cols_num: self.left_cols_num, + right_cols_num: self.right_cols_num, + } + } + /// Get a reference to the join predicate's eq keys. /// /// Note: `right_col_index` starts from `left_cols_num` diff --git a/src/frontend/src/optimizer/plan_node/logical_join.rs b/src/frontend/src/optimizer/plan_node/logical_join.rs index 208e947ef40d5..180989a2b0467 100644 --- a/src/frontend/src/optimizer/plan_node/logical_join.rs +++ b/src/frontend/src/optimizer/plan_node/logical_join.rs @@ -17,7 +17,9 @@ use std::collections::HashMap; use fixedbitset::FixedBitSet; use itertools::{EitherOrBoth, Itertools}; use pretty_xmlish::{Pretty, XmlNode}; -use risingwave_pb::plan_common::JoinType; +use risingwave_expr::bail; +use risingwave_pb::expr::expr_node::PbType; +use risingwave_pb::plan_common::{AsOfJoinDesc, JoinType, PbAsOfJoinInequalityType}; use risingwave_pb::stream_plan::StreamScanType; use risingwave_sqlparser::ast::AsOf; @@ -1379,7 +1381,7 @@ impl LogicalJoin { let logical_join = self.clone_with_left_right(left, right); let inequality_desc = - StreamAsOfJoin::get_inequality_desc_from_predicate(predicate.clone(), left_len)?; + Self::get_inequality_desc_from_predicate(predicate.other_cond().clone(), left_len)?; Ok(StreamAsOfJoin::new( logical_join.core.clone(), @@ -1387,17 +1389,71 @@ impl LogicalJoin { inequality_desc, )) } -} -impl ToBatch for LogicalJoin { - fn to_batch(&self) -> Result { - if JoinType::AsofInner == self.join_type() || JoinType::AsofLeftOuter == self.join_type() { - return Err(ErrorCode::NotSupported( - "AsOf join in batch query".to_owned(), - "AsOf join is only supported in streaming query".to_owned(), + /// Convert the logical `AsOf` join to a Hash join + a Group top 1. + fn to_batch_asof_join( + &self, + logical_join: generic::Join, + predicate: EqJoinPredicate, + ) -> Result { + use super::batch::prelude::*; + + if predicate.eq_keys().is_empty() { + return Err(ErrorCode::InvalidInputSyntax( + "AsOf join requires at least 1 equal condition".to_owned(), ) .into()); } + + let left_schema_len = logical_join.left.schema().len(); + let asof_desc = + Self::get_inequality_desc_from_predicate(predicate.non_eq_cond(), left_schema_len)?; + + let batch_join = BatchHashJoin::new(logical_join, predicate, Some(asof_desc)); + Ok(batch_join.into()) + } + + pub fn get_inequality_desc_from_predicate( + predicate: Condition, + left_input_len: usize, + ) -> Result { + let expr: ExprImpl = predicate.into(); + if let Some((left_input_ref, expr_type, right_input_ref)) = expr.as_comparison_cond() { + if left_input_ref.index() < left_input_len && right_input_ref.index() >= left_input_len + { + Ok(AsOfJoinDesc { + left_idx: left_input_ref.index() as u32, + right_idx: (right_input_ref.index() - left_input_len) as u32, + inequality_type: Self::expr_type_to_comparison_type(expr_type)?.into(), + }) + } else { + bail!("inequal condition from the same side should be push down in optimizer"); + } + } else { + Err(ErrorCode::InvalidInputSyntax( + "AsOf join requires exactly 1 ineuquality condition".to_owned(), + ) + .into()) + } + } + + fn expr_type_to_comparison_type(expr_type: PbType) -> Result { + match expr_type { + PbType::LessThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLt), + PbType::LessThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLe), + PbType::GreaterThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGt), + PbType::GreaterThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGe), + _ => Err(ErrorCode::InvalidInputSyntax(format!( + "Invalid comparison type: {}", + expr_type.as_str_name() + )) + .into()), + } + } +} + +impl ToBatch for LogicalJoin { + fn to_batch(&self) -> Result { let predicate = EqJoinPredicate::create( self.left().schema().len(), self.right().schema().len(), @@ -1411,7 +1467,9 @@ impl ToBatch for LogicalJoin { let ctx = self.base.ctx(); let config = ctx.session_ctx().config(); - if predicate.has_eq() { + if self.join_type() == JoinType::AsofInner || self.join_type() == JoinType::AsofLeftOuter { + self.to_batch_asof_join(logical_join, predicate) + } else if predicate.has_eq() { if !predicate.eq_keys_are_type_aligned() { return Err(ErrorCode::InternalError(format!( "Join eq keys are not aligned for predicate: {predicate:?}" @@ -1427,7 +1485,7 @@ impl ToBatch for LogicalJoin { } } - Ok(BatchHashJoin::new(logical_join, predicate).into()) + Ok(BatchHashJoin::new(logical_join, predicate, None).into()) } else { // Convert to Nested-loop Join for non-equal joins Ok(BatchNestedLoopJoin::new(logical_join).into()) diff --git a/src/frontend/src/optimizer/plan_node/stream_asof_join.rs b/src/frontend/src/optimizer/plan_node/stream_asof_join.rs index 49d6e9c43aabc..b0c417e749cd4 100644 --- a/src/frontend/src/optimizer/plan_node/stream_asof_join.rs +++ b/src/frontend/src/optimizer/plan_node/stream_asof_join.rs @@ -15,9 +15,7 @@ use itertools::Itertools; use pretty_xmlish::{Pretty, XmlNode}; use risingwave_common::util::sort_util::OrderType; -use risingwave_expr::bail; -use risingwave_pb::expr::expr_node::PbType; -use risingwave_pb::plan_common::{AsOfJoinDesc, AsOfJoinType, JoinType, PbAsOfJoinInequalityType}; +use risingwave_pb::plan_common::{AsOfJoinDesc, AsOfJoinType, JoinType}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::AsOfJoinNode; @@ -26,10 +24,10 @@ use super::utils::{ childless_record, plan_node_name, watermark_pretty, Distill, TableCatalogBuilder, }; use super::{ - generic, ExprRewritable, PlanBase, PlanRef, PlanTreeNodeBinary, StreamJoinCommon, StreamNode, + generic, ExprRewritable, LogicalJoin, PlanBase, PlanRef, PlanTreeNodeBinary, StreamJoinCommon, + StreamNode, }; -use crate::error::{ErrorCode, Result}; -use crate::expr::{ExprImpl, ExprRewriter, ExprVisitor}; +use crate::expr::{ExprRewriter, ExprVisitor}; use crate::optimizer::plan_node::expr_visitable::ExprVisitable; use crate::optimizer::plan_node::utils::IndicesDisplay; use crate::optimizer::plan_node::{EqJoinPredicate, EqJoinPredicateDisplay}; @@ -97,44 +95,6 @@ impl StreamAsOfJoin { } } - pub fn get_inequality_desc_from_predicate( - predicate: EqJoinPredicate, - left_input_len: usize, - ) -> Result { - let expr: ExprImpl = predicate.other_cond().clone().into(); - if let Some((left_input_ref, expr_type, right_input_ref)) = expr.as_comparison_cond() { - if left_input_ref.index() < left_input_len && right_input_ref.index() >= left_input_len - { - Ok(AsOfJoinDesc { - left_idx: left_input_ref.index() as u32, - right_idx: (right_input_ref.index() - left_input_len) as u32, - inequality_type: Self::expr_type_to_comparison_type(expr_type)?.into(), - }) - } else { - bail!("inequal condition from the same side should be push down in optimizer"); - } - } else { - Err(ErrorCode::InvalidInputSyntax( - "AsOf join requires exactly 1 ineuquality condition".to_owned(), - ) - .into()) - } - } - - fn expr_type_to_comparison_type(expr_type: PbType) -> Result { - match expr_type { - PbType::LessThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLt), - PbType::LessThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeLe), - PbType::GreaterThan => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGt), - PbType::GreaterThanOrEqual => Ok(PbAsOfJoinInequalityType::AsOfInequalityTypeGe), - _ => Err(ErrorCode::InvalidInputSyntax(format!( - "Invalid comparison type: {}", - expr_type.as_str_name() - )) - .into()), - } - } - /// Get join type pub fn join_type(&self) -> JoinType { self.core.join_type @@ -332,8 +292,8 @@ impl ExprRewritable for StreamAsOfJoin { let mut core = self.core.clone(); core.rewrite_exprs(r); let eq_join_predicate = self.eq_join_predicate.rewrite_exprs(r); - let desc = Self::get_inequality_desc_from_predicate( - eq_join_predicate.clone(), + let desc = LogicalJoin::get_inequality_desc_from_predicate( + eq_join_predicate.other_cond().clone(), core.left.schema().len(), ) .unwrap(); diff --git a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs index e52ef3a2965a7..7146fdf889cc2 100644 --- a/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_hash_agg.rs @@ -216,7 +216,7 @@ impl StreamNode for StreamHashAgg { .collect(), row_count_index: self.row_count_idx as u32, emit_on_window_close: self.base.emit_on_window_close(), - version: PbAggNodeVersion::Issue13465 as _, + version: PbAggNodeVersion::LATEST as _, })) } } diff --git a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs index 9a67fab72d682..3c9102a9434d5 100644 --- a/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/stream_simple_agg.rs @@ -148,7 +148,7 @@ impl StreamNode for StreamSimpleAgg { }) .collect(), row_count_index: self.row_count_idx as u32, - version: PbAggNodeVersion::Issue13465 as _, + version: PbAggNodeVersion::LATEST as _, must_output_per_barrier: self.must_output_per_barrier, })) } diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index cf6319c894a0e..f1915dc7e5748 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -657,7 +657,7 @@ pub(crate) mod tests { let eq_join_predicate = EqJoinPredicate::new(Condition::true_cond(), vec![eq_key_1, eq_key_2], 2, 2); let hash_join_node: PlanRef = - BatchHashJoin::new(logical_join_node, eq_join_predicate).into(); + BatchHashJoin::new(logical_join_node, eq_join_predicate, None).into(); let batch_exchange_node: PlanRef = BatchExchange::new( hash_join_node.clone(), Order::default(), diff --git a/src/license/src/cpu.rs b/src/license/src/cpu.rs index 5dd279f58ec6c..f65687ba7fd5f 100644 --- a/src/license/src/cpu.rs +++ b/src/license/src/cpu.rs @@ -12,41 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::num::NonZeroU64; - -use thiserror::Error; - -use crate::{LicenseKeyError, LicenseManager}; - -/// The error type for CPU core limit exceeded as per the license key. -#[derive(Debug, Clone, Error)] -#[error("invalid license key")] -pub enum CpuCoreLimitExceeded { - #[error("cannot check CPU core limit due to license key error")] - LicenseKeyError(#[from] LicenseKeyError), - - #[error( - "CPU core limit exceeded as per the license key, \ - requesting {actual} while the maximum allowed is {limit}" - )] - Exceeded { limit: NonZeroU64, actual: u64 }, -} - -impl LicenseManager { - /// Check if the given CPU core count exceeds the limit as per the license key. - pub fn check_cpu_core_limit(&self, cpu_core_count: u64) -> Result<(), CpuCoreLimitExceeded> { - let license = self.license()?; - - match license.cpu_core_limit { - Some(limit) if cpu_core_count > limit.get() => Err(CpuCoreLimitExceeded::Exceeded { - limit, - actual: cpu_core_count, - }), - _ => Ok(()), - } - } -} - // Tests below only work in debug mode. #[cfg(debug_assertions)] #[cfg(test)] @@ -54,14 +19,14 @@ mod tests { use expect_test::expect; use thiserror_ext::AsReport as _; - use super::*; - use crate::{LicenseKey, TEST_PAID_LICENSE_KEY_CONTENT}; + use crate::{Feature, LicenseKey, LicenseManager, TEST_PAID_LICENSE_KEY_CONTENT}; - fn do_test(key: &str, cpu_core_count: u64, expect: expect_test::Expect) { + fn do_test(key: &str, cpu_core_count: usize, expect: expect_test::Expect) { let manager = LicenseManager::new(); manager.refresh(LicenseKey(key)); + manager.update_cpu_core_count(cpu_core_count); - match manager.check_cpu_core_limit(cpu_core_count) { + match Feature::TestPaid.check_available_with(&manager) { Ok(_) => expect.assert_eq("ok"), Err(error) => expect.assert_eq(&error.to_report_string()), } @@ -72,28 +37,31 @@ mod tests { do_test(TEST_PAID_LICENSE_KEY_CONTENT, 114514, expect!["ok"]); } - #[test] - fn test_no_license_key_no_limit() { - do_test("", 114514, expect!["ok"]); - } - #[test] fn test_invalid_license_key() { const KEY: &str = "invalid"; - do_test(KEY, 0, expect!["cannot check CPU core limit due to license key error: invalid license key: InvalidToken"]); - do_test(KEY, 114514, expect!["cannot check CPU core limit due to license key error: invalid license key: InvalidToken"]); + do_test( + KEY, + 0, + expect!["feature TestPaid is not available due to license error: invalid license key: InvalidToken"], + ); + do_test( + KEY, + 114514, + expect!["feature TestPaid is not available due to license error: invalid license key: InvalidToken"], + ); } #[test] fn test_limit() { const KEY: &str = "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.\ - eyJzdWIiOiJmcmVlLXRlc3QtMzIiLCJpc3MiOiJwcm9kLnJpc2luZ3dhdmUuY29tIiwidGllciI6ImZyZWUiLCJleHAiOjE4NTI1NTk5OTksImlhdCI6MTcyMzcwMTk5NCwiY3B1X2NvcmVfbGltaXQiOjMyfQ.\ - rsATtzlduLUkGQeXkOROtyGUpafdDhi18iKdYAzAldWQuO9KevNcnD8a6geCShZSGte65bI7oYtv7GHx8i66ge3B1SVsgGgYr10ebphPUNUQenYoN0mpD4Wn0prPStOgANzYZOI2ntMGAaeWStji1x67_iho6r0W9r6RX3kMvzFSbiObSIfvTdrMULeg-xeHc3bT_ErRhaXq7MAa2Oiq3lcK2sNgEvc9KYSP9YbhSik9CBkc8lcyeVoc48SSWEaBU-c8-Ge0fzjgWHI9KIsUV5Ihe66KEfs0PqdRoSWbgskYGzA3o8wHIbtJbJiPzra373kkFH9MGY0HOsw9QeJLGQ"; + eyJzdWIiOiJwYWlkLXRlc3QtMzIiLCJpc3MiOiJ0ZXN0LnJpc2luZ3dhdmUuY29tIiwidGllciI6InBhaWQiLCJleHAiOjIxNTA0OTU5OTksImlhdCI6MTczNzYxMjQ5NSwiY3B1X2NvcmVfbGltaXQiOjMyfQ.\ + SQpX2Dmon5Mb04VUbHyxsU7owJhcdLZHqUefxAXBwG5AqgKdpfS0XUePW5E4D-EfxtH_cWJiD4QDFsfdRUz88g_n_KvfNUObMW7NV5TUoRs_ImtS4ySugExNX3JzJi71QqgI8kugStQ7uOR9kZ_C-cCc_IG2CwwEmhhW1Ij0vX7qjhG5JNMit_bhxPY7Rh27ppgPTqWxJFTTsw-9B7O5WR_yIlaDjxVzk0ALm_j6DPB249gG3dkeK0rP0AK_ip2cK6iQdy8Cge7ATD6yUh4c_aR6GILDF6-vyB7QdWU6DdQS4KhdkPNWoe_Z9psotcXQJ7NhQ39hk8tdLzmTfGDDBA"; do_test(KEY, 31, expect!["ok"]); do_test(KEY, 32, expect!["ok"]); - do_test(KEY, 33, expect!["CPU core limit exceeded as per the license key, requesting 33 while the maximum allowed is 32"]); + do_test(KEY, 33, expect!["feature TestPaid is not available due to license error: the license key is currently not effective because the CPU core in the cluster (33) exceeds the maximum allowed by the license key (32); consider removing some nodes or acquiring a new license key with a higher limit"]); } } diff --git a/src/license/src/feature.rs b/src/license/src/feature.rs index 5243001634e22..0d10d76869b0f 100644 --- a/src/license/src/feature.rs +++ b/src/license/src/feature.rs @@ -14,7 +14,7 @@ use thiserror::Error; -use super::{report_telemetry, License, LicenseKeyError, LicenseManager, Tier}; +use super::{report_telemetry, LicenseError, LicenseManager, Tier}; /// Define all features that are available based on the tier of the license. /// @@ -113,14 +113,17 @@ pub enum FeatureNotAvailable { #[error("feature {feature:?} is not available due to license error")] LicenseError { feature: Feature, - source: LicenseKeyError, + source: LicenseError, }, } impl Feature { - /// Check whether the feature is available based on the current license. - pub fn check_available(self) -> Result<(), FeatureNotAvailable> { - let check_res = match LicenseManager::get().license() { + /// Check whether the feature is available based on the given license manager. + pub(crate) fn check_available_with( + self, + manager: &LicenseManager, + ) -> Result<(), FeatureNotAvailable> { + let check_res = match manager.license() { Ok(license) => { if license.tier >= self.min_tier() { Ok(()) @@ -131,22 +134,19 @@ impl Feature { }) } } - Err(error) => { - // If there's a license key error, we still try against the default license first - // to see if the feature is available for free. - if License::default().tier >= self.min_tier() { - Ok(()) - } else { - Err(FeatureNotAvailable::LicenseError { - feature: self, - source: error, - }) - } - } + Err(error) => Err(FeatureNotAvailable::LicenseError { + feature: self, + source: error, + }), }; report_telemetry(&self, self.get_feature_name(), check_res.is_ok()); check_res } + + /// Check whether the feature is available based on the current license. + pub fn check_available(self) -> Result<(), FeatureNotAvailable> { + self.check_available_with(LicenseManager::get()) + } } diff --git a/src/license/src/lib.rs b/src/license/src/lib.rs index 56a98f97ba5b2..1f30cc0875727 100644 --- a/src/license/src/lib.rs +++ b/src/license/src/lib.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![feature(let_chains)] + mod cpu; mod feature; mod key; diff --git a/src/license/src/manager.rs b/src/license/src/manager.rs index f8d9d8d42a627..ab176bf1db37e 100644 --- a/src/license/src/manager.rs +++ b/src/license/src/manager.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::num::NonZeroU64; +use std::num::NonZeroUsize; use std::sync::{LazyLock, RwLock}; use jsonwebtoken::{Algorithm, DecodingKey, Validation}; @@ -81,7 +81,7 @@ pub struct License { pub tier: Tier, /// Maximum number of compute-node CPU cores allowed to use. Typically used for the paid tier. - pub cpu_core_limit: Option, + pub cpu_core_limit: Option, /// Expiration time in seconds since UNIX epoch. /// @@ -106,11 +106,21 @@ impl Default for License { /// The error type for invalid license key when verifying as JWT. #[derive(Debug, Clone, Error)] -#[error("invalid license key")] -pub struct LicenseKeyError(#[source] jsonwebtoken::errors::Error); +pub enum LicenseError { + #[error("invalid license key")] + InvalidKey(#[source] jsonwebtoken::errors::Error), + + #[error( + "the license key is currently not effective because the CPU core in the cluster \ + ({actual}) exceeds the maximum allowed by the license key ({limit}); \ + consider removing some nodes or acquiring a new license key with a higher limit" + )] + CpuCoreLimitExceeded { limit: NonZeroUsize, actual: usize }, +} struct Inner { - license: Result, + license: Result, + cached_cpu_core_count: usize, } /// The singleton license manager. @@ -129,6 +139,7 @@ impl LicenseManager { Self { inner: RwLock::new(Inner { license: Ok(License::default()), + cached_cpu_core_count: 0, }), } } @@ -162,7 +173,7 @@ impl LicenseManager { inner.license = match jsonwebtoken::decode(license_key, &PUBLIC_KEY, &validation) { Ok(data) => Ok(data.claims), - Err(error) => Err(LicenseKeyError(error)), + Err(error) => Err(LicenseError::InvalidKey(error)), }; match &inner.license { @@ -171,22 +182,40 @@ impl LicenseManager { } } + /// Update the cached CPU core count. + pub fn update_cpu_core_count(&self, cpu_core_count: usize) { + let mut inner = self.inner.write().unwrap(); + inner.cached_cpu_core_count = cpu_core_count; + } + /// Get the current license if it is valid. /// /// Since the license can expire, the returned license should not be cached by the caller. /// /// Prefer calling [`crate::Feature::check_available`] to check the availability of a feature, /// other than directly calling this method and checking the content of the license. - pub fn license(&self) -> Result { - let license = self.inner.read().unwrap().license.clone()?; + pub fn license(&self) -> Result { + let inner = self.inner.read().unwrap(); + let license = inner.license.clone()?; // Check the expiration time additionally. if license.exp < jsonwebtoken::get_current_timestamp() { - return Err(LicenseKeyError( + return Err(LicenseError::InvalidKey( jsonwebtoken::errors::ErrorKind::ExpiredSignature.into(), )); } + // Check the CPU core limit. + let actual_cpu_core = inner.cached_cpu_core_count; + if let Some(limit) = license.cpu_core_limit + && actual_cpu_core > limit.get() + { + return Err(LicenseError::CpuCoreLimitExceeded { + limit, + actual: actual_cpu_core, + }); + } + Ok(license) } } diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index b766689cc9bf3..91a8c5aae6292 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -207,8 +207,16 @@ impl NotificationServiceImpl { Ok((tables, notification_version)) } + async fn get_compute_node_total_cpu_count(&self) -> usize { + self.metadata_manager + .cluster_controller + .compute_node_total_cpu_count() + .await + } + async fn compactor_subscribe(&self) -> MetaResult { let (tables, catalog_version) = self.get_tables_and_creating_tables_snapshot().await?; + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; Ok(MetaSnapshot { tables, @@ -216,6 +224,7 @@ impl NotificationServiceImpl { catalog_version, ..Default::default() }), + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } @@ -266,6 +275,8 @@ impl NotificationServiceImpl { .context("failed to encode session params")?, }); + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; + Ok(MetaSnapshot { databases, schemas, @@ -289,6 +300,7 @@ impl NotificationServiceImpl { serving_worker_slot_mappings, streaming_worker_slot_mappings, session_params, + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } @@ -301,6 +313,7 @@ impl NotificationServiceImpl { .await; let hummock_write_limits = self.hummock_manager.write_limits().await; let meta_backup_manifest_id = self.backup_manager.manifest().manifest_id; + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; Ok(MetaSnapshot { tables, @@ -315,18 +328,22 @@ impl NotificationServiceImpl { hummock_write_limits: Some(WriteLimits { write_limits: hummock_write_limits, }), + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } async fn compute_subscribe(&self) -> MetaResult { let (secrets, catalog_version) = self.get_decrypted_secret_snapshot().await?; + let compute_node_total_cpu_count = self.get_compute_node_total_cpu_count().await; + Ok(MetaSnapshot { secrets, version: Some(SnapshotVersion { catalog_version, ..Default::default() }), + compute_node_total_cpu_count: compute_node_total_cpu_count as _, ..Default::default() }) } diff --git a/src/meta/src/barrier/context/recovery.rs b/src/meta/src/barrier/context/recovery.rs index c13eaabf90822..2dd75e1bb57c9 100644 --- a/src/meta/src/barrier/context/recovery.rs +++ b/src/meta/src/barrier/context/recovery.rs @@ -427,13 +427,13 @@ impl GlobalBarrierWorkerContextImpl { .collect(); if expired_worker_slots.is_empty() { - debug!("no expired worker slots, skipping."); + info!("no expired worker slots, skipping."); return self.resolve_graph_info(None).await; } - debug!("start migrate actors."); + info!("start migrate actors."); let mut to_migrate_worker_slots = expired_worker_slots.into_iter().rev().collect_vec(); - debug!("got to migrate worker slots {:#?}", to_migrate_worker_slots); + info!("got to migrate worker slots {:#?}", to_migrate_worker_slots); let mut inuse_worker_slots: HashSet<_> = all_inuse_worker_slots .intersection(&active_worker_slots) @@ -535,6 +535,8 @@ impl GlobalBarrierWorkerContextImpl { warn!(?changed, "get worker changed or timed out. Retry migrate"); } + info!("migration plan {:?}", plan); + mgr.catalog_controller.migrate_actors(plan).await?; info!("migrate actors succeed."); diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index bde42304d58bd..d5b144ce54765 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -39,8 +39,8 @@ use risingwave_pb::meta::update_worker_node_schedulability_request::Schedulabili use sea_orm::prelude::Expr; use sea_orm::ActiveValue::Set; use sea_orm::{ - ActiveModelTrait, ColumnTrait, DatabaseConnection, DatabaseTransaction, EntityTrait, - QueryFilter, QuerySelect, TransactionTrait, + ActiveModelTrait, ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter, QuerySelect, + TransactionTrait, }; use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; @@ -88,7 +88,7 @@ impl From for PbWorkerNode { 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, + resource: Some(info.2.resource), started_at: info.2.started_at, } } @@ -119,6 +119,24 @@ impl ClusterController { self.inner.read().await.count_worker_by_type().await } + pub async fn compute_node_total_cpu_count(&self) -> usize { + self.inner.read().await.compute_node_total_cpu_count() + } + + async fn update_compute_node_total_cpu_count(&self) -> MetaResult<()> { + let total_cpu_cores = self.compute_node_total_cpu_count().await; + + // Update local license manager. + LicenseManager::get().update_cpu_core_count(total_cpu_cores); + // Notify all other nodes. + self.env.notification_manager().notify_all_without_version( + Operation::Update, // unused + Info::ComputeNodeTotalCpuCount(total_cpu_cores as _), + ); + + Ok(()) + } + /// A worker node will immediately register itself to meta when it bootstraps. /// The meta will assign it with a unique ID and set its state as `Starting`. /// When the worker node is fully ready to serve, it will request meta again @@ -130,7 +148,8 @@ impl ClusterController { property: AddNodeProperty, resource: PbResource, ) -> MetaResult { - self.inner + let worker_id = self + .inner .write() .await .add_worker( @@ -140,7 +159,13 @@ impl ClusterController { resource, self.max_heartbeat_interval, ) - .await + .await?; + + if r#type == PbWorkerType::ComputeNode { + self.update_compute_node_total_cpu_count().await?; + } + + Ok(worker_id) } pub async fn activate_worker(&self, worker_id: WorkerId) -> MetaResult<()> { @@ -164,13 +189,15 @@ impl ClusterController { } pub async fn delete_worker(&self, host_address: HostAddress) -> MetaResult { - let mut inner = self.inner.write().await; - let worker = inner.delete_worker(host_address).await?; + let worker = self.inner.write().await.delete_worker(host_address).await?; + if worker.r#type() == PbWorkerType::ComputeNode { self.env .notification_manager() .notify_frontend(Operation::Delete, Info::Node(worker.clone())) .await; + + self.update_compute_node_total_cpu_count().await?; } // Notify local subscribers. @@ -432,7 +459,8 @@ pub struct WorkerExtraInfo { // Unix timestamp that the worker will expire at. expire_at: Option, started_at: Option, - resource: Option, + resource: PbResource, + r#type: PbWorkerType, } impl WorkerExtraInfo { @@ -561,7 +589,7 @@ impl ClusterControllerInner { resource: PbResource, ) -> MetaResult<()> { if let Some(info) = self.worker_extra_info.get_mut(&worker_id) { - info.resource = Some(resource); + info.resource = resource; info.update_started_at(); Ok(()) } else { @@ -585,41 +613,12 @@ impl ClusterControllerInner { } } - /// Check if the total CPU cores in the cluster exceed the license limit, after counting the - /// newly joined compute node. - pub async fn check_cpu_core_limit_on_newly_joined_compute_node( - &self, - txn: &DatabaseTransaction, - host_address: &HostAddress, - resource: &PbResource, - ) -> MetaResult<()> { - let this = resource.total_cpu_cores; - - let other_worker_ids: Vec = Worker::find() - .filter( - (worker::Column::Host - .eq(host_address.host.clone()) - .and(worker::Column::Port.eq(host_address.port))) - .not() - .and(worker::Column::WorkerType.eq(WorkerType::ComputeNode)), - ) - .select_only() - .column(worker::Column::WorkerId) - .into_tuple() - .all(txn) - .await?; - - let others = other_worker_ids - .into_iter() - .flat_map(|id| self.worker_extra_info.get(&id)) - .flat_map(|info| info.resource.as_ref().map(|r| r.total_cpu_cores)) - .sum::(); - - LicenseManager::get() - .check_cpu_core_limit(this + others) - .map_err(anyhow::Error::from)?; - - Ok(()) + fn compute_node_total_cpu_count(&self) -> usize { + self.worker_extra_info + .values() + .filter(|info| info.r#type == PbWorkerType::ComputeNode) + .map(|info| info.resource.total_cpu_cores as usize) + .sum() } pub async fn add_worker( @@ -632,11 +631,6 @@ impl ClusterControllerInner { ) -> MetaResult { let txn = self.db.begin().await?; - if let PbWorkerType::ComputeNode = r#type { - self.check_cpu_core_limit_on_newly_joined_compute_node(&txn, &host_address, &resource) - .await?; - } - let worker = Worker::find() .filter( worker::Column::Host @@ -768,8 +762,9 @@ impl ClusterControllerInner { } let extra_info = WorkerExtraInfo { started_at: Some(timestamp_now_sec()), - resource: Some(resource), - ..Default::default() + expire_at: None, + resource, + r#type, }; self.worker_extra_info.insert(worker_id, extra_info); @@ -834,7 +829,9 @@ impl ClusterControllerInner { if let Some(txn_id) = &worker.transaction_id { self.available_transactional_ids.push_back(*txn_id); } - Ok(WorkerInfo(worker, property, extra_info).into()) + let worker: PbWorkerNode = WorkerInfo(worker, property, extra_info).into(); + + Ok(worker) } pub fn heartbeat(&mut self, worker_id: WorkerId, ttl: Duration) -> MetaResult<()> { diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index f2c95b0f59e3b..b41e0cd39ba01 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -20,7 +20,7 @@ use anyhow::Context; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::hash::{VnodeCount, VnodeCountCompat, WorkerSlotId}; -use risingwave_common::util::stream_graph_visitor::visit_stream_node; +use risingwave_common::util::stream_graph_visitor::visit_stream_node_mut; use risingwave_meta_model::actor::ActorStatus; use risingwave_meta_model::fragment::DistributionType; use risingwave_meta_model::object::ObjectType; @@ -227,7 +227,7 @@ impl CatalogController { let stream_node = { let actor_template = pb_actors.first().cloned().unwrap(); let mut stream_node = actor_template.nodes.unwrap(); - visit_stream_node(&mut stream_node, |body| { + visit_stream_node_mut(&mut stream_node, |body| { if let NodeBody::Merge(m) = body { m.upstream_actor_id = vec![]; } @@ -244,7 +244,7 @@ impl CatalogController { let node = actor.nodes.as_mut().context("nodes are empty")?; - visit_stream_node(node, |body| { + visit_stream_node_mut(node, |body| { if let NodeBody::Merge(m) = body { let mut upstream_actor_ids = vec![]; swap(&mut m.upstream_actor_id, &mut upstream_actor_ids); @@ -435,7 +435,7 @@ impl CatalogController { let pb_nodes = { let mut nodes = stream_node_template.clone(); - visit_stream_node(&mut nodes, |body| { + visit_stream_node_mut(&mut nodes, |body| { if let NodeBody::Merge(m) = body && let Some(upstream_actor_ids) = upstream_fragment_actors.get(&(m.upstream_fragment_id as _)) @@ -1125,12 +1125,17 @@ impl CatalogController { .insert(*actor_id); } - let expired_workers: HashSet<_> = plan.keys().map(|k| k.worker_id() as WorkerId).collect(); + let expired_or_changed_workers: HashSet<_> = + plan.keys().map(|k| k.worker_id() as WorkerId).collect(); let mut actor_migration_plan = HashMap::new(); for (worker, fragment) in actor_locations { - if expired_workers.contains(&worker) { - for (_, actors) in fragment { + if expired_or_changed_workers.contains(&worker) { + for (fragment_id, actors) in fragment { + debug!( + "worker {} expired or changed, migrating fragment {}", + worker, fragment_id + ); let worker_slot_to_actor: HashMap<_, _> = actors .iter() .enumerate() @@ -1140,8 +1145,9 @@ impl CatalogController { .collect(); for (worker_slot, actor) in worker_slot_to_actor { - actor_migration_plan - .insert(actor, plan[&worker_slot].worker_id() as WorkerId); + if let Some(target) = plan.get(&worker_slot) { + actor_migration_plan.insert(actor, target.worker_id() as WorkerId); + } } } } @@ -1625,7 +1631,7 @@ mod tests { use itertools::Itertools; use risingwave_common::hash::{ActorMapping, VirtualNode, VnodeCount}; use risingwave_common::util::iter_util::ZipEqDebug; - use risingwave_common::util::stream_graph_visitor::visit_stream_node; + use risingwave_common::util::stream_graph_visitor::{visit_stream_node, visit_stream_node_mut}; use risingwave_meta_model::actor::ActorStatus; use risingwave_meta_model::fragment::DistributionType; use risingwave_meta_model::{ @@ -1800,7 +1806,7 @@ mod tests { let nodes = nodes.unwrap(); let actor_upstream_actor_ids = upstream_actor_ids.get(&(actor_id as _)).cloned().unwrap(); - visit_stream_node(&mut template_node, |body| { + visit_stream_node_mut(&mut template_node, |body| { if let NodeBody::Merge(m) = body { m.upstream_actor_id = actor_upstream_actor_ids .get(&(m.upstream_fragment_id as _)) @@ -1978,9 +1984,7 @@ mod tests { assert_eq!(mview_definition, ""); - let mut pb_nodes = pb_nodes.unwrap(); - - visit_stream_node(&mut pb_nodes, |body| { + visit_stream_node(pb_nodes.as_ref().unwrap(), |body| { if let PbNodeBody::Merge(m) = body { let upstream_actor_ids = upstream_actor_ids .get(&(m.upstream_fragment_id as _)) diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 29d900663443f..669c6494667b7 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -20,7 +20,7 @@ use itertools::Itertools; use risingwave_common::config::DefaultParallelism; use risingwave_common::hash::VnodeCountCompat; use risingwave_common::util::column_index_mapping::ColIndexMapping; -use risingwave_common::util::stream_graph_visitor::visit_stream_node; +use risingwave_common::util::stream_graph_visitor::{visit_stream_node, visit_stream_node_mut}; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::WithPropertiesExt; use risingwave_meta_model::actor::ActorStatus; @@ -1183,7 +1183,7 @@ impl CatalogController { .await? .map(|(id, node, upstream)| (id, node.to_protobuf(), upstream)) .ok_or_else(|| MetaError::catalog_id_not_found("fragment", fragment_id))?; - visit_stream_node(&mut stream_node, |body| { + visit_stream_node_mut(&mut stream_node, |body| { if let PbNodeBody::Merge(m) = body && let Some((new_fragment_id, new_actor_ids)) = fragment_replace_map.get(&m.upstream_fragment_id) @@ -1356,7 +1356,7 @@ impl CatalogController { fragments.retain_mut(|(_, fragment_type_mask, stream_node)| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::Source as i32 != 0 { - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::Source(node) = node { if let Some(node_inner) = &mut node.source_inner && node_inner.source_id == source_id as u32 @@ -1370,7 +1370,7 @@ impl CatalogController { if is_fs_source { // in older versions, there's no fragment type flag for `FsFetch` node, // so we just scan all fragments for StreamFsFetch node if using fs connector - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::StreamFsFetch(node) = node { *fragment_type_mask |= PbFragmentTypeFlag::FsFetch as i32; if let Some(node_inner) = &mut node.node_inner @@ -1486,7 +1486,7 @@ impl CatalogController { |fragment_type_mask: &mut i32, stream_node: &mut PbStreamNode| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::backfill_rate_limit_fragments() != 0 { - visit_stream_node(stream_node, |node| match node { + visit_stream_node_mut(stream_node, |node| match node { PbNodeBody::StreamCdcScan(node) => { node.rate_limit = rate_limit; found = true; @@ -1528,7 +1528,7 @@ impl CatalogController { |fragment_type_mask: &mut i32, stream_node: &mut PbStreamNode| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::sink_rate_limit_fragments() != 0 { - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::Sink(node) = node { node.rate_limit = rate_limit; found = true; @@ -1551,7 +1551,7 @@ impl CatalogController { |fragment_type_mask: &mut i32, stream_node: &mut PbStreamNode| { let mut found = false; if *fragment_type_mask & PbFragmentTypeFlag::dml_rate_limit_fragments() != 0 { - visit_stream_node(stream_node, |node| { + visit_stream_node_mut(stream_node, |node| { if let PbNodeBody::Dml(node) = node { node.rate_limit = rate_limit; found = true; @@ -1635,7 +1635,7 @@ impl CatalogController { PbStreamActor { actor_id, fragment_id, - mut nodes, + nodes, dispatcher, upstream_actor_id, vnode_bitmap, @@ -1648,7 +1648,7 @@ impl CatalogController { let mut actor_upstreams = BTreeMap::>::new(); let mut new_actor_dispatchers = vec![]; - if let Some(nodes) = &mut nodes { + if let Some(nodes) = &nodes { visit_stream_node(nodes, |node| { if let PbNodeBody::Merge(node) = node { actor_upstreams @@ -1922,15 +1922,15 @@ impl CatalogController { let mut rate_limits = Vec::new(); for (fragment_id, job_id, fragment_type_mask, stream_node) in fragments { - let mut stream_node = stream_node.to_protobuf(); + let stream_node = stream_node.to_protobuf(); let mut rate_limit = None; let mut node_name = None; - visit_stream_node(&mut stream_node, |node| { + visit_stream_node(&stream_node, |node| { match node { // source rate limit PbNodeBody::Source(node) => { - if let Some(node_inner) = &mut node.source_inner { + if let Some(node_inner) = &node.source_inner { debug_assert!( rate_limit.is_none(), "one fragment should only have 1 rate limit node" @@ -1940,7 +1940,7 @@ impl CatalogController { } } PbNodeBody::StreamFsFetch(node) => { - if let Some(node_inner) = &mut node.node_inner { + if let Some(node_inner) = &node.node_inner { debug_assert!( rate_limit.is_none(), "one fragment should only have 1 rate limit node" 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 f346902bc242d..15600367fffb3 100644 --- a/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs +++ b/src/meta/src/hummock/manager/compaction/compaction_group_schedule.rs @@ -458,7 +458,7 @@ impl HummockManager { parent_group_id, new_sst_start_id, table_ids: vec![], - version: CompatibilityVersion::SplitGroupByTableId as i32, // for compatibility + version: CompatibilityVersion::LATEST as _, // for compatibility split_key: Some(split_key.into()), })], }, diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 335c8e8d157e5..29d06552e2ee7 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -135,7 +135,7 @@ impl<'a> HummockVersionTransaction<'a> { 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, + version: CompatibilityVersion::LATEST as _, split_key: None, })); } diff --git a/src/meta/src/manager/notification.rs b/src/meta/src/manager/notification.rs index dce9fabc7ff11..b31f22993ae5a 100644 --- a/src/meta/src/manager/notification.rs +++ b/src/meta/src/manager/notification.rs @@ -170,6 +170,17 @@ impl NotificationManager { ) } + pub fn notify_all_without_version(&self, operation: Operation, info: Info) { + for subscribe_type in [ + SubscribeType::Frontend, + SubscribeType::Hummock, + SubscribeType::Compactor, + SubscribeType::Compute, + ] { + self.notify_without_version(subscribe_type.into(), operation, info.clone()); + } + } + pub async fn notify_frontend(&self, operation: Operation, info: Info) -> NotificationVersion { self.notify_with_version(SubscribeType::Frontend.into(), operation, info) .await diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 4c2eb57fe6cb7..e11dfb5d27117 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -457,9 +457,9 @@ impl StreamJobFragments { /// Panics if not found. pub fn union_fragment_for_table(&mut self) -> &mut Fragment { let mut union_fragment_id = None; - for (fragment_id, fragment) in &mut self.fragments { - for actor in &mut fragment.actors { - if let Some(node) = &mut actor.nodes { + for (fragment_id, fragment) in &self.fragments { + for actor in &fragment.actors { + if let Some(node) = &actor.nodes { visit_stream_node(node, |body| { if let NodeBody::Union(_) = body { if let Some(union_fragment_id) = union_fragment_id.as_mut() { diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 5212a73ccd0c8..362e888a90fac 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -752,9 +752,9 @@ impl DdlController { } // check if the union fragment is fully assigned. - for fragment in stream_job_fragments.fragments.values_mut() { - for actor in &mut fragment.actors { - if let Some(node) = &mut actor.nodes { + for fragment in stream_job_fragments.fragments.values() { + for actor in &fragment.actors { + if let Some(node) = &actor.nodes { visit_stream_node(node, |node| { if let NodeBody::Merge(merge_node) = node { assert!(!merge_node.upstream_actor_id.is_empty(), "All the mergers for the union should have been fully assigned beforehand."); diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index 16264becc05d6..f472265c41db7 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -132,7 +132,7 @@ impl BuildingFragment { let fragment_id = fragment.fragment_id; let mut has_job = false; - stream_graph_visitor::visit_fragment(fragment, |node_body| match node_body { + stream_graph_visitor::visit_fragment_mut(fragment, |node_body| match node_body { NodeBody::Materialize(materialize_node) => { materialize_node.table_id = job_id; diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index f80ff57599b9c..84965eb2dda78 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -293,7 +293,7 @@ impl Scheduler { // Vnode count requirements: if a fragment is going to look up an existing table, // it must have the same vnode count as that table. for (&id, fragment) in graph.building_fragments() { - visit_fragment(&mut (*fragment).clone(), |node| { + visit_fragment(fragment, |node| { use risingwave_pb::stream_plan::stream_node::NodeBody; let vnode_count = match node { NodeBody::StreamScan(node) => { diff --git a/src/prost/Cargo.toml b/src/prost/Cargo.toml index c9fcb69dafd9b..f279098962542 100644 --- a/src/prost/Cargo.toml +++ b/src/prost/Cargo.toml @@ -25,7 +25,7 @@ workspace-hack = { path = "../workspace-hack" } static_assertions = "1" [build-dependencies] -fs-err = "3.0" +fs-err = "3.1" pbjson-build = "0.7" prost-build = { workspace = true } tonic-build = { workspace = true } diff --git a/src/prost/build.rs b/src/prost/build.rs index faf284dff7c8f..867d11bc918a0 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -246,7 +246,19 @@ fn main() -> Result<(), Box> { .type_attribute( "hummock.TableWatermarks.EpochNewWatermarks", "#[derive(Eq)]", - ); + ) + // proto version enums + .type_attribute("stream_plan.AggNodeVersion", "#[derive(prost_helpers::Version)]") + .type_attribute( + "plan_common.ColumnDescVersion", + "#[derive(prost_helpers::Version)]", + ) + .type_attribute( + "hummock.CompatibilityVersion", + "#[derive(prost_helpers::Version)]", + ) + // end + ; // If any configuration for `prost_build` is not exposed by `tonic_build`, specify it here. let mut prost_config = prost_build::Config::new(); diff --git a/src/prost/helpers/src/lib.rs b/src/prost/helpers/src/lib.rs index 0b6fd3f13b94f..34c0b765df503 100644 --- a/src/prost/helpers/src/lib.rs +++ b/src/prost/helpers/src/lib.rs @@ -16,9 +16,9 @@ #![feature(iterator_try_collect)] use proc_macro::TokenStream; -use proc_macro2::TokenStream as TokenStream2; +use proc_macro2::{Span, TokenStream as TokenStream2}; use quote::{format_ident, quote}; -use syn::{parse_macro_input, DataStruct, DeriveInput, Result}; +use syn::{parse_macro_input, Data, DataStruct, DeriveInput, Result}; mod generate; @@ -69,3 +69,40 @@ fn produce(ast: &DeriveInput) -> Result { #struct_get }) } + +#[cfg_attr(coverage, coverage(off))] +#[proc_macro_derive(Version)] +pub fn version(input: TokenStream) -> TokenStream { + fn version_inner(ast: &DeriveInput) -> syn::Result { + let last_variant = match &ast.data { + Data::Enum(v) => v.variants.iter().last().ok_or_else(|| { + syn::Error::new( + Span::call_site(), + "This macro requires at least one variant in the enum.", + ) + })?, + _ => { + return Err(syn::Error::new( + Span::call_site(), + "This macro only supports enums.", + )); + } + }; + + let enum_name = &ast.ident; + let last_variant_name = &last_variant.ident; + + Ok(quote! { + impl #enum_name { + pub const LATEST: Self = Self::#last_variant_name; + } + }) + } + + let ast = parse_macro_input!(input as DeriveInput); + + match version_inner(&ast) { + Ok(tokens) => tokens.into(), + Err(e) => e.to_compile_error().into(), + } +} diff --git a/src/risedevtool/Cargo.toml b/src/risedevtool/Cargo.toml index 95cbdca6cb01d..7205e6f7ac1b6 100644 --- a/src/risedevtool/Cargo.toml +++ b/src/risedevtool/Cargo.toml @@ -15,7 +15,7 @@ chrono = { version = "0.4", default-features = false, features = [ ] } clap = { workspace = true } console = "0.15" -fs-err = "3.0.0" +fs-err = "3.1.0" glob = "0.3" google-cloud-pubsub = "0.30" indicatif = "0.17" diff --git a/src/risedevtool/config/Cargo.toml b/src/risedevtool/config/Cargo.toml index 331b4d788276d..88e64c837504f 100644 --- a/src/risedevtool/config/Cargo.toml +++ b/src/risedevtool/config/Cargo.toml @@ -13,7 +13,7 @@ clap = { workspace = true } console = "0.15" dialoguer = "0.11" enum-iterator = "2" -fs-err = "3.0.0" +fs-err = "3.1.0" itertools = { workspace = true } [lints] diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index fbd8dde9a7c89..2ee4c63f4b7f5 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -12,23 +12,27 @@ // See the License for the specific language governing permissions and // limitations under the License. +#![feature(trait_alias)] + +use std::collections::HashMap; use std::env; use std::fmt::Write; use std::path::{Path, PathBuf}; use std::sync::Arc; +use std::thread::JoinHandle; use std::time::{Duration, Instant}; use anyhow::{anyhow, Context, Result}; use console::style; use fs_err::OpenOptions; -use indicatif::ProgressBar; -use risedev::util::{complete_spin, fail_spin}; +use indicatif::{MultiProgress, ProgressBar}; +use risedev::util::{begin_spin, complete_spin, fail_spin}; use risedev::{ generate_risedev_env, preflight_check, CompactorService, ComputeNodeService, ConfigExpander, ConfigureTmuxTask, DummyService, EnsureStopService, ExecuteContext, FrontendService, GrafanaService, KafkaService, MetaNodeService, MinioService, MySqlService, PostgresService, PrometheusService, PubsubService, RedisService, SchemaRegistryService, ServiceConfig, - SqlServerService, SqliteConfig, Task, TempoService, RISEDEV_NAME, + SqlServerService, SqliteConfig, Task, TaskGroup, TempoService, RISEDEV_NAME, }; use tempfile::tempdir; use thiserror_ext::AsReport; @@ -36,32 +40,22 @@ use tracing::level_filters::LevelFilter; use tracing_subscriber::EnvFilter; use yaml_rust::YamlEmitter; -#[derive(Default)] pub struct ProgressManager { - pa: Option, + pa: MultiProgress, } impl ProgressManager { pub fn new() -> Self { - Self::default() + let pa = MultiProgress::default(); + pa.set_move_cursor(true); + Self { pa } } /// Create a new progress bar from task pub fn new_progress(&mut self) -> ProgressBar { - if let Some(ref pa) = self.pa { - pa.finish(); - } - let pb = risedev::util::new_spinner(); + let pb = risedev::util::new_spinner().with_finish(indicatif::ProgressFinish::AndLeave); pb.enable_steady_tick(Duration::from_millis(100)); - self.pa = Some(pb.clone()); - pb - } - - /// Finish all progress bars. - pub fn finish_all(&self) { - if let Some(ref pa) = self.pa { - pa.finish(); - } + self.pa.add(pb) } } @@ -114,267 +108,312 @@ fn task_main( // Then, start services one by one - let mut stat = vec![]; + let mut tasks = TaskScheduler::new(); for service in services { - let start_time = Instant::now(); - - match service { - ServiceConfig::Minio(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = MinioService::new(c.clone())?; - service.execute(&mut ctx)?; - - let mut task = risedev::ConfigureMinioTask::new(c.clone())?; - task.execute(&mut ctx)?; - } - ServiceConfig::Sqlite(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - - struct SqliteService(SqliteConfig); - impl Task for SqliteService { - fn execute( - &mut self, - _ctx: &mut ExecuteContext, - ) -> anyhow::Result<()> { - Ok(()) + if let ServiceConfig::Frontend(c) = service { + writeln!( + log_buffer, + "* Run {} to start Postgres interactive shell.", + style(format_args!( + "psql -h localhost -p {} -d dev -U root", + c.port + )) + .blue() + .bold() + )?; + } + let service_ = service.clone(); + let progress_bar = manager.new_progress(); + progress_bar.set_prefix(service.id().to_owned()); + progress_bar.set_message("waiting for previous service to start...".to_owned()); + let status_dir = status_dir.clone(); + let closure = move || { + let mut log = Vec::new(); + let start_time = Instant::now(); + let mut ctx = ExecuteContext::new(&mut log, progress_bar, status_dir); + let service = service_; + let id = service.id().to_owned(); + match service { + ServiceConfig::Minio(c) => { + let mut service = MinioService::new(c.clone())?; + service.execute(&mut ctx)?; + + let mut task = risedev::ConfigureMinioTask::new(c.clone())?; + task.execute(&mut ctx)?; + } + ServiceConfig::Sqlite(c) => { + struct SqliteService(SqliteConfig); + impl Task for SqliteService { + fn execute( + &mut self, + _ctx: &mut ExecuteContext, + ) -> anyhow::Result<()> { + Ok(()) + } + + fn id(&self) -> String { + self.0.id.clone() + } } - fn id(&self) -> String { - self.0.id.clone() - } - } + let prefix_data = env::var("PREFIX_DATA")?; + let file_dir = PathBuf::from(&prefix_data).join(&c.id); + std::fs::create_dir_all(&file_dir)?; + let file_path = file_dir.join(&c.file); - let prefix_data = env::var("PREFIX_DATA")?; - let file_dir = PathBuf::from(&prefix_data).join(&c.id); - std::fs::create_dir_all(&file_dir)?; - let file_path = file_dir.join(&c.file); + ctx.service(&SqliteService(c.clone())); + ctx.complete_spin(); + ctx.pb + .set_message(format!("using local sqlite: {:?}", file_path)); + } + ServiceConfig::Prometheus(c) => { + let mut service = PrometheusService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; + task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("api http://{}:{}/", c.address, c.port)); + } + ServiceConfig::ComputeNode(c) => { + let mut service = ComputeNodeService::new(c.clone())?; + service.execute(&mut ctx)?; - ctx.service(&SqliteService(c.clone())); - ctx.complete_spin(); - ctx.pb - .set_message(format!("using local sqlite: {:?}", file_path)); - } - ServiceConfig::Prometheus(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = PrometheusService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api http://{}:{}/", c.address, c.port)); - } - ServiceConfig::ComputeNode(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = ComputeNodeService::new(c.clone())?; - service.execute(&mut ctx)?; - - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api grpc://{}:{}/", c.address, c.port)); - } - ServiceConfig::MetaNode(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = MetaNodeService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb.set_message(format!( - "api grpc://{}:{}/, dashboard http://{}:{}/", - c.address, c.port, c.address, c.dashboard_port - )); - } - ServiceConfig::Frontend(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = FrontendService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api postgres://{}:{}/", c.address, c.port)); - - writeln!( - log_buffer, - "* Run {} to start Postgres interactive shell.", - style(format_args!( - "psql -h localhost -p {} -d dev -U root", - c.port - )) - .blue() - .bold() - )?; - } - ServiceConfig::Compactor(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = CompactorService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("compactor {}:{}", c.address, c.port)); - } - ServiceConfig::Grafana(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = GrafanaService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("dashboard http://{}:{}/", c.address, c.port)); - } - ServiceConfig::Tempo(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = TempoService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = - risedev::TcpReadyCheckTask::new(c.listen_address.clone(), c.port, false)?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("api http://{}:{}/", c.listen_address, c.port)); - } - ServiceConfig::AwsS3(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - DummyService::new(&c.id).execute(&mut ctx)?; - ctx.pb - .set_message(format!("using AWS s3 bucket {}", c.bucket)); - } - ServiceConfig::Opendal(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - DummyService::new(&c.id).execute(&mut ctx)?; - ctx.pb - .set_message(format!("using Opendal, namenode = {}", c.namenode)); - } - ServiceConfig::Kafka(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = KafkaService::new(c.clone()); - service.execute(&mut ctx)?; - let mut task = risedev::KafkaReadyCheckTask::new(c.clone())?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("kafka {}:{}", c.address, c.port)); - } - ServiceConfig::SchemaRegistry(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = SchemaRegistryService::new(c.clone()); - service.execute(&mut ctx)?; - if c.user_managed { let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; - } else { + ctx.pb + .set_message(format!("api grpc://{}:{}/", c.address, c.port)); + } + ServiceConfig::MetaNode(c) => { + let mut service = MetaNodeService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = - risedev::LogReadyCheckTask::new("Server started, listening for requests")?; + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; + ctx.pb.set_message(format!( + "api grpc://{}:{}/, dashboard http://{}:{}/", + c.address, c.port, c.address, c.dashboard_port + )); } - ctx.pb - .set_message(format!("schema registry http://{}:{}", c.address, c.port)); - } - - ServiceConfig::Pubsub(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = PubsubService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::PubsubReadyTaskCheck::new(c.clone())?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("pubsub {}:{}", c.address, c.port)); - } - ServiceConfig::RedPanda(_) => { - return Err(anyhow!("redpanda is only supported in RiseDev compose.")); - } - ServiceConfig::Redis(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - let mut service = RedisService::new(c.clone())?; - service.execute(&mut ctx)?; - let mut task = risedev::RedisReadyCheckTask::new(c.clone())?; - task.execute(&mut ctx)?; - ctx.pb - .set_message(format!("redis {}:{}", c.address, c.port)); - } - ServiceConfig::MySql(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - MySqlService::new(c.clone()).execute(&mut ctx)?; - if c.user_managed { + ServiceConfig::Frontend(c) => { + let mut service = FrontendService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; - } else { - // When starting a MySQL container, the MySQL process is set as the main process. - // Since the first process in a container always gets PID 1, the MySQL log always shows - // "starting as process 1". - let mut task = risedev::LogReadyCheckTask::new("starting as process 1\n")?; - task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("api postgres://{}:{}/", c.address, c.port)); } - ctx.pb - .set_message(format!("mysql {}:{}", c.address, c.port)); - } - ServiceConfig::Postgres(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - PostgresService::new(c.clone()).execute(&mut ctx)?; - if c.user_managed { + ServiceConfig::Compactor(c) => { + let mut service = CompactorService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; task.execute(&mut ctx)?; - } else { - let mut task = risedev::LogReadyCheckTask::new_all([ - "ready to accept connections", // also appears in init process - "listening on IPv4 address", // only appears when ready - ])?; + ctx.pb + .set_message(format!("compactor {}:{}", c.address, c.port)); + } + ServiceConfig::Grafana(c) => { + let mut service = GrafanaService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = + risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, false)?; task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("dashboard http://{}:{}/", c.address, c.port)); } - ctx.pb - .set_message(format!("postgres {}:{}", c.address, c.port)); - } - ServiceConfig::SqlServer(c) => { - let mut ctx = - ExecuteContext::new(&mut logger, manager.new_progress(), status_dir.clone()); - // only `c.password` will be used in `SqlServerService` as the password for user `sa`. - SqlServerService::new(c.clone()).execute(&mut ctx)?; - if c.user_managed { + ServiceConfig::Tempo(c) => { + let mut service = TempoService::new(c.clone())?; + service.execute(&mut ctx)?; let mut task = - risedev::TcpReadyCheckTask::new(c.address.clone(), c.port, c.user_managed)?; + risedev::TcpReadyCheckTask::new(c.listen_address.clone(), c.port, false)?; + task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("api http://{}:{}/", c.listen_address, c.port)); + } + ServiceConfig::AwsS3(c) => { + DummyService::new(&c.id).execute(&mut ctx)?; + ctx.pb + .set_message(format!("using AWS s3 bucket {}", c.bucket)); + } + ServiceConfig::Opendal(c) => { + DummyService::new(&c.id).execute(&mut ctx)?; + ctx.pb + .set_message(format!("using Opendal, namenode = {}", c.namenode)); + } + ServiceConfig::Kafka(c) => { + let mut service = KafkaService::new(c.clone()); + service.execute(&mut ctx)?; + let mut task = risedev::KafkaReadyCheckTask::new(c.clone())?; + task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("kafka {}:{}", c.address, c.port)); + } + ServiceConfig::SchemaRegistry(c) => { + let mut service = SchemaRegistryService::new(c.clone()); + service.execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + let mut task = risedev::LogReadyCheckTask::new( + "Server started, listening for requests", + )?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("schema registry http://{}:{}", c.address, c.port)); + } + + ServiceConfig::Pubsub(c) => { + let mut service = PubsubService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = risedev::PubsubReadyTaskCheck::new(c.clone())?; task.execute(&mut ctx)?; - } else { - let mut task = risedev::LogReadyCheckTask::new( - "SQL Server is now ready for client connections.", - )?; + ctx.pb + .set_message(format!("pubsub {}:{}", c.address, c.port)); + } + ServiceConfig::RedPanda(_) => { + return Err(anyhow!("redpanda is only supported in RiseDev compose.")); + } + ServiceConfig::Redis(c) => { + let mut service = RedisService::new(c.clone())?; + service.execute(&mut ctx)?; + let mut task = risedev::RedisReadyCheckTask::new(c.clone())?; task.execute(&mut ctx)?; + ctx.pb + .set_message(format!("redis {}:{}", c.address, c.port)); + } + ServiceConfig::MySql(c) => { + MySqlService::new(c.clone()).execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + // When starting a MySQL container, the MySQL process is set as the main process. + // Since the first process in a container always gets PID 1, the MySQL log always shows + // "starting as process 1". + let mut task = risedev::LogReadyCheckTask::new("starting as process 1\n")?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("mysql {}:{}", c.address, c.port)); + } + ServiceConfig::Postgres(c) => { + PostgresService::new(c.clone()).execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + let mut task = risedev::LogReadyCheckTask::new_all([ + "ready to accept connections", // also appears in init process + "listening on IPv4 address", // only appears when ready + ])?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("postgres {}:{}", c.address, c.port)); + } + ServiceConfig::SqlServer(c) => { + // only `c.password` will be used in `SqlServerService` as the password for user `sa`. + SqlServerService::new(c.clone()).execute(&mut ctx)?; + if c.user_managed { + let mut task = risedev::TcpReadyCheckTask::new( + c.address.clone(), + c.port, + c.user_managed, + )?; + task.execute(&mut ctx)?; + } else { + let mut task = risedev::LogReadyCheckTask::new( + "SQL Server is now ready for client connections.", + )?; + task.execute(&mut ctx)?; + } + ctx.pb + .set_message(format!("sqlserver {}:{}", c.address, c.port)); } - ctx.pb - .set_message(format!("sqlserver {}:{}", c.address, c.port)); } - } - let service_id = service.id().to_owned(); - let duration = Instant::now() - start_time; - stat.push((service_id, duration)); + let duration = Instant::now() - start_time; + Ok(TaskResult { + id, + time: duration, + log: String::from_utf8(log)?, + }) + }; + tasks.add(service, closure); } + let stat = tasks.run(&mut logger)?; + Ok((stat, log_buffer)) } +struct TaskResult { + id: String, + time: Duration, + log: String, +} +trait TaskFn = FnOnce() -> anyhow::Result + Send + 'static; +struct TaskScheduler { + /// In each group, the tasks are executed in sequence. + task_groups: HashMap>>, +} + +impl TaskScheduler { + fn new() -> Self { + Self { + task_groups: HashMap::new(), + } + } + + fn add(&mut self, config: &ServiceConfig, task: impl TaskFn) { + self.task_groups + .entry(config.task_group()) + .or_default() + .push(Box::new(task)); + } + + fn run(self, logger: &mut impl std::io::Write) -> anyhow::Result> { + let mut handles: Vec>>> = vec![]; + let mut stats = vec![]; + + let task_groups = self.task_groups; + for (_, tasks) in task_groups { + handles.push(std::thread::spawn(move || { + let mut res = vec![]; + for task in tasks { + let res_ = task()?; + res.push(res_); + } + Ok(res) + })); + } + for handle in handles { + for TaskResult { id, time, log } in handle.join().unwrap()? { + stats.push((id, time)); + write!(logger, "{}", log)?; + } + } + Ok(stats) + } +} + fn main() -> Result<()> { // Intentionally disable backtrace to provide more compact error message for `risedev dev`. // Backtraces for RisingWave components are enabled in `Task::execute`. @@ -386,17 +425,19 @@ fn main() -> Result<()> { EnvFilter::builder() .with_default_directive(LevelFilter::INFO.into()) .with_env_var("RISEDEV_RUST_LOG") - .from_env_lossy(), + .from_env_lossy() + // This log may pollute the progress bar. + .add_directive("librdkafka=off".parse().unwrap()), ) .init(); preflight_check()?; - let task_name = std::env::args() + let profile = std::env::args() .nth(1) .unwrap_or_else(|| "default".to_owned()); - let (config_path, env, risedev_config) = ConfigExpander::expand(".", &task_name)?; + let (config_path, env, risedev_config) = ConfigExpander::expand(".", &profile)?; if let Some(config_path) = &config_path { let target = Path::new(&env::var("PREFIX_CONFIG")?).join("risingwave.toml"); @@ -418,11 +459,12 @@ fn main() -> Result<()> { // Always create a progress before calling `task_main`. Otherwise the progress bar won't be // shown. let p = manager.new_progress(); + begin_spin(&p); p.set_prefix("dev cluster"); p.set_message(format!( "starting {} services for {}...", services.len(), - task_name + profile )); let task_result = task_main(&mut manager, &services, env); @@ -430,19 +472,19 @@ fn main() -> Result<()> { Ok(_) => { p.set_message(format!( "done bootstrapping with config {}", - style(task_name).bold() + style(profile).bold() )); complete_spin(&p); } Err(_) => { p.set_message(format!( "failed to bootstrap with config {}", - style(task_name).bold() + style(profile).bold() )); fail_spin(&p); } } - manager.finish_all(); + p.finish(); use risedev::util::stylized_risedev_subcmd as r; diff --git a/src/risedevtool/src/service_config.rs b/src/risedevtool/src/service_config.rs index c1070d43261c6..f30e69701f438 100644 --- a/src/risedevtool/src/service_config.rs +++ b/src/risedevtool/src/service_config.rs @@ -432,6 +432,18 @@ pub enum ServiceConfig { SqlServer(SqlServerConfig), } +#[derive(PartialEq, Eq, Hash)] +pub enum TaskGroup { + RisingWave, + Observability, + Kafka, + Pubsub, + MySql, + Postgres, + SqlServer, + Redis, +} + impl ServiceConfig { pub fn id(&self) -> &str { match self { @@ -505,6 +517,42 @@ impl ServiceConfig { Self::SchemaRegistry(c) => c.user_managed, } } + + pub fn task_group(&self) -> TaskGroup { + use TaskGroup::*; + match self { + ServiceConfig::ComputeNode(_) + | ServiceConfig::MetaNode(_) + | ServiceConfig::Frontend(_) + | ServiceConfig::Compactor(_) + | ServiceConfig::Minio(_) + | ServiceConfig::Sqlite(_) => RisingWave, + ServiceConfig::Prometheus(_) | ServiceConfig::Grafana(_) | ServiceConfig::Tempo(_) => { + Observability + } + ServiceConfig::Opendal(_) | ServiceConfig::AwsS3(_) => RisingWave, + ServiceConfig::Kafka(_) + | ServiceConfig::SchemaRegistry(_) + | ServiceConfig::RedPanda(_) => Kafka, + ServiceConfig::Pubsub(_) => Pubsub, + ServiceConfig::Redis(_) => Redis, + ServiceConfig::MySql(my_sql_config) => { + if matches!(my_sql_config.application, Application::Metastore) { + RisingWave + } else { + MySql + } + } + ServiceConfig::Postgres(postgres_config) => { + if matches!(postgres_config.application, Application::Metastore) { + RisingWave + } else { + Postgres + } + } + ServiceConfig::SqlServer(_) => SqlServer, + } + } } mod string { diff --git a/src/risedevtool/src/task.rs b/src/risedevtool/src/task.rs index 65c34c649ed68..c4dcab8426aba 100644 --- a/src/risedevtool/src/task.rs +++ b/src/risedevtool/src/task.rs @@ -76,7 +76,7 @@ pub use self::task_pubsub_emu_ready_check::*; pub use self::task_redis_ready_check::*; pub use self::task_tcp_ready_check::*; pub use self::tempo_service::*; -use crate::util::{complete_spin, get_program_args, get_program_name}; +use crate::util::{begin_spin, complete_spin, get_program_args, get_program_name}; use crate::wait::{wait, wait_tcp_available}; pub trait Task: 'static + Send { @@ -134,6 +134,7 @@ where pub fn service(&mut self, task: &impl Task) { let id = task.id(); if !id.is_empty() { + begin_spin(&self.pb); self.pb.set_prefix(id.clone()); self.id = Some(id.clone()); self.status_file = Some(self.status_dir.path().join(format!("{}.status", id))); diff --git a/src/risedevtool/src/util.rs b/src/risedevtool/src/util.rs index d25ad3d92788f..79bab7977d209 100644 --- a/src/risedevtool/src/util.rs +++ b/src/risedevtool/src/util.rs @@ -47,12 +47,20 @@ pub fn new_spinner() -> ProgressBar { let pb = ProgressBar::new(0); pb.set_style( ProgressStyle::default_spinner() - .template("{spinner} {prefix}: {msg}") + .template("🟡 {prefix}: {msg}") .unwrap(), ); pb } +pub fn begin_spin(pb: &ProgressBar) { + pb.set_style( + ProgressStyle::default_spinner() + .template("{spinner} {prefix}: {msg}") + .unwrap(), + ); +} + pub fn complete_spin(pb: &ProgressBar) { pb.set_style( ProgressStyle::default_spinner() diff --git a/src/storage/compactor/src/compactor_observer/observer_manager.rs b/src/storage/compactor/src/compactor_observer/observer_manager.rs index 03eb165fd5dfc..12ca3371bd087 100644 --- a/src/storage/compactor/src/compactor_observer/observer_manager.rs +++ b/src/storage/compactor/src/compactor_observer/observer_manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::license::LicenseManager; use risingwave_common::system_param::local_manager::LocalSystemParamsManagerRef; use risingwave_common_service::ObserverState; use risingwave_pb::catalog::Table; @@ -60,6 +61,9 @@ impl ObserverState for CompactorObserverNode { Info::SystemParams(p) => { self.system_params_manager.try_set_params(p); } + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } _ => { panic!("error type notification"); } @@ -73,6 +77,7 @@ impl ObserverState for CompactorObserverNode { self.handle_catalog_snapshot(snapshot.tables); let snapshot_version = snapshot.version.unwrap(); self.version = snapshot_version.catalog_version; + LicenseManager::get().update_cpu_core_count(snapshot.compute_node_total_cpu_count as _); } } diff --git a/src/storage/hummock_sdk/src/lib.rs b/src/storage/hummock_sdk/src/lib.rs index f4da340b0a90f..fdf739fbe8375 100644 --- a/src/storage/hummock_sdk/src/lib.rs +++ b/src/storage/hummock_sdk/src/lib.rs @@ -318,17 +318,19 @@ pub fn full_key_can_concat(ssts: &[SstableInfo]) -> bool { let sst_2 = &ssts[i]; if sst_1.key_range.right_exclusive { - if sst_1 - .key_range - .compare_right_with(&sst_2.key_range.left) - .is_gt() + if KeyComparator::compare_encoded_full_key( + &sst_1.key_range.right, + &sst_2.key_range.left, + ) + .is_gt() { return false; } - } else if sst_1 - .key_range - .compare_right_with(&sst_2.key_range.left) - .is_ge() + } else if KeyComparator::compare_encoded_full_key( + &sst_1.key_range.right, + &sst_2.key_range.left, + ) + .is_ge() { return false; } @@ -440,6 +442,9 @@ pub fn get_object_id_from_path(path: &str) -> HummockSstableObjectId { #[cfg(test)] mod tests { + use bytes::Bytes; + use sstable_info::SstableInfoInner; + use super::*; #[test] @@ -447,4 +452,41 @@ mod tests { let len = HummockSstableObjectId::MAX.to_string().len(); assert_eq!(len, HUMMOCK_SSTABLE_OBJECT_ID_MAX_DECIMAL_LENGTH) } + + #[test] + fn test_full_key_concat() { + let key1 = b"\0\0\0\x08\0\0\0\x0112-3\0\0\0\0\x04\0\x1c\x16l'\xe2\0\0"; + let key2 = b"\0\0\0\x08\0\0\0\x0112-3\0\0\0\0\x04\0\x1c\x16l \x12\0\0"; + + let sst_1 = SstableInfoInner { + key_range: key_range::KeyRange { + left: Bytes::from(key1.to_vec()), + right: Bytes::from(key1.to_vec()), + right_exclusive: false, + }, + ..Default::default() + }; + + let sst_2 = SstableInfoInner { + key_range: key_range::KeyRange { + left: Bytes::from(key2.to_vec()), + right: Bytes::from(key2.to_vec()), + right_exclusive: false, + }, + ..Default::default() + }; + + let sst_3 = SstableInfoInner { + key_range: key_range::KeyRange { + left: Bytes::from(key1.to_vec()), + right: Bytes::from(key2.to_vec()), + right_exclusive: false, + }, + ..Default::default() + }; + + assert!(full_key_can_concat(&[sst_1.clone().into(), sst_2.into()])); + + assert!(!full_key_can_concat(&[sst_1.into(), sst_3.into()])); + } } diff --git a/src/storage/src/hummock/observer_manager.rs b/src/storage/src/hummock/observer_manager.rs index 1c208e23f474c..6604ad10e3573 100644 --- a/src/storage/src/hummock/observer_manager.rs +++ b/src/storage/src/hummock/observer_manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::license::LicenseManager; use risingwave_common_service::ObserverState; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_trace::TraceSpan; @@ -91,6 +92,10 @@ impl ObserverState for HummockObserverNode { .update_write_limits(write_limits.write_limits); } + Info::ComputeNodeTotalCpuCount(count) => { + LicenseManager::get().update_cpu_core_count(count as _); + } + _ => { panic!("error type notification"); } @@ -132,6 +137,7 @@ impl ObserverState for HummockObserverNode { }); let snapshot_version = snapshot.version.unwrap(); self.version = snapshot_version.catalog_version; + LicenseManager::get().update_cpu_core_count(snapshot.compute_node_total_cpu_count as _); } } diff --git a/src/stream/src/executor/aggregation/minput.rs b/src/stream/src/executor/aggregation/minput.rs index 4967696c01cfb..eee4f2d23a596 100644 --- a/src/stream/src/executor/aggregation/minput.rs +++ b/src/stream/src/executor/aggregation/minput.rs @@ -399,7 +399,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -457,7 +457,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -505,7 +505,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -563,7 +563,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -627,7 +627,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state_1 = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call_1, &PkIndices::new(), // unused &order_columns_1, @@ -642,7 +642,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state_2 = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call_2, &PkIndices::new(), // unused &order_columns_2, @@ -730,7 +730,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -787,7 +787,7 @@ mod tests { { // test recovery (cold start) let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -837,7 +837,7 @@ mod tests { ColumnOrder::new(1, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -945,7 +945,7 @@ mod tests { ColumnOrder::new(1, OrderType::ascending()), // _row_id ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -1064,7 +1064,7 @@ mod tests { ColumnOrder::new(4, OrderType::ascending()), // _row_id ASC ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, @@ -1153,7 +1153,7 @@ mod tests { ColumnOrder::new(3, OrderType::ascending()), // _row_id ASC ]; let mut state = MaterializedInputState::new( - PbAggNodeVersion::Max, + PbAggNodeVersion::LATEST, &agg_call, &PkIndices::new(), // unused &order_columns, diff --git a/src/stream/src/executor/backfill/utils.rs b/src/stream/src/executor/backfill/utils.rs index 06f0a13ed97fb..7128c55986a47 100644 --- a/src/stream/src/executor/backfill/utils.rs +++ b/src/stream/src/executor/backfill/utils.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cmp::Ordering; +use std::borrow::Cow; use std::collections::HashMap; use std::ops::Bound; @@ -341,31 +341,37 @@ pub(crate) fn mark_chunk_ref_by_vnode( let chunk = chunk.clone(); let (data, ops) = chunk.into_parts(); let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); - // Use project to avoid allocation. - for row in data.rows() { + + let mut new_ops: Cow<'_, [Op]> = Cow::Borrowed(ops.as_ref()); + let mut unmatched_update_delete = false; + let mut visible_update_delete = false; + for (i, (op, row)) in ops.iter().zip_eq_debug(data.rows()).enumerate() { let pk = row.project(pk_in_output_indices); let vnode = upstream_table.compute_vnode_by_pk(pk); - let v = match backfill_state.get_progress(&vnode)? { + let visible = match backfill_state.get_progress(&vnode)? { // We want to just forward the row, if the vnode has finished backfill. BackfillProgressPerVnode::Completed { .. } => true, // If not started, no need to forward. BackfillProgressPerVnode::NotStarted => false, // If in progress, we need to check row <= current_pos. BackfillProgressPerVnode::InProgress { current_pos, .. } => { - match cmp_datum_iter(pk.iter(), current_pos.iter(), pk_order.iter().copied()) { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } + cmp_datum_iter(pk.iter(), current_pos.iter(), pk_order.iter().copied()).is_le() } }; - new_visibility.append(v); + new_visibility.append(visible); + + normalize_unmatched_updates( + &mut new_ops, + &mut unmatched_update_delete, + &mut visible_update_delete, + visible, + i, + op, + ); } let (columns, _) = data.into_parts(); - Ok(StreamChunk::with_visibility( - ops, - columns, - new_visibility.finish(), - )) + let chunk = StreamChunk::with_visibility(new_ops, columns, new_visibility.finish()); + Ok(chunk) } /// Mark chunk: @@ -379,22 +385,74 @@ fn mark_chunk_inner( ) -> StreamChunk { let (data, ops) = chunk.into_parts(); let mut new_visibility = BitmapBuilder::with_capacity(ops.len()); - // Use project to avoid allocation. - for v in data.rows().map(|row| { + let mut new_ops: Cow<'_, [Op]> = Cow::Borrowed(ops.as_ref()); + let mut unmatched_update_delete = false; + let mut visible_update_delete = false; + for (i, (op, row)) in ops.iter().zip_eq_debug(data.rows()).enumerate() { let lhs = row.project(pk_in_output_indices); let rhs = current_pos; - let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); - match order { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } - }) { - new_visibility.append(v); + let visible = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()).is_le(); + new_visibility.append(visible); + + normalize_unmatched_updates( + &mut new_ops, + &mut unmatched_update_delete, + &mut visible_update_delete, + visible, + i, + op, + ); } let (columns, _) = data.into_parts(); StreamChunk::with_visibility(ops, columns, new_visibility.finish()) } +/// We will rewrite unmatched U-/U+ into +/- ops. +/// They can be unmatched because while they will always have the same stream key, +/// their storage pk might be different. Here we use storage pk (`current_pos`) to filter them, +/// as such, a U+ might be filtered out, but their corresponding U- could be kept, and vice versa. +/// +/// This hanging U-/U+ can lead to issues downstream, since we work with an assumption in the +/// system that there's never hanging U-/U+. +fn normalize_unmatched_updates( + normalized_ops: &mut Cow<'_, [Op]>, + unmatched_update_delete: &mut bool, + visible_update_delete: &mut bool, + current_visibility: bool, + current_op_index: usize, + current_op: &Op, +) { + if *unmatched_update_delete { + assert_eq!(*current_op, Op::UpdateInsert); + let visible_update_insert = current_visibility; + match (visible_update_delete, visible_update_insert) { + (true, false) => { + // Lazily clone the ops here. + let ops = normalized_ops.to_mut(); + ops[current_op_index - 1] = Op::Delete; + } + (false, true) => { + // Lazily clone the ops here. + let ops = normalized_ops.to_mut(); + ops[current_op_index] = Op::Insert; + } + (true, true) | (false, false) => {} + } + *unmatched_update_delete = false; + } else { + match current_op { + Op::UpdateDelete => { + *unmatched_update_delete = true; + *visible_update_delete = current_visibility; + } + Op::UpdateInsert => { + unreachable!("UpdateInsert should not be present without UpdateDelete") + } + _ => {} + } + } +} + fn mark_cdc_chunk_inner( offset_parse_func: &CdcOffsetParseFunc, chunk: StreamChunk, @@ -422,11 +480,7 @@ fn mark_cdc_chunk_inner( if in_binlog_range { let lhs = row.project(pk_in_output_indices); let rhs = current_pos; - let order = cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()); - match order { - Ordering::Less | Ordering::Equal => true, - Ordering::Greater => false, - } + cmp_datum_iter(lhs.iter(), rhs.iter(), pk_order.iter().copied()).is_le() } else { false } diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fs_fetch_executor.rs similarity index 100% rename from src/stream/src/executor/source/fetch_executor.rs rename to src/stream/src/executor/source/fs_fetch_executor.rs diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/fs_list_executor.rs similarity index 98% rename from src/stream/src/executor/source/list_executor.rs rename to src/stream/src/executor/source/fs_list_executor.rs index 97f3ba12aa9fb..b418300588bcc 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/fs_list_executor.rs @@ -26,7 +26,6 @@ use super::{barrier_to_message_stream, StreamSourceCore}; use crate::executor::prelude::*; use crate::executor::stream_reader::StreamReaderWithPause; -#[allow(dead_code)] pub struct FsListExecutor { actor_ctx: ActorContextRef, @@ -34,15 +33,18 @@ pub struct FsListExecutor { stream_source_core: Option>, /// Metrics for monitor. + #[expect(dead_code)] metrics: Arc, /// Receiver of barrier channel. barrier_receiver: Option>, /// System parameter reader to read barrier interval + #[expect(dead_code)] system_params: SystemParamsReaderRef, /// Rate limit in rows/s. + #[expect(dead_code)] rate_limit_rps: Option, } @@ -65,7 +67,6 @@ impl FsListExecutor { } } - #[allow(clippy::disallowed_types)] fn build_chunked_paginate_stream( &self, source_desc: &SourceDesc, diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/legacy_fs_source_executor.rs similarity index 97% rename from src/stream/src/executor/source/fs_source_executor.rs rename to src/stream/src/executor/source/legacy_fs_source_executor.rs index 2e62865ecff99..8774d5c468b7f 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/legacy_fs_source_executor.rs @@ -25,7 +25,7 @@ use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::epoch::EpochPair; use risingwave_connector::error::ConnectorError; -use risingwave_connector::source::reader::desc::{FsSourceDesc, SourceDescBuilder}; +use risingwave_connector::source::reader::desc::{LegacyFsSourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ BoxSourceChunkStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, @@ -47,9 +47,9 @@ use crate::executor::UpdateMutation; /// some latencies in network and cost in meta. const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; -/// [`FsSourceExecutor`] is a streaming source, fir external file systems +/// [`LegacyFsSourceExecutor`] is a streaming source, fir external file systems /// such as s3. -pub struct FsSourceExecutor { +pub struct LegacyFsSourceExecutor { actor_ctx: ActorContextRef, /// Streaming source for external @@ -68,7 +68,7 @@ pub struct FsSourceExecutor { rate_limit_rps: Option, } -impl FsSourceExecutor { +impl LegacyFsSourceExecutor { pub fn new( actor_ctx: ActorContextRef, stream_source_core: StreamSourceCore, @@ -89,7 +89,7 @@ impl FsSourceExecutor { async fn build_stream_source_reader( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, state: ConnectorState, ) -> StreamExecutorResult { let column_ids = source_desc @@ -121,7 +121,7 @@ impl FsSourceExecutor { async fn rebuild_stream_reader( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, stream: &mut StreamReaderWithPause, ) -> StreamExecutorResult<()> { let target_state: Vec = self @@ -141,7 +141,7 @@ impl FsSourceExecutor { async fn apply_split_change( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, stream: &mut StreamReaderWithPause, mapping: &HashMap>, ) -> StreamExecutorResult<()> { @@ -207,7 +207,7 @@ impl FsSourceExecutor { async fn replace_stream_reader_with_target_state( &mut self, - source_desc: &FsSourceDesc, + source_desc: &LegacyFsSourceDesc, stream: &mut StreamReaderWithPause, target_state: Vec, ) -> StreamExecutorResult<()> { @@ -506,15 +506,15 @@ impl FsSourceExecutor { } } -impl Execute for FsSourceExecutor { +impl Execute for LegacyFsSourceExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } } -impl Debug for FsSourceExecutor { +impl Debug for LegacyFsSourceExecutor { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { - f.debug_struct("FsSourceExecutor") + f.debug_struct("LegacyFsSourceExecutor") .field("source_id", &self.stream_source_core.source_id) .field("column_ids", &self.stream_source_core.column_ids) .finish() diff --git a/src/stream/src/executor/source/mod.rs b/src/stream/src/executor/source/mod.rs index 6aabca713c7c2..94a8ab623ab1e 100644 --- a/src/stream/src/executor/source/mod.rs +++ b/src/stream/src/executor/source/mod.rs @@ -30,17 +30,17 @@ pub use state_table_handler::*; mod executor_core; pub use executor_core::StreamSourceCore; -mod fs_source_executor; +mod legacy_fs_source_executor; #[expect(deprecated)] -pub use fs_source_executor::*; +pub use legacy_fs_source_executor::*; mod source_executor; pub use source_executor::*; mod source_backfill_executor; pub use source_backfill_executor::*; -mod list_executor; -pub use list_executor::*; -mod fetch_executor; -pub use fetch_executor::*; +mod fs_list_executor; +pub use fs_list_executor::*; +mod fs_fetch_executor; +pub use fs_fetch_executor::*; mod source_backfill_state_table; pub use source_backfill_state_table::BackfillStateTableHandler; diff --git a/src/stream/src/executor/source/state_table_handler.rs b/src/stream/src/executor/source/state_table_handler.rs index 02a07255d7588..d03a978f6b8b3 100644 --- a/src/stream/src/executor/source/state_table_handler.rs +++ b/src/stream/src/executor/source/state_table_handler.rs @@ -86,7 +86,7 @@ impl SourceStateTableHandler { .map_err(StreamExecutorError::from) } - /// this method should only be used by [`FsSourceExecutor`](super::FsSourceExecutor) + /// this method should only be used by [`LegacyFsSourceExecutor`](super::LegacyFsSourceExecutor) pub(crate) async fn get_all_completed(&self) -> StreamExecutorResult> { let start = Bound::Excluded(row::once(Some(Self::string_to_scalar( COMPLETE_SPLIT_PREFIX, @@ -137,7 +137,7 @@ impl SourceStateTableHandler { } /// set all complete - /// can only used by [`FsSourceExecutor`](super::FsSourceExecutor) + /// can only used by [`LegacyFsSourceExecutor`](super::LegacyFsSourceExecutor) pub(crate) async fn set_all_complete( &mut self, states: Vec, diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index bdc057341a65d..c15f648660436 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -484,7 +484,7 @@ pub mod agg_executor { }; let exec = HashAggExecutor::::new(AggExecutorArgs { - version: PbAggNodeVersion::Max, + version: PbAggNodeVersion::LATEST, input, actor_ctx: ActorContext::for_test(123), @@ -552,7 +552,7 @@ pub mod agg_executor { }; let exec = SimpleAggExecutor::new(AggExecutorArgs { - version: PbAggNodeVersion::Max, + version: PbAggNodeVersion::LATEST, input, actor_ctx, diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 42cf0e1c1da23..89ec19b84f4b6 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -34,7 +34,6 @@ use crate::executor::source::{ }; use crate::executor::TroublemakerExecutor; -const FS_CONNECTORS: &[&str] = &["s3"]; pub struct SourceExecutorBuilder; pub fn create_source_desc_builder( @@ -197,13 +196,12 @@ impl ExecutorBuilder for SourceExecutorBuilder { state_table_handler, ); - let connector = get_connector_name(&source.with_properties); - let is_fs_connector = FS_CONNECTORS.contains(&connector.as_str()); + let is_fs_connector = source.with_properties.is_legacy_fs_connector(); let is_fs_v2_connector = source.with_properties.is_new_fs_connector(); if is_fs_connector { #[expect(deprecated)] - crate::executor::source::FsSourceExecutor::new( + crate::executor::source::LegacyFsSourceExecutor::new( params.actor_context.clone(), stream_source_core, params.executor_stats, diff --git a/src/tests/simulation/tests/integration_tests/license_cpu_limit.rs b/src/tests/simulation/tests/integration_tests/license_cpu_limit.rs new file mode 100644 index 0000000000000..fbce9dfa3fc71 --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/license_cpu_limit.rs @@ -0,0 +1,89 @@ +// Copyright 2025 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::Result; +use risingwave_common::error::AsReport; +use risingwave_simulation::cluster::{Cluster, Configuration}; +use risingwave_simulation::utils::AssertResult; + +/// Paid-tier key with CPU limit 20. +const KEY_20: &str = "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiJwYWlkLXRlc3QtMzIiLCJpc3MiOiJ0ZXN0LnJpc2luZ3dhdmUuY29tIiwidGllciI6InBhaWQiLCJleHAiOjIxNTA0OTU5OTksImlhdCI6MTczNzYxNTI2NywiY3B1X2NvcmVfbGltaXQiOjIwfQ.V8546BDZydv1aNk8IlVaSVlCriDtMC_75nq8CIaRPKlrltcwRJYKfK-Ru3WbKj-MDFebmW_3CqA4jR77BBtmmmtj-lPHa4qrdgrMItxm9RC_qoSU1YbI8Kb_ClYkrnFug5MAbK3wGlO8CrrjqOOt-Q5ggKChtl0uFj4zgI-S80d8Hse5LYSKHv8cU-ECKvEFe451kXE9x7nN_f8MqTSOqBwfY5o17gTD8oU3XH2k1mpesdci18kDmJPK5DeLPDYht_nRt7WGbVQvx7iiol1nzj5OBjdH_eVbX7pfk9M-JNwqZKaqfOmBbwV2F-Sf7-tK33O-XqSfXjnLAzflfjkoLQ"; +/// Paid-tier key with CPU limit 100. +const KEY_100: &str = "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiJwYWlkLXRlc3QtMzIiLCJpc3MiOiJ0ZXN0LnJpc2luZ3dhdmUuY29tIiwidGllciI6InBhaWQiLCJleHAiOjIxNTA0OTU5OTksImlhdCI6MTczNzYyMzc2MywiY3B1X2NvcmVfbGltaXQiOjEwMH0.ZGQjZa6t3va5MHMHKvgMgOXVLymEvvy1Yvd6teRUgCIF7en5BYaKXWuXwwtWLpLxr7LXyIQ3LQeDXag4k_fQOUTwV4oYTLTFVF8GcJ8JvGdTjBfjnM_2helLEhjZFgXSnhEy-xTOj5yM0BbqKCwanYlSODXQtp5owalt7a0JDwpId9_O8Pl24CjImZzPZLevJ_wSu4wv2IhVjK5QhkfBKBeaOxCeJaKfMVT5AzDQ-WwtJwahr1Dk0H2BxD6Hmp4KKBFRlVwGxq9-8uKBpbrmlClSuxPreBJ_xhP3SHtBFbVfcr38uaT_Bdh-gPRPgi-59tKOWPCY2FytO-Ls1U2l7w"; + +#[tokio::test] +async fn test_license_cpu_limit() -> Result<()> { + // Now 8 * 3 = 24 cores in total. + let mut cluster = Cluster::start(Configuration { + compute_nodes: 3, + compute_node_cores: 8, + ..Default::default() + }) + .await?; + + let mut session = cluster.start_session(); + + macro_rules! set_license_key { + ($key:expr) => { + session + .run(format!("ALTER SYSTEM SET license_key TO '{}';", $key)) + .await?; + tokio::time::sleep(std::time::Duration::from_secs(10)).await; + }; + } + + macro_rules! test_paid_tier { + () => { + session.run("SELECT rw_test_paid_tier();").await + }; + } + + set_license_key!(""); + let error = test_paid_tier!().unwrap_err().to_report_string(); + assert!(error.contains("feature TestPaid is only available for tier Paid and above, while the current tier is Free"), "{error}"); + + // Set a license key with CPU limit 100, it should work. + set_license_key!(KEY_100); + test_paid_tier!().unwrap().assert_result_eq("t"); + + // Set a license key with CPU limit 20, which is lower than the current CPU cores. + // Paid-tier features should not be available. + set_license_key!(KEY_20); + let error = test_paid_tier!().unwrap_err().to_report_string(); + assert!( + error.contains("the license key is currently not effective"), + "{error}" + ); + + // Kill a compute node, the total cores will be reduced to 16, which is under the limit. + // The paid-tier features should be available again. + cluster.simple_kill_nodes(["compute-2"]).await; + tokio::time::sleep(std::time::Duration::from_secs(100)).await; + test_paid_tier!().unwrap().assert_result_eq("t"); + + // Add it back, will be unavailable again. + cluster.simple_restart_nodes(["compute-2"]).await; + tokio::time::sleep(std::time::Duration::from_secs(100)).await; + let error = test_paid_tier!().unwrap_err().to_report_string(); + assert!( + error.contains("the license key is currently not effective"), + "{error}" + ); + + // Set a license key with CPU limit 100, it should work again. + set_license_key!(KEY_100); + test_paid_tier!().unwrap().assert_result_eq("t"); + + Ok(()) +} diff --git a/src/tests/simulation/tests/integration_tests/main.rs b/src/tests/simulation/tests/integration_tests/main.rs index ad8e854a30e5f..1bb58688192c8 100644 --- a/src/tests/simulation/tests/integration_tests/main.rs +++ b/src/tests/simulation/tests/integration_tests/main.rs @@ -22,12 +22,12 @@ mod backfill_tests; mod batch; +mod compaction; +mod default_parallelism; +mod license_cpu_limit; mod recovery; mod scale; mod sink; mod storage; mod throttle; - -mod compaction; -mod default_parallelism; mod utils;