diff --git a/.github/workflows/auto-create-doc-issue-by-pr.yml b/.github/workflows/auto-create-doc-issue-by-pr.yml index 533b5c6c09e4..d5b641de5664 100644 --- a/.github/workflows/auto-create-doc-issue-by-pr.yml +++ b/.github/workflows/auto-create-doc-issue-by-pr.yml @@ -39,7 +39,7 @@ jobs: - name: Create issue in other repository if: steps.check_merged.outputs.merged == 'true' && steps.check_documentation_update.outputs.documentation_update == 'true' run: | - ISSUE_CONTENT="This issue tracks the documentation update needed for the merged PR #$PR_ID.\n\nSource PR URL: $PR_URL\nSource PR Merged At: $PR_MERGED_AT" + ISSUE_CONTENT="This issue tracks the documentation update needed for the merged PR #$PR_ID.\n\nSource PR URL: $PR_URL\nSource PR Merged At: $PR_MERGED_AT\n\nIf it is a major improvement that deserves a new page or a new section in the documentation, please check if we should label it as an experiment feature." curl -X POST \ -H "Authorization: Bearer ${{ secrets.ACCESS_TOKEN }}" \ diff --git a/Cargo.lock b/Cargo.lock index 1328536f9185..d007b6c6f4f3 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -230,6 +230,30 @@ dependencies = [ "zstd 0.12.4", ] +[[package]] +name = "apache-avro" +version = "0.17.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "1aef82843a0ec9f8b19567445ad2421ceeb1d711514384bdd3d49fe37102ee13" +dependencies = [ + "bigdecimal 0.4.5", + "digest", + "libflate", + "log", + "num-bigint", + "quad-rand", + "rand", + "regex-lite", + "serde", + "serde_bytes", + "serde_json", + "strum 0.26.3", + "strum_macros 0.26.4", + "thiserror", + "typed-builder 0.19.1", + "uuid", +] + [[package]] name = "apache-avro" version = "0.17.0" @@ -246,7 +270,7 @@ dependencies = [ "regex-lite", "serde", "serde_json", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "thiserror", "typed-builder 0.18.2", @@ -5930,11 +5954,11 @@ dependencies = [ [[package]] name = "iceberg" -version = "0.2.0" -source = "git+https://github.com/risingwavelabs/iceberg-rust.git?rev=24bd5869f2779a8b9786b5a6e1f9723844f5a82c#24bd5869f2779a8b9786b5a6e1f9723844f5a82c" +version = "0.3.0" +source = "git+https://github.com/apache/iceberg-rust.git?rev=4440af69a354d9af56f239a6126a7f4b7945d58b#4440af69a354d9af56f239a6126a7f4b7945d58b" dependencies = [ "anyhow", - "apache-avro 0.17.0", + "apache-avro 0.17.0 (registry+https://github.com/rust-lang/crates.io-index)", "array-init", "arrow-arith 52.0.0", "arrow-array 52.0.0", @@ -5942,22 +5966,18 @@ dependencies = [ "arrow-schema 52.0.0", "arrow-select 52.0.0", "arrow-string 52.0.0", - "async-stream", "async-trait", "bimap", "bitvec", "bytes", "chrono", "derive_builder 0.20.0", - "either", "fnv", "futures", "itertools 0.13.0", - "lazy_static", - "log", "murmur3", "once_cell", - "opendal", + "opendal 0.49.0", "ordered-float 4.1.1", "parquet 52.0.0", "reqwest 0.12.4", @@ -5969,19 +5989,19 @@ dependencies = [ "serde_repr", "serde_with 3.8.0", "tokio", - "typed-builder 0.18.2", + "typed-builder 0.19.1", "url", - "urlencoding", "uuid", ] [[package]] name = "iceberg-catalog-rest" -version = "0.2.0" -source = "git+https://github.com/risingwavelabs/iceberg-rust.git?rev=24bd5869f2779a8b9786b5a6e1f9723844f5a82c#24bd5869f2779a8b9786b5a6e1f9723844f5a82c" +version = "0.3.0" +source = "git+https://github.com/apache/iceberg-rust.git?rev=4440af69a354d9af56f239a6126a7f4b7945d58b#4440af69a354d9af56f239a6126a7f4b7945d58b" dependencies = [ "async-trait", "chrono", + "http 1.1.0", "iceberg", "itertools 0.13.0", "log", @@ -5989,8 +6009,8 @@ dependencies = [ "serde", "serde_derive", "serde_json", - "typed-builder 0.18.2", - "urlencoding", + "tokio", + "typed-builder 0.19.1", "uuid", ] @@ -6000,7 +6020,7 @@ version = "0.3.141592654" source = "git+https://github.com/risingwavelabs/icelake.git?rev=1860eb315183a5f3f72b4097c1e40d49407f8373#1860eb315183a5f3f72b4097c1e40d49407f8373" dependencies = [ "anyhow", - "apache-avro 0.17.0", + "apache-avro 0.17.0 (git+https://github.com/apache/avro.git)", "arrow-arith 52.0.0", "arrow-array 52.0.0", "arrow-buffer 52.0.0", @@ -6023,7 +6043,7 @@ dependencies = [ "log", "murmur3", "once_cell", - "opendal", + "opendal 0.47.2", "ordered-float 3.9.1", "parquet 52.0.0", "prometheus", @@ -6700,9 +6720,9 @@ dependencies = [ [[package]] name = "log" -version = "0.4.21" +version = "0.4.22" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "90ed8c1e510134f979dbc4f070f87d4313098b704861a105fe34231c70a3901c" +checksum = "a7a70ba024b9dc04c27ea2f0c0548feb474ec5c54bba33a7f72f873a39d07b24" dependencies = [ "value-bag", ] @@ -7558,9 +7578,9 @@ dependencies = [ [[package]] name = "num-bigint" -version = "0.4.5" +version = "0.4.6" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c165a9ab64cf766f73521c0dd2cfdff64f488b8f0b3e621face3462d3db536d7" +checksum = "a5e44f723f1133c9deac646763579fdb3ac745e418f2a7af9cd0c431da1f20b9" dependencies = [ "num-integer", "num-traits", @@ -7809,7 +7829,7 @@ dependencies = [ "percent-encoding", "prometheus", "quick-xml 0.31.0", - "reqsign", + "reqsign 0.15.2", "reqwest 0.12.4", "serde", "serde_json", @@ -7818,6 +7838,36 @@ dependencies = [ "uuid", ] +[[package]] +name = "opendal" +version = "0.49.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "39d516adf7db912c38af382c3e92c27cd62fbbc240e630920555d784c2ab1494" +dependencies = [ + "anyhow", + "async-trait", + "backon", + "base64 0.22.0", + "bytes", + "chrono", + "crc32c", + "flagset", + "futures", + "getrandom", + "http 1.1.0", + "log", + "md-5", + "once_cell", + "percent-encoding", + "quick-xml 0.36.1", + "reqsign 0.16.0", + "reqwest 0.12.4", + "serde", + "serde_json", + "tokio", + "uuid", +] + [[package]] name = "openidconnect" version = "3.4.0" @@ -9353,6 +9403,26 @@ dependencies = [ "serde", ] +[[package]] +name = "quick-xml" +version = "0.35.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "86e446ed58cef1bbfe847bc2fda0e2e4ea9f0e57b90c507d4781292590d72a4e" +dependencies = [ + "memchr", + "serde", +] + +[[package]] +name = "quick-xml" +version = "0.36.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "96a05e2e8efddfa51a84ca47cec303fac86c8541b686d37cac5efc0e094417bc" +dependencies = [ + "memchr", + "serde", +] + [[package]] name = "quote" version = "1.0.36" @@ -9647,6 +9717,34 @@ dependencies = [ "sha2", ] +[[package]] +name = "reqsign" +version = "0.16.0" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "03dd4ba7c3901dd43e6b8c7446a760d45bc1ea4301002e1a6fa48f97c3a796fa" +dependencies = [ + "anyhow", + "async-trait", + "base64 0.22.0", + "chrono", + "form_urlencoded", + "getrandom", + "hex", + "hmac", + "home", + "http 1.1.0", + "log", + "percent-encoding", + "quick-xml 0.35.0", + "rand", + "reqwest 0.12.4", + "rust-ini", + "serde", + "serde_json", + "sha1", + "sha2", +] + [[package]] name = "reqwest" version = "0.11.20" @@ -9934,7 +10032,7 @@ dependencies = [ "madsim-tokio", "madsim-tonic", "memcomparable", - "opendal", + "opendal 0.47.2", "parking_lot 0.12.1", "parquet 52.0.0", "paste", @@ -10051,7 +10149,7 @@ dependencies = [ "risingwave_meta_node", "risingwave_rt", "shell-words", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "tempfile", "thiserror-ext", @@ -10160,7 +10258,7 @@ dependencies = [ "speedate", "stacker", "static_assertions", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "sysinfo", "tempfile", @@ -10468,7 +10566,7 @@ dependencies = [ "mysql_common", "nexmark", "num-bigint", - "opendal", + "opendal 0.47.2", "openssl", "parking_lot 0.12.1", "parquet 52.0.0", @@ -10510,7 +10608,7 @@ dependencies = [ "serde_yaml", "simd-json", "sqlx", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "syn 1.0.109", "tempfile", @@ -11081,7 +11179,7 @@ dependencies = [ "sea-orm", "serde", "serde_json", - "strum 0.26.2", + "strum 0.26.3", "sync-point", "thiserror", "thiserror-ext", @@ -11228,7 +11326,7 @@ dependencies = [ "madsim", "madsim-aws-sdk-s3", "madsim-tokio", - "opendal", + "opendal 0.47.2", "prometheus", "reqwest 0.12.4", "risingwave_common", @@ -11256,7 +11354,7 @@ dependencies = [ "prost-helpers", "risingwave_error", "serde", - "strum 0.26.2", + "strum 0.26.3", "thiserror", "walkdir", "workspace-hack", @@ -11615,7 +11713,7 @@ dependencies = [ "serde_yaml", "smallvec", "static_assertions", - "strum 0.26.2", + "strum 0.26.3", "strum_macros 0.26.4", "thiserror", "thiserror-ext", @@ -12495,9 +12593,9 @@ checksum = "a3f0bf26fd526d2a95683cd0f87bf103b8539e2ca1ef48ce002d67aad59aa0b4" [[package]] name = "serde" -version = "1.0.203" +version = "1.0.208" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "7253ab4de971e72fb7be983802300c30b5a7f0c2e56fab8abfc6a214307c0094" +checksum = "cff085d2cb684faa248efb494c39b68e522822ac0de72ccf08109abde717cfb2" dependencies = [ "serde_derive", ] @@ -12523,9 +12621,9 @@ dependencies = [ [[package]] name = "serde_bytes" -version = "0.11.12" +version = "0.11.15" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "ab33ec92f677585af6d88c65593ae2375adde54efdbf16d597f2cbc7a6d368ff" +checksum = "387cc504cb06bb40a96c8e04e951fe01854cf6bc921053c954e4a606d9675c6a" dependencies = [ "serde", ] @@ -12544,9 +12642,9 @@ dependencies = [ [[package]] name = "serde_derive" -version = "1.0.203" +version = "1.0.208" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "500cbc0ebeb6f46627f50f3f5811ccf6bf00643be300b4c3eabc0ef55dc5b5ba" +checksum = "24008e81ff7613ed8e5ba0cfaf24e2c2f1e5b8a0495711e44fcd4882fca62bcf" dependencies = [ "proc-macro2", "quote", @@ -12566,11 +12664,12 @@ dependencies = [ [[package]] name = "serde_json" -version = "1.0.117" +version = "1.0.125" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "455182ea6142b14f93f4bc5320a2b31c1f266b66a4a5c858b013302a5d8cbfc3" +checksum = "83c8e735a073ccf5be70aa8066aa984eaf2fa000db6c8d0100ae605b366d31ed" dependencies = [ "itoa", + "memchr", "ryu", "serde", ] @@ -13457,9 +13556,9 @@ dependencies = [ [[package]] name = "strum" -version = "0.26.2" +version = "0.26.3" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "5d8cec3501a5194c432b2b7976db6b7d10ec95c253208b45f83f7136aa985e29" +checksum = "8fec0f0aef304996cf250b31b5a10dee7980c85da9d759361292b8bca5a18f06" dependencies = [ "strum_macros 0.26.4", ] @@ -13704,9 +13803,9 @@ checksum = "3369f5ac52d5eb6ab48c6b4ffdc8efbcad6b89c765749064ba298f2c68a16a76" [[package]] name = "thiserror" -version = "1.0.61" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "c546c80d6be4bc6a00c0f01730c08df82eaa7a7a61f11d656526506112cc1709" +checksum = "c0342370b38b6a11b6cc11d6a805569958d54cfa061a29969c3b5ce2ea405724" dependencies = [ "thiserror-impl", ] @@ -13735,9 +13834,9 @@ dependencies = [ [[package]] name = "thiserror-impl" -version = "1.0.61" +version = "1.0.63" source = "registry+https://github.com/rust-lang/crates.io-index" -checksum = "46c3384250002a6d5af4d114f2845d37b57521033f30d5c3f46c4d70e1197533" +checksum = "a4558b58466b9ad7ca0f102865eccc95938dca1a74a856f2b57b6629050da261" dependencies = [ "proc-macro2", "quote", @@ -14629,6 +14728,15 @@ dependencies = [ "typed-builder-macro 0.18.2", ] +[[package]] +name = "typed-builder" +version = "0.19.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "a06fbd5b8de54c5f7c91f6fe4cebb949be2125d7758e630bb58b1d831dbce600" +dependencies = [ + "typed-builder-macro 0.19.1", +] + [[package]] name = "typed-builder-macro" version = "0.16.2" @@ -14651,6 +14759,17 @@ dependencies = [ "syn 2.0.66", ] +[[package]] +name = "typed-builder-macro" +version = "0.19.1" +source = "registry+https://github.com/rust-lang/crates.io-index" +checksum = "f9534daa9fd3ed0bd911d462a37f172228077e7abf18c18a5f67199d959205f8" +dependencies = [ + "proc-macro2", + "quote", + "syn 2.0.66", +] + [[package]] name = "typenum" version = "1.16.0" diff --git a/Cargo.toml b/Cargo.toml index 26c48279900d..8c468bf059d5 100644 --- a/Cargo.toml +++ b/Cargo.toml @@ -141,10 +141,8 @@ arrow-array-iceberg = { package = "arrow-array", version = "52" } arrow-schema-iceberg = { package = "arrow-schema", version = "52" } arrow-buffer-iceberg = { package = "arrow-buffer", version = "52" } arrow-cast-iceberg = { package = "arrow-cast", version = "52" } -# TODO -# After apache/iceberg-rust#411 is merged, we move to the upstream version. -iceberg = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "24bd5869f2779a8b9786b5a6e1f9723844f5a82c" } -iceberg-catalog-rest = { git = "https://github.com/risingwavelabs/iceberg-rust.git", rev = "24bd5869f2779a8b9786b5a6e1f9723844f5a82c" } +iceberg = { git = "https://github.com/apache/iceberg-rust.git", rev = "4440af69a354d9af56f239a6126a7f4b7945d58b" } +iceberg-catalog-rest = { git = "https://github.com/apache/iceberg-rust.git", rev = "4440af69a354d9af56f239a6126a7f4b7945d58b" } opendal = "0.47" arrow-array = "50" arrow-arith = "50" diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index b6eff317a79e..5cdd4b9e3999 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -130,7 +130,7 @@ echo "> inserted new rows into postgres" # start cluster w/o clean-data unset RISINGWAVE_CI -export RUST_LOG="events::stream::message::chunk=trace,risingwave_stream=debug,risingwave_batch=info,risingwave_storage=info" \ +export RUST_LOG="risingwave_stream=debug,risingwave_batch=info,risingwave_storage=info" \ risedev dev ci-1cn-1fe-with-recovery echo "> wait for cluster recovery finish" diff --git a/e2e_test/batch/types/map.slt.part b/e2e_test/batch/types/map.slt.part index 5f68bcad2274..b4b4be7e5cba 100644 --- a/e2e_test/batch/types/map.slt.part +++ b/e2e_test/batch/types/map.slt.part @@ -8,66 +8,66 @@ create table t (m map (float, float)); db error: ERROR: Failed to run the query Caused by: - invalid map key type: double precision + Bind error: invalid map key type: double precision query error -select map_from_entries(array[1.0,2.0,3.0], array[1,2,3]); +select map_from_key_values(array[1.0,2.0,3.0], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): - 1: Failed to bind expression: map_from_entries(ARRAY[1.0, 2.0, 3.0], ARRAY[1, 2, 3]) + 1: Failed to bind expression: map_from_key_values(ARRAY[1.0, 2.0, 3.0], ARRAY[1, 2, 3]) 2: Expr error 3: invalid map key type: numeric query error -select map_from_entries(array[1,1,3], array[1,2,3]); +select map_from_key_values(array[1,1,3], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{1,1,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_key_values('{1,1,3}', '{1,2,3}')` 3: map keys must be unique query ? -select map_from_entries(array[1,2,3], array[1,null,3]); +select map_from_key_values(array[1,2,3], array[1,null,3]); ---- {1:1,2:NULL,3:3} query error -select map_from_entries(array[1,null,3], array[1,2,3]); +select map_from_key_values(array[1,null,3], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{1,NULL,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_key_values('{1,NULL,3}', '{1,2,3}')` 3: map keys must not be NULL query error -select map_from_entries(array[1,3], array[1,2,3]); +select map_from_key_values(array[1,3], array[1,2,3]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{1,3}', '{1,2,3}')` + 2: error while evaluating expression `map_from_key_values('{1,3}', '{1,2,3}')` 3: map keys and values have different length query error -select map_from_entries(array[1,2], array[1,2]) = map_from_entries(array[2,1], array[2,1]); +select map_from_key_values(array[1,2], array[1,2]) = map_from_key_values(array[2,1], array[2,1]); ---- db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): - 1: Failed to bind expression: map_from_entries(ARRAY[1, 2], ARRAY[1, 2]) = map_from_entries(ARRAY[2, 1], ARRAY[2, 1]) + 1: Failed to bind expression: map_from_key_values(ARRAY[1, 2], ARRAY[1, 2]) = map_from_key_values(ARRAY[2, 1], ARRAY[2, 1]) 2: function equal(map(integer,integer), map(integer,integer)) does not exist @@ -83,32 +83,32 @@ create table t ( statement ok insert into t values ( - map_from_entries(array['a','b','c'], array[1.0,2.0,3.0]::float[]), - map_from_entries(array[1,2,3], array[true,false,true]), - map_from_entries(array['a','b'], + map_from_key_values(array['a','b','c'], array[1.0,2.0,3.0]::float[]), + map_from_key_values(array[1,2,3], array[true,false,true]), + map_from_key_values(array['a','b'], array[ - map_from_entries(array['a1'], array['a2']), - map_from_entries(array['b1'], array['b2']) + map_from_key_values(array['a1'], array['a2']), + map_from_key_values(array['b1'], array['b2']) ] ), array[ - map_from_entries(array['a','b','c'], array[1,2,3]), - map_from_entries(array['d','e','f'], array[4,5,6]) + map_from_key_values(array['a','b','c'], array[1,2,3]), + map_from_key_values(array['d','e','f'], array[4,5,6]) ], row( - map_from_entries(array['a','b','c'], array[row(1),row(2),row(3)]::struct[]) + map_from_key_values(array['a','b','c'], array[row(1),row(2),row(3)]::struct[]) ) ); # cast(map(character varying,integer)) -> map(character varying,double precision) query ? -select map_from_entries(array['a','b','c'], array[1,2,3])::map(varchar,float); +select map_from_key_values(array['a','b','c'], array[1,2,3])::map(varchar,float); ---- {a:1,b:2,c:3} statement ok -insert into t(m1) values (map_from_entries(array['a','b','c'], array[1,2,3])); +insert into t(m1) values (map_from_key_values(array['a','b','c'], array[1,2,3])); query ????? rowsort select * from t; @@ -144,7 +144,7 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: Expr error - 2: error while evaluating expression `map_from_entries('{a,a}', '{1,2}')` + 2: error while evaluating expression `map_from_key_values('{a,a}', '{1,2}')` 3: map keys must be unique @@ -165,3 +165,96 @@ select MAP{1:'a',2:'b'}::MAP(VARCHAR,VARCHAR) ---- {} {1:a,2:b} + +query error +select map_from_entries(array[]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(ARRAY[]) + 2: Bind error: cannot determine type of empty array +HINT: Explicitly cast to the desired type, for example ARRAY[]::integer[]. + + +query error +select map_from_entries(array[]::int[]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(CAST(ARRAY[] AS INT[])) + 2: Expr error + 3: invalid map entries type, expected struct, got: integer + + +query error +select map_from_entries(array[]::struct[]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(CAST(ARRAY[] AS STRUCT[])) + 2: Expr error + 3: invalid map key type: double precision + + +query ? +select map_from_entries(array[]::struct[]); +---- +{} + + +query ? +select map_from_entries(array[row('a',1), row('b',2), row('c',3)]); +---- +{a:1,b:2,c:3} + + +query error +select map_from_entries(array[row('a',1), row('a',2), row('c',3)]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Expr error + 2: error while evaluating expression `map_from_entries('{"(a,1)","(a,2)","(c,3)"}')` + 3: map keys must be unique + + +query error +select map_from_entries(array[row('a',1,2)]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(ARRAY[ROW('a', 1, 2)]) + 2: Expr error + 3: the underlying struct for map must have exactly two fields, got: StructType { field_names: [], field_types: [Varchar, Int32, Int32] } + + +query error +select map_from_entries(array[row(1.0,1)]); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(ARRAY[ROW(1.0, 1)]) + 2: Expr error + 3: invalid map key type: numeric + + +query error +select map_from_entries(null); +---- +db error: ERROR: Failed to run the query + +Caused by these errors (recent errors listed first): + 1: Failed to bind expression: map_from_entries(NULL) + 2: Bind error: Cannot implicitly cast 'null:Varchar' to polymorphic type AnyArray + + +query ? +select map_from_entries(null::struct[]); +---- +NULL diff --git a/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt b/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt index efc377f8aed4..3f2c400b91f0 100644 --- a/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt +++ b/e2e_test/streaming/aggregate/shuffle_approx_percentile.slt @@ -58,19 +58,6 @@ select * from m1; ---- -982.5779489474152 -804.4614206837127 0 804.4614206837127 982.5779489474152 -# Test state encode / decode -onlyif can-use-recover -statement ok -recover; - -onlyif can-use-recover -sleep 10s - -query I -select * from m1; ----- --982.5779489474152 -804.4614206837127 0 804.4614206837127 982.5779489474152 - # Test 0 1000) // ------------------------ diff --git a/proto/meta.proto b/proto/meta.proto index bcb6c331549f..4f51522cc006 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -321,9 +321,6 @@ message AddWorkerNodeRequest { } message AddWorkerNodeResponse { - reserved 3; - reserved "system_params"; - common.Status status = 1; optional uint32 node_id = 2; string cluster_id = 4; } diff --git a/src/common/src/array/iterator.rs b/src/common/src/array/iterator.rs index 78c89f422aa0..31518150e6f8 100644 --- a/src/common/src/array/iterator.rs +++ b/src/common/src/array/iterator.rs @@ -56,10 +56,10 @@ mod tests { use super::*; use crate::array::{ArrayBuilder, ArrayImpl}; - use crate::for_all_array_variants; + use crate::for_all_variants; macro_rules! test_trusted_len { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( paste! { #[test] @@ -91,5 +91,5 @@ mod tests { }; } - for_all_array_variants! { test_trusted_len } + for_all_variants! { test_trusted_len } } diff --git a/src/common/src/array/list_array.rs b/src/common/src/array/list_array.rs index c30229852c0a..745b1f6bbab0 100644 --- a/src/common/src/array/list_array.rs +++ b/src/common/src/array/list_array.rs @@ -372,11 +372,11 @@ impl ListValue { /// Creates a new `ListValue` from an iterator of `Datum`. pub fn from_datum_iter( - datatype: &DataType, + elem_datatype: &DataType, iter: impl IntoIterator, ) -> Self { let iter = iter.into_iter(); - let mut builder = datatype.create_array_builder(iter.size_hint().0); + let mut builder = elem_datatype.create_array_builder(iter.size_hint().0); for datum in iter { builder.append(datum); } diff --git a/src/common/src/array/map_array.rs b/src/common/src/array/map_array.rs index f0904211f5ed..2f0da9bbf816 100644 --- a/src/common/src/array/map_array.rs +++ b/src/common/src/array/map_array.rs @@ -24,6 +24,7 @@ use serde::Serializer; use super::{ Array, ArrayBuilder, ArrayImpl, ArrayResult, DatumRef, DefaultOrdered, ListArray, ListArrayBuilder, ListRef, ListValue, MapType, ScalarRef, ScalarRefImpl, StructArray, + StructRef, }; use crate::bitmap::Bitmap; use crate::types::{DataType, Scalar, ToText}; @@ -162,7 +163,7 @@ impl Array for MapArray { fn data_type(&self) -> DataType { let list_value_type = self.inner.values().data_type(); - DataType::Map(MapType::from_list_entries(list_value_type)) + DataType::Map(MapType::from_entries(list_value_type)) } } @@ -193,7 +194,10 @@ pub use scalar::{MapRef, MapValue}; /// We only check the invariants in the constructors. /// After they are constructed, we assume the invariants holds. mod scalar { + use std::collections::HashSet; + use super::*; + use crate::array::{Datum, ScalarImpl, StructValue}; /// Refer to [`MapArray`] for the invariants of a map value. #[derive(Clone, Eq, EstimateSize)] @@ -221,20 +225,33 @@ mod scalar { /// # Panics /// Panics if [map invariants](`super::MapArray`) are violated. - pub fn from_list_entries(list: ListValue) -> Self { + pub fn from_entries(entries: ListValue) -> Self { + Self::try_from_entries(entries).unwrap() + } + + /// Returns error if [map invariants](`super::MapArray`) are violated. + pub fn try_from_entries(entries: ListValue) -> Result { // validates list type is valid - _ = MapType::from_list_entries(list.data_type()); - // TODO: validate the values is valid - MapValue(list) + let _ = MapType::try_from_entries(entries.data_type())?; + let mut keys = HashSet::with_capacity(entries.len()); + let struct_array = entries.into_array(); + for key in struct_array.as_struct().field_at(0).iter() { + let Some(key) = key else { + return Err("map keys must not be NULL".to_string()); + }; + if !keys.insert(key) { + return Err("map keys must be unique".to_string()); + } + } + Ok(MapValue(ListValue::new(struct_array))) } - /// # Panics - /// Panics if [map invariants](`super::MapArray`) are violated. + /// Returns error if [map invariants](`super::MapArray`) are violated. pub fn try_from_kv(key: ListValue, value: ListValue) -> Result { if key.len() != value.len() { return Err("map keys and values have different length".to_string()); } - let unique_keys = key.iter().unique().collect_vec(); + let unique_keys: HashSet<_> = key.iter().unique().collect(); if unique_keys.len() != key.len() { return Err("map keys must be unique".to_string()); } @@ -252,6 +269,46 @@ mod scalar { ); Ok(MapValue(ListValue::new(struct_array.into()))) } + + /// # Panics + /// Panics if `m1` and `m2` have different types. + pub fn concat(m1: MapRef<'_>, m2: MapRef<'_>) -> Self { + debug_assert_eq!(m1.inner().data_type(), m2.inner().data_type()); + let m2_keys = m2.keys(); + let l = ListValue::from_datum_iter( + &m1.inner().data_type(), + m1.iter_struct() + .filter(|s| !m2_keys.contains(&s.field_at(0).expect("map key is not null"))) + .chain(m2.iter_struct()) + .map(|s| Some(ScalarRefImpl::Struct(s))), + ); + Self::from_entries(l) + } + + pub fn insert(m: MapRef<'_>, key: ScalarImpl, value: Datum) -> Self { + let l = ListValue::from_datum_iter( + &m.inner().data_type(), + m.iter_struct() + .filter(|s| { + key.as_scalar_ref_impl() != s.field_at(0).expect("map key is not null") + }) + .chain(std::iter::once( + StructValue::new(vec![Some(key.clone()), value]).as_scalar_ref(), + )) + .map(|s| Some(ScalarRefImpl::Struct(s))), + ); + Self::from_entries(l) + } + + pub fn delete(m: MapRef<'_>, key: ScalarRefImpl<'_>) -> Self { + let l = ListValue::from_datum_iter( + &m.inner().data_type(), + m.iter_struct() + .filter(|s| key != s.field_at(0).expect("map key is not null")) + .map(|s| Some(ScalarRefImpl::Struct(s))), + ); + Self::from_entries(l) + } } impl<'a> MapRef<'a> { @@ -272,6 +329,14 @@ mod scalar { pub fn into_kv(self) -> (ListRef<'a>, ListRef<'a>) { self.0.as_map_kv() } + + pub fn keys(&self) -> HashSet> { + self.iter().map(|(k, _v)| k).collect() + } + + pub fn to_owned(self) -> MapValue { + MapValue(self.0.to_owned()) + } } impl Scalar for MapValue { @@ -379,6 +444,15 @@ impl<'a> MapRef<'a> { }) } + pub fn iter_struct( + self, + ) -> impl DoubleEndedIterator + ExactSizeIterator> + 'a { + self.inner().iter().map(|list_elem| { + let list_elem = list_elem.expect("the list element in map should not be null"); + list_elem.into_struct() + }) + } + pub fn iter_sorted( self, ) -> impl DoubleEndedIterator + ExactSizeIterator, DatumRef<'a>)> + 'a @@ -411,7 +485,7 @@ impl MapValue { deserializer: &mut memcomparable::Deserializer, ) -> memcomparable::Result { let list = ListValue::memcmp_deserialize(&datatype.clone().into_struct(), deserializer)?; - Ok(Self::from_list_entries(list)) + Ok(Self::from_entries(list)) } } diff --git a/src/common/src/array/mod.rs b/src/common/src/array/mod.rs index ae6f7d0fa144..b34e5f9b9c47 100644 --- a/src/common/src/array/mod.rs +++ b/src/common/src/array/mod.rs @@ -68,7 +68,7 @@ pub use self::error::ArrayError; pub use crate::array::num256_array::{Int256Array, Int256ArrayBuilder}; use crate::bitmap::Bitmap; use crate::types::*; -use crate::{dispatch_array_builder_variants, dispatch_array_variants, for_all_array_variants}; +use crate::{dispatch_array_builder_variants, dispatch_array_variants, for_all_variants}; pub type ArrayResult = Result; pub type I64Array = PrimitiveArray; @@ -325,7 +325,7 @@ impl CompactableArray for A { /// Define `ArrayImpl` with macro. macro_rules! array_impl_enum { - ( $( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ( $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { /// `ArrayImpl` embeds all possible array in `array` module. #[derive(Debug, Clone, EstimateSize)] pub enum ArrayImpl { @@ -334,7 +334,7 @@ macro_rules! array_impl_enum { }; } -for_all_array_variants! { array_impl_enum } +for_all_variants! { array_impl_enum } // We cannot put the From implementations in impl_convert, // because then we can't prove for all `T: PrimitiveArrayItemType`, @@ -401,7 +401,7 @@ impl From for ArrayImpl { /// * `ArrayImpl -> Array` with `From` trait. /// * `ArrayBuilder -> ArrayBuilderImpl` with `From` trait. macro_rules! impl_convert { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( paste! { impl ArrayImpl { @@ -455,11 +455,11 @@ macro_rules! impl_convert { }; } -for_all_array_variants! { impl_convert } +for_all_variants! { impl_convert } /// Define `ArrayImplBuilder` with macro. macro_rules! array_builder_impl_enum { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { /// `ArrayBuilderImpl` embeds all possible array in `array` module. #[derive(Debug, Clone, EstimateSize)] pub enum ArrayBuilderImpl { @@ -468,7 +468,7 @@ macro_rules! array_builder_impl_enum { }; } -for_all_array_variants! { array_builder_impl_enum } +for_all_variants! { array_builder_impl_enum } /// Implements all `ArrayBuilder` functions with `for_all_variant`. impl ArrayBuilderImpl { diff --git a/src/common/src/array/struct_array.rs b/src/common/src/array/struct_array.rs index 9c3bd2365381..ebf224f58161 100644 --- a/src/common/src/array/struct_array.rs +++ b/src/common/src/array/struct_array.rs @@ -393,6 +393,15 @@ impl<'a> StructRef<'a> { iter_fields_ref!(self, it, { Either::Left(it) }, { Either::Right(it) }) } + /// # Panics + /// Panics if the index is out of bounds. + pub fn field_at(&self, i: usize) -> DatumRef<'a> { + match self { + StructRef::Indexed { arr, idx } => arr.field_at(i).value_at(*idx), + StructRef::ValueRef { val } => val.fields[i].to_datum_ref(), + } + } + pub fn memcmp_serialize( self, serializer: &mut memcomparable::Serializer, diff --git a/src/common/src/test_utils/rand_array.rs b/src/common/src/test_utils/rand_array.rs index a7c13e3178f2..33cf42bc403e 100644 --- a/src/common/src/test_utils/rand_array.rs +++ b/src/common/src/test_utils/rand_array.rs @@ -154,7 +154,7 @@ impl RandValue for ListValue { impl RandValue for MapValue { fn rand_value(_rand: &mut R) -> Self { // dummy value - MapValue::from_list_entries(ListValue::empty(&DataType::Struct( + MapValue::from_entries(ListValue::empty(&DataType::Struct( MapType::struct_type_for_map(DataType::Varchar, DataType::Varchar), ))) } @@ -201,12 +201,12 @@ where #[cfg(test)] mod tests { use super::*; - use crate::for_all_array_variants; + use crate::for_all_variants; #[test] fn test_create_array() { macro_rules! gen_rand_array { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( { let array = seed_rand_array::<$array>(10, 1024, 0.5); @@ -216,6 +216,6 @@ mod tests { }; } - for_all_array_variants! { gen_rand_array } + for_all_variants! { gen_rand_array } } } diff --git a/src/common/src/types/macros.rs b/src/common/src/types/macros.rs index 1dd29156dd65..9fa9c7ffe5ec 100644 --- a/src/common/src/types/macros.rs +++ b/src/common/src/types/macros.rs @@ -64,63 +64,6 @@ macro_rules! for_all_variants { }; } -/// The projected version of `for_all_variants` for handling scalar variants. -/// -/// Arguments are `$variant_name`, `$suffix_name`, `$scalar`, `$scalar_ref`. -#[macro_export(local_inner_macros)] -macro_rules! for_all_scalar_variants { - ($macro:ident $(, $x:tt)*) => { - for_all_variants! { project_scalar_variants, $macro, [ $($x, )* ] } - }; -} -#[macro_export] -macro_rules! project_scalar_variants { - ($macro:ident, [ $($x:tt, )* ], $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { - $macro! { - $($x, )* - $( { $variant_name, $suffix_name, $scalar, $scalar_ref } ),* - } - }; -} - -/// The projected version of `for_all_variants` for handling array variants. -/// -/// Arguments are `$variant_name`, `$suffix_name`, `$array`, `$builder`. -#[macro_export(local_inner_macros)] -macro_rules! for_all_array_variants { - ($macro:ident $(, $x:tt)*) => { - for_all_variants! { project_array_variants, $macro, [ $($x, )* ] } - }; -} -#[macro_export] -macro_rules! project_array_variants { - ($macro:ident, [ $($x:tt, )* ], $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { - $macro! { - $($x, )* - $( { $variant_name, $suffix_name, $array, $builder } ),* - } - }; -} - -/// The projected version of `for_all_variants` for handling mapping of data types and array types. -/// -/// Arguments are `$data_type`, `$variant_name`. -#[macro_export(local_inner_macros)] -macro_rules! for_all_type_pairs { - ($macro:ident $(, $x:tt)*) => { - for_all_variants! { project_type_pairs, $macro, [ $($x, )* ] } - }; -} -#[macro_export] -macro_rules! project_type_pairs { - ($macro:ident, [ $($x:tt, )* ], $( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { - $macro! { - $($x, )* - $( { $data_type, $variant_name } ),* - } - }; -} - /// Helper macro for expanding type aliases and constants. Internally used by `dispatch_` macros. #[macro_export] macro_rules! do_expand_alias { diff --git a/src/common/src/types/map_type.rs b/src/common/src/types/map_type.rs index 4d9ec3dc5f14..e0dae8d9bc10 100644 --- a/src/common/src/types/map_type.rs +++ b/src/common/src/types/map_type.rs @@ -36,26 +36,37 @@ impl MapType { Self(Box::new((key, value))) } - pub fn try_from_kv(key: DataType, value: DataType) -> Result { + pub fn try_from_kv(key: DataType, value: DataType) -> Result { Self::check_key_type_valid(&key)?; Ok(Self(Box::new((key, value)))) } + pub fn try_from_entries(list_entries_type: DataType) -> Result { + match list_entries_type { + DataType::Struct(s) => { + let Some((k, v)) = s.iter().collect_tuple() else { + return Err(format!( + "the underlying struct for map must have exactly two fields, got: {s:?}" + )); + }; + // the field names are not strictly enforced + // Currently this panics for SELECT * FROM t + // if cfg!(debug_assertions) { + // itertools::assert_equal(struct_type.names(), ["key", "value"]); + // } + Self::try_from_kv(k.1.clone(), v.1.clone()) + } + _ => Err(format!( + "invalid map entries type, expected struct, got: {list_entries_type}" + )), + } + } + /// # Panics /// Panics if the key type is not valid for a map, or the /// entries type is not a valid struct type. - pub fn from_list_entries(list_entries_type: DataType) -> Self { - let struct_type = list_entries_type.as_struct(); - let (k, v) = struct_type - .iter() - .collect_tuple() - .expect("the underlying struct for map must have exactly two fields"); - // the field names are not strictly enforced - // Currently this panics for SELECT * FROM t - // if cfg!(debug_assertions) { - // itertools::assert_equal(struct_type.names(), ["key", "value"]); - // } - Self::from_kv(k.1.clone(), v.1.clone()) + pub fn from_entries(list_entries_type: DataType) -> Self { + Self::try_from_entries(list_entries_type).unwrap() } /// # Panics @@ -89,7 +100,7 @@ impl MapType { /// /// Note that this isn't definitive. /// Just be conservative at the beginning, but not too restrictive (like only allowing strings). - pub fn check_key_type_valid(data_type: &DataType) -> anyhow::Result<()> { + pub fn check_key_type_valid(data_type: &DataType) -> Result<(), String> { let ok = match data_type { DataType::Int16 | DataType::Int32 | DataType::Int64 => true, DataType::Varchar => true, @@ -111,7 +122,7 @@ impl MapType { | DataType::Map(_) => false, }; if !ok { - Err(anyhow::anyhow!("invalid map key type: {data_type}")) + Err(format!("invalid map key type: {data_type}")) } else { Ok(()) } @@ -128,7 +139,7 @@ impl FromStr for MapType { if let Some((key, value)) = s[4..s.len() - 1].split(',').collect_tuple() { let key = key.parse().context("failed to parse map key type")?; let value = value.parse().context("failed to parse map value type")?; - MapType::try_from_kv(key, value) + MapType::try_from_kv(key, value).map_err(|e| anyhow::anyhow!(e)) } else { Err(anyhow::anyhow!("expect map(...,...)")) } diff --git a/src/common/src/types/mod.rs b/src/common/src/types/mod.rs index e76bfaba384f..1fe1f3958e33 100644 --- a/src/common/src/types/mod.rs +++ b/src/common/src/types/mod.rs @@ -42,8 +42,7 @@ pub use crate::array::{ListRef, ListValue, MapRef, MapValue, StructRef, StructVa use crate::cast::{str_to_bool, str_to_bytea}; use crate::error::BoxedError; use crate::{ - dispatch_data_types, dispatch_scalar_ref_variants, dispatch_scalar_variants, - for_all_scalar_variants, for_all_type_pairs, + dispatch_data_types, dispatch_scalar_ref_variants, dispatch_scalar_variants, for_all_variants, }; mod cow; @@ -252,7 +251,7 @@ impl From<&PbDataType> for DataType { // Map is physically the same as a list. // So the first (and only) item is the list element type. let list_entries_type: DataType = (&proto.field_type[0]).into(); - DataType::Map(MapType::from_list_entries(list_entries_type)) + DataType::Map(MapType::from_entries(list_entries_type)) } PbTypeName::Int256 => DataType::Int256, } @@ -552,7 +551,7 @@ pub trait ScalarRef<'a>: private::ScalarBounds> + 'a + Copy { /// Define `ScalarImpl` and `ScalarRefImpl` with macro. macro_rules! scalar_impl_enum { - ($( { $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { /// `ScalarImpl` embeds all possible scalars in the evaluation framework. /// /// Note: `ScalarImpl` doesn't contain all information of its `DataType`, @@ -580,7 +579,7 @@ macro_rules! scalar_impl_enum { }; } -for_all_scalar_variants! { scalar_impl_enum } +for_all_variants! { scalar_impl_enum } // We MUST NOT implement `Ord` for `ScalarImpl` because that will make `Datum` derive an incorrect // default `Ord`. To get a default-ordered `ScalarImpl`/`ScalarRefImpl`/`Datum`/`DatumRef`, you can @@ -686,7 +685,7 @@ macro_rules! for_all_native_types { /// * `&ScalarImpl -> &Scalar` with `impl.as_int16()`. /// * `ScalarImpl -> Scalar` with `impl.into_int16()`. macro_rules! impl_convert { - ($( { $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( impl From<$scalar> for ScalarImpl { fn from(val: $scalar) -> Self { @@ -758,7 +757,7 @@ macro_rules! impl_convert { }; } -for_all_scalar_variants! { impl_convert } +for_all_variants! { impl_convert } // Implement `From` for `ScalarImpl::Float` as a sugar. impl From for ScalarImpl { @@ -850,6 +849,12 @@ impl From for ScalarImpl { } } +impl From> for ScalarImpl { + fn from(list: ListRef<'_>) -> Self { + Self::List(list.to_owned_scalar()) + } +} + impl ScalarImpl { /// Creates a scalar from pgwire "BINARY" format. /// @@ -1092,16 +1097,16 @@ pub fn literal_type_match(data_type: &DataType, literal: Option<&ScalarImpl>) -> match literal { Some(scalar) => { macro_rules! matches { - ($( { $DataType:ident, $PhysicalType:ident }),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty }),*) => { match (data_type, scalar) { $( - (DataType::$DataType { .. }, ScalarImpl::$PhysicalType(_)) => true, - (DataType::$DataType { .. }, _) => false, // so that we won't forget to match a new logical type + (DataType::$data_type { .. }, ScalarImpl::$variant_name(_)) => true, + (DataType::$data_type { .. }, _) => false, // so that we won't forget to match a new logical type )* } } } - for_all_type_pairs! { matches } + for_all_variants! { matches } } None => true, } diff --git a/src/common/src/util/schema_check.rs b/src/common/src/util/schema_check.rs index 850329766577..f035ed713793 100644 --- a/src/common/src/util/schema_check.rs +++ b/src/common/src/util/schema_check.rs @@ -15,7 +15,7 @@ use itertools::Itertools; use crate::array::{ArrayImpl, ArrayRef}; -use crate::for_all_type_pairs; +use crate::for_all_variants; use crate::types::DataType; /// Check if the schema of `columns` matches the expected `data_types`. Used for debugging. @@ -30,9 +30,9 @@ where .enumerate() { macro_rules! matches { - ($( { $DataType:ident, $PhysicalType:ident }),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty }),*) => { match (pair.as_ref().left(), pair.as_ref().right()) { - $( (Some(DataType::$DataType { .. }), Some(ArrayImpl::$PhysicalType(_))) => continue, )* + $( (Some(DataType::$data_type { .. }), Some(ArrayImpl::$variant_name(_))) => continue, )* (data_type, array) => { let array_ident = array.map(|a| a.get_ident()); return Err(format!( @@ -43,7 +43,7 @@ where } } - for_all_type_pairs! { matches } + for_all_variants! { matches } } Ok(()) diff --git a/src/common/src/util/value_encoding/mod.rs b/src/common/src/util/value_encoding/mod.rs index 3b4167331cb7..3fdb8078fdef 100644 --- a/src/common/src/util/value_encoding/mod.rs +++ b/src/common/src/util/value_encoding/mod.rs @@ -360,7 +360,7 @@ fn deserialize_value(ty: &DataType, data: &mut impl Buf) -> Result { DataType::Map(map_type) => { // FIXME: clone type everytime here is inefficient let list = deserialize_list(&map_type.clone().into_struct(), data)?.into_list(); - ScalarImpl::Map(MapValue::from_list_entries(list)) + ScalarImpl::Map(MapValue::from_entries(list)) } }) } diff --git a/src/compute/src/server.rs b/src/compute/src/server.rs index 3270897c29d3..c850f839e620 100644 --- a/src/compute/src/server.rs +++ b/src/compute/src/server.rs @@ -130,8 +130,7 @@ pub async fn compute_node_serve( }, &config.meta, ) - .await - .unwrap(); + .await; let state_store_url = system_params.state_store(); diff --git a/src/connector/codec/src/decoder/avro/mod.rs b/src/connector/codec/src/decoder/avro/mod.rs index 738535ec9410..dc4dae49ca7c 100644 --- a/src/connector/codec/src/decoder/avro/mod.rs +++ b/src/connector/codec/src/decoder/avro/mod.rs @@ -344,7 +344,7 @@ impl<'a> AvroParseOptions<'a> { ); } let list = ListValue::new(builder.finish()); - MapValue::from_list_entries(list).into() + MapValue::from_entries(list).into() } (_expected, _got) => Err(create_error())?, diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index a17c98985de0..540fea13b6c0 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -501,7 +501,7 @@ impl IcebergConfig { .map_err(|e| SinkError::Iceberg(anyhow!(e))) } - async fn create_catalog_v2(&self) -> ConnectorResult> { + fn create_catalog_v2(&self) -> ConnectorResult> { match self.catalog_type() { "storage" => { let config = StorageCatalogConfig::builder() @@ -536,7 +536,7 @@ impl IcebergConfig { })?) .props(iceberg_configs) .build(); - let catalog = iceberg_catalog_rest::RestCatalog::new(config).await?; + let catalog = iceberg_catalog_rest::RestCatalog::new(config); Ok(Arc::new(catalog)) } catalog_type @@ -570,7 +570,6 @@ impl IcebergConfig { pub async fn load_table_v2(&self) -> ConnectorResult { let catalog = self .create_catalog_v2() - .await .context("Unable to load iceberg catalog")?; let table_id = self diff --git a/src/ctl/src/common/meta_service.rs b/src/ctl/src/common/meta_service.rs index ac539b9233ea..6d70bdf94283 100644 --- a/src/ctl/src/common/meta_service.rs +++ b/src/ctl/src/common/meta_service.rs @@ -62,7 +62,7 @@ Note: the default value of `RW_META_ADDR` is 'http://127.0.0.1:5690'."; Property::default(), &MetaConfig::default(), ) - .await?; + .await; let worker_id = client.worker_id(); tracing::info!("registered as RiseCtl worker, worker_id = {}", worker_id); Ok(client) diff --git a/src/expr/core/src/expr/value.rs b/src/expr/core/src/expr/value.rs index c4ec77268488..00aa2bf71ce1 100644 --- a/src/expr/core/src/expr/value.rs +++ b/src/expr/core/src/expr/value.rs @@ -14,7 +14,7 @@ use either::Either; use risingwave_common::array::*; -use risingwave_common::for_all_array_variants; +use risingwave_common::for_all_variants; use risingwave_common::types::{Datum, DatumRef, Scalar, ToDatumRef}; /// The type-erased return value of an expression. @@ -79,7 +79,7 @@ impl<'a, A: Array> ValueRef<'a, A> { } macro_rules! impl_convert { - ($( { $variant_name:ident, $suffix_name:ident, $array:ty, $builder:ty } ),*) => { + ($( { $data_type:ident, $variant_name:ident, $suffix_name:ident, $scalar:ty, $scalar_ref:ty, $array:ty, $builder:ty } ),*) => { $( paste::paste! { /// Converts a type-erased value to a reference of a specific array type. @@ -102,4 +102,4 @@ macro_rules! impl_convert { }; } -for_all_array_variants! { impl_convert } +for_all_variants! { impl_convert } diff --git a/src/expr/impl/src/scalar/array.rs b/src/expr/impl/src/scalar/array.rs index cee7de36c717..d5f53213bf27 100644 --- a/src/expr/impl/src/scalar/array.rs +++ b/src/expr/impl/src/scalar/array.rs @@ -32,8 +32,14 @@ fn row_(row: impl Row) -> StructValue { StructValue::new(row.iter().map(|d| d.to_owned_datum()).collect()) } -fn map_type_infer(args: &[DataType]) -> Result { - let map = MapType::try_from_kv(args[0].as_list().clone(), args[1].as_list().clone())?; +fn map_from_key_values_type_infer(args: &[DataType]) -> Result { + let map = MapType::try_from_kv(args[0].as_list().clone(), args[1].as_list().clone()) + .map_err(ExprError::Custom)?; + Ok(map.into()) +} + +fn map_from_entries_type_infer(args: &[DataType]) -> Result { + let map = MapType::try_from_entries(args[0].as_list().clone()).map_err(ExprError::Custom)?; Ok(map.into()) } @@ -41,62 +47,70 @@ fn map_type_infer(args: &[DataType]) -> Result { /// /// ```slt /// query T -/// select map_from_entries(null::int[], array[1,2,3]); +/// select map_from_key_values(null::int[], array[1,2,3]); /// ---- /// NULL /// /// query T -/// select map_from_entries(array['a','b','c'], array[1,2,3]); +/// select map_from_key_values(array['a','b','c'], array[1,2,3]); /// ---- /// {a:1,b:2,c:3} /// ``` #[function( - "map_from_entries(anyarray, anyarray) -> anymap", - type_infer = "map_type_infer" + "map_from_key_values(anyarray, anyarray) -> anymap", + type_infer = "map_from_key_values_type_infer" )] -fn map_from_entries(key: ListRef<'_>, value: ListRef<'_>) -> Result { +fn map_from_key_values(key: ListRef<'_>, value: ListRef<'_>) -> Result { MapValue::try_from_kv(key.to_owned(), value.to_owned()).map_err(ExprError::Custom) } +#[function( + "map_from_entries(anyarray) -> anymap", + type_infer = "map_from_entries_type_infer" +)] +fn map_from_entries(entries: ListRef<'_>) -> Result { + MapValue::try_from_entries(entries.to_owned()).map_err(ExprError::Custom) +} + /// # Example /// /// ```slt /// query T -/// select map_access(map_from_entries(array[1,2,3], array[100,200,300]), 3); +/// select map_access(map_from_key_values(array[1,2,3], array[100,200,300]), 3); /// ---- /// 300 /// /// query T -/// select map_access(map_from_entries(array[1,2,3], array[100,200,300]), '3'); +/// select map_access(map_from_key_values(array[1,2,3], array[100,200,300]), '3'); /// ---- /// 300 /// /// query error -/// select map_access(map_from_entries(array[1,2,3], array[100,200,300]), 1.0); +/// select map_access(map_from_key_values(array[1,2,3], array[100,200,300]), 1.0); /// ---- /// db error: ERROR: Failed to run the query /// /// Caused by these errors (recent errors listed first): -/// 1: Failed to bind expression: map_access(map_from_entries(ARRAY[1, 2, 3], ARRAY[100, 200, 300]), 1.0) +/// 1: Failed to bind expression: map_access(map_from_key_values(ARRAY[1, 2, 3], ARRAY[100, 200, 300]), 1.0) /// 2: Bind error: Cannot access numeric in map(integer,integer) /// /// /// query T -/// select map_access(map_from_entries(array['a','b','c'], array[1,2,3]), 'a'); +/// select map_access(map_from_key_values(array['a','b','c'], array[1,2,3]), 'a'); /// ---- /// 1 /// /// query T -/// select map_access(map_from_entries(array['a','b','c'], array[1,2,3]), 'd'); +/// select map_access(map_from_key_values(array['a','b','c'], array[1,2,3]), 'd'); /// ---- /// NULL /// /// query T -/// select map_access(map_from_entries(array['a','b','c'], array[1,2,3]), null); +/// select map_access(map_from_key_values(array['a','b','c'], array[1,2,3]), null); /// ---- /// NULL /// ``` -#[function("map_access(anymap, any) -> any")] +#[function("map_access(anymap, any) -> any", type_infer = "unreachable")] fn map_access<'a>( map: MapRef<'a>, key: ScalarRefImpl<'_>, @@ -111,6 +125,122 @@ fn map_access<'a>( } } +/// ```slt +/// query T +/// select +/// map_contains(MAP{1:1}, 1), +/// map_contains(MAP{1:1}, 2), +/// map_contains(MAP{1:1}, NULL::varchar), +/// map_contains(MAP{1:1}, 1.0) +/// ---- +/// t f NULL f +/// ``` +#[function("map_contains(anymap, any) -> boolean")] +fn map_contains(map: MapRef<'_>, key: ScalarRefImpl<'_>) -> Result { + let (keys, _values) = map.into_kv(); + let idx = array_position(keys, Some(key))?; + Ok(idx.is_some()) +} + +/// ```slt +/// query I +/// select +/// map_length(NULL::map(int,int)), +/// map_length(MAP {}::map(int,int)), +/// map_length(MAP {1:1,2:2}::map(int,int)) +/// ---- +/// NULL 0 2 +/// ``` +#[function("map_length(anymap) -> int4")] +fn map_length>(map: MapRef<'_>) -> Result { + map.inner() + .len() + .try_into() + .map_err(|_| ExprError::NumericOverflow) +} + +/// If both `m1` and `m2` have a value with the same key, then the output map contains the value from `m2`. +/// +/// ```slt +/// query T +/// select map_cat(MAP{'a':1,'b':2},null::map(varchar,int)); +/// ---- +/// {a:1,b:2} +/// +/// query T +/// select map_cat(MAP{'a':1,'b':2},MAP{'b':3,'c':4}); +/// ---- +/// {a:1,b:3,c:4} +/// +/// # implicit type cast +/// query T +/// select map_cat(MAP{'a':1,'b':2},MAP{'b':3.0,'c':4.0}); +/// ---- +/// {a:1,b:3.0,c:4.0} +/// ``` +#[function("map_cat(anymap, anymap) -> anymap")] +fn map_cat(m1: Option>, m2: Option>) -> Result, ExprError> { + match (m1, m2) { + (None, None) => Ok(None), + (Some(m), None) | (None, Some(m)) => Ok(Some(m.to_owned())), + (Some(m1), Some(m2)) => Ok(Some(MapValue::concat(m1, m2))), + } +} + +/// Inserts a key-value pair into the map. If the key already exists, the value is updated. +/// +/// # Example +/// +/// ```slt +/// query T +/// select map_insert(map{'a':1, 'b':2}, 'c', 3); +/// ---- +/// {a:1,b:2,c:3} +/// +/// query T +/// select map_insert(map{'a':1, 'b':2}, 'b', 4); +/// ---- +/// {a:1,b:4} +/// ``` +/// +/// TODO: support variadic arguments +#[function("map_insert(anymap, any, any) -> anymap")] +fn map_insert( + map: MapRef<'_>, + key: Option>, + value: Option>, +) -> MapValue { + let Some(key) = key else { + return map.to_owned(); + }; + MapValue::insert(map, key.into_scalar_impl(), value.to_owned_datum()) +} + +/// Deletes a key-value pair from the map. +/// +/// # Example +/// +/// ```slt +/// query T +/// select map_delete(map{'a':1, 'b':2, 'c':3}, 'b'); +/// ---- +/// {a:1,c:3} +/// +/// query T +/// select map_delete(map{'a':1, 'b':2, 'c':3}, 'd'); +/// ---- +/// {a:1,b:2,c:3} +/// ``` +/// +/// TODO: support variadic arguments +#[function("map_delete(anymap, any) -> anymap")] +fn map_delete(map: MapRef<'_>, key: Option>) -> MapValue { + let Some(key) = key else { + return map.to_owned(); + }; + MapValue::delete(map, key) +} + #[cfg(test)] mod tests { use risingwave_common::array::DataChunk; diff --git a/src/expr/impl/src/scalar/cast.rs b/src/expr/impl/src/scalar/cast.rs index e0dd1a8bb3fc..41c51d95445e 100644 --- a/src/expr/impl/src/scalar/cast.rs +++ b/src/expr/impl/src/scalar/cast.rs @@ -249,7 +249,7 @@ fn map_cast(map: MapRef<'_>, ctx: &Context) -> Result { return_type: ctx.return_type.as_map().clone().into_list(), variadic: ctx.variadic, }; - list_cast(map.into_inner(), &new_ctx).map(MapValue::from_list_entries) + list_cast(map.into_inner(), &new_ctx).map(MapValue::from_entries) } #[cfg(test)] diff --git a/src/frontend/planner_test/tests/testdata/input/agg.yaml b/src/frontend/planner_test/tests/testdata/input/agg.yaml index 1979e4ea1fb7..de22c4b5ee91 100644 --- a/src/frontend/planner_test/tests/testdata/input/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/input/agg.yaml @@ -1013,6 +1013,7 @@ SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t; expected_outputs: - logical_plan + - batch_plan - stream_plan - name: test simple approx_percentile with other simple aggs sql: | @@ -1020,6 +1021,7 @@ SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1) from t; expected_outputs: - logical_plan + - batch_plan - stream_plan - name: test simple approx_percentile with other simple aggs (sum, count) sql: | @@ -1027,6 +1029,7 @@ SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1), sum(v1) as s2, count(v1) from t; expected_outputs: - logical_plan + - batch_plan - stream_plan - name: test simple approx_percentile with duplicate approx_percentile sql: | @@ -1041,6 +1044,7 @@ SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; expected_outputs: - logical_plan + - batch_plan - stream_plan - name: test simple approx_percentile with different approx_percentile interleaved with stateless simple aggs sql: | @@ -1048,6 +1052,7 @@ SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, count(*), sum(v2) + approx_percentile(0.9, 0.01) WITHIN GROUP (order by v2) as y from t; expected_outputs: - logical_plan + - batch_plan - stream_plan - name: test simple approx_percentile with duplicated approx_percentile interleaved with stateless simple aggs sql: | @@ -1055,6 +1060,7 @@ SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, count(*), sum(v2) + approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; expected_outputs: - logical_plan + - batch_plan - stream_plan - name: test simple approx_percentile with descending order sql: | @@ -1062,6 +1068,7 @@ SELECT sum(v1) as s1, approx_percentile(0.2, 0.01) WITHIN GROUP (order by v1 desc) from t; expected_outputs: - logical_plan + - batch_plan - stream_plan - name: test simple approx_percentile with different approx_percentile interleaved with stateless + stateful simple aggs sql: | @@ -1069,4 +1076,28 @@ SELECT sum(v1) as s1, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) as x, count(*), max(v2) as m2, approx_percentile(0.5, 0.01) WITHIN GROUP (order by v2) as y from t; expected_outputs: - logical_plan + - batch_plan + - stream_plan +- name: test hash approx_percentile + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t group by v2; + expected_outputs: + - logical_plan + - batch_plan + - stream_plan +- name: test approx_percentile hash_agg forced should use single phase agg + sql: | + SET RW_FORCE_TWO_PHASE_AGG=true; + create table t (v1 int, grp_col int); + select approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t group by grp_col; + expected_outputs: + - stream_error +- name: test approx percentile with default relative_error + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5) WITHIN GROUP (order by v1) from t; + expected_outputs: + - logical_plan + - batch_plan - stream_plan \ No newline at end of file diff --git a/src/frontend/planner_test/tests/testdata/output/agg.yaml b/src/frontend/planner_test/tests/testdata/output/agg.yaml index eca739788bf6..33f8266c7974 100644 --- a/src/frontend/planner_test/tests/testdata/output/agg.yaml +++ b/src/frontend/planner_test/tests/testdata/output/agg.yaml @@ -1888,6 +1888,11 @@ └─LogicalAgg { aggs: [approx_percentile($expr1)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + batch_plan: |- + BatchSimpleAgg { aggs: [approx_percentile($expr1)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1::Float64 as $expr1] } + └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } @@ -1904,6 +1909,11 @@ └─LogicalAgg { aggs: [approx_percentile($expr1), sum(t.v1)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + batch_plan: |- + BatchSimpleAgg { aggs: [approx_percentile($expr1), sum(t.v1)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1::Float64 as $expr1, t.v1] } + └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [approx_percentile, sum], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamRowMerge { output: [approx_percentile:Float64, sum(sum(t.v1)):Int64] } @@ -1928,6 +1938,12 @@ └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count(t.v1)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + batch_plan: |- + BatchProject { exprs: [sum(t.v1), approx_percentile($expr1), sum(t.v1), count(t.v1)] } + └─BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count(t.v1)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1, t.v1::Float64 as $expr1] } + └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, approx_percentile, s2, count], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum(sum(t.v1)), sum0(count(t.v1))] } @@ -1970,6 +1986,11 @@ └─LogicalAgg { aggs: [approx_percentile($expr1), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + batch_plan: |- + BatchSimpleAgg { aggs: [approx_percentile($expr1), approx_percentile($expr2)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1::Float64 as $expr1, t.v2::Float64 as $expr2] } + └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [x, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamRowMerge { output: [approx_percentile:Float64, approx_percentile:Float64] } @@ -1994,6 +2015,12 @@ └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + batch_plan: |- + BatchProject { exprs: [sum(t.v1), approx_percentile($expr1), count, (sum(t.v2)::Float64 + approx_percentile($expr2)) as $expr3] } + └─BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } + └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, x, count, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum0(count), (sum(sum(t.v2))::Float64 + approx_percentile) as $expr3] } @@ -2026,6 +2053,12 @@ └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + batch_plan: |- + BatchProject { exprs: [sum(t.v1), approx_percentile($expr1), count, (sum(t.v2)::Float64 + approx_percentile($expr2)) as $expr3] } + └─BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, sum(t.v2), approx_percentile($expr2)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } + └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, x, count, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamProject { exprs: [sum(sum(t.v1)), approx_percentile, sum0(count), (sum(sum(t.v2))::Float64 + approx_percentile) as $expr3] } @@ -2058,6 +2091,11 @@ └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + batch_plan: |- + BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1, t.v1::Float64 as $expr1] } + └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64] } @@ -2082,6 +2120,11 @@ └─LogicalAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, max(t.v2), approx_percentile($expr2)] } └─LogicalProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + batch_plan: |- + BatchSimpleAgg { aggs: [sum(t.v1), approx_percentile($expr1), count, max(t.v2), approx_percentile($expr2)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1, t.v1::Float64 as $expr1, t.v2, t.v2::Float64 as $expr2] } + └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } stream_plan: |- StreamMaterialize { columns: [s1, x, count, m2, y], stream_key: [], pk_columns: [], pk_conflict: NoCheck } └─StreamRowMerge { output: [sum(sum(t.v1)):Int64, approx_percentile:Float64, sum0(count):Int64, max(max(t.v2)):Int32, approx_percentile:Float64] } @@ -2103,3 +2146,55 @@ └─StreamHashAgg { group_key: [$expr5], aggs: [sum(t.v1), count, max(t.v2)] } └─StreamProject { exprs: [t.v1, t.v1::Float64 as $expr3, t.v2, t.v2::Float64 as $expr4, t._row_id, Vnode(t._row_id) as $expr5] } └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test hash approx_percentile + sql: | + CREATE TABLE t (v1 int, v2 int); + SELECT approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t group by v2; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1)] } + └─LogicalAgg { group_key: [t.v2], aggs: [approx_percentile($expr1)] } + └─LogicalProject { exprs: [t.v2, t.v1::Float64 as $expr1] } + └─LogicalScan { table: t, columns: [t.v1, t.v2, t._row_id] } + batch_plan: |- + BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [approx_percentile($expr1)] } + └─BatchHashAgg { group_key: [t.v2], aggs: [approx_percentile($expr1)] } + └─BatchExchange { order: [], dist: HashShard(t.v2) } + └─BatchProject { exprs: [t.v2, t.v1::Float64 as $expr1] } + └─BatchScan { table: t, columns: [t.v1, t.v2], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [approx_percentile, t.v2(hidden)], stream_key: [t.v2], pk_columns: [t.v2], pk_conflict: NoCheck } + └─StreamProject { exprs: [approx_percentile($expr1), t.v2] } + └─StreamHashAgg { group_key: [t.v2], aggs: [approx_percentile($expr1), count] } + └─StreamExchange { dist: HashShard(t.v2) } + └─StreamProject { exprs: [t.v2, t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t.v2, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } +- name: test approx_percentile hash_agg forced should use single phase agg + sql: | + SET RW_FORCE_TWO_PHASE_AGG=true; + create table t (v1 int, grp_col int); + select approx_percentile(0.5, 0.01) WITHIN GROUP (order by v1) from t group by grp_col; + stream_error: |- + Feature is not yet implemented: two-phase streaming approx percentile aggregation with group key, please use single phase aggregation instead + No tracking issue yet. Feel free to submit a feature request at https://github.com/risingwavelabs/risingwave/issues/new?labels=type%2Ffeature&template=feature_request.yml +- name: test approx percentile with default relative_error + sql: | + CREATE TABLE t (v1 int); + SELECT approx_percentile(0.5) WITHIN GROUP (order by v1) from t; + logical_plan: |- + LogicalProject { exprs: [approx_percentile($expr1)] } + └─LogicalAgg { aggs: [approx_percentile($expr1)] } + └─LogicalProject { exprs: [t.v1::Float64 as $expr1] } + └─LogicalScan { table: t, columns: [t.v1, t._row_id] } + batch_plan: |- + BatchSimpleAgg { aggs: [approx_percentile($expr1)] } + └─BatchExchange { order: [], dist: Single } + └─BatchProject { exprs: [t.v1::Float64 as $expr1] } + └─BatchScan { table: t, columns: [t.v1], distribution: SomeShard } + stream_plan: |- + StreamMaterialize { columns: [approx_percentile], stream_key: [], pk_columns: [], pk_conflict: NoCheck } + └─StreamGlobalApproxPercentile { quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamExchange { dist: Single } + └─StreamLocalApproxPercentile { percentile_col: $expr1, quantile: 0.5:Float64, relative_error: 0.01:Float64 } + └─StreamProject { exprs: [t.v1::Float64 as $expr1, t._row_id] } + └─StreamTableScan { table: t, columns: [t.v1, t._row_id], stream_scan_type: ArrangementBackfill, stream_key: [t._row_id], pk: [_row_id], dist: UpstreamHashShard(t._row_id) } diff --git a/src/frontend/src/binder/expr/function/aggregate.rs b/src/frontend/src/binder/expr/function/aggregate.rs index 1e7b76bf7629..d6410616c1d9 100644 --- a/src/frontend/src/binder/expr/function/aggregate.rs +++ b/src/frontend/src/binder/expr/function/aggregate.rs @@ -14,7 +14,7 @@ use itertools::Itertools; use risingwave_common::bail_not_implemented; -use risingwave_common::types::DataType; +use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_expr::aggregate::{agg_kinds, AggKind, PbAggKind}; use risingwave_sqlparser::ast::{Function, FunctionArgExpr}; @@ -139,12 +139,9 @@ impl Binder { let order_by = OrderBy::new(vec![self.bind_order_by_expr(within_group)?]); // check signature and do implicit cast - match (&kind, direct_args.as_mut_slice(), args.as_mut_slice()) { - ( - AggKind::Builtin(PbAggKind::PercentileCont | PbAggKind::PercentileDisc), - [fraction], - [arg], - ) => { + match (&kind, direct_args.len(), args.as_mut_slice()) { + (AggKind::Builtin(PbAggKind::PercentileCont | PbAggKind::PercentileDisc), 1, [arg]) => { + let fraction = &mut direct_args[0]; decimal_to_float64(fraction, &kind)?; if matches!(&kind, AggKind::Builtin(PbAggKind::PercentileCont)) { arg.cast_implicit_mut(DataType::Float64).map_err(|_| { @@ -155,14 +152,30 @@ impl Binder { })?; } } - (AggKind::Builtin(PbAggKind::Mode), [], [_arg]) => {} - ( - AggKind::Builtin(PbAggKind::ApproxPercentile), - [percentile, relative_error], - [_percentile_col], - ) => { + (AggKind::Builtin(PbAggKind::Mode), 0, [_arg]) => {} + (AggKind::Builtin(PbAggKind::ApproxPercentile), 1..=2, [_percentile_col]) => { + let percentile = &mut direct_args[0]; decimal_to_float64(percentile, &kind)?; - decimal_to_float64(relative_error, &kind)?; + match direct_args.len() { + 2 => { + let relative_error = &mut direct_args[1]; + decimal_to_float64(relative_error, &kind)?; + } + 1 => { + let relative_error: ExprImpl = Literal::new( + ScalarImpl::Float64(0.01.into()).into(), + DataType::Float64, + ) + .into(); + direct_args.push(relative_error); + } + _ => { + return Err(ErrorCode::InvalidInputSyntax( + "invalid direct args for approx_percentile aggregation".to_string(), + ) + .into()) + } + } } _ => { return Err(ErrorCode::InvalidInputSyntax(format!( diff --git a/src/frontend/src/binder/expr/function/builtin_scalar.rs b/src/frontend/src/binder/expr/function/builtin_scalar.rs index 824f08cf36b7..73eb722b2601 100644 --- a/src/frontend/src/binder/expr/function/builtin_scalar.rs +++ b/src/frontend/src/binder/expr/function/builtin_scalar.rs @@ -402,6 +402,15 @@ impl Binder { // map ("map_from_entries", raw_call(ExprType::MapFromEntries)), ("map_access",raw_call(ExprType::MapAccess)), + ("map_keys", raw_call(ExprType::MapKeys)), + ("map_values", raw_call(ExprType::MapValues)), + ("map_entries", raw_call(ExprType::MapEntries)), + ("map_from_key_values", raw_call(ExprType::MapFromKeyValues)), + ("map_cat", raw_call(ExprType::MapCat)), + ("map_contains", raw_call(ExprType::MapContains)), + ("map_delete", raw_call(ExprType::MapDelete)), + ("map_insert", raw_call(ExprType::MapInsert)), + ("map_length", raw_call(ExprType::MapLength)), // Functions that return a constant value ("pi", pi()), // greatest and least diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 3c127c7da7c4..85ed93c7dc0c 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -1017,7 +1017,7 @@ pub fn bind_data_type(data_type: &AstDataType) -> Result { AstDataType::Map(kv) => { let key = bind_data_type(&kv.0)?; let value = bind_data_type(&kv.1)?; - DataType::Map(MapType::try_from_kv(key, value)?) + DataType::Map(MapType::try_from_kv(key, value).map_err(ErrorCode::BindError)?) } AstDataType::Custom(qualified_type_name) => { let idents = qualified_type_name diff --git a/src/frontend/src/binder/expr/value.rs b/src/frontend/src/binder/expr/value.rs index 711aa6bbb697..961306408a43 100644 --- a/src/frontend/src/binder/expr/value.rs +++ b/src/frontend/src/binder/expr/value.rs @@ -159,7 +159,7 @@ impl Binder { .into(); let expr: ExprImpl = FunctionCall::new_unchecked( - ExprType::MapFromEntries, + ExprType::MapFromKeyValues, vec![keys, values], DataType::Map(MapType::from_kv(key_type, value_type)), ) @@ -209,7 +209,7 @@ impl Binder { .into(); let expr: ExprImpl = FunctionCall::new_unchecked( - ExprType::MapFromEntries, + ExprType::MapFromKeyValues, vec![keys, values], DataType::Map(map_type), ) diff --git a/src/frontend/src/expr/mod.rs b/src/frontend/src/expr/mod.rs index 73becd7bc86c..f650fa3cb521 100644 --- a/src/frontend/src/expr/mod.rs +++ b/src/frontend/src/expr/mod.rs @@ -17,7 +17,7 @@ use fixedbitset::FixedBitSet; use futures::FutureExt; use paste::paste; use risingwave_common::array::ListValue; -use risingwave_common::types::{DataType, Datum, JsonbVal, Scalar, ScalarImpl}; +use risingwave_common::types::{DataType, Datum, JsonbVal, MapType, Scalar, ScalarImpl}; use risingwave_expr::aggregate::PbAggKind; use risingwave_expr::expr::build_from_prost; use risingwave_pb::expr::expr_node::RexNode; @@ -324,6 +324,19 @@ impl ExprImpl { } } + /// Ensure the return type of this expression is a map of some type. + pub fn try_into_map_type(&self) -> Result { + if self.is_untyped() { + return Err(ErrorCode::BindError( + "could not determine polymorphic type because input has type unknown".into(), + )); + } + match self.return_type() { + DataType::Map(m) => Ok(m), + t => Err(ErrorCode::BindError(format!("expects map but got {t}"))), + } + } + /// Shorthand to enforce implicit cast to boolean pub fn enforce_bool_clause(self, clause: &str) -> RwResult { if self.is_untyped() { diff --git a/src/frontend/src/expr/pure.rs b/src/frontend/src/expr/pure.rs index 59f087672417..3e6c83d8330f 100644 --- a/src/frontend/src/expr/pure.rs +++ b/src/frontend/src/expr/pure.rs @@ -251,7 +251,16 @@ impl ExprVisitor for ImpureAnalyzer { | Type::QuoteLiteral | Type::QuoteNullable | Type::MapFromEntries - | Type::MapAccess => + | Type::MapAccess + | Type::MapKeys + | Type::MapValues + | Type::MapEntries + | Type::MapFromKeyValues + | Type::MapCat + | Type::MapContains + | Type::MapDelete + | Type::MapInsert + | Type::MapLength => // expression output is deterministic(same result for the same input) { func_call diff --git a/src/frontend/src/expr/type_inference/func.rs b/src/frontend/src/expr/type_inference/func.rs index 746460e2b636..9ed753049992 100644 --- a/src/frontend/src/expr/type_inference/func.rs +++ b/src/frontend/src/expr/type_inference/func.rs @@ -613,7 +613,7 @@ fn infer_type_for_special( } ExprType::MapAccess => { ensure_arity!("map_access", | inputs | == 2); - let map_type = inputs[0].return_type().into_map(); + let map_type = inputs[0].try_into_map_type()?; // We do not align the map's key type with the input type here, but cast the latter to the former instead. // e.g., for {1:'a'}[1.0], if we align them, we will get "numeric" as the key type, which violates the map type's restriction. match inputs[1].cast_implicit_mut(map_type.key().clone()) { @@ -626,6 +626,40 @@ fn infer_type_for_special( .into()), } } + ExprType::MapCat => { + ensure_arity!("map_contains", | inputs | == 2); + Ok(Some(align_types(inputs.iter_mut())?)) + } + ExprType::MapInsert => { + ensure_arity!("map_insert", | inputs | == 3); + let map_type = inputs[0].try_into_map_type()?; + let rk = inputs[1].cast_implicit_mut(map_type.key().clone()); + let rv = inputs[2].cast_implicit_mut(map_type.value().clone()); + match (rk, rv) { + (Ok(()), Ok(())) => Ok(Some(map_type.into())), + _ => Err(ErrorCode::BindError(format!( + "Cannot insert ({},{}) to {}", + inputs[1].return_type(), + inputs[2].return_type(), + inputs[0].return_type(), + )) + .into()), + } + } + ExprType::MapDelete => { + ensure_arity!("map_delete", | inputs | == 2); + let map_type = inputs[0].try_into_map_type()?; + let rk = inputs[1].cast_implicit_mut(map_type.key().clone()); + match rk { + Ok(()) => Ok(Some(map_type.into())), + _ => Err(ErrorCode::BindError(format!( + "Cannot delete {} from {}", + inputs[1].return_type(), + inputs[0].return_type(), + )) + .into()), + } + } ExprType::Vnode => { ensure_arity!("vnode", 1 <= | inputs |); Ok(Some(VirtualNode::RW_TYPE)) diff --git a/src/frontend/src/handler/alter_parallelism.rs b/src/frontend/src/handler/alter_parallelism.rs index e5d2d3303789..3c6ab52f51e3 100644 --- a/src/frontend/src/handler/alter_parallelism.rs +++ b/src/frontend/src/handler/alter_parallelism.rs @@ -14,8 +14,9 @@ use pgwire::pg_response::StatementType; use risingwave_common::bail; +use risingwave_common::hash::VirtualNode; use risingwave_pb::meta::table_parallelism::{ - AdaptiveParallelism, FixedParallelism, PbParallelism, + AdaptiveParallelism, FixedParallelism, Parallelism, PbParallelism, }; use risingwave_pb::meta::{PbTableParallelism, TableParallelism}; use risingwave_sqlparser::ast::{ObjectName, SetVariableValue, SetVariableValueSingle, Value}; @@ -92,15 +93,43 @@ pub async fn handle_alter_parallelism( } }; - let target_parallelism = extract_table_parallelism(parallelism)?; + let mut target_parallelism = extract_table_parallelism(parallelism)?; + + let available_parallelism = session + .env() + .worker_node_manager() + .list_worker_nodes() + .iter() + .filter(|w| w.is_streaming_schedulable()) + .map(|w| w.parallelism) + .sum::(); + + let mut builder = RwPgResponse::builder(stmt_type); + + match &target_parallelism.parallelism { + Some(Parallelism::Adaptive(_)) | Some(Parallelism::Auto(_)) => { + if available_parallelism > VirtualNode::COUNT as u32 { + builder = builder.notice(format!("Available parallelism exceeds the maximum parallelism limit, the actual parallelism will be limited to {}", VirtualNode::COUNT)); + } + } + Some(Parallelism::Fixed(FixedParallelism { parallelism })) => { + if *parallelism > VirtualNode::COUNT as u32 { + builder = builder.notice(format!("Provided parallelism exceeds the maximum parallelism limit, resetting to FIXED({})", VirtualNode::COUNT)); + target_parallelism = PbTableParallelism { + parallelism: Some(PbParallelism::Fixed(FixedParallelism { + parallelism: VirtualNode::COUNT as u32, + })), + }; + } + } + _ => {} + }; let catalog_writer = session.catalog_writer()?; catalog_writer .alter_parallelism(table_id, target_parallelism, deferred) .await?; - let mut builder = RwPgResponse::builder(stmt_type); - if deferred { builder = builder.notice("DEFERRED is used, please ensure that automatic parallelism control is enabled on the meta, otherwise, the alter will not take effect.".to_string()); } diff --git a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs index 5e9a3ce05392..2c14fc730877 100644 --- a/src/frontend/src/optimizer/plan_expr_visitor/strong.rs +++ b/src/frontend/src/optimizer/plan_expr_visitor/strong.rs @@ -293,6 +293,15 @@ impl Strong { | ExprType::JsonbSet | ExprType::MapFromEntries | ExprType::MapAccess + | ExprType::MapKeys + | ExprType::MapValues + | ExprType::MapEntries + | ExprType::MapFromKeyValues + | ExprType::MapCat + | ExprType::MapContains + | ExprType::MapDelete + | ExprType::MapInsert + | ExprType::MapLength | ExprType::Vnode | ExprType::TestPaidTier | ExprType::Proctime diff --git a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs index ff20df7a4d17..894ad9201100 100644 --- a/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs +++ b/src/frontend/src/optimizer/plan_node/batch_simple_agg.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_expr::aggregate::{AggKind, PbAggKind}; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::batch_plan::SortAggNode; @@ -51,7 +52,15 @@ impl BatchSimpleAgg { } pub(crate) fn can_two_phase_agg(&self) -> bool { - self.core.can_two_phase_agg() && self.two_phase_agg_enabled() + self.core.can_two_phase_agg() + && self + .core + // Ban two phase approx percentile. + .agg_calls + .iter() + .map(|agg_call| &agg_call.agg_kind) + .all(|agg_kind| !matches!(agg_kind, AggKind::Builtin(PbAggKind::ApproxPercentile))) + && self.two_phase_agg_enabled() } } diff --git a/src/frontend/src/optimizer/plan_node/logical_agg.rs b/src/frontend/src/optimizer/plan_node/logical_agg.rs index e63b7d760a68..cf7025be9087 100644 --- a/src/frontend/src/optimizer/plan_node/logical_agg.rs +++ b/src/frontend/src/optimizer/plan_node/logical_agg.rs @@ -311,7 +311,10 @@ impl LogicalAgg { col_mapping: approx_percentile_col_mapping, } = approx; if !self.group_key().is_empty() && !approx_percentile_agg_calls.is_empty() { - bail_not_implemented!("two-phase approx percentile agg with group key, please use single phase agg for approx_percentile with group key"); + bail_not_implemented!( + "two-phase streaming approx percentile aggregation with group key, \ + please use single phase aggregation instead" + ); } // Either we have approx percentile aggs and non_approx percentile aggs, diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index b970899ef080..685000dbbe65 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -267,7 +267,7 @@ impl FrontendEnv { Default::default(), &config.meta, ) - .await?; + .await; let worker_id = meta_client.worker_id(); info!("Assigned worker node id {}", worker_id); diff --git a/src/license/src/cpu.rs b/src/license/src/cpu.rs new file mode 100644 index 000000000000..b2c23733ce75 --- /dev/null +++ b/src/license/src/cpu.rs @@ -0,0 +1,99 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::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)] +mod tests { + use expect_test::expect; + use thiserror_ext::AsReport as _; + + use super::*; + use crate::{LicenseKey, TEST_PAID_LICENSE_KEY_CONTENT}; + + fn do_test(key: &str, cpu_core_count: u64, expect: expect_test::Expect) { + let manager = LicenseManager::new(); + manager.refresh(LicenseKey(key)); + + match manager.check_cpu_core_limit(cpu_core_count) { + Ok(_) => expect.assert_eq("ok"), + Err(error) => expect.assert_eq(&error.to_report_string()), + } + } + + #[test] + fn test_no_limit() { + 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"]); + } + + #[test] + fn test_limit() { + const KEY: &str = + "eyJhbGciOiJSUzUxMiIsInR5cCI6IkpXVCJ9.\ + eyJzdWIiOiJmcmVlLXRlc3QtMzIiLCJpc3MiOiJwcm9kLnJpc2luZ3dhdmUuY29tIiwidGllciI6ImZyZWUiLCJleHAiOjE4NTI1NTk5OTksImlhdCI6MTcyMzcwMTk5NCwiY3B1X2NvcmVfbGltaXQiOjMyfQ.\ + rsATtzlduLUkGQeXkOROtyGUpafdDhi18iKdYAzAldWQuO9KevNcnD8a6geCShZSGte65bI7oYtv7GHx8i66ge3B1SVsgGgYr10ebphPUNUQenYoN0mpD4Wn0prPStOgANzYZOI2ntMGAaeWStji1x67_iho6r0W9r6RX3kMvzFSbiObSIfvTdrMULeg-xeHc3bT_ErRhaXq7MAa2Oiq3lcK2sNgEvc9KYSP9YbhSik9CBkc8lcyeVoc48SSWEaBU-c8-Ge0fzjgWHI9KIsUV5Ihe66KEfs0PqdRoSWbgskYGzA3o8wHIbtJbJiPzra373kkFH9MGY0HOsw9QeJLGQ"; + + 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"]); + } +} diff --git a/src/license/src/lib.rs b/src/license/src/lib.rs index bdcac9044104..e2a327578009 100644 --- a/src/license/src/lib.rs +++ b/src/license/src/lib.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +mod cpu; mod feature; mod key; mod manager; diff --git a/src/license/src/manager.rs b/src/license/src/manager.rs index cac51105358a..5c1bc298388d 100644 --- a/src/license/src/manager.rs +++ b/src/license/src/manager.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::num::NonZeroU64; use std::sync::{LazyLock, RwLock}; use jsonwebtoken::{Algorithm, DecodingKey, Validation}; @@ -76,6 +77,9 @@ pub(super) struct License { /// Tier of the 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, + /// Expiration time in seconds since UNIX epoch. /// /// See . @@ -91,6 +95,7 @@ impl Default for License { sub: "default".to_owned(), tier: Tier::Free, iss: Issuer::Prod, + cpu_core_limit: None, exp: u64::MAX, } } @@ -117,7 +122,7 @@ static PUBLIC_KEY: LazyLock = LazyLock::new(|| { impl LicenseManager { /// Create a new license manager with the default license. - fn new() -> Self { + pub(crate) fn new() -> Self { Self { inner: RwLock::new(Inner { license: Ok(License::default()), @@ -208,6 +213,7 @@ mod tests { sub: "rw-test", iss: Test, tier: Paid, + cpu_core_limit: None, exp: 9999999999, } "#]], @@ -228,6 +234,7 @@ mod tests { sub: "rw-test", iss: Test, tier: Free, + cpu_core_limit: None, exp: 9999999999, } "#]], @@ -244,6 +251,7 @@ mod tests { sub: "default", iss: Prod, tier: Free, + cpu_core_limit: None, exp: 18446744073709551615, } "#]], diff --git a/src/meta/model_v2/src/actor_dispatcher.rs b/src/meta/model_v2/src/actor_dispatcher.rs index 81211cc57270..7d40af6967d3 100644 --- a/src/meta/model_v2/src/actor_dispatcher.rs +++ b/src/meta/model_v2/src/actor_dispatcher.rs @@ -12,13 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::hash::Hash; + use risingwave_pb::stream_plan::{PbDispatcher, PbDispatcherType}; use sea_orm::entity::prelude::*; use serde::{Deserialize, Serialize}; use crate::{ActorId, ActorMapping, FragmentId, I32Array}; -#[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Deserialize, Serialize)] +#[derive( + Hash, Copy, Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Serialize, Deserialize, +)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum DispatcherType { #[sea_orm(string_value = "HASH")] diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index 23f97e777851..5a386f1ecac3 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -91,7 +91,7 @@ pub type HummockSstableObjectId = i64; pub type FragmentId = i32; pub type ActorId = i32; -#[derive(Clone, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Serialize, Deserialize)] +#[derive(Clone, Copy, Debug, PartialEq, Eq, EnumIter, DeriveActiveEnum, Serialize, Deserialize)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum JobStatus { #[sea_orm(string_value = "INITIAL")] diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 825d40b5172d..049519372c81 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -296,7 +296,6 @@ pub fn start( ), }, }; - validate_config(&config); let total_memory_bytes = resource_util::memory::system_memory_available_bytes(); diff --git a/src/meta/service/src/cluster_service.rs b/src/meta/service/src/cluster_service.rs index 39cd40ed3740..6f1bfd68e5b3 100644 --- a/src/meta/service/src/cluster_service.rs +++ b/src/meta/service/src/cluster_service.rs @@ -25,7 +25,6 @@ use risingwave_pb::meta::{ ListAllNodesResponse, UpdateWorkerNodeSchedulabilityRequest, UpdateWorkerNodeSchedulabilityResponse, }; -use thiserror_ext::AsReport; use tonic::{Request, Response, Status}; use crate::MetaError; @@ -58,31 +57,16 @@ impl ClusterService for ClusterServiceImpl { .property .ok_or_else(|| MetaError::invalid_parameter("worker node property is not provided"))?; let resource = req.resource.unwrap_or_default(); - let result = self + let worker_id = self .metadata_manager .add_worker_node(worker_type, host, property, resource) - .await; + .await?; let cluster_id = self.metadata_manager.cluster_id().to_string(); - match result { - Ok(worker_id) => Ok(Response::new(AddWorkerNodeResponse { - status: None, - node_id: Some(worker_id), - cluster_id, - })), - Err(e) => { - if e.is_invalid_worker() { - return Ok(Response::new(AddWorkerNodeResponse { - status: Some(risingwave_pb::common::Status { - code: risingwave_pb::common::status::Code::UnknownWorker as i32, - message: e.to_report_string(), - }), - node_id: None, - cluster_id, - })); - } - Err(e.into()) - } - } + + Ok(Response::new(AddWorkerNodeResponse { + node_id: Some(worker_id), + cluster_id, + })) } /// Update schedulability of a compute node. Will not affect actors which are already running on diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 7d5c954fde73..e349fadf4837 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -309,7 +309,7 @@ impl CheckpointControl { ); } - debug!( + tracing::trace!( prev_epoch = command_ctx.prev_epoch.value().0, ?jobs_to_wait, "enqueue command" @@ -354,7 +354,7 @@ impl CheckpointControl { fn barrier_collected(&mut self, resp: BarrierCompleteResponse) { let worker_id = resp.worker_id; let prev_epoch = resp.epoch; - debug!( + tracing::trace!( worker_id, prev_epoch, partial_graph_id = resp.partial_graph_id, @@ -1005,7 +1005,7 @@ impl GlobalBarrierManager { BarrierKind::Barrier }; - debug!(prev_epoch = prev_epoch.value().0, "inject barrier"); + tracing::trace!(prev_epoch = prev_epoch.value().0, "inject barrier"); // Collect the jobs to finish if let (BarrierKind::Checkpoint(_), Command::Plain(None)) = (&kind, &command) @@ -1206,7 +1206,7 @@ impl GlobalBarrierManagerContext { mut finished_jobs: Vec, backfill_pinned_log_epoch: HashMap)>, ) -> MetaResult> { - debug!( + tracing::trace!( prev_epoch = node.command_ctx.prev_epoch.value().0, kind = ?node.command_ctx.kind, "complete barrier" diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 6a7ca826f160..10d4a947e2a9 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -26,6 +26,7 @@ use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::HummockSstableObjectId; +use risingwave_license::LicenseManager; use risingwave_meta_model_v2::prelude::{Worker, WorkerProperty}; use risingwave_meta_model_v2::worker::{WorkerStatus, WorkerType}; use risingwave_meta_model_v2::{worker, worker_property, TransactionId, WorkerId}; @@ -38,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, EntityTrait, QueryFilter, QuerySelect, - TransactionTrait, + ActiveModelTrait, ColumnTrait, DatabaseConnection, DatabaseTransaction, EntityTrait, + QueryFilter, QuerySelect, TransactionTrait, }; use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver}; @@ -581,6 +582,43 @@ 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 as i32)), + ) + .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(()) + } + pub async fn add_worker( &mut self, r#type: PbWorkerType, @@ -591,6 +629,11 @@ 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 diff --git a/src/meta/src/controller/mod.rs b/src/meta/src/controller/mod.rs index 43078ea81272..6b56113aa38a 100644 --- a/src/meta/src/controller/mod.rs +++ b/src/meta/src/controller/mod.rs @@ -38,6 +38,7 @@ pub mod cluster; pub mod fragment; pub mod id; pub mod rename; +pub mod scale; pub mod session_params; pub mod streaming_job; pub mod system_param; diff --git a/src/meta/src/controller/scale.rs b/src/meta/src/controller/scale.rs new file mode 100644 index 000000000000..658ebaef47f6 --- /dev/null +++ b/src/meta/src/controller/scale.rs @@ -0,0 +1,345 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::{HashMap, HashSet}; + +use risingwave_meta_model_migration::{ + Alias, CommonTableExpression, Expr, IntoColumnRef, QueryStatementBuilder, SelectStatement, + UnionType, WithClause, WithQuery, +}; +use risingwave_meta_model_v2::actor_dispatcher::DispatcherType; +use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, StreamingJob}; +use risingwave_meta_model_v2::{ + actor, actor_dispatcher, fragment, streaming_job, ActorId, FragmentId, ObjectId, +}; +use sea_orm::{ + ColumnTrait, ConnectionTrait, DbErr, EntityTrait, JoinType, QueryFilter, QuerySelect, + RelationTrait, Statement, TransactionTrait, +}; + +use crate::controller::catalog::CatalogController; +use crate::{MetaError, MetaResult}; + +/// This function will construct a query using recursive cte to find `no_shuffle` upstream relation graph for target fragments. +/// +/// # Examples +/// +/// ``` +/// use risingwave_meta::controller::scale::construct_no_shuffle_upstream_traverse_query; +/// use sea_orm::sea_query::*; +/// use sea_orm::*; +/// +/// let query = construct_no_shuffle_upstream_traverse_query(vec![2, 3]); +/// +/// assert_eq!(query.to_string(MysqlQueryBuilder), r#"WITH RECURSIVE `shuffle_deps` (`fragment_id`, `dispatcher_type`, `dispatcher_id`) AS (SELECT DISTINCT `actor`.`fragment_id`, `actor_dispatcher`.`dispatcher_type`, `actor_dispatcher`.`dispatcher_id` FROM `actor` INNER JOIN `actor_dispatcher` ON `actor`.`actor_id` = `actor_dispatcher`.`actor_id` WHERE `actor_dispatcher`.`dispatcher_type` = 'NO_SHUFFLE' AND `actor_dispatcher`.`dispatcher_id` IN (2, 3) UNION ALL (SELECT DISTINCT `actor`.`fragment_id`, `actor_dispatcher`.`dispatcher_type`, `actor_dispatcher`.`dispatcher_id` FROM `actor` INNER JOIN `actor_dispatcher` ON `actor`.`actor_id` = `actor_dispatcher`.`actor_id` INNER JOIN `shuffle_deps` ON `shuffle_deps`.`fragment_id` = `actor_dispatcher`.`dispatcher_id` WHERE `actor_dispatcher`.`dispatcher_type` = 'NO_SHUFFLE')) SELECT DISTINCT `fragment_id`, `dispatcher_type`, `dispatcher_id` FROM `shuffle_deps`"#); +/// assert_eq!(query.to_string(PostgresQueryBuilder), r#"WITH RECURSIVE "shuffle_deps" ("fragment_id", "dispatcher_type", "dispatcher_id") AS (SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE' AND "actor_dispatcher"."dispatcher_id" IN (2, 3) UNION ALL (SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" INNER JOIN "shuffle_deps" ON "shuffle_deps"."fragment_id" = "actor_dispatcher"."dispatcher_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE')) SELECT DISTINCT "fragment_id", "dispatcher_type", "dispatcher_id" FROM "shuffle_deps""#); +/// assert_eq!(query.to_string(SqliteQueryBuilder), r#"WITH RECURSIVE "shuffle_deps" ("fragment_id", "dispatcher_type", "dispatcher_id") AS (SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE' AND "actor_dispatcher"."dispatcher_id" IN (2, 3) UNION ALL SELECT DISTINCT "actor"."fragment_id", "actor_dispatcher"."dispatcher_type", "actor_dispatcher"."dispatcher_id" FROM "actor" INNER JOIN "actor_dispatcher" ON "actor"."actor_id" = "actor_dispatcher"."actor_id" INNER JOIN "shuffle_deps" ON "shuffle_deps"."fragment_id" = "actor_dispatcher"."dispatcher_id" WHERE "actor_dispatcher"."dispatcher_type" = 'NO_SHUFFLE') SELECT DISTINCT "fragment_id", "dispatcher_type", "dispatcher_id" FROM "shuffle_deps""#); +/// ``` +pub fn construct_no_shuffle_upstream_traverse_query(fragment_ids: Vec) -> WithQuery { + construct_no_shuffle_traverse_query_helper(fragment_ids, NoShuffleResolveDirection::Upstream) +} + +pub fn construct_no_shuffle_downstream_traverse_query(fragment_ids: Vec) -> WithQuery { + construct_no_shuffle_traverse_query_helper(fragment_ids, NoShuffleResolveDirection::Downstream) +} + +enum NoShuffleResolveDirection { + Upstream, + Downstream, +} + +fn construct_no_shuffle_traverse_query_helper( + fragment_ids: Vec, + direction: NoShuffleResolveDirection, +) -> WithQuery { + let cte_alias = Alias::new("shuffle_deps"); + + // If we need to look upwards + // resolve by fragment_id -> dispatcher_id + // and if downwards + // resolve by dispatcher_id -> fragment_id + let (cte_ref_column, compared_column) = match direction { + NoShuffleResolveDirection::Upstream => ( + (cte_alias.clone(), actor::Column::FragmentId).into_column_ref(), + (ActorDispatcher, actor_dispatcher::Column::DispatcherId).into_column_ref(), + ), + NoShuffleResolveDirection::Downstream => ( + (cte_alias.clone(), actor_dispatcher::Column::DispatcherId).into_column_ref(), + (Actor, actor::Column::FragmentId).into_column_ref(), + ), + }; + + let mut base_query = SelectStatement::new() + .column((Actor, actor::Column::FragmentId)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherId)) + .distinct() + .from(Actor) + .inner_join( + ActorDispatcher, + Expr::col((Actor, actor::Column::ActorId)).eq(Expr::col(( + ActorDispatcher, + actor_dispatcher::Column::ActorId, + ))), + ) + .and_where( + Expr::col((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .eq(DispatcherType::NoShuffle), + ) + .and_where(Expr::col(compared_column.clone()).is_in(fragment_ids.clone())) + .to_owned(); + + let cte_referencing = SelectStatement::new() + .column((Actor, actor::Column::FragmentId)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .column((ActorDispatcher, actor_dispatcher::Column::DispatcherId)) + .distinct() + .from(Actor) + .inner_join( + ActorDispatcher, + Expr::col((Actor, actor::Column::ActorId)).eq(Expr::col(( + ActorDispatcher, + actor_dispatcher::Column::ActorId, + ))), + ) + .inner_join( + cte_alias.clone(), + Expr::col(cte_ref_column).eq(Expr::col(compared_column)), + ) + .and_where( + Expr::col((ActorDispatcher, actor_dispatcher::Column::DispatcherType)) + .eq(DispatcherType::NoShuffle), + ) + .to_owned(); + + let common_table_expr = CommonTableExpression::new() + .query(base_query.union(UnionType::All, cte_referencing).to_owned()) + .column(actor::Column::FragmentId) + .column(actor_dispatcher::Column::DispatcherType) + .column(actor_dispatcher::Column::DispatcherId) + .table_name(cte_alias.clone()) + .to_owned(); + + SelectStatement::new() + .column(actor::Column::FragmentId) + .column(actor_dispatcher::Column::DispatcherType) + .column(actor_dispatcher::Column::DispatcherId) + .distinct() + .from(cte_alias.clone()) + .to_owned() + .with( + WithClause::new() + .recursive(true) + .cte(common_table_expr) + .to_owned(), + ) + .to_owned() +} + +#[derive(Debug, Clone)] +pub struct RescheduleWorkingSet { + pub fragments: HashMap, + pub actors: HashMap, + pub actor_dispatchers: HashMap>, + + pub fragment_downstreams: HashMap>, + pub fragment_upstreams: HashMap>, + + pub related_jobs: HashMap, +} + +async fn resolve_no_shuffle_query( + txn: &C, + query: WithQuery, +) -> MetaResult> +where + C: ConnectionTrait, +{ + let (sql, values) = query.build_any(&*txn.get_database_backend().get_query_builder()); + + let result = txn + .query_all(Statement::from_sql_and_values( + txn.get_database_backend(), + sql, + values, + )) + .await? + .into_iter() + .map(|res| res.try_get_many_by_index()) + .collect::, DbErr>>() + .map_err(MetaError::from)?; + + Ok(result) +} + +impl CatalogController { + pub async fn resolve_working_set_for_reschedule_fragments( + &self, + fragment_ids: Vec, + ) -> MetaResult { + let inner = self.inner.read().await; + self.resolve_working_set_for_reschedule_helper(&inner.db, fragment_ids) + .await + } + + pub async fn resolve_working_set_for_reschedule_tables( + &self, + table_ids: Vec, + ) -> MetaResult { + let inner = self.inner.read().await; + let txn = inner.db.begin().await?; + + let fragment_ids: Vec = Fragment::find() + .filter(fragment::Column::JobId.is_in(table_ids)) + .all(&txn) + .await? + .into_iter() + .map(|fragment| fragment.fragment_id) + .collect(); + + self.resolve_working_set_for_reschedule_helper(&txn, fragment_ids) + .await + } + + pub async fn resolve_working_set_for_reschedule_helper( + &self, + txn: &C, + fragment_ids: Vec, + ) -> MetaResult + where + C: ConnectionTrait, + { + // NO_SHUFFLE related multi-layer upstream fragments + let no_shuffle_related_upstream_fragment_ids = resolve_no_shuffle_query( + txn, + construct_no_shuffle_upstream_traverse_query(fragment_ids.clone()), + ) + .await?; + + // NO_SHUFFLE related multi-layer downstream fragments + let no_shuffle_related_downstream_fragment_ids = resolve_no_shuffle_query( + txn, + construct_no_shuffle_downstream_traverse_query(fragment_ids.clone()), + ) + .await?; + + // We need to identify all other types of dispatchers that are Leaves in the NO_SHUFFLE dependency tree. + let extended_fragment_ids: HashSet<_> = no_shuffle_related_upstream_fragment_ids + .iter() + .chain(no_shuffle_related_downstream_fragment_ids.iter()) + .flat_map(|(src, _, dst)| [*src, *dst]) + .chain(fragment_ids.iter().cloned()) + .collect(); + + let query = Actor::find() + .select_only() + .column(actor::Column::FragmentId) + .column(actor_dispatcher::Column::DispatcherType) + .column(actor_dispatcher::Column::DispatcherId) + .distinct() + .join(JoinType::InnerJoin, actor::Relation::ActorDispatcher.def()); + + // single-layer upstream fragment ids + let upstream_fragments: Vec<(FragmentId, DispatcherType, FragmentId)> = query + .clone() + .filter(actor_dispatcher::Column::DispatcherId.is_in(extended_fragment_ids.clone())) + .into_tuple() + .all(txn) + .await?; + + // single-layer downstream fragment ids + let downstream_fragments: Vec<(FragmentId, DispatcherType, FragmentId)> = query + .clone() + .filter(actor::Column::FragmentId.is_in(extended_fragment_ids.clone())) + .into_tuple() + .all(txn) + .await?; + + let all_fragment_relations: HashSet<_> = no_shuffle_related_upstream_fragment_ids + .into_iter() + .chain(no_shuffle_related_downstream_fragment_ids.into_iter()) + .chain(upstream_fragments.into_iter()) + .chain(downstream_fragments.into_iter()) + .collect(); + + let mut fragment_upstreams: HashMap> = + HashMap::new(); + let mut fragment_downstreams: HashMap> = + HashMap::new(); + + for (src, dispatcher_type, dst) in &all_fragment_relations { + fragment_upstreams + .entry(*dst) + .or_default() + .push((*src, *dispatcher_type)); + fragment_downstreams + .entry(*src) + .or_default() + .push((*dst, *dispatcher_type)); + } + + let all_fragment_ids: HashSet<_> = all_fragment_relations + .iter() + .flat_map(|(src, _, dst)| [*src, *dst]) + .chain(extended_fragment_ids.into_iter()) + .collect(); + + let fragments: Vec<_> = Fragment::find() + .filter(fragment::Column::FragmentId.is_in(all_fragment_ids.clone())) + .all(txn) + .await?; + + let actor_and_dispatchers: Vec<(_, _)> = Actor::find() + .filter(actor::Column::FragmentId.is_in(all_fragment_ids.clone())) + .find_with_related(ActorDispatcher) + .all(txn) + .await?; + + let mut actors = HashMap::with_capacity(actor_and_dispatchers.len()); + let mut actor_dispatchers = HashMap::with_capacity(actor_and_dispatchers.len()); + + for (actor, dispatchers) in actor_and_dispatchers { + let actor_id = actor.actor_id; + actors.insert(actor_id, actor); + actor_dispatchers.insert(actor_id, dispatchers); + } + + let fragments: HashMap = fragments + .into_iter() + .map(|fragment| (fragment.fragment_id, fragment)) + .collect(); + + let related_job_ids: HashSet<_> = + fragments.values().map(|fragment| fragment.job_id).collect(); + + let related_jobs = StreamingJob::find() + .filter(streaming_job::Column::JobId.is_in(related_job_ids)) + .all(txn) + .await?; + + let related_jobs = related_jobs + .into_iter() + .map(|job| (job.job_id, job)) + .collect(); + + Ok(RescheduleWorkingSet { + fragments, + actors, + actor_dispatchers, + fragment_downstreams, + fragment_upstreams, + related_jobs, + }) + } +} diff --git a/src/meta/src/hummock/manager/time_travel.rs b/src/meta/src/hummock/manager/time_travel.rs index 61bcdfe7e8de..142c48e9bc0f 100644 --- a/src/meta/src/hummock/manager/time_travel.rs +++ b/src/meta/src/hummock/manager/time_travel.rs @@ -395,9 +395,13 @@ impl HummockManager { version_id: Set(version_id.try_into().unwrap()), }; hummock_epoch_to_version::Entity::insert(m) + .on_conflict( + OnConflict::column(hummock_epoch_to_version::Column::Epoch) + .update_columns([hummock_epoch_to_version::Column::VersionId]) + .to_owned(), + ) .exec(txn) .await?; - let mut version_sst_ids = None; let select_groups = group_parents .iter() @@ -483,14 +487,8 @@ fn replay_archive( deltas: impl Iterator, ) -> HummockVersion { let mut last_version = HummockVersion::from_persisted_protobuf(&version); - let mut mce = last_version.visible_table_committed_epoch(); for d in deltas { let d = HummockVersionDelta::from_persisted_protobuf(&d); - assert!( - d.visible_table_committed_epoch() > mce, - "time travel expects delta from commit_epoch only" - ); - mce = d.visible_table_committed_epoch(); // Need to work around the assertion in `apply_version_delta`. // Because compaction deltas are not included in time travel archive. while last_version.id < d.prev_id { diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 93e50dec3706..d5c12c70a0b9 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -24,7 +24,8 @@ use risingwave_common::util::addr::HostAddr; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; -use risingwave_pb::common::worker_node::{Property, State}; +use risingwave_license::LicenseManager; +use risingwave_pb::common::worker_node::{Property, Resource, State}; use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; use risingwave_pb::meta::heartbeat_request; @@ -114,6 +115,13 @@ impl ClusterManager { let mut property = self.parse_property(r#type, property); let mut core = self.core.write().await; + if let WorkerType::ComputeNode = r#type { + core.check_cpu_core_limit_on_newly_joined_compute_node( + host_address.clone(), + &resource, + )?; + } + if let Some(worker) = core.get_worker_by_host_mut(host_address.clone()) { tracing::info!("worker {} re-joined the cluster", worker.worker_id()); worker.update_resource(Some(resource)); @@ -631,6 +639,29 @@ impl ClusterManagerCore { .map(|(_, worker)| worker.clone()) } + /// Check if the total CPU cores in the cluster exceed the license limit, after counting the + /// newly joined compute node. + pub fn check_cpu_core_limit_on_newly_joined_compute_node( + &self, + host_address: HostAddress, + resource: &Resource, + ) -> MetaResult<()> { + let this_key = WorkerKey(host_address); + + let this = resource.total_cpu_cores; + let others = (self.workers.iter()) + .filter(|(k, _v)| k != &&this_key) + .filter(|(_k, v)| v.worker_node.r#type == WorkerType::ComputeNode as i32) + .flat_map(|(_k, v)| v.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 add_worker_node(&mut self, worker: Worker) { if let Some(transactional_id) = worker.worker_node.transactional_id { self.available_transactional_ids diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 72d135faa6a0..8062675156fe 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -1523,8 +1523,6 @@ impl DdlController { specified_parallelism: Option, cluster_info: &StreamingClusterInfo, ) -> MetaResult { - const MAX_PARALLELISM: NonZeroUsize = NonZeroUsize::new(VirtualNode::COUNT).unwrap(); - let available_parallelism = cluster_info.parallelism(); if available_parallelism == 0 { return Err(MetaError::unavailable("No available slots to schedule")); @@ -1546,12 +1544,7 @@ impl DdlController { ))); } - if available_parallelism > MAX_PARALLELISM { - tracing::warn!("Too many parallelism, use {} instead", MAX_PARALLELISM); - Ok(MAX_PARALLELISM) - } else { - Ok(parallelism) - } + Ok(parallelism) } /// Builds the actor graph: @@ -1617,6 +1610,15 @@ impl DdlController { let parallelism = self.resolve_stream_parallelism(specified_parallelism, &cluster_info)?; + const MAX_PARALLELISM: NonZeroUsize = NonZeroUsize::new(VirtualNode::COUNT).unwrap(); + + let parallelism_limited = parallelism > MAX_PARALLELISM; + if parallelism_limited { + tracing::warn!("Too many parallelism, use {} instead", MAX_PARALLELISM); + } + + let parallelism = parallelism.min(MAX_PARALLELISM); + let actor_graph_builder = ActorGraphBuilder::new(id, complete_graph, cluster_info, parallelism)?; @@ -1638,6 +1640,10 @@ impl DdlController { // If the frontend does not specify the degree of parallelism and the default_parallelism is set to full, then set it to ADAPTIVE. // Otherwise, it defaults to FIXED based on deduction. let table_parallelism = match (specified_parallelism, &self.env.opts.default_parallelism) { + (None, DefaultParallelism::Full) if parallelism_limited => { + tracing::warn!("Parallelism limited to 256 in ADAPTIVE mode"); + TableParallelism::Adaptive + } (None, DefaultParallelism::Full) => TableParallelism::Adaptive, _ => TableParallelism::Fixed(parallelism.get()), }; diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index f61789c8f937..42ed98b372c7 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -30,7 +30,7 @@ use risingwave_common::bitmap::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableId; use risingwave_common::hash::{ActorMapping, VirtualNode}; use risingwave_common::util::iter_util::ZipEqDebug; -use risingwave_meta_model_v2::StreamingParallelism; +use risingwave_meta_model_v2::{actor, fragment, ObjectId, StreamingParallelism}; use risingwave_pb::common::{ActorInfo, Buffer, PbActorLocation, WorkerNode, WorkerType}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; @@ -41,7 +41,7 @@ use risingwave_pb::meta::table_fragments::{self, ActorStatus, PbFragment, State} use risingwave_pb::meta::FragmentWorkerSlotMappings; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ - Dispatcher, DispatcherType, FragmentTypeFlag, PbStreamActor, StreamNode, + Dispatcher, DispatcherType, FragmentTypeFlag, PbDispatcher, PbStreamActor, StreamNode, }; use risingwave_pb::stream_service::build_actor_info::SubscriptionIds; use risingwave_pb::stream_service::BuildActorInfo; @@ -52,8 +52,10 @@ use tokio::task::JoinHandle; use tokio::time::{Instant, MissedTickBehavior}; use crate::barrier::{Command, Reschedule, StreamRpcManager}; +use crate::controller::scale::RescheduleWorkingSet; use crate::manager::{ - IdCategory, IdGenManagerImpl, LocalNotification, MetaSrvEnv, MetadataManager, WorkerId, + IdCategory, IdGenManagerImpl, LocalNotification, MetaSrvEnv, MetadataManager, + MetadataManagerV2, WorkerId, }; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism}; use crate::serving::{ @@ -61,7 +63,7 @@ use crate::serving::{ }; use crate::storage::{MetaStore, MetaStoreError, MetaStoreRef, Transaction, DEFAULT_COLUMN_FAMILY}; use crate::stream::{GlobalStreamManager, SourceManagerRef}; -use crate::{model, MetaError, MetaResult}; +use crate::{MetaError, MetaResult}; #[derive(Default, Copy, Clone, Debug, Eq, PartialEq, Ord, PartialOrd)] pub struct TableRevision(u64); @@ -120,7 +122,7 @@ pub struct CustomFragmentInfo { pub actors: Vec, } -#[derive(Default)] +#[derive(Default, Clone)] pub struct CustomActorInfo { pub actor_id: u32, pub fragment_id: u32, @@ -540,6 +542,141 @@ impl ScaleController { ); } + async fn fulfill_index_by_fragment_ids( + actor_map: &mut HashMap, + fragment_map: &mut HashMap, + actor_status: &mut BTreeMap, + fragment_state: &mut HashMap, + fragment_to_table: &mut HashMap, + mgr: &MetadataManagerV2, + fragment_ids: Vec, + ) -> Result<(), MetaError> { + let RescheduleWorkingSet { + fragments, + actors, + mut actor_dispatchers, + fragment_downstreams: _, + fragment_upstreams: _, + related_jobs, + } = mgr + .catalog_controller + .resolve_working_set_for_reschedule_fragments(fragment_ids) + .await?; + + let mut fragment_actors: HashMap< + risingwave_meta_model_v2::FragmentId, + Vec, + > = HashMap::new(); + + let mut expr_contexts = HashMap::new(); + for ( + _, + actor::Model { + actor_id, + fragment_id, + status: _, + splits: _, + worker_id, + upstream_actor_ids, + vnode_bitmap, + expr_context, + }, + ) in actors + { + let dispatchers = actor_dispatchers + .remove(&actor_id) + .unwrap_or_default() + .into_iter() + .map(PbDispatcher::from) + .collect(); + + let actor_info = CustomActorInfo { + actor_id: actor_id as _, + fragment_id: fragment_id as _, + dispatcher: dispatchers, + upstream_actor_id: upstream_actor_ids + .into_inner() + .values() + .flatten() + .map(|id| *id as _) + .collect(), + vnode_bitmap: vnode_bitmap.map(|bitmap| bitmap.to_protobuf()), + }; + + actor_map.insert(actor_id as _, actor_info.clone()); + + fragment_actors + .entry(fragment_id as _) + .or_default() + .push(actor_info); + + actor_status.insert(actor_id as _, worker_id as WorkerId); + + expr_contexts.insert(actor_id as u32, expr_context); + } + + for ( + _, + fragment::Model { + fragment_id, + job_id, + fragment_type_mask, + distribution_type, + stream_node, + state_table_ids, + upstream_fragment_id, + }, + ) in fragments + { + let actors = fragment_actors + .remove(&(fragment_id as _)) + .unwrap_or_default(); + + let CustomActorInfo { + actor_id, + fragment_id, + dispatcher, + upstream_actor_id, + vnode_bitmap, + } = actors.first().unwrap().clone(); + + let fragment = CustomFragmentInfo { + fragment_id: fragment_id as _, + fragment_type_mask: fragment_type_mask as _, + distribution_type: distribution_type.into(), + state_table_ids: state_table_ids.into_u32_array(), + upstream_fragment_ids: upstream_fragment_id.into_u32_array(), + actor_template: PbStreamActor { + nodes: Some(stream_node.to_protobuf()), + actor_id, + fragment_id: fragment_id as _, + dispatcher, + upstream_actor_id, + vnode_bitmap, + // todo, we need to fill this part + mview_definition: "".to_string(), + expr_context: expr_contexts + .get(&actor_id) + .cloned() + .map(|expr_context| expr_context.to_protobuf()), + }, + actors, + }; + + fragment_map.insert(fragment_id as _, fragment); + + fragment_to_table.insert(fragment_id as _, TableId::from(job_id as u32)); + + let related_job = related_jobs.get(&job_id).expect("job not found"); + + fragment_state.insert( + fragment_id, + table_fragments::PbState::from(related_job.job_status), + ); + } + Ok(()) + } + match &self.metadata_manager { MetadataManager::V1(mgr) => { let guard = mgr.fragment_manager.get_fragment_read_guard().await; @@ -555,19 +692,19 @@ impl ScaleController { ); } } - MetadataManager::V2(_) => { - let all_table_fragments = self.list_all_table_fragments().await?; - - for table_fragments in &all_table_fragments { - fulfill_index_by_table_fragments_ref( - &mut actor_map, - &mut fragment_map, - &mut actor_status, - &mut fragment_state, - &mut fragment_to_table, - table_fragments, - ); - } + MetadataManager::V2(mgr) => { + let fragment_ids = reschedule.keys().map(|id| *id as _).collect(); + + fulfill_index_by_fragment_ids( + &mut actor_map, + &mut fragment_map, + &mut actor_status, + &mut fragment_state, + &mut fragment_to_table, + mgr, + fragment_ids, + ) + .await?; } }; @@ -1814,23 +1951,6 @@ impl ScaleController { Ok(()) } - // FIXME: should be removed - pub(crate) async fn list_all_table_fragments(&self) -> MetaResult> { - use crate::model::MetadataModel; - let all_table_fragments = match &self.metadata_manager { - MetadataManager::V1(mgr) => mgr.fragment_manager.list_table_fragments().await, - MetadataManager::V2(mgr) => mgr - .catalog_controller - .table_fragments() - .await? - .into_values() - .map(model::TableFragments::from_protobuf) - .collect(), - }; - - Ok(all_table_fragments) - } - pub async fn generate_table_resize_plan( &self, policy: TableResizePolicy, @@ -1863,7 +1983,7 @@ impl ScaleController { .map(|worker| (worker.id, worker)) .collect(); - let worker_slots = workers + let schedulable_worker_slots = workers .values() .map(|worker| (worker.id, worker.parallelism as usize)) .collect::>(); @@ -1962,6 +2082,62 @@ impl ScaleController { Ok(()) } + async fn build_index_v2( + no_shuffle_source_fragment_ids: &mut HashSet, + no_shuffle_target_fragment_ids: &mut HashSet, + fragment_distribution_map: &mut HashMap, + actor_location: &mut HashMap, + table_fragment_id_map: &mut HashMap>, + fragment_actor_id_map: &mut HashMap>, + mgr: &MetadataManagerV2, + table_ids: Vec, + ) -> Result<(), MetaError> { + let RescheduleWorkingSet { + fragments, + actors, + actor_dispatchers: _actor_dispatchers, + fragment_downstreams, + fragment_upstreams: _fragment_upstreams, + related_jobs: _related_jobs, + } = mgr + .catalog_controller + .resolve_working_set_for_reschedule_tables(table_ids) + .await?; + + for (fragment_id, downstreams) in fragment_downstreams { + for (downstream_fragment_id, dispatcher_type) in downstreams { + if let risingwave_meta_model_v2::actor_dispatcher::DispatcherType::NoShuffle = + dispatcher_type + { + no_shuffle_source_fragment_ids.insert(fragment_id as FragmentId); + no_shuffle_target_fragment_ids.insert(downstream_fragment_id as FragmentId); + } + } + } + + for (fragment_id, fragment) in fragments { + fragment_distribution_map.insert( + fragment_id as FragmentId, + FragmentDistributionType::from(fragment.distribution_type), + ); + + table_fragment_id_map + .entry(fragment.job_id as u32) + .or_default() + .insert(fragment_id as FragmentId); + } + + for (actor_id, actor) in actors { + actor_location.insert(actor_id as ActorId, actor.worker_id as WorkerId); + fragment_actor_id_map + .entry(actor.fragment_id as FragmentId) + .or_default() + .insert(actor_id as ActorId); + } + + Ok(()) + } + match &self.metadata_manager { MetadataManager::V1(mgr) => { let guard = mgr.fragment_manager.get_fragment_read_guard().await; @@ -1975,22 +2151,24 @@ impl ScaleController { guard.table_fragments(), )?; } - MetadataManager::V2(_) => { - let all_table_fragments = self.list_all_table_fragments().await?; - let all_table_fragments = all_table_fragments - .into_iter() - .map(|table_fragments| (table_fragments.table_id(), table_fragments)) - .collect::>(); - build_index( + MetadataManager::V2(mgr) => { + let table_ids = table_parallelisms + .keys() + .map(|id| *id as ObjectId) + .collect(); + + build_index_v2( &mut no_shuffle_source_fragment_ids, &mut no_shuffle_target_fragment_ids, &mut fragment_distribution_map, &mut actor_location, &mut table_fragment_id_map, &mut fragment_actor_id_map, - &all_table_fragments, - )?; + mgr, + table_ids, + ) + .await?; } } @@ -2013,7 +2191,7 @@ impl ScaleController { *fragment_slots.entry(*worker_id).or_default() += 1; } - let all_available_slots: usize = worker_slots.values().cloned().sum(); + let all_available_slots: usize = schedulable_worker_slots.values().cloned().sum(); if all_available_slots == 0 { bail!( @@ -2030,12 +2208,13 @@ impl ScaleController { assert_eq!(*should_be_one, 1); - if worker_slots.contains_key(single_worker_id) { + if schedulable_worker_slots.contains_key(single_worker_id) { // NOTE: shall we continue? continue; } - let units = schedule_units_for_slots(&worker_slots, 1, table_id)?; + let units = + schedule_units_for_slots(&schedulable_worker_slots, 1, table_id)?; let (chosen_target_worker_id, should_be_one) = units.iter().exactly_one().ok().with_context(|| { @@ -2059,14 +2238,41 @@ impl ScaleController { } FragmentDistributionType::Hash => match parallelism { TableParallelism::Adaptive => { - target_plan.insert( - fragment_id, - Self::diff_worker_slot_changes(&fragment_slots, &worker_slots), - ); + if all_available_slots > VirtualNode::COUNT { + tracing::warn!("available parallelism for table {table_id} is larger than VirtualNode::COUNT, force limit to VirtualNode::COUNT"); + // force limit to VirtualNode::COUNT + let target_worker_slots = schedule_units_for_slots( + &schedulable_worker_slots, + VirtualNode::COUNT, + table_id, + )?; + + target_plan.insert( + fragment_id, + Self::diff_worker_slot_changes( + &fragment_slots, + &target_worker_slots, + ), + ); + } else { + target_plan.insert( + fragment_id, + Self::diff_worker_slot_changes( + &fragment_slots, + &schedulable_worker_slots, + ), + ); + } } - TableParallelism::Fixed(n) => { + TableParallelism::Fixed(mut n) => { + if n > VirtualNode::COUNT { + // This should be unreachable, but we still intercept it to prevent accidental modifications. + tracing::warn!("parallelism {n} for table {table_id} is larger than VirtualNode::COUNT, force limit to VirtualNode::COUNT"); + n = VirtualNode::COUNT + } + let target_worker_slots = - schedule_units_for_slots(&worker_slots, n, table_id)?; + schedule_units_for_slots(&schedulable_worker_slots, n, table_id)?; target_plan.insert( fragment_id, diff --git a/src/prost/src/lib.rs b/src/prost/src/lib.rs index b8cab9006dea..0b416af39b83 100644 --- a/src/prost/src/lib.rs +++ b/src/prost/src/lib.rs @@ -23,6 +23,7 @@ pub use prost::Message; use risingwave_error::tonic::ToTonicStatus; use thiserror::Error; + #[rustfmt::skip] #[cfg_attr(madsim, path = "sim/catalog.rs")] pub mod catalog; @@ -242,6 +243,14 @@ impl meta::table_fragments::ActorStatus { } } +impl common::WorkerNode { + pub fn is_streaming_schedulable(&self) -> bool { + let property = self.property.as_ref(); + property.map_or(false, |p| p.is_streaming) + && !property.map_or(false, |p| p.is_unschedulable) + } +} + impl common::ActorLocation { pub fn from_worker(worker_node_id: u32) -> Option { Some(Self { worker_node_id }) diff --git a/src/rpc_client/src/error.rs b/src/rpc_client/src/error.rs index c5c5613a32a4..26204cc1908c 100644 --- a/src/rpc_client/src/error.rs +++ b/src/rpc_client/src/error.rs @@ -77,3 +77,23 @@ macro_rules! impl_from_status { } impl_from_status!(stream, batch, meta, compute, compactor, connector); + +impl RpcError { + /// Returns `true` if the error is a connection error. Typically used to determine if + /// the error is transient and can be retried. + pub fn is_connection_error(&self) -> bool { + match self { + RpcError::TransportError(_) => true, + RpcError::GrpcStatus(status) => matches!( + status.inner().code(), + tonic::Code::Unavailable // server not started + | tonic::Code::Unknown // could be transport error + | tonic::Code::Unimplemented // meta leader service not started + ), + RpcError::MetaAddressParse(_) => false, + RpcError::Internal(anyhow) => anyhow + .downcast_ref::() // this skips all contexts attached to the error + .map_or(false, Self::is_connection_error), + } + } +} diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 5a45e0752c9d..ae99c57afd7c 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -36,6 +36,7 @@ use risingwave_common::util::meta_addr::MetaAddressStrategy; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; +use risingwave_error::bail; use risingwave_hummock_sdk::compaction_group::StateTableId; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::{ @@ -220,12 +221,33 @@ impl MetaClient { } /// Register the current node to the cluster and set the corresponding worker id. + /// + /// Retry if there's connection issue with the meta node. Exit the process if the registration fails. pub async fn register_new( addr_strategy: MetaAddressStrategy, worker_type: WorkerType, addr: &HostAddr, property: Property, meta_config: &MetaConfig, + ) -> (Self, SystemParamsReader) { + let ret = + Self::register_new_inner(addr_strategy, worker_type, addr, property, meta_config).await; + + match ret { + Ok(ret) => ret, + Err(err) => { + tracing::error!(error = %err.as_report(), "failed to register worker, exiting..."); + std::process::exit(1); + } + } + } + + async fn register_new_inner( + addr_strategy: MetaAddressStrategy, + worker_type: WorkerType, + addr: &HostAddr, + property: Property, + meta_config: &MetaConfig, ) -> Result<(Self, SystemParamsReader)> { tracing::info!("register meta client using strategy: {}", addr_strategy); @@ -238,34 +260,35 @@ impl MetaClient { if property.is_unschedulable { tracing::warn!("worker {:?} registered as unschedulable", addr.clone()); } - let init_result: Result<_> = tokio_retry::Retry::spawn(retry_strategy, || async { - let grpc_meta_client = GrpcMetaClient::new(&addr_strategy, meta_config.clone()).await?; - - let add_worker_resp = grpc_meta_client - .add_worker_node(AddWorkerNodeRequest { - worker_type: worker_type as i32, - host: Some(addr.to_protobuf()), - property: Some(property), - resource: Some(risingwave_pb::common::worker_node::Resource { - rw_version: RW_VERSION.to_string(), - total_memory_bytes: system_memory_available_bytes() as _, - total_cpu_cores: total_cpu_available() as _, - }), - }) - .await?; - if let Some(status) = &add_worker_resp.status - && status.code() == risingwave_pb::common::status::Code::UnknownWorker - { - tracing::error!("invalid worker: {}", status.message); - std::process::exit(1); - } + let init_result: Result<_> = tokio_retry::RetryIf::spawn( + retry_strategy, + || async { + let grpc_meta_client = + GrpcMetaClient::new(&addr_strategy, meta_config.clone()).await?; + + let add_worker_resp = grpc_meta_client + .add_worker_node(AddWorkerNodeRequest { + worker_type: worker_type as i32, + host: Some(addr.to_protobuf()), + property: Some(property), + resource: Some(risingwave_pb::common::worker_node::Resource { + rw_version: RW_VERSION.to_string(), + total_memory_bytes: system_memory_available_bytes() as _, + total_cpu_cores: total_cpu_available() as _, + }), + }) + .await + .context("failed to add worker node")?; - let system_params_resp = grpc_meta_client - .get_system_params(GetSystemParamsRequest {}) - .await?; + let system_params_resp = grpc_meta_client + .get_system_params(GetSystemParamsRequest {}) + .await + .context("failed to get initial system params")?; - Ok((add_worker_resp, system_params_resp, grpc_meta_client)) - }) + Ok((add_worker_resp, system_params_resp, grpc_meta_client)) + }, + RpcError::is_connection_error, + ) .await; let (add_worker_resp, system_params_resp, grpc_meta_client) = init_result?; @@ -1708,38 +1731,40 @@ impl MetaMemberManagement { let mut fetched_members = None; for (addr, client) in &mut member_group.members { - let client: Result = try { - match client { + let members: Result<_> = try { + let mut client = match client { Some(cached_client) => cached_client.to_owned(), None => { let endpoint = GrpcMetaClient::addr_to_endpoint(addr.clone()); - let channel = GrpcMetaClient::connect_to_endpoint(endpoint).await?; + let channel = GrpcMetaClient::connect_to_endpoint(endpoint) + .await + .context("failed to create client")?; let new_client: MetaMemberClient = MetaMemberServiceClient::new(channel); *client = Some(new_client.clone()); new_client } - } + }; + + let resp = client + .members(MembersRequest {}) + .await + .context("failed to fetch members")?; + + resp.into_inner().members }; - if let Err(err) = client { - tracing::warn!(%addr, error = %err.as_report(), "failed to create client"); - continue; - } - match client.unwrap().members(MembersRequest {}).await { - Err(err) => { - tracing::warn!(%addr, error = %err.as_report(), "failed to fetch members"); - continue; - } - Ok(resp) => { - fetched_members = Some(resp.into_inner().members); - break; - } + + let fetched = members.is_ok(); + fetched_members = Some(members); + if fetched { + break; } } - let members = - fetched_members.ok_or_else(|| anyhow!("could not refresh members"))?; + let members = fetched_members + .context("no member available in the list")? + .context("could not refresh members")?; // find new leader let mut leader = None; @@ -1916,7 +1941,7 @@ impl GrpcMetaClient { .map(|addr| (Self::addr_to_endpoint(addr.clone()), addr)) .collect(); - let endpoints = endpoints.clone(); + let mut last_error = None; for (endpoint, addr) in endpoints { match Self::connect_to_endpoint(endpoint).await { @@ -1929,14 +1954,19 @@ impl GrpcMetaClient { error = %e.as_report(), "Failed to connect to meta server {}, trying again", addr, - ) + ); + last_error = Some(e); } } } - Err(RpcError::Internal(anyhow!( - "Failed to connect to meta server" - ))) + if let Some(last_error) = last_error { + Err(anyhow::anyhow!(last_error) + .context("failed to connect to all meta servers") + .into()) + } else { + bail!("no meta server address provided") + } } async fn connect_to_endpoint(endpoint: Endpoint) -> Result { diff --git a/src/sqlparser/src/ast/mod.rs b/src/sqlparser/src/ast/mod.rs index 73c174769eb4..eeb985967971 100644 --- a/src/sqlparser/src/ast/mod.rs +++ b/src/sqlparser/src/ast/mod.rs @@ -1840,7 +1840,7 @@ impl fmt::Display for Statement { if !include_column_options.is_empty() { // (Ident, Option) write!(f, "{}", display_comma_separated( include_column_options.iter().map(|option_item: &IncludeOptionItem| { - format!("INCLUDE {}{}{}", + format!(" INCLUDE {}{}{}", option_item.column_type, if let Some(inner_field) = &option_item.inner_field { format!(" {}", inner_field) diff --git a/src/sqlparser/tests/testdata/create.yaml b/src/sqlparser/tests/testdata/create.yaml index 317130b79d5e..13f5e2dad5bf 100644 --- a/src/sqlparser/tests/testdata/create.yaml +++ b/src/sqlparser/tests/testdata/create.yaml @@ -79,6 +79,8 @@ ^ - input: CREATE TABLE T ("FULL" INT) ON CONFLICT DO UPDATE IF NOT NULL formatted_sql: CREATE TABLE T ("FULL" INT) ON CONFLICT DO UPDATE IF NOT NULL +- input: CREATE TABLE t (a int, b int, ts timestamptz as proctime(), primary key (a)) ON CONFLICT DO UPDATE IF NOT NULL INCLUDE timestamp AS ts_col WITH (connector = 'kafka', properties.bootstrap.server = '127.0.0.1:9092', topic = 'test_topic') FORMAT PLAIN ENCODE JSON + formatted_sql: CREATE TABLE t (a INT, b INT, ts timestamptz AS proctime(), PRIMARY KEY (a)) ON CONFLICT DO UPDATE IF NOT NULL INCLUDE timestamp AS ts_col WITH (connector = 'kafka', properties.bootstrap.server = '127.0.0.1:9092', topic = 'test_topic') FORMAT PLAIN ENCODE JSON - input: CREATE USER user WITH SUPERUSER CREATEDB PASSWORD 'password' formatted_sql: CREATE USER user WITH SUPERUSER CREATEDB PASSWORD 'password' - input: CREATE SINK snk diff --git a/src/storage/compactor/src/server.rs b/src/storage/compactor/src/server.rs index e139bc201cd4..0b086ee06fe1 100644 --- a/src/storage/compactor/src/server.rs +++ b/src/storage/compactor/src/server.rs @@ -197,8 +197,7 @@ pub async fn compactor_serve( Default::default(), &config.meta, ) - .await - .unwrap(); + .await; info!("Assigned compactor id {}", meta_client.worker_id()); diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index f7305fa0a150..13a0bcc08adf 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -565,7 +565,7 @@ impl HummockVersion { && self.visible_table_committed_epoch() < version_delta.visible_table_committed_epoch() { is_commit_epoch = true; - warn!("max committed epoch bumped but no table committed epoch is changed"); + tracing::trace!("max committed epoch bumped but no table committed epoch is changed"); } // apply to `levels`, which is different compaction groups diff --git a/src/tests/compaction_test/src/compaction_test_runner.rs b/src/tests/compaction_test/src/compaction_test_runner.rs index 0aa7d1d83c8d..328c23f8fbe8 100644 --- a/src/tests/compaction_test/src/compaction_test_runner.rs +++ b/src/tests/compaction_test/src/compaction_test_runner.rs @@ -238,7 +238,7 @@ async fn init_metadata_for_replay( std::process::exit(0); }, ret = MetaClient::register_new(cluster_meta_endpoint.parse()?, WorkerType::RiseCtl, advertise_addr, Default::default(), &meta_config) => { - (meta_client, _) = ret.unwrap(); + (meta_client, _) = ret; }, } let worker_id = meta_client.worker_id(); @@ -254,7 +254,7 @@ async fn init_metadata_for_replay( Default::default(), &meta_config, ) - .await?; + .await; new_meta_client.activate(advertise_addr).await.unwrap(); if ci_mode { let table_to_check = tables.iter().find(|t| t.name == "nexmark_q7").unwrap(); @@ -286,7 +286,7 @@ async fn pull_version_deltas( Default::default(), &MetaConfig::default(), ) - .await?; + .await; let worker_id = meta_client.worker_id(); tracing::info!("Assigned pull worker id {}", worker_id); meta_client.activate(advertise_addr).await.unwrap(); @@ -335,7 +335,7 @@ async fn start_replay( Default::default(), &config.meta, ) - .await?; + .await; let worker_id = meta_client.worker_id(); tracing::info!("Assigned replay worker id {}", worker_id); meta_client.activate(&advertise_addr).await.unwrap(); diff --git a/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs b/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs index 8e2b0f446550..b54a5bd3d6b3 100644 --- a/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs +++ b/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs @@ -25,7 +25,7 @@ use risingwave_simulation::utils::AssertResult; use tokio::time::sleep; /// Please ensure that this value is the same as the one in the `risingwave-auto-scale.toml` file. -const MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE: u64 = 15; +pub const MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE: u64 = 15; #[tokio::test] async fn test_passive_online_and_offline() -> Result<()> { diff --git a/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs b/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs index 625a9bc6a8bf..bef6b1ca1f1c 100644 --- a/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs +++ b/src/tests/simulation/tests/integration_tests/scale/streaming_parallelism.rs @@ -12,9 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::time::Duration; + use anyhow::Result; +use madsim::time::sleep; +use risingwave_common::hash::VirtualNode; use risingwave_simulation::cluster::{Cluster, Configuration}; use risingwave_simulation::ctl_ext::predicate::identity_contains; +use risingwave_simulation::utils::AssertResult; + +use crate::scale::auto_parallelism::MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE; #[tokio::test] async fn test_streaming_parallelism_default() -> Result<()> { @@ -135,3 +142,91 @@ async fn test_streaming_parallelism_index() -> Result<()> { ); Ok(()) } + +#[tokio::test] +async fn test_parallelism_exceed_virtual_node_max_create() -> Result<()> { + let vnode_max = VirtualNode::COUNT; + let mut configuration = Configuration::for_auto_parallelism( + MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE, + true, + ); + + configuration.compute_nodes = 1; + configuration.compute_node_cores = vnode_max + 1; + let mut cluster = Cluster::start(configuration).await?; + + sleep(Duration::from_secs( + MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE * 2, + )) + .await; + + let mut session = cluster.start_session(); + session.run("create table t(v int)").await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("ADAPTIVE"); + + session + .run("select distinct parallelism from rw_fragment_parallelism where name = 't'") + .await? + .assert_result_eq(format!("{}", vnode_max)); + + Ok(()) +} + +#[tokio::test] +async fn test_parallelism_exceed_virtual_node_max_alter_fixed() -> Result<()> { + let vnode_max = VirtualNode::COUNT; + let mut configuration = Configuration::for_scale(); + configuration.compute_nodes = 1; + configuration.compute_node_cores = vnode_max + 100; + let mut cluster = Cluster::start(configuration).await?; + let mut session = cluster.start_session(); + session.run("set streaming_parallelism = 1").await?; + session.run("create table t(v int)").await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("FIXED(1)"); + + session + .run(format!("alter table t set parallelism = {}", vnode_max + 1)) + .await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq(format!("FIXED({})", vnode_max)); + Ok(()) +} + +#[tokio::test] +async fn test_parallelism_exceed_virtual_node_max_alter_adaptive() -> Result<()> { + let vnode_max = VirtualNode::COUNT; + let mut configuration = Configuration::for_scale(); + configuration.compute_nodes = 1; + configuration.compute_node_cores = vnode_max + 100; + let mut cluster = Cluster::start(configuration).await?; + let mut session = cluster.start_session(); + session.run("set streaming_parallelism = 1").await?; + session.run("create table t(v int)").await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("FIXED(1)"); + + session + .run("alter table t set parallelism = adaptive") + .await?; + session + .run("select parallelism from rw_streaming_parallelism where name = 't'") + .await? + .assert_result_eq("ADAPTIVE"); + + session + .run("select distinct parallelism from rw_fragment_parallelism where name = 't'") + .await? + .assert_result_eq(format!("{}", vnode_max)); + + Ok(()) +}