diff --git a/Cargo.lock b/Cargo.lock index 1c074e276553a..1e711a211887b 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8685,6 +8685,7 @@ dependencies = [ "strum_macros 0.26.1", "task_stats_alloc", "tempfile", + "thiserror-ext", "tikv-jemallocator", "tracing", "vergen", @@ -9195,6 +9196,7 @@ dependencies = [ "async-trait", "auto_enums", "chrono", + "chrono-tz", "criterion", "expect-test", "fancy-regex", diff --git a/backwards-compat-tests/scripts/utils.sh b/backwards-compat-tests/scripts/utils.sh index 1afbf08dd4441..5990aac026077 100644 --- a/backwards-compat-tests/scripts/utils.sh +++ b/backwards-compat-tests/scripts/utils.sh @@ -103,19 +103,21 @@ insert_json_kafka() { local JSON=$1 echo "$JSON" | "$KAFKA_PATH"/bin/kafka-console-producer.sh \ --topic backwards_compat_test_kafka_source \ - --bootstrap-server localhost:29092 + --bootstrap-server localhost:29092 \ + --property "parse.key=true" \ + --property "key.separator=," } seed_json_kafka() { - insert_json_kafka '{"timestamp": "2023-07-28 07:11:00", "user_id": 1, "page_id": 1, "action": "gtrgretrg"}' - insert_json_kafka '{"timestamp": "2023-07-28 07:11:00", "user_id": 2, "page_id": 1, "action": "fsdfgerrg"}' - insert_json_kafka '{"timestamp": "2023-07-28 07:11:00", "user_id": 3, "page_id": 1, "action": "sdfergtth"}' - insert_json_kafka '{"timestamp": "2023-07-28 06:54:00", "user_id": 4, "page_id": 2, "action": "erwerhghj"}' - insert_json_kafka '{"timestamp": "2023-07-28 06:54:00", "user_id": 5, "page_id": 2, "action": "kiku7ikkk"}' - insert_json_kafka '{"timestamp": "2023-07-28 06:54:00", "user_id": 6, "page_id": 3, "action": "6786745ge"}' - insert_json_kafka '{"timestamp": "2023-07-28 06:54:00", "user_id": 7, "page_id": 3, "action": "fgbgfnyyy"}' - insert_json_kafka '{"timestamp": "2023-07-28 06:54:00", "user_id": 8, "page_id": 4, "action": "werwerwwe"}' - insert_json_kafka '{"timestamp": "2023-07-28 06:54:00", "user_id": 9, "page_id": 4, "action": "yjtyjtyyy"}' + insert_json_kafka '{"user_id": 1},{"timestamp": "2023-07-28 07:11:00", "user_id": 1, "page_id": 1, "action": "gtrgretrg"}' + insert_json_kafka '{"user_id": 2},{"timestamp": "2023-07-28 07:11:00", "user_id": 2, "page_id": 1, "action": "fsdfgerrg"}' + insert_json_kafka '{"user_id": 3},{"timestamp": "2023-07-28 07:11:00", "user_id": 3, "page_id": 1, "action": "sdfergtth"}' + insert_json_kafka '{"user_id": 4},{"timestamp": "2023-07-28 06:54:00", "user_id": 4, "page_id": 2, "action": "erwerhghj"}' + insert_json_kafka '{"user_id": 5},{"timestamp": "2023-07-28 06:54:00", "user_id": 5, "page_id": 2, "action": "kiku7ikkk"}' + insert_json_kafka '{"user_id": 6},{"timestamp": "2023-07-28 06:54:00", "user_id": 6, "page_id": 3, "action": "6786745ge"}' + insert_json_kafka '{"user_id": 7},{"timestamp": "2023-07-28 06:54:00", "user_id": 7, "page_id": 3, "action": "fgbgfnyyy"}' + insert_json_kafka '{"user_id": 8},{"timestamp": "2023-07-28 06:54:00", "user_id": 8, "page_id": 4, "action": "werwerwwe"}' + insert_json_kafka '{"user_id": 9},{"timestamp": "2023-07-28 06:54:00", "user_id": 9, "page_id": 4, "action": "yjtyjtyyy"}' } # https://stackoverflow.com/a/4024263 @@ -225,6 +227,12 @@ seed_old_cluster() { create_kafka_topic seed_json_kafka sqllogictest -d dev -h localhost -p 4566 "$TEST_DIR/kafka/seed.slt" + # use the old syntax for version at most 1.5.4 + if version_le "$OLD_VERSION" "1.5.4" ; then + sqllogictest -d dev -h localhost -p 4566 "$TEST_DIR/kafka/upsert/deprecate_upsert.slt" + else + sqllogictest -d dev -h localhost -p 4566 "$TEST_DIR/kafka/upsert/include_key_as.slt" + fi echo "--- KAFKA TEST: wait 5s for kafka to process data" sleep 5 diff --git a/backwards-compat-tests/slt/kafka/upsert/deprecate_upsert.slt b/backwards-compat-tests/slt/kafka/upsert/deprecate_upsert.slt new file mode 100644 index 0000000000000..55cfce886455d --- /dev/null +++ b/backwards-compat-tests/slt/kafka/upsert/deprecate_upsert.slt @@ -0,0 +1,16 @@ +statement ok +CREATE TABLE IF NOT EXISTS kafka_table +( + action varchar, + user_id integer, + obj_id integer, + name varchar, + page_id integer, + age integer +) +WITH ( + connector='kafka', + topic='backwards_compat_test_kafka_source', + properties.bootstrap.server='localhost:29092', + scan.startup.mode='earliest', +) FORMAT UPSERT ENCODE JSON; \ No newline at end of file diff --git a/backwards-compat-tests/slt/kafka/upsert/include_key_as.slt b/backwards-compat-tests/slt/kafka/upsert/include_key_as.slt new file mode 100644 index 0000000000000..36ef426574223 --- /dev/null +++ b/backwards-compat-tests/slt/kafka/upsert/include_key_as.slt @@ -0,0 +1,18 @@ +statement ok +CREATE TABLE IF NOT EXISTS kafka_table +( + action varchar, + user_id integer, + obj_id integer, + name varchar, + page_id integer, + age integer, + primary key (_rw_key) +) +INCLUDE key as _rw_key +WITH ( + connector='kafka', + topic='backwards_compat_test_kafka_source', + properties.bootstrap.server='localhost:29092', + scan.startup.mode='earliest', +) FORMAT UPSERT ENCODE JSON; \ No newline at end of file diff --git a/backwards-compat-tests/slt/kafka/validate_restart.slt b/backwards-compat-tests/slt/kafka/validate_restart.slt index 7058b118f4d20..6d853007b9829 100644 --- a/backwards-compat-tests/slt/kafka/validate_restart.slt +++ b/backwards-compat-tests/slt/kafka/validate_restart.slt @@ -50,3 +50,16 @@ werwerwwe 8 NULL NULL 4 NULL yjtyjtyyy 9 NULL NULL 4 NULL yjtyjtyyy 9 NULL NULL 4 NULL +# kafka_table should do the upsert and overwrite the existing records +query I rowsort +SELECT action, user_id, obj_id, name, page_id, age, _rw_key FROM kafka_table; +---- +6786745ge 6 NULL NULL 3 NULL \x7b22757365725f6964223a20367d +erwerhghj 4 NULL NULL 2 NULL \x7b22757365725f6964223a20347d +fgbgfnyyy 7 NULL NULL 3 NULL \x7b22757365725f6964223a20377d +fsdfgerrg 2 NULL NULL 1 NULL \x7b22757365725f6964223a20327d +gtrgretrg 1 NULL NULL 1 NULL \x7b22757365725f6964223a20317d +kiku7ikkk 5 NULL NULL 2 NULL \x7b22757365725f6964223a20357d +sdfergtth 3 NULL NULL 1 NULL \x7b22757365725f6964223a20337d +werwerwwe 8 NULL NULL 4 NULL \x7b22757365725f6964223a20387d +yjtyjtyyy 9 NULL NULL 4 NULL \x7b22757365725f6964223a20397d diff --git a/ci/scripts/e2e-clickhouse-sink-test.sh b/ci/scripts/e2e-clickhouse-sink-test.sh index c14d83e8c4281..5443d4e53b7fd 100755 --- a/ci/scripts/e2e-clickhouse-sink-test.sh +++ b/ci/scripts/e2e-clickhouse-sink-test.sh @@ -31,7 +31,7 @@ sleep 1 echo "--- create clickhouse table" curl https://clickhouse.com/ | sh sleep 2 -./clickhouse client --host=clickhouse-server --port=9000 --query="CREATE table demo_test(v1 Int32,v2 Int64,v3 String)ENGINE = ReplacingMergeTree PRIMARY KEY (v1);" +./clickhouse client --host=clickhouse-server --port=9000 --query="CREATE table demo_test(v1 Int32,v2 Int64,v3 String,v4 Enum16('A'=1,'B'=2))ENGINE = ReplacingMergeTree PRIMARY KEY (v1);" echo "--- testing sinks" sqllogictest -p 4566 -d dev './e2e_test/sink/clickhouse_sink.slt' @@ -41,13 +41,13 @@ sleep 5 # check sink destination using shell if cat ./query_result.csv | sort | awk -F "," '{ -if ($1 == 1 && $2 == 50 && $3 == "\"1-50\"") c1++; - if ($1 == 13 && $2 == 2 && $3 == "\"13-2\"") c2++; - if ($1 == 2 && $2 == 2 && $3 == "\"2-2\"") c3++; - if ($1 == 21 && $2 == 2 && $3 == "\"21-2\"") c4++; - if ($1 == 3 && $2 == 2 && $3 == "\"3-2\"") c5++; - if ($1 == 5 && $2 == 2 && $3 == "\"5-2\"") c6++; - if ($1 == 8 && $2 == 2 && $3 == "\"8-2\"") c7++; } +if ($1 == 1 && $2 == 50 && $3 == "\"1-50\"" && $4 == "\"A\"") c1++; + if ($1 == 13 && $2 == 2 && $3 == "\"13-2\"" && $4 == "\"B\"") c2++; + if ($1 == 2 && $2 == 2 && $3 == "\"2-2\"" && $4 == "\"B\"") c3++; + if ($1 == 21 && $2 == 2 && $3 == "\"21-2\"" && $4 == "\"A\"") c4++; + if ($1 == 3 && $2 == 2 && $3 == "\"3-2\"" && $4 == "\"A\"") c5++; + if ($1 == 5 && $2 == 2 && $3 == "\"5-2\"" && $4 == "\"B\"") c6++; + if ($1 == 8 && $2 == 2 && $3 == "\"8-2\"" && $4 == "\"A\"") c7++; } END { exit !(c1 == 1 && c2 == 1 && c3 == 1 && c4 == 1 && c5 == 1 && c6 == 1 && c7 == 1); }'; then echo "Clickhouse sink check passed" else diff --git a/e2e_test/batch/basic/make_time.slt.part b/e2e_test/batch/basic/make_time.slt.part index 7a11b837c4fdb..ff1d4453e0efd 100644 --- a/e2e_test/batch/basic/make_time.slt.part +++ b/e2e_test/batch/basic/make_time.slt.part @@ -9,7 +9,12 @@ SELECT make_timestamptz(1973, 07, 15, 08, 15, 55.33); query T SELECT make_timestamptz(-1973, 07, 15, 08, 15, 55.33); ---- --1972-07-15 08:15:55.330+00:00 +1973-07-15 08:15:55.330+00:00 BC + +query T +SELECT make_timestamptz(20240, 1, 26, 14, 20, 26); +---- +20240-01-26 14:20:26+00:00 query error Invalid parameter year, month, day: invalid date: -3-2-29 SELECT make_timestamptz(-4, 02, 29, 08, 15, 55.33); @@ -17,7 +22,7 @@ SELECT make_timestamptz(-4, 02, 29, 08, 15, 55.33); query T SELECT make_timestamptz(-5, 02, 29, 08, 15, 55.33); ---- --0004-02-29 08:15:55.330+00:00 +0005-02-29 08:15:55.330+00:00 BC query error Invalid parameter sec: invalid sec: -55.33 SELECT make_timestamptz(1973, 07, 15, 08, 15, -55.33); @@ -105,6 +110,11 @@ SELECT make_date(2024, 1, 26); ---- 2024-01-26 +query T +SELECT make_date(20240, 1, 26); +---- +20240-01-26 + query T SELECT make_date(-2024, 1, 26); ---- @@ -146,10 +156,15 @@ SELECT make_timestamp(2024, 1, 26, 14, 20, 26); ---- 2024-01-26 14:20:26 +query T +SELECT make_timestamp(20240, 1, 26, 14, 20, 26); +---- +20240-01-26 14:20:26 + query T SELECT make_timestamp(-1973, 07, 15, 08, 15, 55.33); ---- --1972-07-15 08:15:55.330 +1973-07-15 08:15:55.330 BC query error Invalid parameter year, month, day: invalid date: -3-2-29 SELECT make_timestamp(-4, 02, 29, 08, 15, 55.33); @@ -157,4 +172,14 @@ SELECT make_timestamp(-4, 02, 29, 08, 15, 55.33); query T SELECT make_timestamp(-5, 02, 29, 08, 15, 55.33); ---- --0004-02-29 08:15:55.330 +0005-02-29 08:15:55.330 BC + +query T +select '0001-01-01 12:34:56'::timestamp - '10 year'::interval; +---- +0010-01-01 12:34:56 BC + +query T +select '0001-01-01 12:34:56'::timestamptz - '10 year'::interval; +---- +0010-01-01 12:34:56+00:00 BC diff --git a/e2e_test/batch/catalog/pg_class.slt.part b/e2e_test/batch/catalog/pg_class.slt.part index 2f2ffbe016e3a..ff31c27dcc17d 100644 --- a/e2e_test/batch/catalog/pg_class.slt.part +++ b/e2e_test/batch/catalog/pg_class.slt.part @@ -11,7 +11,7 @@ SELECT oid,relname,relowner,relkind FROM pg_catalog.pg_class ORDER BY oid limit 8 pg_cast 1 r 9 pg_class 1 v 10 pg_collation 1 v -11 pg_constraint 1 v +11 pg_constraint 1 r 12 pg_conversion 1 v 13 pg_database 1 v 14 pg_depend 1 v @@ -20,4 +20,4 @@ SELECT oid,relname,relowner,relkind FROM pg_catalog.pg_class ORDER BY oid limit query ITIT SELECT oid,relname,relowner,relkind FROM pg_catalog.pg_class WHERE oid = 'pg_namespace'::regclass; ---- -24 pg_namespace 1 v +25 pg_namespace 1 v diff --git a/e2e_test/batch/catalog/pg_constraint.slt.part b/e2e_test/batch/catalog/pg_constraint.slt.part new file mode 100644 index 0000000000000..a2a36e73f5416 --- /dev/null +++ b/e2e_test/batch/catalog/pg_constraint.slt.part @@ -0,0 +1,10 @@ +statement ok +create table t(a int, b int, c varchar, primary key(a,b)); + +query TTTT +select conname, contype, conkey from pg_constraint where conname='t_pkey'; +---- +t_pkey p {1,2} + +statement ok +drop table t; diff --git a/e2e_test/ddl/search_path.slt b/e2e_test/ddl/search_path.slt index 06db7f3f45c90..e01ed5d602936 100644 --- a/e2e_test/ddl/search_path.slt +++ b/e2e_test/ddl/search_path.slt @@ -76,6 +76,11 @@ select a from test order by a; 1 2 +# Issue #15195 +# index shall be created in `search_path_test2` (same as table) rather than `search_path_test1` (first in path) +statement ok +create index if not exists index1_test_a on test(a); + statement ok drop table test; diff --git a/e2e_test/sink/clickhouse_sink.slt b/e2e_test/sink/clickhouse_sink.slt index 909bdbfd6356b..9791f484326d7 100644 --- a/e2e_test/sink/clickhouse_sink.slt +++ b/e2e_test/sink/clickhouse_sink.slt @@ -1,11 +1,11 @@ statement ok -CREATE TABLE t6 (v1 int primary key, v2 bigint, v3 varchar); +CREATE TABLE t6 (v1 int primary key, v2 bigint, v3 varchar, v4 smallint); statement ok CREATE MATERIALIZED VIEW mv6 AS SELECT * FROM t6; statement ok -CREATE SINK s6 AS select mv6.v1 as v1, mv6.v2 as v2, mv6.v3 as v3 from mv6 WITH ( +CREATE SINK s6 AS select mv6.v1 as v1, mv6.v2 as v2, mv6.v3 as v3, mv6.v4 as v4 from mv6 WITH ( connector = 'clickhouse', type = 'append-only', force_append_only='true', @@ -17,7 +17,7 @@ CREATE SINK s6 AS select mv6.v1 as v1, mv6.v2 as v2, mv6.v3 as v3 from mv6 WITH ); statement ok -INSERT INTO t6 VALUES (1, 50, '1-50'), (2, 2, '2-2'), (3, 2, '3-2'), (5, 2, '5-2'), (8, 2, '8-2'), (13, 2, '13-2'), (21, 2, '21-2'); +INSERT INTO t6 VALUES (1, 50, '1-50', 1), (2, 2, '2-2', 2), (3, 2, '3-2', 1), (5, 2, '5-2', 2), (8, 2, '8-2', 1), (13, 2, '13-2', 2), (21, 2, '21-2', 1); statement ok FLUSH; diff --git a/e2e_test/source/basic/ddl.slt b/e2e_test/source/basic/ddl.slt index 6e640e047d4c2..402cf129b86ba 100644 --- a/e2e_test/source/basic/ddl.slt +++ b/e2e_test/source/basic/ddl.slt @@ -28,7 +28,8 @@ db error: ERROR: Failed to run the query Caused by these errors (recent errors listed first): 1: gRPC request to meta service failed: Internal error 2: failed to create source worker - 3: missing field `properties.bootstrap.server` + 3: failed to parse json + 4: missing field `properties.bootstrap.server` statement error diff --git a/e2e_test/streaming/rate_limit.slt b/e2e_test/streaming/rate_limit/basic.slt similarity index 100% rename from e2e_test/streaming/rate_limit.slt rename to e2e_test/streaming/rate_limit/basic.slt diff --git a/e2e_test/streaming/rate_limit/upstream_amplification.slt b/e2e_test/streaming/rate_limit/upstream_amplification.slt new file mode 100644 index 0000000000000..71be801a78fc2 --- /dev/null +++ b/e2e_test/streaming/rate_limit/upstream_amplification.slt @@ -0,0 +1,44 @@ +# This test will test that barrier latency does not spike +# when there's rate limit on source. +# The upstream side should backpressure the source reader, +# but still allow barriers to flow through. + +statement ok +SET STREAMING_PARALLELISM=2; + +statement ok +SET STREAMING_RATE_LIMIT=1; + +statement ok +CREATE TABLE source_table (i1 int) +WITH ( + connector = 'datagen', + fields.i1.start = '1', + fields.i1.end = '5', + datagen.rows.per.second = '10000' +) FORMAT PLAIN ENCODE JSON; + +statement ok +CREATE SINK sink AS + SELECT x.i1 as i1 FROM source_table x + JOIN source_table s1 ON x.i1 = s1.i1 + JOIN source_table s2 ON x.i1 = s2.i1 + JOIN source_table s3 ON x.i1 = s3.i1 + WITH (connector = 'blackhole'); + +# The following sequence of FLUSH should be fast, since barrier should be able to bypass sink. +# Otherwise, these FLUSH will take a long time to complete, and trigger timeout. +statement ok +flush; + +statement ok +flush; + +statement ok +flush; + +statement ok +drop sink sink; + +statement ok +drop table source_table; \ No newline at end of file diff --git a/integration_tests/starrocks-sink/docker-compose.yml b/integration_tests/starrocks-sink/docker-compose.yml index 4210206aa7705..81ef7c277dad0 100644 --- a/integration_tests/starrocks-sink/docker-compose.yml +++ b/integration_tests/starrocks-sink/docker-compose.yml @@ -2,7 +2,7 @@ version: "3" services: starrocks-fe: - image: starrocks/fe-ubuntu:latest + image: starrocks/fe-ubuntu:3.1.7 hostname: starrocks-fe container_name: starrocks-fe volumes: @@ -19,7 +19,7 @@ services: timeout: 5s retries: 30 starrocks-be: - image: starrocks/be-ubuntu:latest + image: starrocks/be-ubuntu:3.1.7 command: - /bin/bash - -c @@ -27,6 +27,7 @@ services: sleep 15s; mysql --connect-timeout 2 -h starrocks-fe -P9030 -uroot -e "alter system add backend \"starrocks-be:9050\";" /opt/starrocks/be/bin/start_be.sh ports: + - 9050:9050 - 8040:8040 hostname: starrocks-be container_name: starrocks-be diff --git a/lints/src/format_error.rs b/lints/src/format_error.rs index 0d1df649460e8..8dcbed8cb520d 100644 --- a/lints/src/format_error.rs +++ b/lints/src/format_error.rs @@ -16,7 +16,7 @@ use clippy_utils::diagnostics::span_lint_and_help; use clippy_utils::macros::{ find_format_arg_expr, find_format_args, is_format_macro, macro_backtrace, }; -use clippy_utils::ty::implements_trait; +use clippy_utils::ty::{implements_trait, match_type}; use clippy_utils::{ is_in_cfg_test, is_in_test_function, is_trait_method, match_def_path, match_function_call, }; @@ -64,6 +64,7 @@ const TRACING_FIELD_DEBUG: [&str; 3] = ["tracing_core", "field", "debug"]; const TRACING_FIELD_DISPLAY: [&str; 3] = ["tracing_core", "field", "display"]; const TRACING_MACROS_EVENT: [&str; 3] = ["tracing", "macros", "event"]; const ANYHOW_MACROS_ANYHOW: [&str; 3] = ["anyhow", "macros", "anyhow"]; +const ANYHOW_ERROR: [&str; 2] = ["anyhow", "Error"]; impl<'tcx> LateLintPass<'tcx> for FormatError { fn check_expr(&mut self, cx: &LateContext<'tcx>, expr: &'tcx Expr<'_>) { @@ -143,7 +144,10 @@ fn check_fmt_arg_in_anyhow_error(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { check_fmt_arg_with_help( cx, arg_expr, - "consider directly wrapping the error with `anyhow::anyhow!(..)` instead of formatting it", + ( + "consider directly wrapping the error with `anyhow::anyhow!(..)` instead of formatting it", + "consider removing the redundant wrapping of `anyhow::anyhow!(..)`", + ), ); } @@ -151,12 +155,16 @@ fn check_fmt_arg_in_anyhow_context(cx: &LateContext<'_>, arg_expr: &Expr<'_>) { check_fmt_arg_with_help( cx, arg_expr, - "consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to \ + ( + "consider using `anyhow::Context::(with_)context` to \ attach additional message to the error and make it an error source instead", + "consider using `.context(..)` to \ + attach additional message to the error and make it an error source instead", + ), ); } -fn check_fmt_arg_with_help(cx: &LateContext<'_>, arg_expr: &Expr<'_>, help: &str) { +fn check_fmt_arg_with_help(cx: &LateContext<'_>, arg_expr: &Expr<'_>, help: impl Help) { check_arg(cx, arg_expr, arg_expr.span, help); } @@ -169,27 +177,56 @@ fn check_to_string_call(cx: &LateContext<'_>, receiver: &Expr<'_>, to_string_spa ); } -fn check_arg(cx: &LateContext<'_>, arg_expr: &Expr<'_>, span: Span, help: &str) { +fn check_arg(cx: &LateContext<'_>, arg_expr: &Expr<'_>, span: Span, help: impl Help) { let Some(error_trait_id) = cx.tcx.get_diagnostic_item(sym::Error) else { return; }; let ty = cx.typeck_results().expr_ty(arg_expr).peel_refs(); - if implements_trait(cx, ty, error_trait_id, &[]) { - if let Some(span) = core::iter::successors(Some(span), |s| s.parent_callsite()) - .find(|s| s.can_be_used_for_suggestions()) - { - // TODO: applicable suggestions - span_lint_and_help( - cx, - FORMAT_ERROR, - span, - "should not format error directly", - None, - help, - ); - } + let help = if implements_trait(cx, ty, error_trait_id, &[]) { + help.normal_help() + } else if match_type(cx, ty, &ANYHOW_ERROR) { + help.anyhow_help() + } else { + return; + }; + + if let Some(span) = core::iter::successors(Some(span), |s| s.parent_callsite()) + .find(|s| s.can_be_used_for_suggestions()) + { + // TODO: applicable suggestions + span_lint_and_help( + cx, + FORMAT_ERROR, + span, + "should not format error directly", + None, + help, + ); + } +} + +trait Help { + fn normal_help(&self) -> &str; + fn anyhow_help(&self) -> &str { + self.normal_help() + } +} + +impl Help for &str { + fn normal_help(&self) -> &str { + self + } +} + +impl Help for (&str, &str) { + fn normal_help(&self) -> &str { + self.0 + } + + fn anyhow_help(&self) -> &str { + self.1 } } diff --git a/lints/src/lib.rs b/lints/src/lib.rs index d2c78515272f4..df77538d3cf17 100644 --- a/lints/src/lib.rs +++ b/lints/src/lib.rs @@ -14,6 +14,7 @@ #![feature(rustc_private)] #![feature(let_chains)] +#![feature(lazy_cell)] #![warn(unused_extern_crates)] extern crate rustc_ast; diff --git a/lints/ui/format_error.rs b/lints/ui/format_error.rs index eeead1306ea3f..0e46c72766157 100644 --- a/lints/ui/format_error.rs +++ b/lints/ui/format_error.rs @@ -55,4 +55,22 @@ fn main() { let _ = anyhow!("{:?}", err); let _ = anyhow!("some error occurred: {}", err); let _ = anyhow!("some error occurred: {:?}", err); + + // `anyhow::Error` does not implement `Error` trait, test the special path here. + let make_anyhow_err = || anyhow!("foobar"); + let anyhow_err = make_anyhow_err(); + + let _ = format!("{}", anyhow_err); + let _ = format!("{}", &anyhow_err); + let _ = format!("{}", &&anyhow_err); + let _ = format!("{}", Box::new(&anyhow_err)); // TODO: fail to lint + + tracing::field::display(&anyhow_err); + tracing::field::debug(make_anyhow_err()); + + let _ = anyhow_err.to_string(); + let _ = (&&anyhow_err).to_string(); + + let _ = anyhow!("{}", anyhow_err); + let _ = anyhow!("some error occurred: {:?}", anyhow_err); } diff --git a/lints/ui/format_error.stderr b/lints/ui/format_error.stderr index 8ec6e69b7fcf4..0eb4786380a79 100644 --- a/lints/ui/format_error.stderr +++ b/lints/ui/format_error.stderr @@ -262,7 +262,7 @@ error: should not format error directly LL | let _ = anyhow!("some error occurred: {}", err); | ^^^ | - = help: consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead + = help: consider using `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead error: should not format error directly --> $DIR/format_error.rs:57:50 @@ -270,7 +270,79 @@ error: should not format error directly LL | let _ = anyhow!("some error occurred: {:?}", err); | ^^^ | - = help: consider using `anyhow::Error::context`, `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead + = help: consider using `anyhow::Context::(with_)context` to attach additional message to the error and make it an error source instead -error: aborting due to 34 previous errors +error: should not format error directly + --> $DIR/format_error.rs:63:27 + | +LL | let _ = format!("{}", anyhow_err); + | ^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:64:27 + | +LL | let _ = format!("{}", &anyhow_err); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:65:27 + | +LL | let _ = format!("{}", &&anyhow_err); + | ^^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:68:29 + | +LL | tracing::field::display(&anyhow_err); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:69:27 + | +LL | tracing::field::debug(make_anyhow_err()); + | ^^^^^^^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and recording the error as a field with `error = %.as_report()` instead + +error: should not format error directly + --> $DIR/format_error.rs:71:24 + | +LL | let _ = anyhow_err.to_string(); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.to_report_string()` instead + +error: should not format error directly + --> $DIR/format_error.rs:72:28 + | +LL | let _ = (&&anyhow_err).to_string(); + | ^^^^^^^^^^^ + | + = help: consider importing `thiserror_ext::AsReport` and using `.to_report_string()` instead + +error: should not format error directly + --> $DIR/format_error.rs:74:27 + | +LL | let _ = anyhow!("{}", anyhow_err); + | ^^^^^^^^^^ + | + = help: consider removing the redundant wrapping of `anyhow::anyhow!(..)` + +error: should not format error directly + --> $DIR/format_error.rs:75:50 + | +LL | let _ = anyhow!("some error occurred: {:?}", anyhow_err); + | ^^^^^^^^^^ + | + = help: consider using `.context(..)` to attach additional message to the error and make it an error source instead + +error: aborting due to 43 previous errors diff --git a/proto/plan_common.proto b/proto/plan_common.proto index 1dd45ad08a6ef..79a1b1622704e 100644 --- a/proto/plan_common.proto +++ b/proto/plan_common.proto @@ -54,10 +54,8 @@ message ColumnDesc { // This field is used to represent the connector-spec additional column type. // UNSPECIFIED or unset for normal column. - // deprecated, use AdditionalColumn instead - // AdditionalColumnType additional_column_type = 9; - reserved "additional_column_type"; - reserved 9; + // deprecated, use AdditionalColumn instead, keep for compatibility with v1.6.x + AdditionalColumnType additional_column_type = 9; ColumnDescVersion version = 10; @@ -218,3 +216,14 @@ message AdditionalColumn { AdditionalColumnHeaders headers = 7; } } + +enum AdditionalColumnType { + ADDITIONAL_COLUMN_TYPE_UNSPECIFIED = 0; + ADDITIONAL_COLUMN_TYPE_KEY = 1; + ADDITIONAL_COLUMN_TYPE_TIMESTAMP = 2; + ADDITIONAL_COLUMN_TYPE_PARTITION = 3; + ADDITIONAL_COLUMN_TYPE_OFFSET = 4; + ADDITIONAL_COLUMN_TYPE_HEADER = 5; + ADDITIONAL_COLUMN_TYPE_FILENAME = 6; + ADDITIONAL_COLUMN_TYPE_NORMAL = 7; +} diff --git a/src/batch/src/error.rs b/src/batch/src/error.rs index 5631707e2f422..f4d7341cbc6dd 100644 --- a/src/batch/src/error.rs +++ b/src/batch/src/error.rs @@ -20,6 +20,7 @@ pub use anyhow::anyhow; use risingwave_common::array::ArrayError; use risingwave_common::error::BoxedError; use risingwave_common::util::value_encoding::error::ValueEncodingError; +use risingwave_connector::error::ConnectorError; use risingwave_dml::error::DmlError; use risingwave_expr::ExprError; use risingwave_pb::PbFieldNotFound; @@ -156,3 +157,9 @@ impl From for Status { Self::from(&err) } } + +impl From for BatchError { + fn from(value: ConnectorError) -> Self { + Self::Connector(value.into()) + } +} diff --git a/src/batch/src/executor/iceberg_scan.rs b/src/batch/src/executor/iceberg_scan.rs index caf1289220d48..9c24e554c8e1f 100644 --- a/src/batch/src/executor/iceberg_scan.rs +++ b/src/batch/src/executor/iceberg_scan.rs @@ -116,11 +116,7 @@ impl IcebergScanExecutor { #[try_stream(ok = DataChunk, error = BatchError)] async fn do_execute(self: Box) { - let table = self - .iceberg_config - .load_table() - .await - .map_err(BatchError::Internal)?; + let table = self.iceberg_config.load_table().await?; let table_scan: TableScan = table .new_scan_builder() diff --git a/src/batch/src/executor/source.rs b/src/batch/src/executor/source.rs index e67d4b26f850d..fe053ff63dfc8 100644 --- a/src/batch/src/executor/source.rs +++ b/src/batch/src/executor/source.rs @@ -87,6 +87,7 @@ impl BoxedExecutorBuilder for SourceExecutor { }; let source_ctrl_opts = SourceCtrlOpts { chunk_size: source.context().get_config().developer.chunk_size, + rate_limit: None, }; let column_ids: Vec<_> = source_node diff --git a/src/cmd/src/lib.rs b/src/cmd/src/lib.rs index ce110c9effc17..13d5fcad5ec8c 100644 --- a/src/cmd/src/lib.rs +++ b/src/cmd/src/lib.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use risingwave_common::error::v2::AsReport as _; use risingwave_compactor::CompactorOpts; use risingwave_compute::ComputeNodeOpts; use risingwave_ctl::CliOpts as CtlOpts; @@ -67,13 +68,12 @@ pub fn ctl(opts: CtlOpts) { // Note: Use a simple current thread runtime for ctl. // When there's a heavy workload, multiple thread runtime seems to respond slowly. May need // further investigation. - tokio::runtime::Builder::new_current_thread() + if let Err(e) = tokio::runtime::Builder::new_current_thread() .enable_all() .build() .unwrap() .block_on(risingwave_ctl::start(opts)) - .inspect_err(|e| { - eprintln!("{:#?}", e); - }) - .unwrap(); + { + eprintln!("Error: {:#?}", e.as_report()); + } } diff --git a/src/cmd_all/Cargo.toml b/src/cmd_all/Cargo.toml index bb57fbfe88a09..c5f193ef8a2a3 100644 --- a/src/cmd_all/Cargo.toml +++ b/src/cmd_all/Cargo.toml @@ -58,6 +58,7 @@ workspace-hack = { path = "../workspace-hack" } expect-test = "1" [build-dependencies] +thiserror-ext = { workspace = true } vergen = { version = "8", default-features = false, features = [ "build", "git", diff --git a/src/cmd_all/build.rs b/src/cmd_all/build.rs index a4a7c27e65685..38d9f2d7107a6 100644 --- a/src/cmd_all/build.rs +++ b/src/cmd_all/build.rs @@ -12,11 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +use thiserror_ext::AsReport; use vergen::EmitBuilder; fn main() { if let Err(e) = EmitBuilder::builder().git_sha(true).fail_on_error().emit() { // Leave the environment variable unset if error occurs. - println!("cargo:warning={}", e) + println!("cargo:warning={}", e.as_report()) } } diff --git a/src/common/src/catalog/column.rs b/src/common/src/catalog/column.rs index f82e96a80c0e2..82d2f22f41cb4 100644 --- a/src/common/src/catalog/column.rs +++ b/src/common/src/catalog/column.rs @@ -170,6 +170,7 @@ impl ColumnDesc { type_name: self.type_name.clone(), generated_or_default_column: self.generated_or_default_column.clone(), description: self.description.clone(), + additional_column_type: 0, // deprecated additional_column: Some(self.additional_column.clone()), version: self.version as i32, } @@ -305,6 +306,7 @@ impl From<&ColumnDesc> for PbColumnDesc { type_name: c.type_name.clone(), generated_or_default_column: c.generated_or_default_column.clone(), description: c.description.clone(), + additional_column_type: 0, // deprecated additional_column: c.additional_column.clone().into(), version: c.version as i32, } diff --git a/src/common/src/catalog/test_utils.rs b/src/common/src/catalog/test_utils.rs index 9930a5717b849..ae87b3a881f84 100644 --- a/src/common/src/catalog/test_utils.rs +++ b/src/common/src/catalog/test_utils.rs @@ -60,6 +60,7 @@ impl ColumnDescTestExt for ColumnDesc { field_descs: fields, generated_or_default_column: None, description: None, + additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, } diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 971fb28d208c2..b1415a00b1362 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -391,9 +391,9 @@ impl<'de> Deserialize<'de> for DefaultParallelism { VirtualNode::COUNT )))? } else { - NonZeroUsize::new(i) - .context("default parallelism should be greater than 0") - .map_err(|e| serde::de::Error::custom(e.to_string()))? + NonZeroUsize::new(i).ok_or_else(|| { + serde::de::Error::custom("default parallelism should be greater than 0") + })? })), } } diff --git a/src/common/src/field_generator/mod.rs b/src/common/src/field_generator/mod.rs index 0309798068854..679d60ba1f188 100644 --- a/src/common/src/field_generator/mod.rs +++ b/src/common/src/field_generator/mod.rs @@ -18,6 +18,7 @@ mod varchar; use std::time::Duration; +// TODO(error-handling): use a new error type use anyhow::{anyhow, Result}; use chrono::{DateTime, FixedOffset}; pub use numeric::*; diff --git a/src/common/src/types/datetime.rs b/src/common/src/types/datetime.rs index af6b54b057c82..c609017d06e3f 100644 --- a/src/common/src/types/datetime.rs +++ b/src/common/src/types/datetime.rs @@ -328,17 +328,15 @@ impl ToText for Date { /// ``` fn write(&self, f: &mut W) -> std::fmt::Result { let (ce, year) = self.0.year_ce(); - if ce { - write!(f, "{}", self.0) - } else { - write!( - f, - "{:04}-{:02}-{:02} BC", - year, - self.0.month(), - self.0.day() - ) - } + let suffix = if ce { "" } else { " BC" }; + write!( + f, + "{:04}-{:02}-{:02}{}", + year, + self.0.month(), + self.0.day(), + suffix + ) } fn write_with_type(&self, ty: &DataType, f: &mut W) -> std::fmt::Result { @@ -364,7 +362,17 @@ impl ToText for Time { impl ToText for Timestamp { fn write(&self, f: &mut W) -> std::fmt::Result { - write!(f, "{}", self.0) + let (ce, year) = self.0.year_ce(); + let suffix = if ce { "" } else { " BC" }; + write!( + f, + "{:04}-{:02}-{:02} {}{}", + year, + self.0.month(), + self.0.day(), + self.0.time(), + suffix + ) } fn write_with_type(&self, ty: &DataType, f: &mut W) -> std::fmt::Result { diff --git a/src/common/src/types/timestamptz.rs b/src/common/src/types/timestamptz.rs index 41359bdf84377..f14d5d2edee6e 100644 --- a/src/common/src/types/timestamptz.rs +++ b/src/common/src/types/timestamptz.rs @@ -17,7 +17,7 @@ use std::io::Write; use std::str::FromStr; use bytes::{Bytes, BytesMut}; -use chrono::{TimeZone, Utc}; +use chrono::{DateTime, Datelike, TimeZone, Utc}; use chrono_tz::Tz; use postgres_types::{accepts, to_sql_checked, IsNull, ToSql, Type}; use serde::{Deserialize, Serialize}; @@ -201,6 +201,26 @@ impl std::fmt::Display for Timestamptz { } } +pub fn write_date_time_tz( + instant_local: DateTime, + writer: &mut impl std::fmt::Write, +) -> std::fmt::Result { + let date = instant_local.date_naive(); + let (ce, year) = date.year_ce(); + write!( + writer, + "{:04}-{:02}-{:02} {}", + year, + date.month(), + date.day(), + instant_local.format(if ce { + "%H:%M:%S%.f%:z" + } else { + "%H:%M:%S%.f%:z BC" + }) + ) +} + #[cfg(test)] mod test { use super::*; diff --git a/src/compute/tests/cdc_tests.rs b/src/compute/tests/cdc_tests.rs index 7fc3641770a9e..30849c40d4f84 100644 --- a/src/compute/tests/cdc_tests.rs +++ b/src/compute/tests/cdc_tests.rs @@ -44,31 +44,22 @@ use risingwave_stream::error::StreamResult; use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::executor::test_utils::MockSource; use risingwave_stream::executor::{ - expect_first_barrier, ActorContext, AddMutation, Barrier, BoxedExecutor as StreamBoxedExecutor, - BoxedMessageStream, CdcBackfillExecutor, Executor, ExecutorInfo, ExternalStorageTable, - MaterializeExecutor, Message, Mutation, PkIndices, PkIndicesRef, StreamExecutorError, + expect_first_barrier, ActorContext, AddMutation, Barrier, BoxedMessageStream, + CdcBackfillExecutor, Execute, Executor as StreamExecutor, ExecutorInfo, ExternalStorageTable, + MaterializeExecutor, Message, Mutation, StreamExecutorError, }; // mock upstream binlog offset starting from "1.binlog, pos=0" pub struct MockOffsetGenExecutor { - upstream: Option, - - schema: Schema, - - pk_indices: PkIndices, - - identity: String, + upstream: Option, start_offset: u32, } impl MockOffsetGenExecutor { - pub fn new(upstream: StreamBoxedExecutor, schema: Schema, pk_indices: PkIndices) -> Self { + pub fn new(upstream: StreamExecutor) -> Self { Self { upstream: Some(upstream), - schema, - pk_indices, - identity: "MockOffsetGenExecutor".to_string(), start_offset: 0, } } @@ -131,22 +122,10 @@ impl MockOffsetGenExecutor { } } -impl Executor for MockOffsetGenExecutor { +impl Execute for MockOffsetGenExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.pk_indices - } - - fn identity(&self) -> &str { - &self.identity - } } #[tokio::test] @@ -154,21 +133,26 @@ async fn test_cdc_backfill() -> StreamResult<()> { use risingwave_common::types::DataType; let memory_state_store = MemoryStateStore::new(); - let table_id = TableId::new(1002); - let schema = Schema::new(vec![ - Field::unnamed(DataType::Jsonb), // payload - Field::unnamed(DataType::Varchar), // _rw_offset - ]); - let column_ids = vec![0.into(), 1.into()]; - - let pk_indices = vec![0]; - - let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); - let _actor_ctx = ActorContext::for_test(0x3a3a3a); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor( + Schema::new(vec![ + Field::unnamed(DataType::Jsonb), // payload + ]), + vec![0], + ); // mock upstream offset (start from "1.binlog, pos=0") for ingested chunks - let mock_offset_executor = - MockOffsetGenExecutor::new(Box::new(source), schema.clone(), pk_indices.clone()); + let mock_offset_executor = StreamExecutor::new( + ExecutorInfo { + schema: Schema::new(vec![ + Field::unnamed(DataType::Jsonb), // payload + Field::unnamed(DataType::Varchar), // _rw_offset + ]), + pk_indices: vec![0], + identity: "MockOffsetGenExecutor".to_string(), + }, + MockOffsetGenExecutor::new(source).boxed(), + ); let binlog_file = String::from("1.binlog"); // mock binlog watermarks for backfill @@ -188,13 +172,15 @@ async fn test_cdc_backfill() -> StreamResult<()> { Field::with_name(DataType::Int64, "id"), // primary key Field::with_name(DataType::Float64, "price"), ]); + let table_pk_indices = vec![0]; + let table_pk_order_types = vec![OrderType::ascending()]; let external_table = ExternalStorageTable::new( - table_id, + TableId::new(1234), table_name, ExternalTableReaderImpl::Mock(MockExternalTableReader::new(binlog_watermarks)), table_schema.clone(), - vec![OrderType::ascending()], - pk_indices, + table_pk_order_types, + table_pk_indices.clone(), vec![0, 1], ); @@ -224,32 +210,35 @@ async fn test_cdc_backfill() -> StreamResult<()> { vec![0_usize], ) .await; - let info = ExecutorInfo { - schema: table_schema.clone(), - pk_indices: vec![0], - identity: "CdcBackfillExecutor".to_string(), - }; - let cdc_backfill = CdcBackfillExecutor::new( - ActorContext::for_test(actor_id), - info, - external_table, - Box::new(mock_offset_executor), - vec![0, 1], - None, - Arc::new(StreamingMetrics::unused()), - state_table, - 4, // 4 rows in a snapshot chunk - false, + + let cdc_backfill = StreamExecutor::new( + ExecutorInfo { + schema: table_schema.clone(), + pk_indices: table_pk_indices, + identity: "CdcBackfillExecutor".to_string(), + }, + CdcBackfillExecutor::new( + ActorContext::for_test(actor_id), + external_table, + mock_offset_executor, + vec![0, 1], + None, + Arc::new(StreamingMetrics::unused()), + state_table, + 4, // 4 rows in a snapshot chunk + false, + ) + .boxed(), ); // Create a `MaterializeExecutor` to write the changes to storage. + let materialize_table_id = TableId::new(5678); let mut materialize = MaterializeExecutor::for_test( - Box::new(cdc_backfill), + cdc_backfill, memory_state_store.clone(), - table_id, + materialize_table_id, vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids.clone(), - 4, + vec![0.into(), 1.into()], Arc::new(AtomicU64::new(0)), ConflictBehavior::Overwrite, ) @@ -354,7 +343,7 @@ async fn test_cdc_backfill() -> StreamResult<()> { // Since we have not polled `Materialize`, we cannot scan anything from this table let table = StorageTable::for_test( memory_state_store.clone(), - table_id, + materialize_table_id, column_descs.clone(), vec![OrderType::ascending()], vec![0], diff --git a/src/compute/tests/integration_tests.rs b/src/compute/tests/integration_tests.rs index 21fbe90b514e0..5e9fb3dfea86a 100644 --- a/src/compute/tests/integration_tests.rs +++ b/src/compute/tests/integration_tests.rs @@ -56,7 +56,7 @@ use risingwave_stream::executor::monitor::StreamingMetrics; use risingwave_stream::executor::row_id_gen::RowIdGenExecutor; use risingwave_stream::executor::source_executor::SourceExecutor; use risingwave_stream::executor::{ - ActorContext, Barrier, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices, + ActorContext, Barrier, Execute, Executor, ExecutorInfo, MaterializeExecutor, Message, PkIndices, }; use tokio::sync::mpsc::unbounded_channel; @@ -162,56 +162,58 @@ async fn test_table_materialize() -> StreamResult<()> { let system_params_manager = LocalSystemParamsManager::for_test(); // Create a `SourceExecutor` to read the changes. - let source_executor = SourceExecutor::::new( - actor_ctx.clone(), + let source_executor = Executor::new( ExecutorInfo { schema: all_schema.clone(), pk_indices: pk_indices.clone(), identity: format!("SourceExecutor {:X}", 1), }, - None, // There is no external stream source. - Arc::new(StreamingMetrics::unused()), - barrier_rx, - system_params_manager.get_params(), - SourceCtrlOpts::default(), - ConnectorParams::default(), + SourceExecutor::::new( + actor_ctx.clone(), + None, // There is no external stream source. + Arc::new(StreamingMetrics::unused()), + barrier_rx, + system_params_manager.get_params(), + SourceCtrlOpts::default(), + ConnectorParams::default(), + ) + .boxed(), ); // Create a `DmlExecutor` to accept data change from users. - let dml_executor = DmlExecutor::new( + let dml_executor = Executor::new( ExecutorInfo { schema: all_schema.clone(), pk_indices: pk_indices.clone(), identity: format!("DmlExecutor {:X}", 2), }, - Box::new(source_executor), - dml_manager.clone(), - table_id, - INITIAL_TABLE_VERSION_ID, - column_descs.clone(), - 1024, + DmlExecutor::new( + source_executor, + dml_manager.clone(), + table_id, + INITIAL_TABLE_VERSION_ID, + column_descs.clone(), + 1024, + ) + .boxed(), ); - let row_id_gen_executor = RowIdGenExecutor::new( - actor_ctx, + let row_id_gen_executor = Executor::new( ExecutorInfo { schema: all_schema.clone(), pk_indices: pk_indices.clone(), identity: format!("RowIdGenExecutor {:X}", 3), }, - Box::new(dml_executor), - row_id_index, - vnodes, + RowIdGenExecutor::new(actor_ctx, dml_executor, row_id_index, vnodes).boxed(), ); // Create a `MaterializeExecutor` to write the changes to storage. let mut materialize = MaterializeExecutor::for_test( - Box::new(row_id_gen_executor), + row_id_gen_executor, memory_state_store.clone(), table_id, vec![ColumnOrder::new(0, OrderType::ascending())], all_column_ids.clone(), - 4, Arc::new(AtomicU64::new(0)), ConflictBehavior::NoCheck, ) diff --git a/src/connector/src/aws_utils.rs b/src/connector/src/aws_utils.rs index cf70a90e07cda..1578c7b844422 100644 --- a/src/connector/src/aws_utils.rs +++ b/src/connector/src/aws_utils.rs @@ -21,6 +21,7 @@ use aws_sdk_s3::{client as s3_client, config as s3_config}; use url::Url; use crate::common::AwsAuthProps; +use crate::error::ConnectorResult; const AWS_CUSTOM_CONFIG_KEY: [&str; 3] = ["retry_times", "conn_timeout", "read_timeout"]; @@ -106,7 +107,7 @@ pub fn s3_client( pub async fn load_file_descriptor_from_s3( location: &Url, config: &AwsAuthProps, -) -> anyhow::Result> { +) -> ConnectorResult> { let bucket = location .domain() .with_context(|| format!("illegal file path {}", location))?; diff --git a/src/connector/src/common.rs b/src/connector/src/common.rs index 418155250e74e..d5944eb07fa3c 100644 --- a/src/connector/src/common.rs +++ b/src/connector/src/common.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::io::Write; use std::time::Duration; -use anyhow::{anyhow, Context, Ok}; +use anyhow::{anyhow, Context}; use async_nats::jetstream::consumer::DeliverPolicy; use async_nats::jetstream::{self}; use aws_sdk_kinesis::Client as KinesisClient; @@ -35,6 +35,7 @@ use with_options::WithOptions; use crate::aws_utils::load_file_descriptor_from_s3; use crate::deserialize_duration_from_string; +use crate::error::ConnectorResult; use crate::sink::SinkError; use crate::source::nats::source::NatsOffset; // The file describes the common abstractions for each connector and can be used in both source and @@ -72,7 +73,7 @@ pub struct AwsAuthProps { } impl AwsAuthProps { - async fn build_region(&self) -> anyhow::Result { + async fn build_region(&self) -> ConnectorResult { if let Some(region_name) = &self.region { Ok(Region::new(region_name.clone())) } else { @@ -85,11 +86,11 @@ impl AwsAuthProps { .build() .region() .await - .ok_or_else(|| anyhow::format_err!("region should be provided"))?) + .context("region should be provided")?) } } - fn build_credential_provider(&self) -> anyhow::Result { + fn build_credential_provider(&self) -> ConnectorResult { if self.access_key.is_some() && self.secret_key.is_some() { Ok(SharedCredentialsProvider::new( aws_credential_types::Credentials::from_keys( @@ -99,16 +100,14 @@ impl AwsAuthProps { ), )) } else { - Err(anyhow!( - "Both \"access_key\" and \"secret_access\" are required." - )) + bail!("Both \"access_key\" and \"secret_access\" are required.") } } async fn with_role_provider( &self, credential: SharedCredentialsProvider, - ) -> anyhow::Result { + ) -> ConnectorResult { if let Some(role_name) = &self.arn { let region = self.build_region().await?; let mut role = AssumeRoleProvider::builder(role_name) @@ -124,7 +123,7 @@ impl AwsAuthProps { } } - pub async fn build_config(&self) -> anyhow::Result { + pub async fn build_config(&self) -> ConnectorResult { let region = self.build_region().await?; let credentials_provider = self .with_role_provider(self.build_credential_provider()?) @@ -386,12 +385,19 @@ pub struct PulsarOauthCommon { pub scope: Option, } +fn create_credential_temp_file(credentials: &[u8]) -> std::io::Result { + let mut f = NamedTempFile::new()?; + f.write_all(credentials)?; + f.as_file().sync_all()?; + Ok(f) +} + impl PulsarCommon { pub(crate) async fn build_client( &self, oauth: &Option, aws_auth_props: &AwsAuthProps, - ) -> anyhow::Result> { + ) -> ConnectorResult> { let mut pulsar_builder = Pulsar::builder(&self.service_url, TokioExecutor); let mut temp_file = None; if let Some(oauth) = oauth.as_ref() { @@ -399,10 +405,10 @@ impl PulsarCommon { match url.scheme() { "s3" => { let credentials = load_file_descriptor_from_s3(&url, aws_auth_props).await?; - let mut f = NamedTempFile::new()?; - f.write_all(&credentials)?; - f.as_file().sync_all()?; - temp_file = Some(f); + temp_file = Some( + create_credential_temp_file(&credentials) + .context("failed to create temp file for pulsar credentials")?, + ); } "file" => {} _ => { @@ -477,7 +483,7 @@ pub struct KinesisCommon { } impl KinesisCommon { - pub(crate) async fn build_client(&self) -> anyhow::Result { + pub(crate) async fn build_client(&self) -> ConnectorResult { let config = AwsAuthProps { region: Some(self.stream_region.clone()), endpoint: self.endpoint.clone(), @@ -539,7 +545,7 @@ pub struct NatsCommon { } impl NatsCommon { - pub(crate) async fn build_client(&self) -> anyhow::Result { + pub(crate) async fn build_client(&self) -> ConnectorResult { let mut connect_options = async_nats::ConnectOptions::new(); match self.connect_mode.as_str() { "user_and_password" => { @@ -582,7 +588,7 @@ impl NatsCommon { Ok(client) } - pub(crate) async fn build_context(&self) -> anyhow::Result { + pub(crate) async fn build_context(&self) -> ConnectorResult { let client = self.build_client().await?; let jetstream = async_nats::jetstream::new(client); Ok(jetstream) @@ -593,7 +599,7 @@ impl NatsCommon { stream: String, split_id: String, start_sequence: NatsOffset, - ) -> anyhow::Result< + ) -> ConnectorResult< async_nats::jetstream::consumer::Consumer, > { let context = self.build_context().await?; @@ -612,13 +618,16 @@ impl NatsCommon { NatsOffset::Earliest => DeliverPolicy::All, NatsOffset::Latest => DeliverPolicy::Last, NatsOffset::SequenceNumber(v) => { - let parsed = v.parse::()?; + let parsed = v + .parse::() + .context("failed to parse nats offset as sequence number")?; DeliverPolicy::ByStartSequence { start_sequence: 1 + parsed, } } NatsOffset::Timestamp(v) => DeliverPolicy::ByStartTime { - start_time: OffsetDateTime::from_unix_timestamp_nanos(v * 1_000_000)?, + start_time: OffsetDateTime::from_unix_timestamp_nanos(v * 1_000_000) + .context("invalid timestamp for nats offset")?, }, NatsOffset::None => DeliverPolicy::All, }; @@ -635,7 +644,7 @@ impl NatsCommon { &self, jetstream: jetstream::Context, stream: String, - ) -> anyhow::Result { + ) -> ConnectorResult { let subjects: Vec = self.subject.split(',').map(|s| s.to_string()).collect(); let mut config = jetstream::stream::Config { name: stream, @@ -662,7 +671,7 @@ impl NatsCommon { Ok(stream) } - pub(crate) fn create_credential(&self, seed: &str, jwt: &str) -> anyhow::Result { + pub(crate) fn create_credential(&self, seed: &str, jwt: &str) -> ConnectorResult { let creds = format!( "-----BEGIN NATS USER JWT-----\n{}\n------END NATS USER JWT------\n\n\ ************************* IMPORTANT *************************\n\ diff --git a/src/connector/src/error.rs b/src/connector/src/error.rs index 4cf36e9859d36..3dc10af3d8e7a 100644 --- a/src/connector/src/error.rs +++ b/src/connector/src/error.rs @@ -13,13 +13,57 @@ // limitations under the License. use risingwave_common::error::v2::def_anyhow_newtype; +use risingwave_pb::PbFieldNotFound; +use risingwave_rpc_client::error::RpcError; + +use crate::parser::AccessError; +use crate::schema::schema_registry::{ConcurrentRequestError, WireFormatError}; +use crate::schema::InvalidOptionError; +use crate::sink::SinkError; def_anyhow_newtype! { pub ConnectorError, + // Common errors + std::io::Error => transparent, + + // Fine-grained connector errors + AccessError => transparent, + WireFormatError => transparent, + ConcurrentRequestError => transparent, + InvalidOptionError => transparent, + SinkError => transparent, + PbFieldNotFound => transparent, + // TODO(error-handling): Remove implicit contexts below and specify ad-hoc context for each conversion. + + // Parsing errors + url::ParseError => "failed to parse url", + serde_json::Error => "failed to parse json", + csv::Error => "failed to parse csv", + + // Connector errors + opendal::Error => transparent, // believed to be self-explanatory + mysql_async::Error => "MySQL error", tokio_postgres::Error => "Postgres error", + apache_avro::Error => "Avro error", + rdkafka::error::KafkaError => "Kafka error", + pulsar::Error => "Pulsar error", + async_nats::jetstream::consumer::StreamError => "Nats error", + async_nats::jetstream::consumer::pull::MessagesError => "Nats error", + async_nats::jetstream::context::CreateStreamError => "Nats error", + async_nats::jetstream::stream::ConsumerError => "Nats error", + icelake::Error => "Iceberg error", + redis::RedisError => "Redis error", + arrow_schema::ArrowError => "Arrow error", + google_cloud_pubsub::client::google_cloud_auth::error::Error => "Google Cloud error", } -pub type ConnectorResult = Result; +pub type ConnectorResult = std::result::Result; + +impl From for RpcError { + fn from(value: ConnectorError) -> Self { + RpcError::Internal(value.0) + } +} diff --git a/src/connector/src/macros.rs b/src/connector/src/macros.rs index e34171717ae6c..d4a546c6a00a7 100644 --- a/src/connector/src/macros.rs +++ b/src/connector/src/macros.rs @@ -168,12 +168,12 @@ macro_rules! impl_split { $( impl TryFrom for $split { - type Error = anyhow::Error; + type Error = $crate::error::ConnectorError; fn try_from(split: SplitImpl) -> std::result::Result { match split { SplitImpl::$variant_name(inner) => Ok(inner), - other => Err(anyhow::anyhow!("expect {} but get {:?}", stringify!($split), other)) + other => risingwave_common::bail!("expect {} but get {:?}", stringify!($split), other), } } } diff --git a/src/connector/src/parser/additional_columns.rs b/src/connector/src/parser/additional_columns.rs index c1da30f788b3e..06cc061566690 100644 --- a/src/connector/src/parser/additional_columns.rs +++ b/src/connector/src/parser/additional_columns.rs @@ -27,6 +27,7 @@ use risingwave_pb::plan_common::{ AdditionalColumnTimestamp, }; +use crate::error::ConnectorResult; use crate::source::{ GCS_CONNECTOR, KAFKA_CONNECTOR, KINESIS_CONNECTOR, OPENDAL_S3_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR, @@ -86,7 +87,7 @@ pub fn build_additional_column_catalog( inner_field_name: Option<&str>, data_type: Option<&str>, reject_unknown_connector: bool, -) -> anyhow::Result { +) -> ConnectorResult { let compatible_columns = match ( COMPATIBLE_ADDITIONAL_COLUMNS.get(connector_name), reject_unknown_connector, diff --git a/src/connector/src/parser/avro/parser.rs b/src/connector/src/parser/avro/parser.rs index 5e876d2ce9324..7343f1c43118c 100644 --- a/src/connector/src/parser/avro/parser.rs +++ b/src/connector/src/parser/avro/parser.rs @@ -23,6 +23,7 @@ use risingwave_pb::plan_common::ColumnDesc; use super::schema_resolver::ConfluentSchemaResolver; use super::util::avro_schema_to_column_descs; +use crate::error::ConnectorResult; use crate::parser::unified::avro::{AvroAccess, AvroParseOptions}; use crate::parser::unified::AccessImpl; use crate::parser::util::bytes_from_url; @@ -40,7 +41,7 @@ pub struct AvroAccessBuilder { } impl AccessBuilder for AvroAccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { self.value = self.parse_avro_value(&payload, Some(&*self.schema)).await?; Ok(AccessImpl::Avro(AvroAccess::new( self.value.as_ref().unwrap(), @@ -50,7 +51,7 @@ impl AccessBuilder for AvroAccessBuilder { } impl AvroAccessBuilder { - pub fn new(config: AvroParserConfig, encoding_type: EncodingType) -> anyhow::Result { + pub fn new(config: AvroParserConfig, encoding_type: EncodingType) -> ConnectorResult { let AvroParserConfig { schema, key_schema, @@ -71,7 +72,7 @@ impl AvroAccessBuilder { &self, payload: &[u8], reader_schema: Option<&Schema>, - ) -> anyhow::Result> { + ) -> ConnectorResult> { // parse payload to avro value // if use confluent schema, get writer schema from confluent schema registry if let Some(resolver) = &self.schema_resolver { @@ -87,9 +88,7 @@ impl AvroAccessBuilder { match reader.next() { Some(Ok(v)) => Ok(Some(v)), Some(Err(e)) => Err(e)?, - None => { - anyhow::bail!("avro parse unexpected eof") - } + None => bail!("avro parse unexpected eof"), } } else { unreachable!("both schema_resolver and reader_schema not exist"); @@ -105,7 +104,7 @@ pub struct AvroParserConfig { } impl AvroParserConfig { - pub async fn new(encoding_properties: EncodingProperties) -> anyhow::Result { + pub async fn new(encoding_properties: EncodingProperties) -> ConnectorResult { let avro_config = try_match_expand!(encoding_properties, EncodingProperties::Avro)?; let schema_location = &avro_config.row_schema_location; let enable_upsert = avro_config.enable_upsert; @@ -160,7 +159,7 @@ impl AvroParserConfig { } } - pub fn extract_pks(&self) -> anyhow::Result> { + pub fn extract_pks(&self) -> ConnectorResult> { avro_schema_to_column_descs( self.key_schema .as_deref() @@ -168,7 +167,7 @@ impl AvroParserConfig { ) } - pub fn map_to_columns(&self) -> anyhow::Result> { + pub fn map_to_columns(&self) -> ConnectorResult> { avro_schema_to_column_descs(self.schema.as_ref()) } } @@ -196,6 +195,7 @@ mod test { use super::*; use crate::common::AwsAuthProps; + use crate::error::ConnectorResult; use crate::parser::plain_parser::PlainParser; use crate::parser::unified::avro::unix_epoch_days; use crate::parser::{ @@ -256,7 +256,7 @@ mod test { println!("schema = {:?}", schema.unwrap()); } - async fn new_avro_conf_from_local(file_name: &str) -> anyhow::Result { + async fn new_avro_conf_from_local(file_name: &str) -> ConnectorResult { let schema_path = "file://".to_owned() + &test_data_path(file_name); let info = StreamSourceInfo { row_schema_location: schema_path.clone(), @@ -269,7 +269,7 @@ mod test { AvroParserConfig::new(parser_config.encoding_config).await } - async fn new_avro_parser_from_local(file_name: &str) -> anyhow::Result { + async fn new_avro_parser_from_local(file_name: &str) -> ConnectorResult { let conf = new_avro_conf_from_local(file_name).await?; Ok(PlainParser { diff --git a/src/connector/src/parser/avro/schema_resolver.rs b/src/connector/src/parser/avro/schema_resolver.rs index ef2dd9fc5f731..cdc52de7accee 100644 --- a/src/connector/src/parser/avro/schema_resolver.rs +++ b/src/connector/src/parser/avro/schema_resolver.rs @@ -18,6 +18,7 @@ use anyhow::Context; use apache_avro::Schema; use moka::future::Cache; +use crate::error::ConnectorResult; use crate::schema::schema_registry::{Client, ConfluentSchema}; #[derive(Debug)] @@ -30,7 +31,7 @@ impl ConfluentSchemaResolver { async fn parse_and_cache_schema( &self, raw_schema: ConfluentSchema, - ) -> anyhow::Result> { + ) -> ConnectorResult> { let schema = Schema::parse_str(&raw_schema.content).context("failed to parse avro schema")?; let schema = Arc::new(schema); @@ -48,7 +49,7 @@ impl ConfluentSchemaResolver { } } - pub async fn get_by_subject_name(&self, subject_name: &str) -> anyhow::Result> { + pub async fn get_by_subject_name(&self, subject_name: &str) -> ConnectorResult> { let raw_schema = self.get_raw_schema_by_subject_name(subject_name).await?; self.parse_and_cache_schema(raw_schema).await } @@ -56,7 +57,7 @@ impl ConfluentSchemaResolver { pub async fn get_raw_schema_by_subject_name( &self, subject_name: &str, - ) -> anyhow::Result { + ) -> ConnectorResult { self.confluent_client .get_schema_by_subject(subject_name) .await @@ -64,7 +65,7 @@ impl ConfluentSchemaResolver { } // get the writer schema by id - pub async fn get(&self, schema_id: i32) -> anyhow::Result> { + pub async fn get(&self, schema_id: i32) -> ConnectorResult> { // TODO: use `get_with` if let Some(schema) = self.writer_schemas.get(&schema_id).await { Ok(schema) diff --git a/src/connector/src/parser/avro/util.rs b/src/connector/src/parser/avro/util.rs index 8d2d4265883e6..958f4c9ca5db5 100644 --- a/src/connector/src/parser/avro/util.rs +++ b/src/connector/src/parser/avro/util.rs @@ -16,20 +16,23 @@ use std::sync::LazyLock; use apache_avro::schema::{DecimalSchema, RecordSchema, Schema}; use itertools::Itertools; +use risingwave_common::bail; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{DataType, Decimal}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; -pub fn avro_schema_to_column_descs(schema: &Schema) -> anyhow::Result> { +use crate::error::ConnectorResult; + +pub fn avro_schema_to_column_descs(schema: &Schema) -> ConnectorResult> { if let Schema::Record(RecordSchema { fields, .. }) = schema { let mut index = 0; let fields = fields .iter() .map(|field| avro_field_to_column_desc(&field.name, &field.schema, &mut index)) - .collect::>>()?; + .collect::>>()?; Ok(fields) } else { - anyhow::bail!("schema invalid, record type required at top level of the schema."); + bail!("schema invalid, record type required at top level of the schema."); } } @@ -40,7 +43,7 @@ fn avro_field_to_column_desc( name: &str, schema: &Schema, index: &mut i32, -) -> anyhow::Result { +) -> ConnectorResult { let data_type = avro_type_mapping(schema)?; match schema { Schema::Record(RecordSchema { @@ -51,7 +54,7 @@ fn avro_field_to_column_desc( let vec_column = fields .iter() .map(|f| avro_field_to_column_desc(&f.name, &f.schema, index)) - .collect::>>()?; + .collect::>>()?; *index += 1; Ok(ColumnDesc { column_type: Some(data_type.to_protobuf()), @@ -61,6 +64,7 @@ fn avro_field_to_column_desc( type_name: schema_name.to_string(), generated_or_default_column: None, description: None, + additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, }) @@ -79,7 +83,7 @@ fn avro_field_to_column_desc( } } -fn avro_type_mapping(schema: &Schema) -> anyhow::Result { +fn avro_type_mapping(schema: &Schema) -> ConnectorResult { let data_type = match schema { Schema::String => DataType::Varchar, Schema::Int => DataType::Int32, @@ -122,7 +126,7 @@ fn avro_type_mapping(schema: &Schema) -> anyhow::Result { let struct_fields = fields .iter() .map(|f| avro_type_mapping(&f.schema)) - .collect::>>()?; + .collect::>>()?; let struct_names = fields.iter().map(|f| f.name.clone()).collect_vec(); DataType::new_struct(struct_fields, struct_names) } @@ -147,18 +151,10 @@ fn avro_type_mapping(schema: &Schema) -> anyhow::Result { { DataType::Decimal } else { - return Err(anyhow::format_err!( - "unsupported type in Avro: {:?}", - schema - )); + bail!("unsupported type in Avro: {:?}", schema); } } - _ => { - return Err(anyhow::format_err!( - "unsupported type in Avro: {:?}", - schema - )); - } + _ => bail!("unsupported type in Avro: {:?}", schema), }; Ok(data_type) diff --git a/src/connector/src/parser/bytes_parser.rs b/src/connector/src/parser/bytes_parser.rs index 44c035fa3ff5d..4f353ce2c60e6 100644 --- a/src/connector/src/parser/bytes_parser.rs +++ b/src/connector/src/parser/bytes_parser.rs @@ -17,6 +17,7 @@ use risingwave_common::try_match_expand; use super::unified::bytes::BytesAccess; use super::unified::AccessImpl; use super::{AccessBuilder, EncodingProperties}; +use crate::error::ConnectorResult; #[derive(Debug)] pub struct BytesAccessBuilder { @@ -25,7 +26,7 @@ pub struct BytesAccessBuilder { impl AccessBuilder for BytesAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { Ok(AccessImpl::Bytes(BytesAccess::new( &self.column_name, payload, @@ -34,7 +35,7 @@ impl AccessBuilder for BytesAccessBuilder { } impl BytesAccessBuilder { - pub fn new(encoding_properties: EncodingProperties) -> anyhow::Result { + pub fn new(encoding_properties: EncodingProperties) -> ConnectorResult { let config = try_match_expand!(encoding_properties, EncodingProperties::Bytes)?; Ok(Self { column_name: config.column_name, diff --git a/src/connector/src/parser/canal/simd_json_parser.rs b/src/connector/src/parser/canal/simd_json_parser.rs index 09a00c490a9f5..75e6656fd7a7a 100644 --- a/src/connector/src/parser/canal/simd_json_parser.rs +++ b/src/connector/src/parser/canal/simd_json_parser.rs @@ -18,6 +18,7 @@ use risingwave_common::bail; use simd_json::prelude::{MutableObject, ValueAsScalar, ValueObjectAccess}; use simd_json::BorrowedValue; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::canal::operators::*; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; @@ -44,7 +45,7 @@ impl CanalJsonParser { rw_columns: Vec, source_ctx: SourceContextRef, config: &JsonProperties, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self { rw_columns, source_ctx, @@ -57,9 +58,10 @@ impl CanalJsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let mut event: BorrowedValue<'_> = - simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..])?; + simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) + .context("failed to parse canal json payload")?; let is_ddl = event .get(IS_DDL) @@ -123,7 +125,7 @@ impl ByteStreamSourceParser for CanalJsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/csv_parser.rs b/src/connector/src/parser/csv_parser.rs index 7bb67c9d7b510..8a8bb211da327 100644 --- a/src/connector/src/parser/csv_parser.rs +++ b/src/connector/src/parser/csv_parser.rs @@ -16,6 +16,7 @@ use risingwave_common::types::{Date, Decimal, Time, Timestamp, Timestamptz}; use super::unified::{AccessError, AccessResult}; use super::{ByteStreamSourceParser, CsvProperties}; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::{ParserFormat, SourceStreamChunkRowWriter}; use crate::source::{DataType, SourceColumnDesc, SourceContext, SourceContextRef}; @@ -44,7 +45,7 @@ impl CsvParser { rw_columns: Vec, csv_props: CsvProperties, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let CsvProperties { delimiter, has_header, @@ -58,7 +59,7 @@ impl CsvParser { }) } - fn read_row(&self, buf: &[u8]) -> anyhow::Result> { + fn read_row(&self, buf: &[u8]) -> ConnectorResult> { let mut reader_builder = csv::ReaderBuilder::default(); reader_builder.delimiter(self.delimiter).has_headers(false); let record = reader_builder @@ -102,7 +103,7 @@ impl CsvParser { &mut self, payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let mut fields = self.read_row(&payload)?; if let Some(headers) = &mut self.headers { @@ -158,7 +159,7 @@ impl ByteStreamSourceParser for CsvParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/debezium/avro_parser.rs b/src/connector/src/parser/debezium/avro_parser.rs index 6320ef5cdf3ec..ca1574af3d6b2 100644 --- a/src/connector/src/parser/debezium/avro_parser.rs +++ b/src/connector/src/parser/debezium/avro_parser.rs @@ -21,6 +21,7 @@ use risingwave_common::try_match_expand; use risingwave_pb::catalog::PbSchemaRegistryNameStrategy; use risingwave_pb::plan_common::ColumnDesc; +use crate::error::ConnectorResult; use crate::parser::avro::schema_resolver::ConfluentSchemaResolver; use crate::parser::avro::util::avro_schema_to_column_descs; use crate::parser::unified::avro::{ @@ -48,7 +49,7 @@ pub struct DebeziumAvroAccessBuilder { // TODO: reduce encodingtype match impl AccessBuilder for DebeziumAvroAccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { let (schema_id, mut raw_payload) = extract_schema_id(&payload)?; let schema = self.schema_resolver.get(schema_id).await?; self.value = Some(from_avro_datum(schema.as_ref(), &mut raw_payload, None)?); @@ -70,7 +71,7 @@ impl DebeziumAvroAccessBuilder { pub fn new( config: DebeziumAvroParserConfig, encoding_type: EncodingType, - ) -> anyhow::Result { + ) -> ConnectorResult { let DebeziumAvroParserConfig { outer_schema, schema_resolver, @@ -99,7 +100,7 @@ pub struct DebeziumAvroParserConfig { } impl DebeziumAvroParserConfig { - pub async fn new(encoding_config: EncodingProperties) -> anyhow::Result { + pub async fn new(encoding_config: EncodingProperties) -> ConnectorResult { let avro_config = try_match_expand!(encoding_config, EncodingProperties::Avro)?; let schema_location = &avro_config.row_schema_location; let client_config = &avro_config.client_config; @@ -121,11 +122,11 @@ impl DebeziumAvroParserConfig { }) } - pub fn extract_pks(&self) -> anyhow::Result> { + pub fn extract_pks(&self) -> ConnectorResult> { avro_schema_to_column_descs(&self.key_schema) } - pub fn map_to_columns(&self) -> anyhow::Result> { + pub fn map_to_columns(&self) -> ConnectorResult> { avro_schema_to_column_descs(avro_schema_skip_union(avro_extract_field_schema( &self.outer_schema, Some("before"), @@ -348,7 +349,7 @@ mod tests { #[ignore] #[tokio::test] - async fn test_debezium_avro_parser() -> anyhow::Result<()> { + async fn test_debezium_avro_parser() -> crate::error::ConnectorResult<()> { let props = convert_args!(hashmap!( "kafka.topic" => "dbserver1.inventory.customers" )); diff --git a/src/connector/src/parser/debezium/debezium_parser.rs b/src/connector/src/parser/debezium/debezium_parser.rs index f69c19e691e97..d9c824882e1ea 100644 --- a/src/connector/src/parser/debezium/debezium_parser.rs +++ b/src/connector/src/parser/debezium/debezium_parser.rs @@ -16,6 +16,7 @@ use risingwave_common::bail; use super::simd_json_parser::DebeziumJsonAccessBuilder; use super::{DebeziumAvroAccessBuilder, DebeziumAvroParserConfig}; +use crate::error::ConnectorResult; use crate::extract_key_config; use crate::parser::unified::debezium::DebeziumChangeEvent; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; @@ -37,7 +38,7 @@ pub struct DebeziumParser { async fn build_accessor_builder( config: EncodingProperties, encoding_type: EncodingType, -) -> anyhow::Result { +) -> ConnectorResult { match config { EncodingProperties::Avro(_) => { let config = DebeziumAvroParserConfig::new(config).await?; @@ -60,7 +61,7 @@ impl DebeziumParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let (key_config, key_type) = extract_key_config!(props); let key_builder = build_accessor_builder(key_config, key_type).await?; let payload_builder = @@ -73,7 +74,7 @@ impl DebeziumParser { }) } - pub async fn new_for_test(rw_columns: Vec) -> anyhow::Result { + pub async fn new_for_test(rw_columns: Vec) -> ConnectorResult { let props = SpecificParserConfig { key_encoding_config: None, encoding_config: EncodingProperties::Json(JsonProperties { @@ -89,7 +90,7 @@ impl DebeziumParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result { + ) -> ConnectorResult { // tombetone messages are handled implicitly by these accessors let key_accessor = match key { None => None, @@ -137,7 +138,7 @@ impl ByteStreamSourceParser for DebeziumParser { _key: Option>, _payload: Option>, _writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { unreachable!("should call `parse_one_with_txn` instead") } @@ -146,7 +147,7 @@ impl ByteStreamSourceParser for DebeziumParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result { + ) -> ConnectorResult { self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/debezium/mongo_json_parser.rs b/src/connector/src/parser/debezium/mongo_json_parser.rs index 74c8a4ee8cfbc..aa4263ace7cbc 100644 --- a/src/connector/src/parser/debezium/mongo_json_parser.rs +++ b/src/connector/src/parser/debezium/mongo_json_parser.rs @@ -20,6 +20,7 @@ use risingwave_common::types::DataType; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::unified::debezium::{DebeziumChangeEvent, MongoProjection}; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; @@ -39,7 +40,7 @@ impl DebeziumMongoJsonParser { pub fn new( rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let id_column = rw_columns .iter() .find(|desc| { @@ -78,8 +79,9 @@ impl DebeziumMongoJsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { - let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload)?; + ) -> ConnectorResult<()> { + let mut event: BorrowedValue<'_> = simd_json::to_borrowed_value(&mut payload) + .context("failed to parse debezium mongo json payload")?; // Event can be configured with and without the "payload" field present. // See https://github.com/risingwavelabs/risingwave/issues/10178 @@ -115,7 +117,7 @@ impl ByteStreamSourceParser for DebeziumMongoJsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/debezium/simd_json_parser.rs b/src/connector/src/parser/debezium/simd_json_parser.rs index 3ad3609c78063..bc516ebce4f23 100644 --- a/src/connector/src/parser/debezium/simd_json_parser.rs +++ b/src/connector/src/parser/debezium/simd_json_parser.rs @@ -14,9 +14,11 @@ use std::fmt::Debug; +use anyhow::Context; use simd_json::prelude::MutableObject; use simd_json::BorrowedValue; +use crate::error::ConnectorResult; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; use crate::parser::unified::AccessImpl; use crate::parser::AccessBuilder; @@ -27,17 +29,18 @@ pub struct DebeziumJsonAccessBuilder { } impl DebeziumJsonAccessBuilder { - pub fn new() -> anyhow::Result { + pub fn new() -> ConnectorResult { Ok(Self { value: None }) } } impl AccessBuilder for DebeziumJsonAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { self.value = Some(payload); let mut event: BorrowedValue<'_> = - simd_json::to_borrowed_value(self.value.as_mut().unwrap())?; + simd_json::to_borrowed_value(self.value.as_mut().unwrap()) + .context("failed to parse debezium json payload")?; let payload = if let Some(payload) = event.get_mut("payload") { std::mem::take(payload) @@ -64,12 +67,14 @@ mod tests { DataType, Date, Interval, Scalar, ScalarImpl, StructType, Time, Timestamp, }; use serde_json::Value; + use thiserror_ext::AsReport; use crate::parser::{ DebeziumParser, EncodingProperties, JsonProperties, ProtocolProperties, SourceColumnDesc, SourceStreamChunkBuilder, SpecificParserConfig, }; use crate::source::SourceContextRef; + fn assert_json_eq(parse_result: &Option, json_str: &str) { if let Some(ScalarImpl::Jsonb(json_val)) = parse_result { let mut json_string = String::new(); @@ -491,7 +496,7 @@ mod tests { } else { // For f64 overflow, the parsing fails let e = res.unwrap_err(); - assert!(e.to_string().contains("InvalidNumber"), "{i}: {e}"); + assert!(e.to_report_string().contains("InvalidNumber"), "{i}: {e}"); } } } diff --git a/src/connector/src/parser/json_parser.rs b/src/connector/src/parser/json_parser.rs index 74c82d14d8065..47db36a0b2b8f 100644 --- a/src/connector/src/parser/json_parser.rs +++ b/src/connector/src/parser/json_parser.rs @@ -24,6 +24,7 @@ use risingwave_pb::plan_common::ColumnDesc; use super::avro::schema_resolver::ConfluentSchemaResolver; use super::util::{bytes_from_url, get_kafka_topic}; use super::{EncodingProperties, SchemaRegistryAuth, SpecificParserConfig}; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::avro::util::avro_schema_to_column_descs; use crate::parser::unified::json::{JsonAccess, JsonParseOptions}; @@ -43,7 +44,7 @@ pub struct JsonAccessBuilder { impl AccessBuilder for JsonAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { if payload.is_empty() { self.value = Some("{}".into()); } else { @@ -51,7 +52,8 @@ impl AccessBuilder for JsonAccessBuilder { } let value = simd_json::to_borrowed_value( &mut self.value.as_mut().unwrap()[self.payload_start_idx..], - )?; + ) + .context("failed to parse json payload")?; Ok(AccessImpl::Json(JsonAccess::new_with_options( value, // Debezium and Canal have their special json access builder and will not @@ -62,7 +64,7 @@ impl AccessBuilder for JsonAccessBuilder { } impl JsonAccessBuilder { - pub fn new(use_schema_registry: bool) -> anyhow::Result { + pub fn new(use_schema_registry: bool) -> ConnectorResult { Ok(Self { value: None, payload_start_idx: if use_schema_registry { 5 } else { 0 }, @@ -84,7 +86,7 @@ impl JsonParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let json_config = try_match_expand!(props.encoding_config, EncodingProperties::Json)?; let payload_start_idx = if json_config.use_schema_registry { 5 @@ -98,7 +100,7 @@ impl JsonParser { }) } - pub fn new_for_test(rw_columns: Vec) -> anyhow::Result { + pub fn new_for_test(rw_columns: Vec) -> ConnectorResult { Ok(Self { rw_columns, source_ctx: Default::default(), @@ -111,8 +113,9 @@ impl JsonParser { &self, mut payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { - let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..])?; + ) -> ConnectorResult<()> { + let value = simd_json::to_borrowed_value(&mut payload[self.payload_start_idx..]) + .context("failed to parse json payload")?; let values = if let simd_json::BorrowedValue::Array(arr) = value { Either::Left(arr.into_iter()) } else { @@ -145,7 +148,7 @@ pub async fn schema_to_columns( schema_location: &str, schema_registry_auth: Option, props: &HashMap, -) -> anyhow::Result> { +) -> ConnectorResult> { let url = handle_sr_list(schema_location)?; let json_schema = if let Some(schema_registry_auth) = schema_registry_auth { let client = Client::new(url, &schema_registry_auth)?; @@ -185,7 +188,7 @@ impl ByteStreamSourceParser for JsonParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { only_parse_payload!(self, payload, writer) } } diff --git a/src/connector/src/parser/maxwell/maxwell_parser.rs b/src/connector/src/parser/maxwell/maxwell_parser.rs index aa661585fa1ff..8ba95ad212130 100644 --- a/src/connector/src/parser/maxwell/maxwell_parser.rs +++ b/src/connector/src/parser/maxwell/maxwell_parser.rs @@ -14,6 +14,7 @@ use risingwave_common::bail; +use crate::error::ConnectorResult; use crate::only_parse_payload; use crate::parser::unified::maxwell::MaxwellChangeEvent; use crate::parser::unified::util::apply_row_operation_on_stream_chunk_writer; @@ -35,7 +36,7 @@ impl MaxwellParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { match props.encoding_config { EncodingProperties::Json(_) => { let payload_builder = @@ -55,7 +56,7 @@ impl MaxwellParser { &mut self, payload: Vec, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let payload_accessor = self.payload_builder.generate_accessor(payload).await?; let row_op = MaxwellChangeEvent::new(payload_accessor); @@ -81,7 +82,7 @@ impl ByteStreamSourceParser for MaxwellParser { _key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { // restrict the behaviours since there is no corresponding // key/value test for maxwell yet. only_parse_payload!(self, payload, writer) diff --git a/src/connector/src/parser/mod.rs b/src/connector/src/parser/mod.rs index c5b470db966ab..721cfa2f241c7 100644 --- a/src/connector/src/parser/mod.rs +++ b/src/connector/src/parser/mod.rs @@ -44,10 +44,11 @@ pub use self::mysql::mysql_row_to_owned_row; use self::plain_parser::PlainParser; pub use self::postgres::postgres_row_to_owned_row; use self::simd_json_parser::DebeziumJsonAccessBuilder; -use self::unified::{AccessImpl, AccessResult}; +use self::unified::AccessImpl; use self::upsert_parser::UpsertParser; use self::util::get_kafka_topic; use crate::common::AwsAuthProps; +use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::maxwell::MaxwellParser; use crate::parser::util::{ extract_header_inner_from_meta, extract_headers_from_meta, extreact_timestamp_from_meta, @@ -56,7 +57,7 @@ use crate::schema::schema_registry::SchemaRegistryAuth; use crate::source::monitor::GLOBAL_SOURCE_METRICS; use crate::source::{ extract_source_struct, BoxSourceStream, ChunkSourceStream, SourceColumnDesc, SourceColumnType, - SourceContext, SourceContextRef, SourceEncode, SourceFormat, SourceMeta, + SourceContext, SourceContextRef, SourceEncode, SourceFormat, SourceMessage, SourceMeta, }; pub mod additional_columns; @@ -76,6 +77,8 @@ mod unified; mod upsert_parser; mod util; +pub use unified::{AccessError, AccessResult}; + /// A builder for building a [`StreamChunk`] from [`SourceColumnDesc`]. pub struct SourceStreamChunkBuilder { descs: Vec, @@ -536,7 +539,7 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a; + ) -> impl Future> + Send + 'a; /// Parse one record from the given `payload`, either write rows to the `writer` or interpret it /// as a transaction control message. @@ -550,12 +553,27 @@ pub trait ByteStreamSourceParser: Send + Debug + Sized + 'static { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> impl Future> + Send + 'a { + ) -> impl Future> + Send + 'a { self.parse_one(key, payload, writer) .map_ok(|_| ParseResult::Rows) } } +#[try_stream(ok = Vec, error = ConnectorError)] +async fn ensure_largest_at_rate_limit(stream: BoxSourceStream, rate_limit: u32) { + #[for_await] + for batch in stream { + let mut batch = batch?; + let mut start = 0; + let end = batch.len(); + while start < end { + let next = std::cmp::min(start + rate_limit as usize, end); + yield std::mem::take(&mut batch[start..next].as_mut()).to_vec(); + start = next; + } + } +} + #[easy_ext::ext(SourceParserIntoStreamExt)] impl P { /// Parse a data stream of one source split into a stream of [`StreamChunk`]. @@ -568,9 +586,15 @@ impl P { /// /// A [`ChunkSourceStream`] which is a stream of parsed messages. pub fn into_stream(self, data_stream: BoxSourceStream) -> impl ChunkSourceStream { - // Enable tracing to provide more information for parsing failures. let source_info = self.source_ctx().source_info.clone(); + // Ensure chunk size is smaller than rate limit + let data_stream = if let Some(rate_limit) = &self.source_ctx().source_ctrl_opts.rate_limit { + Box::pin(ensure_largest_at_rate_limit(data_stream, *rate_limit)) + } else { + data_stream + }; + // The parser stream will be long-lived. We use `instrument_with` here to create // a new span for the polling of each chunk. into_chunk_stream(self, data_stream).instrument_with(move || { @@ -589,7 +613,7 @@ const MAX_ROWS_FOR_TRANSACTION: usize = 4096; // TODO: when upsert is disabled, how to filter those empty payload // Currently, an err is returned for non upsert with empty payload -#[try_stream(ok = StreamChunk, error = anyhow::Error)] +#[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(mut parser: P, data_stream: BoxSourceStream) { let columns = parser.columns().to_vec(); @@ -679,14 +703,14 @@ async fn into_chunk_stream(mut parser: P, data_stream LazyLock::new(LogSuppresser::default); if let Ok(suppressed_count) = LOG_SUPPERSSER.check() { tracing::error!( - %error, + error = %error.as_report(), split_id = &*msg.split_id, offset = msg.offset, suppressed_count, "failed to parse message, skipping" ); } - parser.source_ctx().report_user_source_error(&*error); + parser.source_ctx().report_user_source_error(&error); } } @@ -729,7 +753,7 @@ async fn into_chunk_stream(mut parser: P, data_stream } pub trait AccessBuilder { - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result>; + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult>; } #[derive(Debug)] @@ -749,7 +773,10 @@ pub enum AccessBuilderImpl { } impl AccessBuilderImpl { - pub async fn new_default(config: EncodingProperties, kv: EncodingType) -> anyhow::Result { + pub async fn new_default( + config: EncodingProperties, + kv: EncodingType, + ) -> ConnectorResult { let accessor = match config { EncodingProperties::Avro(_) => { let config = AvroParserConfig::new(config).await?; @@ -773,7 +800,7 @@ impl AccessBuilderImpl { pub async fn generate_accessor( &mut self, payload: Vec, - ) -> anyhow::Result> { + ) -> ConnectorResult> { let accessor = match self { Self::Avro(builder) => builder.generate_accessor(payload).await?, Self::Protobuf(builder) => builder.generate_accessor(payload).await?, @@ -822,7 +849,7 @@ impl ByteStreamSourceParserImpl { pub async fn create( parser_config: ParserConfig, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let CommonParserConfig { rw_columns } = parser_config.common; let protocol = &parser_config.specific.protocol_config; let encode = &parser_config.specific.encoding_config; @@ -969,7 +996,7 @@ impl SpecificParserConfig { pub fn new( info: &StreamSourceInfo, with_properties: &HashMap, - ) -> anyhow::Result { + ) -> ConnectorResult { let source_struct = extract_source_struct(info)?; let format = source_struct.format; let encode = source_struct.encode; diff --git a/src/connector/src/parser/plain_parser.rs b/src/connector/src/parser/plain_parser.rs index 5fdb9fbf3c6ca..3b5460de1bfb8 100644 --- a/src/connector/src/parser/plain_parser.rs +++ b/src/connector/src/parser/plain_parser.rs @@ -18,6 +18,7 @@ use super::{ AccessBuilderImpl, ByteStreamSourceParser, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; +use crate::error::ConnectorResult; use crate::parser::bytes_parser::BytesAccessBuilder; use crate::parser::simd_json_parser::DebeziumJsonAccessBuilder; use crate::parser::unified::debezium::parse_transaction_meta; @@ -43,7 +44,7 @@ impl PlainParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { Some(AccessBuilderImpl::Bytes(BytesAccessBuilder::new( EncodingProperties::Bytes(BytesProperties { @@ -81,7 +82,7 @@ impl PlainParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result { + ) -> ConnectorResult { // if the message is transaction metadata, parse it and return if let Some(msg_meta) = writer.row_meta && let SourceMeta::DebeziumCdc(cdc_meta) = msg_meta.meta @@ -145,7 +146,7 @@ impl ByteStreamSourceParser for PlainParser { _key: Option>, _payload: Option>, _writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { unreachable!("should call `parse_one_with_txn` instead") } @@ -154,7 +155,7 @@ impl ByteStreamSourceParser for PlainParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result { + ) -> ConnectorResult { self.parse_inner(key, payload, writer).await } } @@ -262,7 +263,7 @@ mod tests { assert_eq!(1, output.len()); } - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn source_message_stream(transactional: bool) { let begin_msg = r#"{"schema":null,"payload":{"status":"BEGIN","id":"35352:3962948040","event_count":null,"data_collections":null,"ts_ms":1704269323180}}"#; let commit_msg = r#"{"schema":null,"payload":{"status":"END","id":"35352:3962950064","event_count":11,"data_collections":[{"data_collection":"public.orders_tx","event_count":5},{"data_collection":"public.person","event_count":6}],"ts_ms":1704269323180}}"#; diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 922705e3d3f8f..4248fa2b7470c 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -28,6 +28,7 @@ use thiserror::Error; use thiserror_ext::{AsReport, Macro}; use super::schema_resolver::*; +use crate::error::ConnectorResult; use crate::parser::unified::protobuf::ProtobufAccess; use crate::parser::unified::{ bail_uncategorized, uncategorized, AccessError, AccessImpl, AccessResult, @@ -47,7 +48,7 @@ pub struct ProtobufAccessBuilder { impl AccessBuilder for ProtobufAccessBuilder { #[allow(clippy::unused_async)] - async fn generate_accessor(&mut self, payload: Vec) -> anyhow::Result> { + async fn generate_accessor(&mut self, payload: Vec) -> ConnectorResult> { let payload = if self.confluent_wire_type { resolve_pb_header(&payload)? } else { @@ -65,7 +66,7 @@ impl AccessBuilder for ProtobufAccessBuilder { } impl ProtobufAccessBuilder { - pub fn new(config: ProtobufParserConfig) -> anyhow::Result { + pub fn new(config: ProtobufParserConfig) -> ConnectorResult { let ProtobufParserConfig { confluent_wire_type, message_descriptor, @@ -89,7 +90,7 @@ pub struct ProtobufParserConfig { } impl ProtobufParserConfig { - pub async fn new(encoding_properties: EncodingProperties) -> anyhow::Result { + pub async fn new(encoding_properties: EncodingProperties) -> ConnectorResult { let protobuf_config = try_match_expand!(encoding_properties, EncodingProperties::Protobuf)?; let location = &protobuf_config.row_schema_location; let message_name = &protobuf_config.message_name; @@ -133,7 +134,7 @@ impl ProtobufParserConfig { } /// Maps the protobuf schema to relational schema. - pub fn map_to_columns(&self) -> anyhow::Result> { + pub fn map_to_columns(&self) -> ConnectorResult> { let mut columns = Vec::with_capacity(self.message_descriptor.fields().len()); let mut index = 0; let mut parse_trace: Vec = vec![]; @@ -153,8 +154,9 @@ impl ProtobufParserConfig { field_descriptor: &FieldDescriptor, index: &mut i32, parse_trace: &mut Vec, - ) -> anyhow::Result { - let field_type = protobuf_type_mapping(field_descriptor, parse_trace)?; + ) -> ConnectorResult { + let field_type = protobuf_type_mapping(field_descriptor, parse_trace) + .context("failed to map protobuf type")?; if let Kind::Message(m) = field_descriptor.kind() { let field_descs = if let DataType::List { .. } = field_type { vec![] @@ -172,6 +174,7 @@ impl ProtobufParserConfig { type_name: m.full_name().to_string(), generated_or_default_column: None, description: None, + additional_column_type: 0, // deprecated additional_column: Some(AdditionalColumn { column_type: None }), version: ColumnDescVersion::Pr13707 as i32, }) @@ -525,7 +528,7 @@ fn protobuf_type_mapping( /// Wire format for Confluent pb header is: /// | 0 | 1-4 | 5-x | x+1-end /// | magic-byte | schema-id | message-indexes | protobuf-payload -pub(crate) fn resolve_pb_header(payload: &[u8]) -> anyhow::Result<&[u8]> { +pub(crate) fn resolve_pb_header(payload: &[u8]) -> ConnectorResult<&[u8]> { // there's a message index array at the front of payload // if it is the first message in proto def, the array is just and `0` // TODO: support parsing more complex index array @@ -575,7 +578,7 @@ mod test { static PRE_GEN_PROTO_DATA: &[u8] = b"\x08\x7b\x12\x0c\x74\x65\x73\x74\x20\x61\x64\x64\x72\x65\x73\x73\x1a\x09\x74\x65\x73\x74\x20\x63\x69\x74\x79\x20\xc8\x03\x2d\x19\x04\x9e\x3f\x32\x0a\x32\x30\x32\x31\x2d\x30\x31\x2d\x30\x31"; #[tokio::test] - async fn test_simple_schema() -> anyhow::Result<()> { + async fn test_simple_schema() -> crate::error::ConnectorResult<()> { let location = schema_dir() + "/simple-schema"; println!("location: {}", location); let message_name = "test.TestRecord"; @@ -620,7 +623,7 @@ mod test { } #[tokio::test] - async fn test_complex_schema() -> anyhow::Result<()> { + async fn test_complex_schema() -> crate::error::ConnectorResult<()> { let location = schema_dir() + "/complex-schema"; let message_name = "test.User"; @@ -912,7 +915,7 @@ mod test { static ANY_GEN_PROTO_DATA: &[u8] = b"\x08\xb9\x60\x12\x32\x0a\x24\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x53\x74\x72\x69\x6e\x67\x56\x61\x6c\x75\x65\x12\x0a\x0a\x08\x4a\x6f\x68\x6e\x20\x44\x6f\x65"; #[tokio::test] - async fn test_any_schema() -> anyhow::Result<()> { + async fn test_any_schema() -> crate::error::ConnectorResult<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); @@ -973,7 +976,7 @@ mod test { static ANY_GEN_PROTO_DATA_1: &[u8] = b"\x08\xb9\x60\x12\x2b\x0a\x23\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x49\x6e\x74\x33\x32\x56\x61\x6c\x75\x65\x12\x04\x08\xd2\xfe\x06"; #[tokio::test] - async fn test_any_schema_1() -> anyhow::Result<()> { + async fn test_any_schema_1() -> crate::error::ConnectorResult<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); @@ -1042,7 +1045,7 @@ mod test { static ANY_RECURSIVE_GEN_PROTO_DATA: &[u8] = b"\x08\xb9\x60\x12\x84\x01\x0a\x21\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x41\x6e\x79\x56\x61\x6c\x75\x65\x12\x5f\x0a\x30\x0a\x24\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x53\x74\x72\x69\x6e\x67\x56\x61\x6c\x75\x65\x12\x08\x0a\x06\x31\x31\x34\x35\x31\x34\x12\x2b\x0a\x23\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x49\x6e\x74\x33\x32\x56\x61\x6c\x75\x65\x12\x04\x08\xd2\xfe\x06"; #[tokio::test] - async fn test_any_recursive() -> anyhow::Result<()> { + async fn test_any_recursive() -> crate::error::ConnectorResult<()> { let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; println!("Current conf: {:#?}", conf); diff --git a/src/connector/src/parser/protobuf/schema_resolver.rs b/src/connector/src/parser/protobuf/schema_resolver.rs index 919413a0c50dc..828843842c785 100644 --- a/src/connector/src/parser/protobuf/schema_resolver.rs +++ b/src/connector/src/parser/protobuf/schema_resolver.rs @@ -22,6 +22,7 @@ use protobuf_native::compiler::{ }; use protobuf_native::MessageLite; +use crate::error::ConnectorResult; use crate::schema::schema_registry::Client; macro_rules! embed_wkts { @@ -54,7 +55,7 @@ const WELL_KNOWN_TYPES: &[(&str, &[u8])] = embed_wkts![ pub(super) async fn compile_file_descriptor_from_schema_registry( subject_name: &str, client: &Client, -) -> anyhow::Result> { +) -> ConnectorResult> { let (primary_subject, dependency_subjects) = client .get_subject_and_references(subject_name) .await diff --git a/src/connector/src/parser/unified/avro.rs b/src/connector/src/parser/unified/avro.rs index f00e26a32faed..f0825d9af4bbe 100644 --- a/src/connector/src/parser/unified/avro.rs +++ b/src/connector/src/parser/unified/avro.rs @@ -22,6 +22,7 @@ use chrono::Datelike; use itertools::Itertools; use num_bigint::{BigInt, Sign}; use risingwave_common::array::{ListValue, StructValue}; +use risingwave_common::bail; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{ DataType, Date, Datum, Interval, JsonbVal, ScalarImpl, Time, Timestamp, Timestamptz, @@ -29,6 +30,7 @@ use risingwave_common::types::{ use risingwave_common::util::iter_util::ZipEqFast; use super::{bail_uncategorized, uncategorized, Access, AccessError, AccessResult}; +use crate::error::ConnectorResult; #[derive(Clone)] /// Options for parsing an `AvroValue` into Datum, with an optional avro schema. pub struct AvroParseOptions<'a> { @@ -384,7 +386,7 @@ pub(crate) fn extract_decimal(bytes: Vec) -> AccessResult<(u32, u32, u32)> { } } -pub fn avro_schema_skip_union(schema: &Schema) -> anyhow::Result<&Schema> { +pub fn avro_schema_skip_union(schema: &Schema) -> ConnectorResult<&Schema> { match schema { Schema::Union(union_schema) => { let inner_schema = union_schema @@ -403,7 +405,7 @@ pub fn avro_schema_skip_union(schema: &Schema) -> anyhow::Result<&Schema> { pub fn avro_extract_field_schema<'a>( schema: &'a Schema, name: Option<&'a str>, -) -> anyhow::Result<&'a Schema> { +) -> ConnectorResult<&'a Schema> { match schema { Schema::Record(RecordSchema { fields, lookup, .. }) => { let name = @@ -418,7 +420,7 @@ pub fn avro_extract_field_schema<'a>( } Schema::Array(schema) => Ok(schema), Schema::Union(_) => avro_schema_skip_union(schema), - _ => Err(anyhow::format_err!("avro schema is not a record or array")), + _ => bail!("avro schema is not a record or array"), } } @@ -481,7 +483,7 @@ mod tests { value: Value, value_schema: &Schema, shape: &DataType, - ) -> anyhow::Result { + ) -> crate::error::ConnectorResult { AvroParseOptions { schema: Some(value_schema), relax_numeric: true, diff --git a/src/connector/src/parser/upsert_parser.rs b/src/connector/src/parser/upsert_parser.rs index b8abf4785bb4a..048fd0beca3ff 100644 --- a/src/connector/src/parser/upsert_parser.rs +++ b/src/connector/src/parser/upsert_parser.rs @@ -23,6 +23,7 @@ use super::{ AccessBuilderImpl, ByteStreamSourceParser, BytesProperties, EncodingProperties, EncodingType, SourceStreamChunkRowWriter, SpecificParserConfig, }; +use crate::error::ConnectorResult; use crate::parser::ParserFormat; use crate::source::{SourceColumnDesc, SourceContext, SourceContextRef}; @@ -37,7 +38,7 @@ pub struct UpsertParser { async fn build_accessor_builder( config: EncodingProperties, encoding_type: EncodingType, -) -> anyhow::Result { +) -> ConnectorResult { match config { EncodingProperties::Json(_) | EncodingProperties::Protobuf(_) @@ -66,7 +67,7 @@ impl UpsertParser { props: SpecificParserConfig, rw_columns: Vec, source_ctx: SourceContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { // check whether columns has Key as AdditionalColumnType, if so, the key accessor should be // bytes let key_builder = if let Some(key_column_name) = get_key_column_name(&rw_columns) { @@ -95,7 +96,7 @@ impl UpsertParser { key: Option>, payload: Option>, mut writer: SourceStreamChunkRowWriter<'_>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { let mut row_op: UpsertChangeEvent, AccessImpl<'_, '_>> = UpsertChangeEvent::default(); let mut change_event_op = ChangeEventOperation::Delete; @@ -133,7 +134,7 @@ impl ByteStreamSourceParser for UpsertParser { key: Option>, payload: Option>, writer: SourceStreamChunkRowWriter<'a>, - ) -> anyhow::Result<()> { + ) -> ConnectorResult<()> { self.parse_inner(key, payload, writer).await } } diff --git a/src/connector/src/parser/util.rs b/src/connector/src/parser/util.rs index ccb0e47310d9a..eeaa09bcc031d 100644 --- a/src/connector/src/parser/util.rs +++ b/src/connector/src/parser/util.rs @@ -22,10 +22,11 @@ use risingwave_pb::data::DataType as PbDataType; use crate::aws_utils::load_file_descriptor_from_s3; use crate::common::AwsAuthProps; +use crate::error::ConnectorResult; use crate::source::SourceMeta; /// get kafka topic name -pub(super) fn get_kafka_topic(props: &HashMap) -> anyhow::Result<&String> { +pub(super) fn get_kafka_topic(props: &HashMap) -> ConnectorResult<&String> { const KAFKA_TOPIC_KEY1: &str = "kafka.topic"; const KAFKA_TOPIC_KEY2: &str = "topic"; @@ -45,7 +46,7 @@ pub(super) fn get_kafka_topic(props: &HashMap) -> anyhow::Result } /// download bytes from http(s) url -pub(super) async fn download_from_http(location: &Url) -> anyhow::Result { +pub(super) async fn download_from_http(location: &Url) -> ConnectorResult { let res = reqwest::get(location.clone()) .await .with_context(|| format!("failed to make request to {location}"))? @@ -95,7 +96,7 @@ macro_rules! extract_key_config { pub(super) async fn bytes_from_url( url: &Url, config: Option<&AwsAuthProps>, -) -> anyhow::Result> { +) -> ConnectorResult> { match (url.scheme(), config) { // TODO(Tao): support local file only when it's compiled in debug mode. ("file", _) => { diff --git a/src/connector/src/schema/mod.rs b/src/connector/src/schema/mod.rs index 0f64ee85e57dd..8d2a9ae780572 100644 --- a/src/connector/src/schema/mod.rs +++ b/src/connector/src/schema/mod.rs @@ -12,6 +12,8 @@ // See the License for the specific language governing permissions and // limitations under the License. +use crate::error::ConnectorError; + pub mod avro; pub mod protobuf; pub mod schema_registry; @@ -46,6 +48,6 @@ pub enum SchemaFetchError { YetToMigrate( #[source] #[backtrace] - anyhow::Error, + ConnectorError, ), } diff --git a/src/connector/src/sink/clickhouse.rs b/src/connector/src/sink/clickhouse.rs index 2a9a2e5a39eb6..4e21659dae49e 100644 --- a/src/connector/src/sink/clickhouse.rs +++ b/src/connector/src/sink/clickhouse.rs @@ -30,6 +30,7 @@ use thiserror_ext::AsReport; use with_options::WithOptions; use super::{DummySinkCommitCoordinator, SinkWriterParam}; +use crate::error::ConnectorResult; use crate::sink::catalog::desc::SinkDesc; use crate::sink::log_store::DeliveryFutureManagerAddFuture; use crate::sink::writer::{ @@ -132,7 +133,7 @@ impl ClickHouseEngine { const POOL_IDLE_TIMEOUT: Duration = Duration::from_secs(5); impl ClickHouseCommon { - pub(crate) fn build_client(&self) -> anyhow::Result { + pub(crate) fn build_client(&self) -> ConnectorResult { use hyper_tls::HttpsConnector; let https = HttpsConnector::new(); @@ -261,9 +262,11 @@ impl ClickHouseSink { ) -> Result<()> { let is_match = match fields_type { risingwave_common::types::DataType::Boolean => Ok(ck_column.r#type.contains("Bool")), - risingwave_common::types::DataType::Int16 => { - Ok(ck_column.r#type.contains("UInt16") | ck_column.r#type.contains("Int16")) - } + risingwave_common::types::DataType::Int16 => Ok(ck_column.r#type.contains("UInt16") + | ck_column.r#type.contains("Int16") + // Allow Int16 to be pushed to Enum16, they share an encoding and value range + // No special care is taken to ensure values are valid. + | ck_column.r#type.contains("Enum16")), risingwave_common::types::DataType::Int32 => { Ok(ck_column.r#type.contains("UInt32") | ck_column.r#type.contains("Int32")) } diff --git a/src/connector/src/sink/iceberg/jni_catalog.rs b/src/connector/src/sink/iceberg/jni_catalog.rs index 11de50e69936a..08a876fb62ac8 100644 --- a/src/connector/src/sink/iceberg/jni_catalog.rs +++ b/src/connector/src/sink/iceberg/jni_catalog.rs @@ -29,6 +29,8 @@ use jni::JavaVM; use risingwave_jni_core::call_method; use risingwave_jni_core::jvm_runtime::{execute_with_jni_env, jobj_to_str, JVM}; +use crate::error::ConnectorResult; + pub struct JniCatalog { java_catalog: GlobalRef, jvm: &'static JavaVM, @@ -142,7 +144,7 @@ impl JniCatalog { name: impl ToString, catalog_impl: impl ToString, java_catalog_props: HashMap, - ) -> anyhow::Result { + ) -> ConnectorResult { let jvm = JVM.get_or_init()?; execute_with_jni_env(jvm, |env| { @@ -182,5 +184,6 @@ impl JniCatalog { config: base_config, }) as CatalogRef) }) + .map_err(Into::into) } } diff --git a/src/connector/src/sink/iceberg/mod.rs b/src/connector/src/sink/iceberg/mod.rs index 326f8586d76eb..0c3840af1f3bb 100644 --- a/src/connector/src/sink/iceberg/mod.rs +++ b/src/connector/src/sink/iceberg/mod.rs @@ -59,6 +59,7 @@ use super::{ Sink, SinkError, SinkWriterParam, SINK_TYPE_APPEND_ONLY, SINK_TYPE_OPTION, SINK_TYPE_UPSERT, }; use crate::deserialize_bool_from_string; +use crate::error::ConnectorResult; use crate::sink::coordinate::CoordinatedSinkWriter; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{Result, SinkCommitCoordinator, SinkParam}; @@ -350,16 +351,19 @@ impl IcebergConfig { Ok((base_catalog_config, java_catalog_configs)) } - async fn create_catalog(&self) -> anyhow::Result { + async fn create_catalog(&self) -> ConnectorResult { match self.catalog_type() { "storage" | "rest" => { let iceberg_configs = self.build_iceberg_configs()?; - let catalog = load_catalog(&iceberg_configs) - .await - .map_err(|e| anyhow!(e))?; + let catalog = load_catalog(&iceberg_configs).await?; Ok(catalog) } - catalog_type if catalog_type == "hive" || catalog_type == "sql" || catalog_type == "glue" || catalog_type == "dynamodb" => { + catalog_type + if catalog_type == "hive" + || catalog_type == "sql" + || catalog_type == "glue" + || catalog_type == "dynamodb" => + { // Create java catalog let (base_catalog_config, java_catalog_props) = self.build_jni_catalog_configs()?; let catalog_impl = match catalog_type { @@ -370,19 +374,24 @@ impl IcebergConfig { _ => unreachable!(), }; - jni_catalog::JniCatalog::build(base_catalog_config, "risingwave", catalog_impl, java_catalog_props) + jni_catalog::JniCatalog::build( + base_catalog_config, + "risingwave", + catalog_impl, + java_catalog_props, + ) } - "mock" => Ok(Arc::new(MockCatalog{})), + "mock" => Ok(Arc::new(MockCatalog {})), _ => { - Err(anyhow!( - "Unsupported catalog type: {}, only support `storage`, `rest`, `hive`, `sql`, `glue`, `dynamodb`", - self.catalog_type() - )) + bail!( + "Unsupported catalog type: {}, only support `storage`, `rest`, `hive`, `sql`, `glue`, `dynamodb`", + self.catalog_type() + ) } } } - pub async fn load_table(&self) -> anyhow::Result { + pub async fn load_table(&self) -> ConnectorResult
{ let catalog = self .create_catalog() .await @@ -395,10 +404,7 @@ impl IcebergConfig { ) .context("Unable to parse table name")?; - catalog - .load_table(&table_id) - .await - .map_err(|err| anyhow!(err)) + catalog.load_table(&table_id).await.map_err(Into::into) } } @@ -428,7 +434,11 @@ impl Debug for IcebergSink { impl IcebergSink { async fn create_and_validate_table(&self) -> Result
{ - let table = self.config.load_table().await.map_err(SinkError::Iceberg)?; + let table = self + .config + .load_table() + .await + .map_err(|err| SinkError::Iceberg(anyhow!(err)))?; let sink_schema = self.param.schema(); let iceberg_schema = table @@ -825,7 +835,7 @@ impl WriteResult { .collect::, icelake::Error>>() .unwrap(); } else { - return Err(anyhow!("icberg sink metadata should have data_files object").into()); + bail!("icberg sink metadata should have data_files object"); } if let serde_json::Value::Array(values) = values .remove(DELETE_FILES) @@ -837,14 +847,14 @@ impl WriteResult { .collect::, icelake::Error>>() .context("Failed to parse data file from json")?; } else { - return Err(anyhow!("icberg sink metadata should have data_files object").into()); + bail!("icberg sink metadata should have data_files object"); } Ok(Self { data_files, delete_files, }) } else { - Err(anyhow!("Can't create iceberg sink write result from empty data!").into()) + bail!("Can't create iceberg sink write result from empty data!") } } } diff --git a/src/connector/src/sink/kinesis.rs b/src/connector/src/sink/kinesis.rs index aca370d7ab1b3..54e9394b511db 100644 --- a/src/connector/src/sink/kinesis.rs +++ b/src/connector/src/sink/kinesis.rs @@ -176,7 +176,7 @@ impl KinesisSinkWriter { .common .build_client() .await - .map_err(SinkError::Kinesis)?; + .map_err(|err| SinkError::Kinesis(anyhow!(err)))?; Ok(Self { config: config.clone(), formatter, diff --git a/src/connector/src/sink/log_store.rs b/src/connector/src/sink/log_store.rs index f74a22d3b80e5..3879d817d1a84 100644 --- a/src/connector/src/sink/log_store.rs +++ b/src/connector/src/sink/log_store.rs @@ -19,9 +19,9 @@ use std::future::{poll_fn, Future}; use std::sync::Arc; use std::task::Poll; -use anyhow::anyhow; use futures::{TryFuture, TryFutureExt}; use risingwave_common::array::StreamChunk; +use risingwave_common::bail; use risingwave_common::buffer::Bitmap; use risingwave_common::util::epoch::{EpochPair, INVALID_EPOCH}; @@ -62,13 +62,13 @@ impl TruncateOffset { } } - pub fn check_next_offset(&self, next_offset: TruncateOffset) -> anyhow::Result<()> { + pub fn check_next_offset(&self, next_offset: TruncateOffset) -> LogStoreResult<()> { if *self >= next_offset { - Err(anyhow!( + bail!( "next offset {:?} should be later than current offset {:?}", next_offset, self - )) + ) } else { Ok(()) } @@ -81,22 +81,22 @@ impl TruncateOffset { .. } => { if epoch != *offset_epoch { - return Err(anyhow!( + bail!( "new item epoch {} not match current chunk offset epoch {}", epoch, offset_epoch - )); + ); } } TruncateOffset::Barrier { epoch: offset_epoch, } => { if epoch <= *offset_epoch { - return Err(anyhow!( + bail!( "new item epoch {} not exceed barrier offset epoch {}", epoch, offset_epoch - )); + ); } } } @@ -534,6 +534,7 @@ mod tests { use tokio::sync::oneshot; use tokio::sync::oneshot::Receiver; + use super::LogStoreResult; use crate::sink::log_store::{DeliveryFutureManager, TruncateOffset}; #[test] @@ -587,7 +588,7 @@ mod tests { } type TestFuture = impl TryFuture + Unpin + 'static; - fn to_test_future(rx: Receiver>) -> TestFuture { + fn to_test_future(rx: Receiver>) -> TestFuture { async move { rx.await.unwrap() }.boxed() } diff --git a/src/connector/src/sink/mod.rs b/src/connector/src/sink/mod.rs index fc6712f17604b..6abe8d93b5956 100644 --- a/src/connector/src/sink/mod.rs +++ b/src/connector/src/sink/mod.rs @@ -61,6 +61,7 @@ pub use tracing; use self::catalog::{SinkFormatDesc, SinkType}; use self::mock_coordination_client::{MockMetaClient, SinkCoordinationRpcClientEnum}; +use crate::error::ConnectorError; use crate::sink::catalog::desc::SinkDesc; use crate::sink::catalog::{SinkCatalog, SinkId}; use crate::sink::log_store::{LogReader, LogStoreReadItem, LogStoreResult, TruncateOffset}; @@ -532,6 +533,12 @@ pub enum SinkError { #[backtrace] anyhow::Error, ), + #[error(transparent)] + Connector( + #[from] + #[backtrace] + ConnectorError, + ), } impl From for SinkError { diff --git a/src/connector/src/sink/nats.rs b/src/connector/src/sink/nats.rs index 7a97771dee8ef..fc6afc379eb76 100644 --- a/src/connector/src/sink/nats.rs +++ b/src/connector/src/sink/nats.rs @@ -107,15 +107,9 @@ impl Sink for NatsSink { "Nats sink only support append-only mode" ))); } - match self.config.common.build_client().await { - Ok(_client) => {} - Err(error) => { - return Err(SinkError::Nats(anyhow!( - "validate nats sink error: {:?}", - error - ))); - } - } + let _client = (self.config.common.build_client().await) + .context("validate nats sink error") + .map_err(SinkError::Nats)?; Ok(()) } @@ -134,7 +128,7 @@ impl NatsSinkWriter { .common .build_context() .await - .map_err(|e| SinkError::Nats(anyhow!("nats sink error: {:?}", e)))?; + .map_err(|e| SinkError::Nats(anyhow!(e)))?; Ok::<_, SinkError>(Self { config: config.clone(), context, diff --git a/src/connector/src/sink/redis.rs b/src/connector/src/sink/redis.rs index d79d67e4adc2e..f1a07b66cb692 100644 --- a/src/connector/src/sink/redis.rs +++ b/src/connector/src/sink/redis.rs @@ -30,6 +30,7 @@ use super::formatter::SinkFormatterImpl; use super::writer::FormattedSink; use super::{SinkError, SinkParam}; use crate::dispatch_sink_formatter_str_key_impl; +use crate::error::ConnectorResult; use crate::sink::log_store::DeliveryFutureManagerAddFuture; use crate::sink::writer::{ AsyncTruncateLogSinkerOf, AsyncTruncateSinkWriter, AsyncTruncateSinkWriterExt, @@ -47,7 +48,7 @@ pub struct RedisCommon { } impl RedisCommon { - pub(crate) fn build_client(&self) -> anyhow::Result { + pub(crate) fn build_client(&self) -> ConnectorResult { let client = RedisClient::open(self.url.clone())?; Ok(client) } diff --git a/src/connector/src/sink/remote.rs b/src/connector/src/sink/remote.rs index dfc3bed0e372c..4c4a662f83178 100644 --- a/src/connector/src/sink/remote.rs +++ b/src/connector/src/sink/remote.rs @@ -26,6 +26,7 @@ use itertools::Itertools; use jni::JavaVM; use prost::Message; use risingwave_common::array::StreamChunk; +use risingwave_common::bail; use risingwave_common::catalog::{ColumnDesc, ColumnId}; use risingwave_common::types::DataType; use risingwave_jni_core::jvm_runtime::JVM; @@ -56,9 +57,10 @@ use tokio_stream::wrappers::ReceiverStream; use tracing::warn; use super::elasticsearch::{StreamChunkConverter, ES_OPTION_DELIMITER}; +use crate::error::ConnectorResult; use crate::sink::catalog::desc::SinkDesc; use crate::sink::coordinate::CoordinatedSinkWriter; -use crate::sink::log_store::{LogStoreReadItem, TruncateOffset}; +use crate::sink::log_store::{LogStoreReadItem, LogStoreResult, TruncateOffset}; use crate::sink::writer::{LogSinkerOf, SinkWriter, SinkWriterExt}; use crate::sink::{ DummySinkCommitCoordinator, LogSinker, Result, Sink, SinkCommitCoordinator, SinkError, @@ -157,14 +159,12 @@ impl Sink for RemoteSink { } } -async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Result<()> { +async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> ConnectorResult<()> { if sink_name == ElasticSearchSink::SINK_NAME && param.downstream_pk.len() > 1 && param.properties.get(ES_OPTION_DELIMITER).is_none() { - return Err(anyhow!( - "Es sink only support single pk or pk with delimiter option" - )); + bail!("Es sink only support single pk or pk with delimiter option"); } // FIXME: support struct and array in stream sink param.columns.iter().map(|col| { @@ -215,7 +215,7 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Res let jvm = JVM.get_or_init()?; let sink_param = param.to_proto(); - spawn_blocking(move || { + spawn_blocking(move || -> anyhow::Result<()> { let mut env = jvm.attach_current_thread()?; let validate_sink_request = ValidateSinkRequest { sink_param: Some(sink_param), @@ -236,16 +236,13 @@ async fn validate_remote_sink(param: &SinkParam, sink_name: &str) -> anyhow::Res validate_sink_response.error.map_or_else( || Ok(()), // If there is no error message, return Ok here. - |err| { - Err(anyhow!(format!( - "sink cannot pass validation: {}", - err.error_message - ))) - }, + |err| bail!("sink cannot pass validation: {}", err.error_message), ) }) .await - .context("JoinHandle returns error")? + .context("JoinHandle returns error")??; + + Ok(()) } pub struct RemoteLogSinker { @@ -338,12 +335,11 @@ impl LogSinker for RemoteLogSinker { anyhow!("get unsent offset {:?} in response", persisted_offset) })?; if sent_offset != persisted_offset { - return Err(anyhow!( + bail!( "new response offset {:?} not match the buffer offset {:?}", persisted_offset, sent_offset - ) - .into()); + ); } if let (TruncateOffset::Barrier { .. }, Some(start_time)) = @@ -366,13 +362,13 @@ impl LogSinker for RemoteLogSinker { loop { let either_result: futures::future::Either< Option, - anyhow::Result<(u64, LogStoreReadItem)>, + LogStoreResult<(u64, LogStoreReadItem)>, > = drop_either_future( select(pin!(response_rx.recv()), pin!(log_reader.next_item())).await, ); match either_result { futures::future::Either::Left(opt) => { - let response = opt.ok_or_else(|| anyhow!("end of response stream"))?; + let response = opt.context("end of response stream")?; match response { SinkWriterStreamResponse { response: @@ -569,7 +565,7 @@ impl CoordinatedRemoteSinkWriter { } fn for_test( - response_receiver: Receiver>, + response_receiver: Receiver>, request_sender: Sender, ) -> CoordinatedRemoteSinkWriter { let properties = HashMap::from([("output.path".to_string(), "/tmp/rw".to_string())]); diff --git a/src/connector/src/source/base.rs b/src/connector/src/source/base.rs index fed8e0263aac4..625f3dedb30bc 100644 --- a/src/connector/src/source/base.rs +++ b/src/connector/src/source/base.rs @@ -15,7 +15,7 @@ use std::collections::{BTreeMap, HashMap}; use std::sync::Arc; -use anyhow::{anyhow, Result}; +use anyhow::anyhow; use async_trait::async_trait; use aws_sdk_s3::types::Object; use bytes::Bytes; @@ -25,6 +25,7 @@ use futures::Stream; use itertools::Itertools; use parking_lot::Mutex; use risingwave_common::array::StreamChunk; +use risingwave_common::bail; use risingwave_common::catalog::TableId; use risingwave_common::error::ErrorSuppressor; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; @@ -45,6 +46,7 @@ use super::kinesis::KinesisMeta; use super::monitor::SourceMetrics; use super::nexmark::source::message::NexmarkMeta; use super::{GCS_CONNECTOR, OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR}; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; pub(crate) use crate::source::common::CommonSplitReader; use crate::source::filesystem::FsPageItem; @@ -69,7 +71,9 @@ pub trait TryFromHashmap: Sized + UnknownFields { /// Each instance should add a `#[derive(with_options::WithOptions)]` marker. pub trait SourceProperties: TryFromHashmap + Clone + WithOptions { const SOURCE_NAME: &'static str; - type Split: SplitMetaData + TryFrom + Into; + type Split: SplitMetaData + + TryFrom + + Into; type SplitEnumerator: SplitEnumerator; type SplitReader: SplitReader; @@ -91,10 +95,10 @@ impl TryFromHashmap for P { if !deny_unknown_fields || res.unknown_fields().is_empty() { Ok(res) } else { - Err(anyhow!( + bail!( "Unknown fields in the WITH clause: {:?}", res.unknown_fields() - )) + ) } } } @@ -133,12 +137,15 @@ pub struct SourceCtrlOpts { // comes from developer::stream_chunk_size in stream scenario and developer::batch_chunk_size // in batch scenario pub chunk_size: usize, + /// Rate limit of source + pub rate_limit: Option, } impl Default for SourceCtrlOpts { fn default() -> Self { Self { chunk_size: MAX_CHUNK_SIZE, + rate_limit: None, } } } @@ -337,21 +344,22 @@ pub fn extract_source_struct(info: &PbStreamSourceInfo) -> Result } (PbFormatType::Plain, PbEncodeType::Bytes) => (SourceFormat::Plain, SourceEncode::Bytes), (format, encode) => { - return Err(anyhow!( + bail!( "Unsupported combination of format {:?} and encode {:?}", format, encode - )); + ); } }; Ok(SourceStruct::new(format, encode)) } -pub type BoxSourceStream = BoxStream<'static, anyhow::Result>>; +pub type BoxSourceStream = BoxStream<'static, crate::error::ConnectorResult>>; -pub trait ChunkSourceStream = Stream> + Send + 'static; -pub type BoxChunkSourceStream = BoxStream<'static, anyhow::Result>; -pub type BoxTryStream = BoxStream<'static, anyhow::Result>; +pub trait ChunkSourceStream = + Stream> + Send + 'static; +pub type BoxChunkSourceStream = BoxStream<'static, crate::error::ConnectorResult>; +pub type BoxTryStream = BoxStream<'static, crate::error::ConnectorResult>; /// [`SplitReader`] is a new abstraction of the external connector read interface which is /// responsible for parsing, it is used to read messages from the outside and transform them into a @@ -367,7 +375,7 @@ pub trait SplitReader: Sized + Send { parser_config: ParserConfig, source_ctx: SourceContextRef, columns: Option>, - ) -> anyhow::Result; + ) -> crate::error::ConnectorResult; fn into_stream(self) -> BoxChunkSourceStream; } @@ -423,7 +431,7 @@ impl ConnectorProperties { PropType, PropType::try_from_hashmap(with_properties, deny_unknown_fields) .map(ConnectorProperties::from), - |other| Err(anyhow!("connector '{}' is not supported", other)) + |other| bail!("connector '{}' is not supported", other) ) } @@ -459,7 +467,7 @@ impl From<&SplitImpl> for ConnectorSplit { } impl TryFrom<&ConnectorSplit> for SplitImpl { - type Error = anyhow::Error; + type Error = crate::error::ConnectorError; fn try_from(split: &ConnectorSplit) -> std::result::Result { match_source_name_str!( @@ -471,7 +479,7 @@ impl TryFrom<&ConnectorSplit> for SplitImpl { ) .map(Into::into) }, - |other| Err(anyhow!("connector '{}' is not supported", other)) + |other| bail!("connector '{}' is not supported", other) ) } } @@ -500,7 +508,7 @@ impl SplitImpl { split_type.to_lowercase().as_str(), PropType, ::Split::restore_from_json(value).map(Into::into), - |other| Err(anyhow!("connector '{}' is not supported", other)) + |other| bail!("connector '{}' is not supported", other) ) } } @@ -618,7 +626,7 @@ pub trait SplitMetaData: Sized { fn encode_to_json(&self) -> JsonbVal; fn restore_from_json(value: JsonbVal) -> Result; - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()>; + fn update_with_offset(&mut self, start_offset: String) -> crate::error::ConnectorResult<()>; } /// [`ConnectorState`] maintains the consuming splits' info. In specific split readers, diff --git a/src/connector/src/source/cdc/enumerator/mod.rs b/src/connector/src/source/cdc/enumerator/mod.rs index 58bc42e537578..b5ac4826921bc 100644 --- a/src/connector/src/source/cdc/enumerator/mod.rs +++ b/src/connector/src/source/cdc/enumerator/mod.rs @@ -27,6 +27,7 @@ use risingwave_pb::connector_service::{ SourceCommonParam, SourceType, ValidateSourceRequest, ValidateSourceResponse, }; +use crate::error::ConnectorResult; use crate::source::cdc::{ CdcProperties, CdcSourceTypeTrait, CdcSplitBase, Citus, DebeziumCdcSplit, MySqlCdcSplit, Mysql, Postgres, PostgresCdcSplit, @@ -54,7 +55,7 @@ where async fn new( props: CdcProperties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let server_addrs = props .properties .get(DATABASE_SERVERS_KEY) @@ -72,7 +73,7 @@ where ); let source_id = context.info.source_id; - tokio::task::spawn_blocking(move || { + tokio::task::spawn_blocking(move || -> anyhow::Result<()> { let mut env = JVM.get_or_init()?.attach_current_thread()?; let validate_source_request = ValidateSourceRequest { @@ -100,15 +101,11 @@ where .deref(), )?; - validate_source_response.error.map_or_else( - || Ok(()), - |err| { - Err(anyhow!(format!( - "source cannot pass validation: {}", - err.error_message - ))) - }, - ) + if let Some(error) = validate_source_response.error { + return Err(anyhow!(error.error_message).context("source cannot pass validation")); + } + + Ok(()) }) .await .context("failed to validate source")??; @@ -121,7 +118,7 @@ where }) } - async fn list_splits(&mut self) -> anyhow::Result>> { + async fn list_splits(&mut self) -> ConnectorResult>> { Ok(self.list_cdc_splits()) } } diff --git a/src/connector/src/source/cdc/external/mod.rs b/src/connector/src/source/cdc/external/mod.rs index f281d1ecea58a..a9808e3a9e1e2 100644 --- a/src/connector/src/source/cdc/external/mod.rs +++ b/src/connector/src/source/cdc/external/mod.rs @@ -407,7 +407,7 @@ impl MySqlExternalTableReader { bail!("primary key {} cannot be null", pk); } }) - .try_collect()?; + .try_collect::<_, _, ConnectorError>()?; let rs_stream = sql .with(Params::from(params)) diff --git a/src/connector/src/source/cdc/external/postgres.rs b/src/connector/src/source/cdc/external/postgres.rs index bd8a0b51c04e7..9f9a055fd7d8f 100644 --- a/src/connector/src/source/cdc/external/postgres.rs +++ b/src/connector/src/source/cdc/external/postgres.rs @@ -199,7 +199,7 @@ impl PostgresExternalTableReader { let stream = client.query_raw(&sql, ¶ms).await?; let row_stream = stream.map(|row| { let row = row?; - Ok::<_, anyhow::Error>(postgres_row_to_owned_row(row, &self.rw_schema)) + Ok::<_, crate::error::ConnectorError>(postgres_row_to_owned_row(row, &self.rw_schema)) }); pin_mut!(row_stream); diff --git a/src/connector/src/source/cdc/mod.rs b/src/connector/src/source/cdc/mod.rs index 5fc6aefdfefdd..b663274c30207 100644 --- a/src/connector/src/source/cdc/mod.rs +++ b/src/connector/src/source/cdc/mod.rs @@ -28,6 +28,7 @@ use risingwave_pb::plan_common::ExternalTableDesc; use simd_json::prelude::ArrayTrait; pub use source::*; +use crate::error::ConnectorResult; use crate::source::{SourceProperties, SplitImpl, TryFromHashmap}; use crate::{for_all_classified_sources, impl_cdc_source_type}; @@ -91,7 +92,7 @@ impl TryFromHashmap for CdcProperties { fn try_from_hashmap( properties: HashMap, _deny_unknown_fields: bool, - ) -> anyhow::Result { + ) -> ConnectorResult { let is_multi_table_shared = properties .get(CDC_SHARING_MODE_KEY) .is_some_and(|v| v == "true"); @@ -107,7 +108,7 @@ impl TryFromHashmap for CdcProperties { impl SourceProperties for CdcProperties where - DebeziumCdcSplit: TryFrom + Into, + DebeziumCdcSplit: TryFrom + Into, DebeziumSplitEnumerator: ListCdcSplits, { type Split = DebeziumCdcSplit; diff --git a/src/connector/src/source/cdc/source/reader.rs b/src/connector/src/source/cdc/source/reader.rs index c21d579df7778..902f113526a2e 100644 --- a/src/connector/src/source/cdc/source/reader.rs +++ b/src/connector/src/source/cdc/source/reader.rs @@ -14,11 +14,12 @@ use std::str::FromStr; -use anyhow::{anyhow, Result}; +use anyhow::{anyhow, Context}; use async_trait::async_trait; use futures_async_stream::try_stream; use itertools::Itertools; use prost::Message; +use risingwave_common::bail; use risingwave_common::metrics::GLOBAL_ERROR_METRICS; use risingwave_common::util::addr::HostAddr; use risingwave_jni_core::jvm_runtime::JVM; @@ -29,6 +30,7 @@ use risingwave_pb::connector_service::{ use thiserror_ext::AsReport; use tokio::sync::mpsc; +use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::cdc::{CdcProperties, CdcSourceType, CdcSourceTypeTrait, DebeziumCdcSplit}; @@ -66,7 +68,7 @@ impl SplitReader for CdcSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> Result { + ) -> ConnectorResult { assert_eq!(splits.len(), 1); let split = splits.into_iter().next().unwrap(); let split_id = split.id(); @@ -77,8 +79,8 @@ impl SplitReader for CdcSplitReader { if matches!(T::source_type(), CdcSourceType::Citus) && let Some(server_addr) = split.server_addr() { - let host_addr = HostAddr::from_str(&server_addr) - .map_err(|err| anyhow!("invalid server address for cdc split. {}", err))?; + let host_addr = + HostAddr::from_str(&server_addr).context("invalid server address for cdc split")?; properties.insert("hostname".to_string(), host_addr.host); properties.insert("port".to_string(), host_addr.port.to_string()); // rewrite table name with suffix to capture all shards in the split @@ -117,10 +119,8 @@ impl SplitReader for CdcSplitReader { let (mut env, get_event_stream_request_bytes) = match result { Ok(inner) => inner, Err(e) => { - let _ = tx.blocking_send(Err(anyhow!( - "err before calling runJniDbzSourceThread: {:?}", - e - ))); + let _ = tx + .blocking_send(Err(e.context("err before calling runJniDbzSourceThread"))); return; } }; @@ -154,7 +154,7 @@ impl SplitReader for CdcSplitReader { } }; if !inited { - return Err(anyhow!("failed to start cdc connector")); + bail!("failed to start cdc connector"); } } tracing::info!(?source_id, "cdc connector started"); @@ -196,7 +196,7 @@ impl SplitReader for CdcSplitReader { } impl CommonSplitReader for CdcSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = ConnectorError)] async fn into_data_stream(self) { let source_type = T::source_type(); let mut rx = self.rx; @@ -218,13 +218,13 @@ impl CommonSplitReader for CdcSplitReader { GLOBAL_ERROR_METRICS.cdc_source_error.report([ source_type.as_str_name().into(), source_id.clone(), - e.to_string(), + e.to_report_string(), ]); Err(e)?; } } } - Err(anyhow!("all senders are dropped"))?; + bail!("all senders are dropped"); } } diff --git a/src/connector/src/source/cdc/split.rs b/src/connector/src/source/cdc/split.rs index a7357d231b78b..30165f1939e0f 100644 --- a/src/connector/src/source/cdc/split.rs +++ b/src/connector/src/source/cdc/split.rs @@ -14,10 +14,11 @@ use std::marker::PhantomData; -use anyhow::{anyhow, Context}; +use anyhow::Context; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::cdc::external::DebeziumOffset; use crate::source::cdc::CdcSourceTypeTrait; use crate::source::{SplitId, SplitMetaData}; @@ -63,7 +64,7 @@ impl MySqlCdcSplit { Self { inner: split } } - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + pub fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let mut snapshot_done = self.inner.snapshot_done; if !snapshot_done { let dbz_offset: DebeziumOffset = @@ -102,7 +103,7 @@ impl PostgresCdcSplit { } } - pub fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + pub fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let mut snapshot_done = self.inner.snapshot_done; if !snapshot_done { let dbz_offset: DebeziumOffset = @@ -154,11 +155,11 @@ impl SplitMetaData for DebeziumCdcSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { // TODO: may check T to get the specific cdc type assert!(self.mysql_split.is_some() || self.pg_split.is_some()); if let Some(split) = &mut self.mysql_split { diff --git a/src/connector/src/source/common.rs b/src/connector/src/source/common.rs index 5d39e303e7590..c145bd7f403da 100644 --- a/src/connector/src/source/common.rs +++ b/src/connector/src/source/common.rs @@ -16,14 +16,15 @@ use futures::{Stream, StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; +use crate::error::{ConnectorError, ConnectorResult}; use crate::parser::ParserConfig; use crate::source::{SourceContextRef, SourceMessage, SplitReader}; pub(crate) trait CommonSplitReader: SplitReader + 'static { - fn into_data_stream(self) -> impl Stream>> + Send; + fn into_data_stream(self) -> impl Stream>> + Send; } -#[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] +#[try_stream(boxed, ok = StreamChunk, error = ConnectorError)] pub(crate) async fn into_chunk_stream( reader: impl CommonSplitReader, parser_config: ParserConfig, diff --git a/src/connector/src/source/datagen/enumerator/mod.rs b/src/connector/src/source/datagen/enumerator/mod.rs index 47eb54cf24005..5b5b473656d09 100644 --- a/src/connector/src/source/datagen/enumerator/mod.rs +++ b/src/connector/src/source/datagen/enumerator/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use anyhow::Context; use async_trait::async_trait; use crate::source::datagen::{DatagenProperties, DatagenSplit}; @@ -30,13 +31,15 @@ impl SplitEnumerator for DatagenSplitEnumerator { async fn new( properties: DatagenProperties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> crate::error::ConnectorResult { let split_num = properties.split_num.unwrap_or_else(|| "1".to_string()); - let split_num = split_num.parse::()?; + let split_num = split_num + .parse::() + .context("failed to parse datagen split num")?; Ok(Self { split_num }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> crate::error::ConnectorResult> { let mut splits = vec![]; for i in 0..self.split_num { splits.push(DatagenSplit { diff --git a/src/connector/src/source/datagen/source/generator.rs b/src/connector/src/source/datagen/source/generator.rs index 18376f3ce73ab..1c05c6b4ffc8f 100644 --- a/src/connector/src/source/datagen/source/generator.rs +++ b/src/connector/src/source/datagen/source/generator.rs @@ -21,6 +21,7 @@ use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::iter_util::ZipEqFast; +use crate::error::ConnectorResult; use crate::parser::{EncodingProperties, ProtocolProperties, SpecificParserConfig}; use crate::source::{SourceMessage, SourceMeta, SplitId}; @@ -59,7 +60,7 @@ impl DatagenEventGenerator { split_id: SplitId, split_num: u64, split_index: u64, - ) -> anyhow::Result { + ) -> ConnectorResult { let partition_rows_per_second = if rows_per_second % split_num > split_index { rows_per_second / split_num + 1 } else { @@ -76,7 +77,7 @@ impl DatagenEventGenerator { }) } - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn into_msg_stream(mut self) { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; @@ -156,7 +157,7 @@ impl DatagenEventGenerator { } } - #[try_stream(ok = StreamChunk, error = anyhow::Error)] + #[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] pub async fn into_native_stream(mut self) { let mut interval = tokio::time::interval(Duration::from_secs(1)); const MAX_ROWS_PER_YIELD: u64 = 1024; diff --git a/src/connector/src/source/datagen/source/reader.rs b/src/connector/src/source/datagen/source/reader.rs index 2e1b5f7917261..0b522c4e7c938 100644 --- a/src/connector/src/source/datagen/source/reader.rs +++ b/src/connector/src/source/datagen/source/reader.rs @@ -14,13 +14,14 @@ use std::collections::HashMap; -use anyhow::{Context, Result}; +use anyhow::Context; use async_trait::async_trait; use futures::{Stream, StreamExt, TryStreamExt}; use risingwave_common::field_generator::{FieldGeneratorImpl, VarcharProperty}; use thiserror_ext::AsReport; use super::generator::DatagenEventGenerator; +use crate::error::{ConnectorResult, ConnectorResult as Result}; use crate::parser::{EncodingProperties, ParserConfig, ProtocolProperties}; use crate::source::data_gen_util::spawn_data_generation_stream; use crate::source::datagen::source::SEQUENCE_FIELD_KIND; @@ -183,7 +184,7 @@ impl SplitReader for DatagenSplitReader { } impl CommonSplitReader for DatagenSplitReader { - fn into_data_stream(self) -> impl Stream, anyhow::Error>> { + fn into_data_stream(self) -> impl Stream>> { // Will buffer at most 4 event chunks. const BUFFER_SIZE: usize = 4; spawn_data_generation_stream(self.generator.into_msg_stream(), BUFFER_SIZE) @@ -253,13 +254,15 @@ fn generator_from_data_type( random_seed, ) } + .map_err(Into::into) } DataType::Varchar => { let length_key = format!("fields.{}.length", name); let length_value = fields_option_map .get(&length_key) .map(|s| s.parse::()) - .transpose()?; + .transpose() + .context("failed to parse the length of varchar field")?; Ok(FieldGeneratorImpl::with_varchar( &VarcharProperty::RandomFixedLength(length_value), random_seed, @@ -280,7 +283,7 @@ fn generator_from_data_type( Ok((field_name.to_string(), gen)) }) .collect::>()?; - FieldGeneratorImpl::with_struct_fields(struct_fields) + FieldGeneratorImpl::with_struct_fields(struct_fields).map_err(Into::into) } DataType::List(datatype) => { let length_key = format!("fields.{}.length", name); @@ -293,7 +296,7 @@ fn generator_from_data_type( split_num, offset, )?; - FieldGeneratorImpl::with_list(generator, length_value) + FieldGeneratorImpl::with_list(generator, length_value).map_err(Into::into) } _ => { let kind_key = format!("fields.{}.kind", name); @@ -312,12 +315,14 @@ fn generator_from_data_type( split_num, offset, ) + .map_err(Into::into) } else { let min_key = format!("fields.{}.min", name); let max_key = format!("fields.{}.max", name); let min_value = fields_option_map.get(&min_key).map(|s| s.to_string()); let max_value = fields_option_map.get(&max_key).map(|s| s.to_string()); FieldGeneratorImpl::with_number_random(data_type, min_value, max_value, random_seed) + .map_err(Into::into) } } } diff --git a/src/connector/src/source/datagen/split.rs b/src/connector/src/source/datagen/split.rs index c2e0bea1f8fb4..6d51cfa7d47ae 100644 --- a/src/connector/src/source/datagen/split.rs +++ b/src/connector/src/source/datagen/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::base::SplitMetaData; use crate::source::SplitId; @@ -32,15 +32,15 @@ impl SplitMetaData for DatagenSplit { format!("{}-{}", self.split_num, self.split_index).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset.as_str().parse::().unwrap()); Ok(()) } diff --git a/src/connector/src/source/filesystem/file_common.rs b/src/connector/src/source/filesystem/file_common.rs index 3a537509edb88..ccff7315491ba 100644 --- a/src/connector/src/source/filesystem/file_common.rs +++ b/src/connector/src/source/filesystem/file_common.rs @@ -15,12 +15,12 @@ use std::fmt::Debug; use std::hash::Hash; use std::marker::PhantomData; -use anyhow::anyhow; use aws_sdk_s3::types::Object; use risingwave_common::types::{JsonbVal, Timestamptz}; use serde::{Deserialize, Serialize}; use super::opendal_source::OpendalSource; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; /// [`FsSplit`] Describes a file or a split of a file. A file is a generic concept, @@ -47,15 +47,15 @@ impl SplitMetaData for FsSplit { self.name.as_str().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let offset = start_offset.parse().unwrap(); self.offset = offset; Ok(()) @@ -98,15 +98,15 @@ impl SplitMetaData for OpendalFsSplit { self.name.as_str().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let offset = start_offset.parse().unwrap(); self.offset = offset; Ok(()) diff --git a/src/connector/src/source/filesystem/nd_streaming.rs b/src/connector/src/source/filesystem/nd_streaming.rs index 7eb8a84c503ad..711e1a15e981c 100644 --- a/src/connector/src/source/filesystem/nd_streaming.rs +++ b/src/connector/src/source/filesystem/nd_streaming.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; +use anyhow::Context as _; use bytes::BytesMut; use futures::io::Cursor; use futures::AsyncBufReadExt; @@ -26,7 +26,7 @@ pub fn need_nd_streaming(encode_config: &EncodingProperties) -> bool { || matches!(encode_config, EncodingProperties::Csv(_)) } -#[try_stream(boxed, ok = Vec, error = anyhow::Error)] +#[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] /// This function splits a byte stream by the newline separator "(\r)\n" into a message stream. /// It can be difficult to split and compute offsets correctly when the bytes are received in /// chunks. There are two cases to consider: @@ -50,7 +50,7 @@ pub async fn split_stream(data_stream: BoxSourceStream) { .map(|msg| (msg.offset.clone(), msg.split_id.clone(), msg.meta.clone())) .unwrap(); - let mut offset: usize = offset.parse()?; + let mut offset: usize = offset.parse().context("failed to parse the offset")?; let mut buf = BytesMut::new(); for msg in batch { let payload = msg.payload.unwrap_or_default(); @@ -108,7 +108,7 @@ pub async fn split_stream(data_stream: BoxSourceStream) { last_message = msgs.pop(); } } - Err(e) => return Err(anyhow!(e)), + Err(e) => return Err(e.into()), } line_cnt += 1; diff --git a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs index d6f7b44bff591..01594af4e4bad 100644 --- a/src/connector/src/source/filesystem/opendal_source/gcs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/gcs_source.rs @@ -21,11 +21,12 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::{GcsProperties, OpendalSource}; +use crate::error::ConnectorResult; use crate::source::filesystem::s3::enumerator::get_prefix; impl OpendalEnumerator { /// create opendal gcs source. - pub fn new_gcs_source(gcs_properties: GcsProperties) -> anyhow::Result { + pub fn new_gcs_source(gcs_properties: GcsProperties) -> ConnectorResult { // Create gcs builder. let mut builder = Gcs::default(); diff --git a/src/connector/src/source/filesystem/opendal_source/mod.rs b/src/connector/src/source/filesystem/opendal_source/mod.rs index e0c5a22f1fd90..15371a0da90a6 100644 --- a/src/connector/src/source/filesystem/opendal_source/mod.rs +++ b/src/connector/src/source/filesystem/opendal_source/mod.rs @@ -27,6 +27,7 @@ use self::opendal_enumerator::OpendalEnumerator; use self::opendal_reader::OpendalReader; use super::s3::S3PropertiesCommon; use super::OpendalFsSplit; +use crate::error::ConnectorResult; use crate::source::{SourceProperties, UnknownFields}; pub const GCS_CONNECTOR: &str = "gcs"; @@ -71,7 +72,7 @@ impl SourceProperties for GcsProperties { pub trait OpendalSource: Send + Sync + 'static + Clone + PartialEq { type Properties: SourceProperties + Send + Sync; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result>; + fn new_enumerator(properties: Self::Properties) -> ConnectorResult>; } #[derive(Debug, Clone, Copy, PartialEq, Eq)] @@ -80,7 +81,7 @@ pub struct OpendalS3; impl OpendalSource for OpendalS3 { type Properties = OpendalS3Properties; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + fn new_enumerator(properties: Self::Properties) -> ConnectorResult> { OpendalEnumerator::new_s3_source(properties.s3_properties, properties.assume_role) } } @@ -91,7 +92,7 @@ pub struct OpendalGcs; impl OpendalSource for OpendalGcs { type Properties = GcsProperties; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + fn new_enumerator(properties: Self::Properties) -> ConnectorResult> { OpendalEnumerator::new_gcs_source(properties) } } @@ -102,7 +103,7 @@ pub struct OpendalPosixFs; impl OpendalSource for OpendalPosixFs { type Properties = PosixFsProperties; - fn new_enumerator(properties: Self::Properties) -> anyhow::Result> { + fn new_enumerator(properties: Self::Properties) -> ConnectorResult> { OpendalEnumerator::new_posix_fs_source(properties) } } diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs index 318467eea6069..96646ade0e1df 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_enumerator.rs @@ -22,6 +22,7 @@ use opendal::{Metakey, Operator}; use risingwave_common::types::Timestamptz; use super::OpendalSource; +use crate::error::ConnectorResult; use crate::source::filesystem::{FsPageItem, OpendalFsSplit}; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -42,11 +43,11 @@ impl SplitEnumerator for OpendalEnumerator { async fn new( properties: Src::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { Src::new_enumerator(properties) } - async fn list_splits(&mut self) -> anyhow::Result>> { + async fn list_splits(&mut self) -> ConnectorResult>> { let empty_split: OpendalFsSplit = OpendalFsSplit::empty_split(); Ok(vec![empty_split]) @@ -54,7 +55,7 @@ impl SplitEnumerator for OpendalEnumerator { } impl OpendalEnumerator { - pub async fn list(&self) -> anyhow::Result { + pub async fn list(&self) -> ConnectorResult { let prefix = match &self.prefix { Some(prefix) => prefix, None => "", @@ -100,4 +101,4 @@ impl OpendalEnumerator { &self.matcher } } -pub type ObjectMetadataIter = BoxStream<'static, anyhow::Result>; +pub type ObjectMetadataIter = BoxStream<'static, ConnectorResult>; diff --git a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs index 8ed913cad64a4..5cb84652fbab6 100644 --- a/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs +++ b/src/connector/src/source/filesystem/opendal_source/opendal_reader.rs @@ -22,6 +22,7 @@ use tokio_util::io::{ReaderStream, StreamReader}; use super::opendal_enumerator::OpendalEnumerator; use super::OpendalSource; +use crate::error::ConnectorResult; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::filesystem::nd_streaming::need_nd_streaming; use crate::source::filesystem::{nd_streaming, OpendalFsSplit}; @@ -50,7 +51,7 @@ impl SplitReader for OpendalReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let connector = Src::new_enumerator(properties)?; let opendal_reader = OpendalReader { connector, @@ -67,7 +68,7 @@ impl SplitReader for OpendalReader { } impl OpendalReader { - #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] + #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); @@ -107,7 +108,7 @@ impl OpendalReader { } } - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn stream_read_object( op: Operator, split: OpendalFsSplit, diff --git a/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs b/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs index 748230ba5a16e..3a4fb7fcfeaa7 100644 --- a/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/posix_fs_source.rs @@ -21,6 +21,7 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::{OpendalSource, PosixFsProperties}; +use crate::error::ConnectorResult; // Posix fs source should only be used for testing. // For a single-CN cluster, the behavior is well-defined. It will read from the local file system. @@ -28,7 +29,7 @@ use super::{OpendalSource, PosixFsProperties}; impl OpendalEnumerator { /// create opendal posix fs source. - pub fn new_posix_fs_source(posix_fs_properties: PosixFsProperties) -> anyhow::Result { + pub fn new_posix_fs_source(posix_fs_properties: PosixFsProperties) -> ConnectorResult { // Create Fs builder. let mut builder = Fs::default(); diff --git a/src/connector/src/source/filesystem/opendal_source/s3_source.rs b/src/connector/src/source/filesystem/opendal_source/s3_source.rs index ef18ffa4b8fec..f4a548306885a 100644 --- a/src/connector/src/source/filesystem/opendal_source/s3_source.rs +++ b/src/connector/src/source/filesystem/opendal_source/s3_source.rs @@ -21,6 +21,7 @@ use opendal::Operator; use super::opendal_enumerator::OpendalEnumerator; use super::OpendalSource; +use crate::error::ConnectorResult; use crate::source::filesystem::s3::enumerator::get_prefix; use crate::source::filesystem::s3::S3PropertiesCommon; @@ -29,7 +30,7 @@ impl OpendalEnumerator { pub fn new_s3_source( s3_properties: S3PropertiesCommon, assume_role: Option, - ) -> anyhow::Result { + ) -> ConnectorResult { // Create s3 builder. let mut builder = S3::default(); builder.bucket(&s3_properties.bucket_name); diff --git a/src/connector/src/source/filesystem/s3/enumerator.rs b/src/connector/src/source/filesystem/s3/enumerator.rs index 7033ebdaf91ba..63a57b8574b6d 100644 --- a/src/connector/src/source/filesystem/s3/enumerator.rs +++ b/src/connector/src/source/filesystem/s3/enumerator.rs @@ -75,7 +75,7 @@ impl SplitEnumerator for S3SplitEnumerator { async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> crate::error::ConnectorResult { let config = AwsAuthProps::from(&properties); let sdk_config = config.build_config().await?; let s3_client = s3_client(&sdk_config, Some(default_conn_config())); @@ -98,7 +98,7 @@ impl SplitEnumerator for S3SplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> crate::error::ConnectorResult> { let mut objects = Vec::new(); loop { let (files, has_finished) = self.get_next_page::().await?; diff --git a/src/connector/src/source/filesystem/s3/source/reader.rs b/src/connector/src/source/filesystem/s3/source/reader.rs index 884f1d19062ce..3f485dc22383a 100644 --- a/src/connector/src/source/filesystem/s3/source/reader.rs +++ b/src/connector/src/source/filesystem/s3/source/reader.rs @@ -32,6 +32,7 @@ use tokio_util::io::ReaderStream; use crate::aws_utils::{default_conn_config, s3_client}; use crate::common::AwsAuthProps; +use crate::error::ConnectorResult; use crate::parser::{ByteStreamSourceParserImpl, ParserConfig}; use crate::source::base::{SplitMetaData, SplitReader}; use crate::source::filesystem::file_common::FsSplit; @@ -54,7 +55,7 @@ pub struct S3FileReader { } impl S3FileReader { - #[try_stream(boxed, ok = Vec, error = anyhow::Error)] + #[try_stream(boxed, ok = Vec, error = crate::error::ConnectorError)] pub async fn stream_read_object( client_for_s3: s3_client::Client, bucket_name: String, @@ -85,7 +86,9 @@ impl S3FileReader { return Ok(()); } Err(e) => { - return Err(anyhow!(e).context(format!("S3 GetObject from {bucket_name} error"))); + return Err(anyhow!(e) + .context(format!("S3 GetObject from {bucket_name} error")) + .into()); } }; @@ -180,7 +183,7 @@ impl SplitReader for S3FileReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let config = AwsAuthProps::from(&props); let sdk_config = config.build_config().await?; @@ -206,7 +209,7 @@ impl SplitReader for S3FileReader { } impl S3FileReader { - #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] + #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_chunk_stream(self) { for split in self.splits { let actor_id = self.source_ctx.source_info.actor_id.to_string(); diff --git a/src/connector/src/source/filesystem/s3_v2/lister.rs b/src/connector/src/source/filesystem/s3_v2/lister.rs index d6a6b6c1e68a3..ccc40f1ccef80 100644 --- a/src/connector/src/source/filesystem/s3_v2/lister.rs +++ b/src/connector/src/source/filesystem/s3_v2/lister.rs @@ -18,6 +18,7 @@ use aws_sdk_s3::error::DisplayErrorContext; use aws_sdk_s3::types::Object; use itertools::Itertools; +use crate::error::ConnectorResult; use crate::source::filesystem::{FsPageItem, S3SplitEnumerator}; use crate::source::{FsFilterCtrlCtx, FsListInner}; @@ -25,7 +26,7 @@ use crate::source::{FsFilterCtrlCtx, FsListInner}; impl FsListInner for S3SplitEnumerator { async fn get_next_page From<&'a Object>>( &mut self, - ) -> anyhow::Result<(Vec, bool)> { + ) -> ConnectorResult<(Vec, bool)> { let mut has_finished = false; let mut req = self .client diff --git a/src/connector/src/source/google_pubsub/enumerator/client.rs b/src/connector/src/source/google_pubsub/enumerator/client.rs index bc1d9d078b66a..25cb28909c479 100644 --- a/src/connector/src/source/google_pubsub/enumerator/client.rs +++ b/src/connector/src/source/google_pubsub/enumerator/client.rs @@ -12,12 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{bail, Context}; +use anyhow::Context; use async_trait::async_trait; use chrono::{TimeZone, Utc}; use google_cloud_pubsub::client::{Client, ClientConfig}; use google_cloud_pubsub::subscription::{SeekTo, SubscriptionConfig}; +use risingwave_common::bail; +use crate::error::ConnectorResult; use crate::source::base::SplitEnumerator; use crate::source::google_pubsub::split::PubsubSplit; use crate::source::google_pubsub::PubsubProperties; @@ -36,7 +38,7 @@ impl SplitEnumerator for PubsubSplitEnumerator { async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let subscription = properties.subscription.to_owned(); if properties.credentials.is_none() && properties.emulator_host.is_none() { @@ -62,7 +64,10 @@ impl SplitEnumerator for PubsubSplitEnumerator { // We need the `retain_acked_messages` configuration to be true to seek back to timestamps // as done in the [`PubsubSplitReader`] and here. - let (_, subscription_config) = sub.config(None).await?; + let (_, subscription_config) = sub + .config(None) + .await + .context("failed to fetch subscription config")?; if let SubscriptionConfig { retain_acked_messages: false, .. @@ -98,7 +103,7 @@ impl SplitEnumerator for PubsubSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { tracing::debug!("enumerating pubsub splits"); let splits: Vec = (0..self.split_count) .map(|i| PubsubSplit { diff --git a/src/connector/src/source/google_pubsub/mod.rs b/src/connector/src/source/google_pubsub/mod.rs index aeec1accd820b..0a49fa6467f66 100644 --- a/src/connector/src/source/google_pubsub/mod.rs +++ b/src/connector/src/source/google_pubsub/mod.rs @@ -101,9 +101,8 @@ impl PubsubProperties { #[cfg(test)] mod tests { - use anyhow::Result; - use super::*; + use crate::error::ConnectorResult as Result; const EMULATOR_HOST: &str = "localhost:8081"; const CREDENTIALS: &str = "{}"; diff --git a/src/connector/src/source/google_pubsub/source/reader.rs b/src/connector/src/source/google_pubsub/source/reader.rs index fd5fab15ed10b..0887cb06594f9 100644 --- a/src/connector/src/source/google_pubsub/source/reader.rs +++ b/src/connector/src/source/google_pubsub/source/reader.rs @@ -12,16 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, ensure, Context, Result}; +use anyhow::{anyhow, Context}; use async_trait::async_trait; use chrono::{NaiveDateTime, TimeZone, Utc}; use futures_async_stream::try_stream; use google_cloud_pubsub::client::{Client, ClientConfig}; use google_cloud_pubsub::subscription::{SeekTo, Subscription}; -use risingwave_common::bail; +use risingwave_common::{bail, ensure}; use tonic::Code; use super::TaggedReceivedMessage; +use crate::error::{ConnectorError, ConnectorResult as Result}; use crate::parser::ParserConfig; use crate::source::google_pubsub::{PubsubProperties, PubsubSplit}; use crate::source::{ @@ -41,7 +42,7 @@ pub struct PubsubSplitReader { } impl CommonSplitReader for PubsubSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = ConnectorError)] async fn into_data_stream(self) { loop { let pull_result = self diff --git a/src/connector/src/source/google_pubsub/split.rs b/src/connector/src/source/google_pubsub/split.rs index e52ffa8ef72a4..f150f7f08038b 100644 --- a/src/connector/src/source/google_pubsub/split.rs +++ b/src/connector/src/source/google_pubsub/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] @@ -36,8 +36,8 @@ pub struct PubsubSplit { } impl SplitMetaData for PubsubSplit { - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { @@ -48,7 +48,7 @@ impl SplitMetaData for PubsubSplit { format!("{}-{}", self.subscription, self.index).into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset); Ok(()) } diff --git a/src/connector/src/source/iceberg/mod.rs b/src/connector/src/source/iceberg/mod.rs index e274f639f15b2..899828322c810 100644 --- a/src/connector/src/source/iceberg/mod.rs +++ b/src/connector/src/source/iceberg/mod.rs @@ -18,6 +18,7 @@ use async_trait::async_trait; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::{ BoxChunkSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, @@ -71,7 +72,7 @@ impl SplitMetaData for IcebergSplit { unimplemented!() } - fn restore_from_json(_value: JsonbVal) -> anyhow::Result { + fn restore_from_json(_value: JsonbVal) -> ConnectorResult { unimplemented!() } @@ -79,7 +80,7 @@ impl SplitMetaData for IcebergSplit { unimplemented!() } - fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, _start_offset: String) -> ConnectorResult<()> { unimplemented!() } } @@ -95,11 +96,11 @@ impl SplitEnumerator for IcebergSplitEnumerator { async fn new( _properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self {}) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { Ok(vec![]) } } @@ -118,7 +119,7 @@ impl SplitReader for IcebergFileReader { _parser_config: ParserConfig, _source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { unimplemented!() } diff --git a/src/connector/src/source/kafka/enumerator/client.rs b/src/connector/src/source/kafka/enumerator/client.rs index 9850a4c244920..4441be8c9db21 100644 --- a/src/connector/src/source/kafka/enumerator/client.rs +++ b/src/connector/src/source/kafka/enumerator/client.rs @@ -15,12 +15,14 @@ use std::collections::HashMap; use std::time::Duration; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use async_trait::async_trait; use rdkafka::consumer::{BaseConsumer, Consumer}; use rdkafka::error::KafkaResult; use rdkafka::{Offset, TopicPartitionList}; +use risingwave_common::bail; +use crate::error::ConnectorResult; use crate::source::base::SplitEnumerator; use crate::source::kafka::split::KafkaSplit; use crate::source::kafka::{KafkaProperties, PrivateLinkConsumerContext, KAFKA_ISOLATION_LEVEL}; @@ -57,7 +59,7 @@ impl SplitEnumerator for KafkaSplitEnumerator { async fn new( properties: KafkaProperties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let mut config = rdkafka::ClientConfig::new(); let common_props = &properties.common; @@ -77,11 +79,9 @@ impl SplitEnumerator for KafkaSplitEnumerator { Some("earliest") => KafkaEnumeratorOffset::Earliest, Some("latest") => KafkaEnumeratorOffset::Latest, None => KafkaEnumeratorOffset::Earliest, - _ => { - return Err(anyhow!( - "properties `scan_startup_mode` only support earliest and latest or leave it empty" - )); - } + _ => bail!( + "properties `scan_startup_mode` only support earliest and latest or leave it empty" + ), }; if let Some(s) = &properties.time_offset { @@ -105,13 +105,14 @@ impl SplitEnumerator for KafkaSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { - let topic_partitions = self.fetch_topic_partition().await.map_err(|e| { - anyhow!(format!( - "failed to fetch metadata from kafka ({}), error: {}", - self.broker_address, e - )) + async fn list_splits(&mut self) -> ConnectorResult> { + let topic_partitions = self.fetch_topic_partition().await.with_context(|| { + format!( + "failed to fetch metadata from kafka ({})", + self.broker_address + ) })?; + let watermarks = self.get_watermarks(topic_partitions.as_ref()).await?; let mut start_offsets = self .fetch_start_offset(topic_partitions.as_ref(), &watermarks) @@ -153,12 +154,12 @@ impl KafkaSplitEnumerator { &mut self, expect_start_timestamp_millis: Option, expect_stop_timestamp_millis: Option, - ) -> anyhow::Result> { - let topic_partitions = self.fetch_topic_partition().await.map_err(|e| { - anyhow!(format!( - "failed to fetch metadata from kafka ({}), error: {}", - self.broker_address, e - )) + ) -> ConnectorResult> { + let topic_partitions = self.fetch_topic_partition().await.with_context(|| { + format!( + "failed to fetch metadata from kafka ({})", + self.broker_address + ) })?; // here we are getting the start offset and end offset for each partition with the given @@ -349,7 +350,7 @@ impl KafkaSplitEnumerator { .is_ok() } - async fn fetch_topic_partition(&self) -> anyhow::Result> { + async fn fetch_topic_partition(&self) -> ConnectorResult> { // for now, we only support one topic let metadata = self .client @@ -358,11 +359,11 @@ impl KafkaSplitEnumerator { let topic_meta = match metadata.topics() { [meta] => meta, - _ => return Err(anyhow!("topic {} not found", self.topic)), + _ => bail!("topic {} not found", self.topic), }; if topic_meta.partitions().is_empty() { - return Err(anyhow!("topic {} not found", self.topic)); + bail!("topic {} not found", self.topic); } Ok(topic_meta diff --git a/src/connector/src/source/kafka/private_link.rs b/src/connector/src/source/kafka/private_link.rs index 645588f457498..3eebacca09f93 100644 --- a/src/connector/src/source/kafka/private_link.rs +++ b/src/connector/src/source/kafka/private_link.rs @@ -16,12 +16,13 @@ use std::collections::{BTreeMap, HashMap}; use std::str::FromStr; use std::sync::Arc; -use anyhow::anyhow; +use anyhow::{anyhow, Context}; use itertools::Itertools; use rdkafka::client::BrokerAddr; use rdkafka::consumer::ConsumerContext; use rdkafka::producer::{DeliveryResult, ProducerContext}; use rdkafka::{ClientContext, Statistics}; +use risingwave_common::bail; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::catalog::connection::PrivateLinkService; @@ -29,6 +30,7 @@ use risingwave_pb::catalog::connection::PrivateLinkService; use crate::common::{ AwsPrivateLinkItem, PRIVATE_LINK_BROKER_REWRITE_MAP_KEY, PRIVATE_LINK_TARGETS_KEY, }; +use crate::error::ConnectorResult; use crate::source::kafka::stats::RdKafkaStats; use crate::source::kafka::{KAFKA_PROPS_BROKER_KEY, KAFKA_PROPS_BROKER_KEY_ALIAS}; use crate::source::KAFKA_CONNECTOR; @@ -68,9 +70,9 @@ impl BrokerAddrRewriter { pub fn new( role: PrivateLinkContextRole, broker_rewrite_map: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { tracing::info!("[{}] rewrite map {:?}", role, broker_rewrite_map); - let rewrite_map: anyhow::Result> = broker_rewrite_map + let rewrite_map: ConnectorResult> = broker_rewrite_map .map_or(Ok(BTreeMap::new()), |addr_map| { addr_map .into_iter() @@ -109,7 +111,7 @@ impl PrivateLinkConsumerContext { broker_rewrite_map: Option>, identifier: Option, metrics: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let inner = BrokerAddrRewriter::new(PrivateLinkContextRole::Consumer, broker_rewrite_map)?; Ok(Self { inner, @@ -152,7 +154,7 @@ impl PrivateLinkProducerContext { broker_rewrite_map: Option>, identifier: Option, metrics: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { let inner = BrokerAddrRewriter::new(PrivateLinkContextRole::Producer, broker_rewrite_map)?; Ok(Self { inner, @@ -195,11 +197,12 @@ fn kafka_props_broker_key(with_properties: &BTreeMap) -> &str { fn get_property_required( with_properties: &BTreeMap, property: &str, -) -> anyhow::Result { +) -> ConnectorResult { with_properties .get(property) .map(|s| s.to_lowercase()) - .ok_or_else(|| anyhow!("Required property \"{property}\" is not provided")) + .with_context(|| format!("Required property \"{property}\" is not provided")) + .map_err(Into::into) } #[inline(always)] @@ -216,7 +219,7 @@ pub fn insert_privatelink_broker_rewrite_map( with_options: &mut BTreeMap, svc: Option<&PrivateLinkService>, privatelink_endpoint: Option, -) -> anyhow::Result<()> { +) -> ConnectorResult<()> { let mut broker_rewrite_map = HashMap::new(); let servers = get_property_required(with_options, kafka_props_broker_key(with_options))?; let broker_addrs = servers.split(',').collect_vec(); @@ -227,11 +230,11 @@ pub fn insert_privatelink_broker_rewrite_map( with_options.remove(PRIVATE_LINK_TARGETS_KEY); if broker_addrs.len() != link_targets.len() { - return Err(anyhow!( + bail!( "The number of broker addrs {} does not match the number of private link targets {}", broker_addrs.len(), link_targets.len() - )); + ); } if let Some(endpoint) = privatelink_endpoint { @@ -241,15 +244,15 @@ pub fn insert_privatelink_broker_rewrite_map( } } else { if svc.is_none() { - return Err(anyhow!("Privatelink endpoint not found.",)); + bail!("Privatelink endpoint not found."); } let svc = svc.unwrap(); for (link, broker) in link_targets.iter().zip_eq_fast(broker_addrs.into_iter()) { if svc.dns_entries.is_empty() { - return Err(anyhow!( + bail!( "No available private link endpoints for Kafka broker {}", broker - )); + ); } // rewrite the broker address to the dns name w/o az // requires the NLB has enabled the cross-zone load balancing diff --git a/src/connector/src/source/kafka/source/reader.rs b/src/connector/src/source/kafka/source/reader.rs index bb8e70471282f..d67e45fee3837 100644 --- a/src/connector/src/source/kafka/source/reader.rs +++ b/src/connector/src/source/kafka/source/reader.rs @@ -17,7 +17,7 @@ use std::collections::HashMap; use std::mem::swap; use std::time::Duration; -use anyhow::{Context, Result}; +use anyhow::Context; use async_trait::async_trait; use futures::StreamExt; use futures_async_stream::try_stream; @@ -27,6 +27,7 @@ use rdkafka::error::KafkaError; use rdkafka::{ClientConfig, Message, Offset, TopicPartitionList}; use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; use crate::source::base::SourceMessage; use crate::source::kafka::{ @@ -168,7 +169,7 @@ impl KafkaSplitReader { } impl CommonSplitReader for KafkaSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn into_data_stream(self) { if self.offsets.values().all(|(start_offset, stop_offset)| { match (start_offset, stop_offset) { diff --git a/src/connector/src/source/kafka/split.rs b/src/connector/src/source/kafka/split.rs index 1043cdd01f2f6..a98707eb4ab3a 100644 --- a/src/connector/src/source/kafka/split.rs +++ b/src/connector/src/source/kafka/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Clone, Serialize, Deserialize, Debug, PartialEq, Hash)] @@ -32,15 +32,15 @@ impl SplitMetaData for KafkaSplit { format!("{}", self.partition).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset.as_str().parse::().unwrap()); Ok(()) } diff --git a/src/connector/src/source/kinesis/enumerator/client.rs b/src/connector/src/source/kinesis/enumerator/client.rs index bbf4e29258260..b8966f99ae1ac 100644 --- a/src/connector/src/source/kinesis/enumerator/client.rs +++ b/src/connector/src/source/kinesis/enumerator/client.rs @@ -12,11 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::Result; +use anyhow::Context as _; use async_trait::async_trait; use aws_sdk_kinesis::types::Shard; use aws_sdk_kinesis::Client as kinesis_client; +use risingwave_common::bail; +use crate::error::ConnectorResult as Result; use crate::source::kinesis::split::{KinesisOffset, KinesisSplit}; use crate::source::kinesis::*; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -56,15 +58,11 @@ impl SplitEnumerator for KinesisSplitEnumerator { .set_next_token(next_token) .stream_name(&self.stream_name) .send() - .await?; + .await + .context("failed to list kinesis shards")?; match list_shard_output.shards { Some(shard) => shard_collect.extend(shard), - None => { - return Err(anyhow::Error::msg(format!( - "no shards in stream {}", - &self.stream_name - ))); - } + None => bail!("no shards in stream {}", &self.stream_name), } match list_shard_output.next_token { diff --git a/src/connector/src/source/kinesis/source/reader.rs b/src/connector/src/source/kinesis/source/reader.rs index 51b3c77710410..363dfb8777b12 100644 --- a/src/connector/src/source/kinesis/source/reader.rs +++ b/src/connector/src/source/kinesis/source/reader.rs @@ -14,7 +14,7 @@ use std::time::Duration; -use anyhow::{anyhow, Result}; +use anyhow::anyhow; use async_trait::async_trait; use aws_sdk_kinesis::error::{DisplayErrorContext, SdkError}; use aws_sdk_kinesis::operation::get_records::{GetRecordsError, GetRecordsOutput}; @@ -22,8 +22,11 @@ use aws_sdk_kinesis::primitives::DateTime; use aws_sdk_kinesis::types::ShardIteratorType; use aws_sdk_kinesis::Client as KinesisClient; use futures_async_stream::try_stream; +use risingwave_common::bail; +use thiserror_ext::AsReport; use tokio_retry; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; use crate::source::kinesis::source::message::from_kinesis_record; use crate::source::kinesis::split::{KinesisOffset, KinesisSplit}; @@ -74,13 +77,11 @@ impl SplitReader for KinesisSplitReader { if let Some(ts) = &properties.timestamp_offset { KinesisOffset::Timestamp(*ts) } else { - return Err(anyhow!("scan.startup.timestamp.millis is required")); + bail!("scan.startup.timestamp.millis is required"); } } _ => { - return Err(anyhow!( - "invalid scan_startup_mode, accept earliest/latest/timestamp" - )) + bail!("invalid scan_startup_mode, accept earliest/latest/timestamp") } }, }, @@ -90,9 +91,7 @@ impl SplitReader for KinesisSplitReader { if !matches!(start_position, KinesisOffset::Timestamp(_)) && properties.timestamp_offset.is_some() { - return Err( - anyhow!("scan.startup.mode need to be set to 'timestamp' if you want to start with a specific timestamp") - ); + bail!("scan.startup.mode need to be set to 'timestamp' if you want to start with a specific timestamp"); } let stream_name = properties.common.stream_name.clone(); @@ -121,7 +120,7 @@ impl SplitReader for KinesisSplitReader { } impl CommonSplitReader for KinesisSplitReader { - #[try_stream(ok = Vec < SourceMessage >, error = anyhow::Error)] + #[try_stream(ok = Vec < SourceMessage >, error = crate::error::ConnectorError)] async fn into_data_stream(mut self) { self.new_shard_iter().await?; loop { @@ -189,14 +188,12 @@ impl CommonSplitReader for KinesisSplitReader { continue; } Err(e) => { - let error_msg = format!( - "Kinesis got a unhandled error: {:?}, stream {:?}, shard {:?}", - DisplayErrorContext(e), - self.stream_name, - self.shard_id, - ); - tracing::error!("{}", error_msg); - return Err(anyhow!("{}", error_msg)); + let error = anyhow!(e).context(format!( + "Kinesis got a unhandled error on stream {:?}, shard {:?}", + self.stream_name, self.shard_id + )); + tracing::error!(error = %error.as_report()); + return Err(error.into()); } } } @@ -251,7 +248,7 @@ impl KinesisSplitReader { if let Some(iter) = resp.shard_iterator() { Ok(iter.to_owned()) } else { - Err(anyhow!("shard iterator is none")) + bail!("shard iterator is none") } } diff --git a/src/connector/src/source/kinesis/split.rs b/src/connector/src/source/kinesis/split.rs index c327e6da61e92..1c7bea61f8744 100644 --- a/src/connector/src/source/kinesis/split.rs +++ b/src/connector/src/source/kinesis/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize, Hash)] @@ -39,15 +39,15 @@ impl SplitMetaData for KinesisSplit { self.shard_id.clone() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let start_offset = if start_offset.is_empty() { KinesisOffset::Earliest } else { diff --git a/src/connector/src/source/nats/enumerator/mod.rs b/src/connector/src/source/nats/enumerator/mod.rs index c5059fdc8186c..557921747b8f0 100644 --- a/src/connector/src/source/nats/enumerator/mod.rs +++ b/src/connector/src/source/nats/enumerator/mod.rs @@ -14,11 +14,12 @@ use std::sync::Arc; -use anyhow; use async_trait::async_trait; +use risingwave_common::bail; use super::source::{NatsOffset, NatsSplit}; use super::NatsProperties; +use crate::error::ConnectorResult; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator, SplitId}; #[derive(Debug, Clone)] @@ -36,7 +37,7 @@ impl SplitEnumerator for NatsSplitEnumerator { async fn new( properties: Self::Properties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let client = properties.common.build_client().await?; Ok(Self { subject: properties.common.subject, @@ -45,14 +46,14 @@ impl SplitEnumerator for NatsSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { // Nats currently does not support list_splits API, if we simple return the default 0 without checking the client status, will result executor crash let state = self.client.connection_state(); if state != async_nats::connection::State::Connected { - return Err(anyhow::anyhow!( + bail!( "Nats connection status is not connected, current status is {:?}", state - )); + ); } // TODO: to simplify the logic, return 1 split for first version let nats_split = NatsSplit { diff --git a/src/connector/src/source/nats/source/reader.rs b/src/connector/src/source/nats/source/reader.rs index 20a8f9c0dbc0b..05954538b91cf 100644 --- a/src/connector/src/source/nats/source/reader.rs +++ b/src/connector/src/source/nats/source/reader.rs @@ -12,14 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result}; +use anyhow::Context as _; use async_nats::jetstream::consumer; use async_trait::async_trait; use futures::StreamExt; use futures_async_stream::try_stream; +use risingwave_common::bail; use super::message::NatsMessage; use super::{NatsOffset, NatsSplit}; +use crate::error::ConnectorResult as Result; use crate::parser::ParserConfig; use crate::source::common::{into_chunk_stream, CommonSplitReader}; use crate::source::nats::NatsProperties; @@ -60,15 +62,15 @@ impl SplitReader for NatsSplitReader { "earliest" => NatsOffset::Earliest, "timestamp_millis" => { if let Some(time) = &properties.start_time { - NatsOffset::Timestamp(time.parse()?) + NatsOffset::Timestamp(time.parse().context( + "failed to parse the start time as nats offset timestamp", + )?) } else { - return Err(anyhow!("scan_startup_timestamp_millis is required")); + bail!("scan_startup_timestamp_millis is required"); } } _ => { - return Err(anyhow!( - "invalid scan_startup_mode, accept earliest/latest/timestamp_millis" - )) + bail!("invalid scan_startup_mode, accept earliest/latest/timestamp_millis") } }, }, @@ -101,7 +103,7 @@ impl SplitReader for NatsSplitReader { } impl CommonSplitReader for NatsSplitReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn into_data_stream(self) { let capacity = self.source_ctx.source_ctrl_opts.chunk_size; let messages = self.consumer.messages().await?; diff --git a/src/connector/src/source/nats/split.rs b/src/connector/src/source/nats/split.rs index 1a176102efb60..d3b4ded019016 100644 --- a/src/connector/src/source/nats/split.rs +++ b/src/connector/src/source/nats/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Ok}; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Debug, Clone, Eq, PartialEq, Serialize, Deserialize, Hash)] @@ -43,15 +43,15 @@ impl SplitMetaData for NatsSplit { format!("{}", self.split_id).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_sequence: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_sequence: String) -> ConnectorResult<()> { let start_sequence = if start_sequence.is_empty() { NatsOffset::Earliest } else { diff --git a/src/connector/src/source/nexmark/enumerator/mod.rs b/src/connector/src/source/nexmark/enumerator/mod.rs index b67a58f548965..7a53f02489fb9 100644 --- a/src/connector/src/source/nexmark/enumerator/mod.rs +++ b/src/connector/src/source/nexmark/enumerator/mod.rs @@ -14,6 +14,7 @@ use async_trait::async_trait; +use crate::error::ConnectorResult; use crate::source::nexmark::split::NexmarkSplit; use crate::source::nexmark::NexmarkProperties; use crate::source::{SourceEnumeratorContextRef, SplitEnumerator}; @@ -32,12 +33,12 @@ impl SplitEnumerator for NexmarkSplitEnumerator { async fn new( properties: NexmarkProperties, _context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { let split_num = properties.split_num; Ok(Self { split_num }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { let mut splits = vec![]; for i in 0..self.split_num { splits.push(NexmarkSplit { @@ -52,9 +53,8 @@ impl SplitEnumerator for NexmarkSplitEnumerator { #[cfg(test)] mod tests { - use anyhow::Result; - use super::*; + use crate::error::ConnectorResult as Result; use crate::source::SplitMetaData; #[tokio::test] diff --git a/src/connector/src/source/nexmark/source/reader.rs b/src/connector/src/source/nexmark/source/reader.rs index e7621e5325524..fd68348d6faf6 100644 --- a/src/connector/src/source/nexmark/source/reader.rs +++ b/src/connector/src/source/nexmark/source/reader.rs @@ -27,6 +27,7 @@ use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, ScalarImpl}; use tokio::time::Instant; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::data_gen_util::spawn_data_generation_stream; use crate::source::nexmark::source::combined_event::{ @@ -64,7 +65,7 @@ impl SplitReader for NexmarkSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { tracing::debug!("Splits for nexmark found! {:?}", splits); assert!(splits.len() == 1); // TODO: currently, assume there's only one split in one reader @@ -163,7 +164,7 @@ impl NexmarkSplitReader { } } - #[try_stream(boxed, ok = StreamChunk, error = anyhow::Error)] + #[try_stream(boxed, ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_native_stream(mut self) { let start_time = Instant::now(); let start_offset = self.generator.global_offset(); @@ -213,7 +214,7 @@ mod tests { use crate::source::{SourceEnumeratorContext, SplitEnumerator}; #[tokio::test] - async fn test_nexmark_split_reader() -> anyhow::Result<()> { + async fn test_nexmark_split_reader() -> crate::error::ConnectorResult<()> { let props = NexmarkProperties { split_num: 2, min_event_gap_in_ns: 0, @@ -247,7 +248,7 @@ mod tests { } #[tokio::test] - async fn test_nexmark_event_num() -> anyhow::Result<()> { + async fn test_nexmark_event_num() -> crate::error::ConnectorResult<()> { let max_chunk_size = 32; let event_num = max_chunk_size * 128 + 1; let props = NexmarkProperties { diff --git a/src/connector/src/source/nexmark/split.rs b/src/connector/src/source/nexmark/split.rs index d68aa2c8e1aa4..5150f1b6a1e1d 100644 --- a/src/connector/src/source/nexmark/split.rs +++ b/src/connector/src/source/nexmark/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::{SplitId, SplitMetaData}; #[derive(Clone, Debug, Default, Serialize, Deserialize, PartialEq, Hash)] @@ -31,15 +31,15 @@ impl SplitMetaData for NexmarkSplit { format!("{}-{}", self.split_num, self.split_index).into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.start_offset = Some(start_offset.as_str().parse::().unwrap()); Ok(()) } diff --git a/src/connector/src/source/pulsar/enumerator/client.rs b/src/connector/src/source/pulsar/enumerator/client.rs index d92e633060616..dddcf927d0c3f 100644 --- a/src/connector/src/source/pulsar/enumerator/client.rs +++ b/src/connector/src/source/pulsar/enumerator/client.rs @@ -12,12 +12,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, bail, Result}; +use anyhow::anyhow; use async_trait::async_trait; use itertools::Itertools; use pulsar::{Pulsar, TokioExecutor}; +use risingwave_common::bail; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::topic::{parse_topic, Topic}; use crate::source::pulsar::PulsarProperties; @@ -45,7 +47,7 @@ impl SplitEnumerator for PulsarSplitEnumerator { async fn new( properties: PulsarProperties, _context: SourceEnumeratorContextRef, - ) -> Result { + ) -> ConnectorResult { let pulsar = properties .common .build_client(&properties.oauth, &properties.aws_auth_props) @@ -80,7 +82,7 @@ impl SplitEnumerator for PulsarSplitEnumerator { }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { let offset = self.start_offset.clone(); // MessageId is only used when recovering from a State assert!(!matches!(offset, PulsarEnumeratorOffset::MessageId(_))); diff --git a/src/connector/src/source/pulsar/source/reader.rs b/src/connector/src/source/pulsar/source/reader.rs index 139af839bd16d..9ed810dfc933a 100644 --- a/src/connector/src/source/pulsar/source/reader.rs +++ b/src/connector/src/source/pulsar/source/reader.rs @@ -31,7 +31,9 @@ use pulsar::{Consumer, ConsumerBuilder, ConsumerOptions, Pulsar, SubType, TokioE use risingwave_common::array::{DataChunk, StreamChunk}; use risingwave_common::catalog::ROWID_PREFIX; use risingwave_common::{bail, ensure}; +use thiserror_ext::AsReport; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::pulsar::split::PulsarSplit; use crate::source::pulsar::{PulsarEnumeratorOffset, PulsarProperties}; @@ -56,7 +58,7 @@ impl SplitReader for PulsarSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { ensure!(splits.len() == 1, "only support single split"); let split = splits.into_iter().next().unwrap(); let topic = split.topic.to_string(); @@ -106,7 +108,7 @@ pub struct PulsarBrokerReader { } // {ledger_id}:{entry_id}:{partition}:{batch_index} -fn parse_message_id(id: &str) -> anyhow::Result { +fn parse_message_id(id: &str) -> ConnectorResult { let splits = id.split(':').collect_vec(); if splits.len() < 2 || splits.len() > 4 { @@ -150,7 +152,7 @@ impl SplitReader for PulsarBrokerReader { parser_config: ParserConfig, source_ctx: SourceContextRef, _columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { ensure!(splits.len() == 1, "only support single split"); let split = splits.into_iter().next().unwrap(); let pulsar = props @@ -233,7 +235,7 @@ impl SplitReader for PulsarBrokerReader { } impl CommonSplitReader for PulsarBrokerReader { - #[try_stream(ok = Vec, error = anyhow::Error)] + #[try_stream(ok = Vec, error = crate::error::ConnectorError)] async fn into_data_stream(self) { let max_chunk_size = self.source_ctx.source_ctrl_opts.chunk_size; #[for_await] @@ -278,7 +280,7 @@ impl PulsarIcebergReader { } } - async fn scan(&self) -> anyhow::Result { + async fn scan(&self) -> ConnectorResult { let table = self.create_iceberg_table().await?; let schema = table.current_table_metadata().current_schema()?; tracing::debug!("Created iceberg pulsar table, schema is: {:?}", schema,); @@ -321,12 +323,13 @@ impl PulsarIcebergReader { .new_scan_builder() .with_partition_value(partition_value) .with_batch_size(max_chunk_size) - .build()? + .build() + .context("failed to build iceberg table scan")? .scan(&table) .await?) } - async fn create_iceberg_table(&self) -> anyhow::Result
{ + async fn create_iceberg_table(&self) -> ConnectorResult
{ let catalog = load_catalog(&self.build_iceberg_configs()?) .await .context("Unable to load iceberg catalog")?; @@ -340,7 +343,7 @@ impl PulsarIcebergReader { Ok(table) } - #[try_stream(ok = (StreamChunk, HashMap), error = anyhow::Error)] + #[try_stream(ok = (StreamChunk, HashMap), error = crate::error::ConnectorError)] async fn as_stream_chunk_stream(&self) { #[for_await] for file_scan in self.scan().await? { @@ -355,7 +358,7 @@ impl PulsarIcebergReader { } } - #[try_stream(ok = StreamChunk, error = anyhow::Error)] + #[try_stream(ok = StreamChunk, error = crate::error::ConnectorError)] async fn into_stream(self) { let (props, mut split, parser_config, source_ctx) = ( self.props.clone(), @@ -368,8 +371,9 @@ impl PulsarIcebergReader { #[for_await] for msg in self.as_stream_chunk_stream() { - let (_chunk, mapping) = - msg.inspect_err(|e| tracing::error!("Failed to read message from iceberg: {}", e))?; + let (_chunk, mapping) = msg.inspect_err( + |e| tracing::error!(error = %e.as_report(), "Failed to read message from iceberg"), + )?; last_msg_id = mapping.get(self.split.topic.to_string().as_str()).cloned(); } @@ -394,7 +398,7 @@ impl PulsarIcebergReader { } } - fn build_iceberg_configs(&self) -> anyhow::Result> { + fn build_iceberg_configs(&self) -> ConnectorResult> { let mut iceberg_configs = HashMap::new(); let bucket = self @@ -451,7 +455,7 @@ impl PulsarIcebergReader { fn convert_record_batch_to_source_with_state( &self, record_batch: &RecordBatch, - ) -> anyhow::Result<(StreamChunk, HashMap)> { + ) -> ConnectorResult<(StreamChunk, HashMap)> { let mut offsets = Vec::with_capacity(record_batch.num_rows()); let ledger_id_array = record_batch @@ -493,7 +497,8 @@ impl PulsarIcebergReader { .iter() .filter(|col| col.name != ROWID_PREFIX) .map(|col| record_batch.schema().index_of(col.name.as_str())) - .try_collect()?; + .try_collect() + .context("failed to look up column name in arrow record batch")?; for row in 0..record_batch.num_rows() { let offset = format!( @@ -507,7 +512,8 @@ impl PulsarIcebergReader { offsets.push(offset); } - let data_chunk = DataChunk::try_from(&record_batch.project(&field_indices)?)?; + let data_chunk = DataChunk::try_from(&record_batch.project(&field_indices)?) + .context("failed to convert arrow record batch to data chunk")?; let stream_chunk = StreamChunk::from(data_chunk); diff --git a/src/connector/src/source/pulsar/split.rs b/src/connector/src/source/pulsar/split.rs index 36f9bc47e3ec5..bf9b63d99d74f 100644 --- a/src/connector/src/source/pulsar/split.rs +++ b/src/connector/src/source/pulsar/split.rs @@ -12,10 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::anyhow; use risingwave_common::types::JsonbVal; use serde::{Deserialize, Serialize}; +use crate::error::ConnectorResult; use crate::source::pulsar::topic::Topic; use crate::source::pulsar::PulsarEnumeratorOffset; use crate::source::{SplitId, SplitMetaData}; @@ -32,15 +32,15 @@ impl SplitMetaData for PulsarSplit { self.topic.to_string().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } fn encode_to_json(&self) -> JsonbVal { serde_json::to_value(self.clone()).unwrap().into() } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { let start_offset = if start_offset.is_empty() { PulsarEnumeratorOffset::Earliest } else { diff --git a/src/connector/src/source/pulsar/topic.rs b/src/connector/src/source/pulsar/topic.rs index 4512662d6252c..352c7e47d8da2 100644 --- a/src/connector/src/source/pulsar/topic.rs +++ b/src/connector/src/source/pulsar/topic.rs @@ -12,10 +12,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -use anyhow::{anyhow, Result}; +use anyhow::anyhow; +use risingwave_common::bail; use serde::{Deserialize, Serialize}; use urlencoding::encode; +use crate::error::ConnectorResult as Result; + const PERSISTENT_DOMAIN: &str = "persistent"; const NON_PERSISTENT_DOMAIN: &str = "non-persistent"; const PUBLIC_TENANT: &str = "public"; @@ -59,7 +62,7 @@ impl Topic { pub fn sub_topic(&self, partition: i32) -> Result { if partition < 0 { - return Err(anyhow!("invalid partition index number")); + bail!("invalid partition index number"); } if self.topic.contains(PARTITIONED_TOPIC_SUFFIX) { @@ -119,11 +122,11 @@ pub fn parse_topic(topic: &str) -> Result { ), 3 => format!("{}://{}", PERSISTENT_DOMAIN, topic), _ => { - return Err(anyhow!( + bail!( "Invalid short topic name '{}', \ it should be in the format of // or ", topic - )); + ); } }; } @@ -133,10 +136,10 @@ pub fn parse_topic(topic: &str) -> Result { let domain = match parts[0] { PERSISTENT_DOMAIN | NON_PERSISTENT_DOMAIN => parts[0], _ => { - return Err(anyhow!( + bail!( "The domain only can be specified as 'persistent' or 'non-persistent'. Input domain is '{}'", parts[0] - )); + ); } }; @@ -144,10 +147,10 @@ pub fn parse_topic(topic: &str) -> Result { let parts: Vec<&str> = rest.splitn(3, '/').collect(); if parts.len() != 3 { - return Err(anyhow!( + bail!( "invalid topic name '{}', it should be in the format of //", rest - )); + ); } let parsed_topic = Topic { @@ -159,7 +162,7 @@ pub fn parse_topic(topic: &str) -> Result { }; if parsed_topic.topic.is_empty() { - return Err(anyhow!("topic name cannot be empty".to_string(),)); + bail!("topic name cannot be empty".to_string()); } Ok(parsed_topic) diff --git a/src/connector/src/source/reader/desc.rs b/src/connector/src/source/reader/desc.rs index a842b091ab928..46107c2d73d0a 100644 --- a/src/connector/src/source/reader/desc.rs +++ b/src/connector/src/source/reader/desc.rs @@ -25,6 +25,7 @@ use risingwave_pb::plan_common::{AdditionalColumn, PbColumnCatalog}; #[expect(deprecated)] use super::fs_reader::FsSourceReader; use super::reader::SourceReader; +use crate::error::ConnectorResult; use crate::parser::additional_columns::{ build_additional_column_catalog, COMMON_COMPATIBLE_ADDITIONAL_COLUMNS, COMPATIBLE_ADDITIONAL_COLUMNS, @@ -176,7 +177,7 @@ impl SourceDescBuilder { columns } - pub fn build(self) -> anyhow::Result { + pub fn build(self) -> ConnectorResult { let columns = self.column_catalogs_to_source_column_descs(); let psrser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; @@ -201,7 +202,7 @@ impl SourceDescBuilder { #[deprecated = "will be replaced by new fs source (list + fetch)"] #[expect(deprecated)] - pub fn build_fs_source_desc(&self) -> anyhow::Result { + pub fn build_fs_source_desc(&self) -> ConnectorResult { let parser_config = SpecificParserConfig::new(&self.source_info, &self.with_properties)?; match ( diff --git a/src/connector/src/source/reader/fs_reader.rs b/src/connector/src/source/reader/fs_reader.rs index f64a9def6aab6..93a0bd2c2d6a8 100644 --- a/src/connector/src/source/reader/fs_reader.rs +++ b/src/connector/src/source/reader/fs_reader.rs @@ -23,6 +23,7 @@ use futures::StreamExt; use risingwave_common::catalog::ColumnId; use crate::dispatch_source_prop; +use crate::error::ConnectorResult; use crate::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use crate::source::{ create_split_reader, BoxChunkSourceStream, ConnectorProperties, ConnectorState, @@ -44,7 +45,7 @@ impl FsSourceReader { columns: Vec, connector_node_addr: Option, parser_config: SpecificParserConfig, - ) -> anyhow::Result { + ) -> ConnectorResult { // Store the connector node address to properties for later use. let mut source_props: HashMap = HashMap::from_iter(properties.clone()); connector_node_addr @@ -62,7 +63,7 @@ impl FsSourceReader { fn get_target_columns( &self, column_ids: Vec, - ) -> anyhow::Result> { + ) -> ConnectorResult> { column_ids .iter() .map(|id| { @@ -75,6 +76,7 @@ impl FsSourceReader { .cloned() }) .try_collect() + .map_err(Into::into) } pub async fn to_stream( @@ -82,7 +84,7 @@ impl FsSourceReader { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> anyhow::Result { + ) -> ConnectorResult { let config = self.config.clone(); let columns = self.get_target_columns(column_ids)?; diff --git a/src/connector/src/source/reader/reader.rs b/src/connector/src/source/reader/reader.rs index ba9bd4dded4d8..833c9661c3ca1 100644 --- a/src/connector/src/source/reader/reader.rs +++ b/src/connector/src/source/reader/reader.rs @@ -24,8 +24,10 @@ use itertools::Itertools; use risingwave_common::bail; use risingwave_common::catalog::ColumnId; use rw_futures_util::select_all; +use thiserror_ext::AsReport as _; use crate::dispatch_source_prop; +use crate::error::ConnectorResult; use crate::parser::{CommonParserConfig, ParserConfig, SpecificParserConfig}; use crate::source::filesystem::opendal_source::opendal_enumerator::OpendalEnumerator; use crate::source::filesystem::opendal_source::{ @@ -51,7 +53,7 @@ impl SourceReader { columns: Vec, connector_message_buffer_size: usize, parser_config: SpecificParserConfig, - ) -> anyhow::Result { + ) -> ConnectorResult { let config = ConnectorProperties::extract(properties, false)?; Ok(Self { @@ -65,7 +67,7 @@ impl SourceReader { fn get_target_columns( &self, column_ids: Vec, - ) -> anyhow::Result> { + ) -> ConnectorResult> { column_ids .iter() .map(|id| { @@ -78,9 +80,10 @@ impl SourceReader { .cloned() }) .try_collect() + .map_err(Into::into) } - pub fn get_source_list(&self) -> anyhow::Result> { + pub fn get_source_list(&self) -> ConnectorResult> { let config = self.config.clone(); match config { ConnectorProperties::Gcs(prop) => { @@ -107,7 +110,7 @@ impl SourceReader { state: ConnectorState, column_ids: Vec, source_ctx: Arc, - ) -> anyhow::Result { + ) -> ConnectorResult { let Some(splits) = state else { return Ok(pending().boxed()); }; @@ -165,7 +168,7 @@ impl SourceReader { } } -#[try_stream(boxed, ok = FsPageItem, error = anyhow::Error)] +#[try_stream(boxed, ok = FsPageItem, error = crate::error::ConnectorError)] async fn build_opendal_fs_list_stream(lister: OpendalEnumerator) { let matcher = lister.get_matcher(); let mut object_metadata_iter = lister.list().await?; @@ -185,7 +188,7 @@ async fn build_opendal_fs_list_stream(lister: OpendalEnumera } } Err(err) => { - tracing::error!("list object fail, err {}", err); + tracing::error!(error = %err.as_report(), "list object fail"); return Err(err); } } diff --git a/src/connector/src/source/test_source.rs b/src/connector/src/source/test_source.rs index e0b901ddbf253..6d224593d7a2e 100644 --- a/src/connector/src/source/test_source.rs +++ b/src/connector/src/source/test_source.rs @@ -15,13 +15,14 @@ use std::collections::HashMap; use std::sync::{Arc, OnceLock}; -use anyhow::anyhow; use async_trait::async_trait; use parking_lot::Mutex; +use risingwave_common::bail; use risingwave_common::types::JsonbVal; use serde_derive::{Deserialize, Serialize}; use with_options::WithOptions; +use crate::error::ConnectorResult; use crate::parser::ParserConfig; use crate::source::{ BoxChunkSourceStream, Column, SourceContextRef, SourceEnumeratorContextRef, SourceProperties, @@ -32,7 +33,7 @@ pub type BoxListSplits = Box< dyn FnMut( TestSourceProperties, SourceEnumeratorContextRef, - ) -> anyhow::Result> + ) -> ConnectorResult> + Send + 'static, >; @@ -59,7 +60,7 @@ impl BoxSource { list_splits: impl FnMut( TestSourceProperties, SourceEnumeratorContextRef, - ) -> anyhow::Result> + ) -> ConnectorResult> + Send + 'static, into_source_stream: impl FnMut( @@ -124,11 +125,11 @@ impl TryFromHashmap for TestSourceProperties { fn try_from_hashmap( props: HashMap, _deny_unknown_fields: bool, - ) -> anyhow::Result { + ) -> ConnectorResult { if cfg!(any(madsim, test)) { Ok(TestSourceProperties { properties: props }) } else { - Err(anyhow!("test source only available at test")) + bail!("test source only available at test") } } } @@ -149,11 +150,11 @@ impl SplitMetaData for TestSourceSplit { serde_json::to_value(self.clone()).unwrap().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, start_offset: String) -> ConnectorResult<()> { self.offset = start_offset; Ok(()) } @@ -172,14 +173,14 @@ impl SplitEnumerator for TestSourceSplitEnumerator { async fn new( properties: Self::Properties, context: SourceEnumeratorContextRef, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self { properties, context, }) } - async fn list_splits(&mut self) -> anyhow::Result> { + async fn list_splits(&mut self) -> ConnectorResult> { (get_registry() .box_source .lock() @@ -208,7 +209,7 @@ impl SplitReader for TestSourceSplitReader { parser_config: ParserConfig, source_ctx: SourceContextRef, columns: Option>, - ) -> anyhow::Result { + ) -> ConnectorResult { Ok(Self { properties, state, diff --git a/src/error/src/lib.rs b/src/error/src/lib.rs index d3364485e8f2f..f7a2611b84a65 100644 --- a/src/error/src/lib.rs +++ b/src/error/src/lib.rs @@ -23,3 +23,6 @@ pub mod anyhow; pub mod tonic; + +// Re-export the `thiserror-ext` crate. +pub use thiserror_ext::*; diff --git a/src/expr/impl/Cargo.toml b/src/expr/impl/Cargo.toml index a8c66be8a2281..e7b765820dfdd 100644 --- a/src/expr/impl/Cargo.toml +++ b/src/expr/impl/Cargo.toml @@ -25,6 +25,7 @@ chrono = { version = "0.4", default-features = false, features = [ "clock", "std", ] } +chrono-tz = { version = "0.8", features = ["case-insensitive"] } fancy-regex = "0.13" futures-async-stream = { workspace = true } futures-util = "0.3" diff --git a/src/expr/impl/src/scalar/make_time.rs b/src/expr/impl/src/scalar/make_time.rs index add8759299197..ae1ff58033eed 100644 --- a/src/expr/impl/src/scalar/make_time.rs +++ b/src/expr/impl/src/scalar/make_time.rs @@ -129,8 +129,6 @@ mod tests { use chrono::{NaiveDate, NaiveDateTime, NaiveTime}; use risingwave_common::types::{Date, Timestamp}; - /// This test is to testify that our `Date` expressess a year `-X` as `X+1 BC`, while `Timestamp` expresses it as `-X`. - /// Can be removed if we change the `ToText` implementation of `Date` or `Timestamp`. #[test] fn test_naive_date_and_time() { let year = -1973; @@ -154,7 +152,7 @@ mod tests { let date_time = Timestamp(NaiveDateTime::new(naive_date, naive_time)); assert_eq!( date_time.to_string(), - String::from("-1973-02-02 12:34:56.789") + String::from("1974-02-02 12:34:56.789 BC") ); } } diff --git a/src/expr/impl/src/scalar/timestamptz.rs b/src/expr/impl/src/scalar/timestamptz.rs index 83e77011ec6be..7e7f1a0164d6a 100644 --- a/src/expr/impl/src/scalar/timestamptz.rs +++ b/src/expr/impl/src/scalar/timestamptz.rs @@ -15,7 +15,9 @@ use std::fmt::Write; use num_traits::CheckedNeg; -use risingwave_common::types::{CheckedAdd, Interval, IntoOrdered, Timestamp, Timestamptz, F64}; +use risingwave_common::types::{ + write_date_time_tz, CheckedAdd, Interval, IntoOrdered, Timestamp, Timestamptz, F64, +}; use risingwave_expr::{function, ExprError, Result}; use thiserror_ext::AsReport; @@ -72,13 +74,7 @@ pub fn timestamptz_to_string( ) -> Result<()> { let time_zone = Timestamptz::lookup_time_zone(time_zone).map_err(time_zone_err)?; let instant_local = elem.to_datetime_in_zone(time_zone); - write!( - writer, - "{}", - instant_local.format("%Y-%m-%d %H:%M:%S%.f%:z") - ) - .map_err(|e| ExprError::Internal(e.into()))?; - Ok(()) + write_date_time_tz(instant_local, writer).map_err(|e| ExprError::Internal(e.into())) } // Tries to interpret the string with a timezone, and if failing, tries to interpret the string as a diff --git a/src/frontend/planner_test/src/lib.rs b/src/frontend/planner_test/src/lib.rs index f7ce8cc04a950..1578fa5cd0dfe 100644 --- a/src/frontend/planner_test/src/lib.rs +++ b/src/frontend/planner_test/src/lib.rs @@ -944,7 +944,7 @@ pub async fn run_test_file(file_path: &Path, file_content: &str) -> Result<()> { "Test #{i} (id: {}) failed, SQL:\n{}\nError: {}", c.id().clone().unwrap_or_else(|| "".to_string()), c.sql(), - e + e.as_report() ); failed_num += 1; } diff --git a/src/frontend/planner_test/tests/planner_test_runner.rs b/src/frontend/planner_test/tests/planner_test_runner.rs index 00adc280ddce3..0ce6bba6d5e66 100644 --- a/src/frontend/planner_test/tests/planner_test_runner.rs +++ b/src/frontend/planner_test/tests/planner_test_runner.rs @@ -43,7 +43,7 @@ fn main() { let file_name = path.file_name().unwrap().to_string_lossy().to_string(); let test_case_name = file_name.split('.').next().unwrap().to_string(); - tests.push(Trial::test(format!("{test_case_name}_test"), move || { + tests.push(Trial::test(test_case_name, move || { let path = test_data_dir().join("input").join(file_name); let file_content = std::fs::read_to_string(&path).unwrap(); diff --git a/src/frontend/planner_test/tests/testdata/input/dbeaver.yaml b/src/frontend/planner_test/tests/testdata/input/dbeaver.yaml new file mode 100644 index 0000000000000..ebe7d16659ca4 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/input/dbeaver.yaml @@ -0,0 +1,43 @@ +- sql: | + SELECT DISTINCT dep.deptype, dep.classid, dep.objid, cl.relkind, attr.attname,pg_get_expr(ad.adbin, ad.adrelid) adefval, + CASE WHEN cl.relkind IS NOT NULL THEN cl.relkind::text || COALESCE(dep.objsubid::text, '')::text + WHEN tg.oid IS NOT NULL THEN 'T'::text + WHEN ty.oid IS NOT NULL THEN 'y'::text + WHEN ns.oid IS NOT NULL THEN 'n'::text + WHEN pr.oid IS NOT NULL THEN 'p'::text + WHEN la.oid IS NOT NULL THEN 'l'::text + WHEN rw.oid IS NOT NULL THEN 'R'::text + WHEN co.oid IS NOT NULL THEN 'C'::text || contype::text + WHEN ad.oid IS NOT NULL THEN 'A'::text + ELSE '' + END AS type, + COALESCE(coc.relname, clrw.relname, tgr.relname) AS ownertable, + CASE WHEN cl.relname IS NOT NULL AND att.attname IS NOT NULL THEN cl.relname || '.' || att.attname + ELSE COALESCE(cl.relname, co.conname, pr.proname, tg.tgname, ty.typname, la.lanname, rw.rulename, ns.nspname) + END AS refname, + COALESCE(nsc.nspname, nso.nspname, nsp.nspname, nst.nspname, nsrw.nspname, tgrn.nspname) AS nspname + FROM pg_depend dep + LEFT JOIN pg_class cl ON dep.objid=cl.oid + LEFT JOIN pg_attribute att ON dep.objid=att.attrelid AND dep.objsubid=att.attnum + LEFT JOIN pg_namespace nsc ON cl.relnamespace=nsc.oid + LEFT JOIN pg_proc pr ON dep.objid=pr.oid + LEFT JOIN pg_namespace nsp ON pr.pronamespace=nsp.oid + LEFT JOIN pg_trigger tg ON dep.objid=tg.oid + LEFT JOIN pg_class tgr ON tg.tgrelid=tgr.oid + LEFT JOIN pg_namespace tgrn ON tgr.relnamespace=tgrn.oid + LEFT JOIN pg_type ty ON dep.objid=ty.oid + LEFT JOIN pg_namespace nst ON ty.typnamespace=nst.oid + LEFT JOIN pg_constraint co ON dep.objid=co.oid + LEFT JOIN pg_class coc ON co.conrelid=coc.oid + LEFT JOIN pg_namespace nso ON co.connamespace=nso.oid + LEFT JOIN pg_rewrite rw ON dep.objid=rw.oid + LEFT JOIN pg_class clrw ON clrw.oid=rw.ev_class + LEFT JOIN pg_namespace nsrw ON clrw.relnamespace=nsrw.oid + LEFT JOIN pg_language la ON dep.objid=la.oid + LEFT JOIN pg_namespace ns ON dep.objid=ns.oid + LEFT JOIN pg_attrdef ad ON ad.oid=dep.objid + LEFT JOIN pg_attribute attr ON attr.attrelid=ad.adrelid and attr.attnum=ad.adnum + WHERE dep.refobjid=$1 + ORDER BY type + expected_outputs: + - batch_plan diff --git a/src/frontend/planner_test/tests/testdata/output/dbeaver.yaml b/src/frontend/planner_test/tests/testdata/output/dbeaver.yaml new file mode 100644 index 0000000000000..d0fcf6db1cfd4 --- /dev/null +++ b/src/frontend/planner_test/tests/testdata/output/dbeaver.yaml @@ -0,0 +1,171 @@ +# This file is automatically generated. See `src/frontend/planner_test/README.md` for more information. +- sql: | + SELECT DISTINCT dep.deptype, dep.classid, dep.objid, cl.relkind, attr.attname,pg_get_expr(ad.adbin, ad.adrelid) adefval, + CASE WHEN cl.relkind IS NOT NULL THEN cl.relkind::text || COALESCE(dep.objsubid::text, '')::text + WHEN tg.oid IS NOT NULL THEN 'T'::text + WHEN ty.oid IS NOT NULL THEN 'y'::text + WHEN ns.oid IS NOT NULL THEN 'n'::text + WHEN pr.oid IS NOT NULL THEN 'p'::text + WHEN la.oid IS NOT NULL THEN 'l'::text + WHEN rw.oid IS NOT NULL THEN 'R'::text + WHEN co.oid IS NOT NULL THEN 'C'::text || contype::text + WHEN ad.oid IS NOT NULL THEN 'A'::text + ELSE '' + END AS type, + COALESCE(coc.relname, clrw.relname, tgr.relname) AS ownertable, + CASE WHEN cl.relname IS NOT NULL AND att.attname IS NOT NULL THEN cl.relname || '.' || att.attname + ELSE COALESCE(cl.relname, co.conname, pr.proname, tg.tgname, ty.typname, la.lanname, rw.rulename, ns.nspname) + END AS refname, + COALESCE(nsc.nspname, nso.nspname, nsp.nspname, nst.nspname, nsrw.nspname, tgrn.nspname) AS nspname + FROM pg_depend dep + LEFT JOIN pg_class cl ON dep.objid=cl.oid + LEFT JOIN pg_attribute att ON dep.objid=att.attrelid AND dep.objsubid=att.attnum + LEFT JOIN pg_namespace nsc ON cl.relnamespace=nsc.oid + LEFT JOIN pg_proc pr ON dep.objid=pr.oid + LEFT JOIN pg_namespace nsp ON pr.pronamespace=nsp.oid + LEFT JOIN pg_trigger tg ON dep.objid=tg.oid + LEFT JOIN pg_class tgr ON tg.tgrelid=tgr.oid + LEFT JOIN pg_namespace tgrn ON tgr.relnamespace=tgrn.oid + LEFT JOIN pg_type ty ON dep.objid=ty.oid + LEFT JOIN pg_namespace nst ON ty.typnamespace=nst.oid + LEFT JOIN pg_constraint co ON dep.objid=co.oid + LEFT JOIN pg_class coc ON co.conrelid=coc.oid + LEFT JOIN pg_namespace nso ON co.connamespace=nso.oid + LEFT JOIN pg_rewrite rw ON dep.objid=rw.oid + LEFT JOIN pg_class clrw ON clrw.oid=rw.ev_class + LEFT JOIN pg_namespace nsrw ON clrw.relnamespace=nsrw.oid + LEFT JOIN pg_language la ON dep.objid=la.oid + LEFT JOIN pg_namespace ns ON dep.objid=ns.oid + LEFT JOIN pg_attrdef ad ON ad.oid=dep.objid + LEFT JOIN pg_attribute attr ON attr.attrelid=ad.adrelid and attr.attnum=ad.adnum + WHERE dep.refobjid=$1 + ORDER BY type + batch_plan: |- + BatchExchange { order: [$expr4 ASC], dist: Single } + └─BatchSort { order: [$expr4 ASC] } + └─BatchHashAgg { group_key: [null:Varchar, null:Int32, null:Int32, $expr1, rw_columns.name, '':Varchar, $expr4, $expr5, $expr6, $expr7], aggs: [] } + └─BatchExchange { order: [], dist: HashShard(null:Varchar, null:Int32, null:Int32, $expr1, rw_columns.name, '':Varchar, $expr4, $expr5, $expr6, $expr7) } + └─BatchProject { exprs: [null:Varchar, null:Int32, null:Int32, $expr1, rw_columns.name, '':Varchar, Case(IsNotNull($expr1), ConcatOp($expr1, Coalesce(null:Int16::Varchar, '':Varchar)), IsNotNull(null:Int32), 'T':Varchar, IsNotNull(rw_types.id), 'y':Varchar, IsNotNull(rw_schemas.id), 'n':Varchar, IsNotNull(null:Int32), 'p':Varchar, IsNotNull(null:Int32), 'l':Varchar, IsNotNull(null:Int32), 'R':Varchar, IsNotNull(pg_constraint.oid), ConcatOp('C':Varchar, pg_constraint.contype), IsNotNull(null:Int32), 'A':Varchar, '':Varchar) as $expr4, Coalesce(rw_tables.name, rw_tables.name, rw_tables.name) as $expr5, Case((IsNotNull(rw_tables.name) AND IsNotNull(rw_columns.name)), ConcatOp(ConcatOp(rw_tables.name, '.':Varchar), rw_columns.name), Coalesce(rw_tables.name, pg_constraint.conname, null:Varchar, null:Varchar, rw_types.name, null:Varchar, null:Varchar, rw_schemas.name)) as $expr6, Coalesce(rw_schemas.name, rw_schemas.name, rw_schemas.name, rw_schemas.name, rw_schemas.name, rw_schemas.name) as $expr7] } + └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_columns.relation_id AND null:Int16 = $expr3, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name, rw_types.id, rw_types.name, rw_schemas.name, pg_constraint.oid, pg_constraint.conname, pg_constraint.contype, rw_tables.name, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.id, rw_schemas.name, null:Int32, rw_columns.name] } + ├─BatchExchange { order: [], dist: HashShard(null:Int32, null:Int16) } + │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = null:Int32, output: all } + │ ├─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_schemas.id, output: all } + │ │ ├─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = null:Int32, output: all } + │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: rw_tables.schema_id = rw_schemas.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name, rw_types.id, rw_types.name, rw_schemas.name, pg_constraint.oid, pg_constraint.conname, pg_constraint.contype, rw_tables.name, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name] } + │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(rw_tables.schema_id) } + │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_tables.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name, rw_types.id, rw_types.name, rw_schemas.name, pg_constraint.oid, pg_constraint.conname, pg_constraint.contype, rw_tables.name, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_tables.schema_id] } + │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = null:Int32, output: all } + │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: pg_constraint.connamespace = rw_schemas.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name, rw_types.id, rw_types.name, rw_schemas.name, pg_constraint.oid, pg_constraint.conname, pg_constraint.contype, rw_tables.name, rw_schemas.name] } + │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(pg_constraint.connamespace) } + │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: pg_constraint.conrelid = rw_tables.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name, rw_types.id, rw_types.name, rw_schemas.name, pg_constraint.oid, pg_constraint.conname, pg_constraint.connamespace, pg_constraint.contype, rw_tables.name] } + │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(pg_constraint.conrelid) } + │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = pg_constraint.oid, output: all } + │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: rw_schemas.id = rw_schemas.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name, rw_types.id, rw_types.name, rw_schemas.name] } + │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_types.id, output: all } + │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: rw_tables.schema_id = rw_schemas.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_schemas.name] } + │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(rw_tables.schema_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_tables.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name, null:Int32, null:Varchar, rw_tables.name, rw_tables.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = null:Int32, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_schemas.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name, null:Int32, null:Varchar, rw_schemas.name] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = null:Int32, output: all } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: rw_tables.schema_id = rw_schemas.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, $expr1, rw_columns.name, rw_schemas.name] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(rw_tables.schema_id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_columns.relation_id AND null:Int16 = $expr2, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, rw_tables.schema_id, $expr1, rw_columns.name] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32, null:Int16) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchHashJoin { type: LeftOuter, predicate: null:Int32 = rw_tables.id, output: [null:Int32, null:Int32, null:Int16, null:Varchar, rw_tables.name, rw_tables.schema_id, $expr1] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchValues { rows: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_tables.id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [rw_tables.id, rw_tables.name, rw_tables.schema_id, Case(('table':Varchar = 'table':Varchar), 'r':Varchar, ('table':Varchar = 'system table':Varchar), 'r':Varchar, ('table':Varchar = 'index':Varchar), 'i':Varchar, ('table':Varchar = 'view':Varchar), 'v':Varchar, ('table':Varchar = 'materialized view':Varchar), 'm':Varchar) as $expr1] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchUnion { all: true } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchProject { exprs: [rw_tables.id, rw_tables.name, 'table':Varchar, rw_tables.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name, rw_tables.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchProject { exprs: [rw_system_tables.id, rw_system_tables.name, 'system table':Varchar, rw_system_tables.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchProject { exprs: [rw_sources.id, rw_sources.name, 'source':Varchar, rw_sources.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchProject { exprs: [rw_indexes.id, rw_indexes.name, 'index':Varchar, rw_indexes.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchProject { exprs: [rw_sinks.id, rw_sinks.name, 'sink':Varchar, rw_sinks.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchProject { exprs: [rw_materialized_views.id, rw_materialized_views.name, 'materialized view':Varchar, rw_materialized_views.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name, rw_materialized_views.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [rw_views.id, rw_views.name, 'view':Varchar, rw_views.schema_id] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_views, columns: [rw_views.id, rw_views.name, rw_views.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_columns.relation_id, $expr2) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [rw_columns.relation_id, rw_columns.name, rw_columns.position::Int16 as $expr2] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchFilter { predicate: (rw_columns.is_hidden = false:Boolean) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_columns, columns: [rw_columns.relation_id, rw_columns.name, rw_columns.position, rw_columns.is_hidden], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchValues { rows: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchValues { rows: [] } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_tables.id) } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchUnion { all: true } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name, rw_tables.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name, rw_materialized_views.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_views, columns: [rw_views.id, rw_views.name, rw_views.schema_id], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_types.id) } + │ │ │ │ │ │ │ │ │ │ │ └─BatchNestedLoopJoin { type: Inner, predicate: true, output: all } + │ │ │ │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_types, columns: [rw_types.id, rw_types.name], distribution: Single } + │ │ │ │ │ │ │ │ │ │ │ └─BatchProject { exprs: [rw_schemas.id] } + │ │ │ │ │ │ │ │ │ │ │ └─BatchFilter { predicate: (rw_schemas.name = 'pg_catalog':Varchar) } + │ │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ │ │ │ │ │ │ │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(pg_constraint.oid) } + │ │ │ │ │ │ │ │ │ └─BatchScan { table: pg_constraint, columns: [pg_constraint.oid, pg_constraint.conname, pg_constraint.connamespace, pg_constraint.contype, pg_constraint.conrelid], distribution: Single } + │ │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_tables.id) } + │ │ │ │ │ │ │ │ └─BatchUnion { all: true } + │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name], distribution: Single } + │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name], distribution: Single } + │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name], distribution: Single } + │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name], distribution: Single } + │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name], distribution: Single } + │ │ │ │ │ │ │ │ ├─BatchScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name], distribution: Single } + │ │ │ │ │ │ │ │ └─BatchScan { table: rw_views, columns: [rw_views.id, rw_views.name], distribution: Single } + │ │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ │ │ │ │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ │ │ │ └─BatchValues { rows: [] } + │ │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_tables.id) } + │ │ │ │ │ └─BatchUnion { all: true } + │ │ │ │ │ ├─BatchScan { table: rw_tables, columns: [rw_tables.id, rw_tables.name, rw_tables.schema_id], distribution: Single } + │ │ │ │ │ ├─BatchScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id], distribution: Single } + │ │ │ │ │ ├─BatchScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id], distribution: Single } + │ │ │ │ │ ├─BatchScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.schema_id], distribution: Single } + │ │ │ │ │ ├─BatchScan { table: rw_sinks, columns: [rw_sinks.id, rw_sinks.name, rw_sinks.schema_id], distribution: Single } + │ │ │ │ │ ├─BatchScan { table: rw_materialized_views, columns: [rw_materialized_views.id, rw_materialized_views.name, rw_materialized_views.schema_id], distribution: Single } + │ │ │ │ │ └─BatchScan { table: rw_views, columns: [rw_views.id, rw_views.name, rw_views.schema_id], distribution: Single } + │ │ │ │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ │ │ └─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ │ │ └─BatchValues { rows: [] } + │ │ └─BatchExchange { order: [], dist: HashShard(rw_schemas.id) } + │ │ └─BatchScan { table: rw_schemas, columns: [rw_schemas.id, rw_schemas.name], distribution: Single } + │ └─BatchExchange { order: [], dist: HashShard(null:Int32) } + │ └─BatchValues { rows: [] } + └─BatchExchange { order: [], dist: HashShard(rw_columns.relation_id, $expr3) } + └─BatchProject { exprs: [rw_columns.relation_id, rw_columns.name, rw_columns.position::Int16 as $expr3] } + └─BatchFilter { predicate: (rw_columns.is_hidden = false:Boolean) } + └─BatchScan { table: rw_columns, columns: [rw_columns.relation_id, rw_columns.name, rw_columns.position, rw_columns.is_hidden], distribution: Single } diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 6196244e193a5..c7e56eec3da13 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -227,7 +227,7 @@ └─LogicalFilter { predicate: In($expr1, 'r':Varchar, 'p':Varchar, 'v':Varchar, 'm':Varchar, 'S':Varchar, 'f':Varchar, '':Varchar) AND (rw_schemas.name <> 'pg_catalog':Varchar) AND Not(RegexpEq(rw_schemas.name, '^pg_toast':Varchar)) AND (rw_schemas.name <> 'information_schema':Varchar) } └─LogicalJoin { type: LeftOuter, on: (rw_schemas.id = rw_tables.schema_id), output: all } ├─LogicalShare { id: 16 } - │ └─LogicalProject { exprs: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, 'p':Varchar, Case(('table':Varchar = 'table':Varchar), 'r':Varchar, ('table':Varchar = 'system table':Varchar), 'r':Varchar, ('table':Varchar = 'index':Varchar), 'i':Varchar, ('table':Varchar = 'view':Varchar), 'v':Varchar, ('table':Varchar = 'materialized view':Varchar), 'm':Varchar) as $expr1, 0:Int32, 0:Int32, Array as $expr2] } + │ └─LogicalProject { exprs: [rw_tables.id, rw_tables.name, rw_tables.schema_id, rw_tables.owner, 'p':Varchar, Case(('table':Varchar = 'table':Varchar), 'r':Varchar, ('table':Varchar = 'system table':Varchar), 'r':Varchar, ('table':Varchar = 'index':Varchar), 'i':Varchar, ('table':Varchar = 'view':Varchar), 'v':Varchar, ('table':Varchar = 'materialized view':Varchar), 'm':Varchar) as $expr1, 0:Int32, 0:Int32, Array as $expr2, null:Varchar] } │ └─LogicalShare { id: 14 } │ └─LogicalUnion { all: true } │ ├─LogicalUnion { all: true } diff --git a/src/frontend/src/binder/expr/function.rs b/src/frontend/src/binder/expr/function.rs index 22fc4ce99c45f..86b8fa93671b2 100644 --- a/src/frontend/src/binder/expr/function.rs +++ b/src/frontend/src/binder/expr/function.rs @@ -1323,6 +1323,7 @@ impl Binder { ("pg_table_is_visible", raw_literal(ExprImpl::literal_bool(true))), ("pg_type_is_visible", raw_literal(ExprImpl::literal_bool(true))), ("pg_get_constraintdef", raw_literal(ExprImpl::literal_null(DataType::Varchar))), + ("pg_get_partkeydef", raw_literal(ExprImpl::literal_null(DataType::Varchar))), ("pg_encoding_to_char", raw_literal(ExprImpl::literal_varchar("UTF8".into()))), ("has_database_privilege", raw_literal(ExprImpl::literal_bool(true))), ("pg_backend_pid", raw(|binder, _inputs| { diff --git a/src/frontend/src/binder/expr/mod.rs b/src/frontend/src/binder/expr/mod.rs index 0bc12545984ca..b1bd37b168010 100644 --- a/src/frontend/src/binder/expr/mod.rs +++ b/src/frontend/src/binder/expr/mod.rs @@ -471,6 +471,60 @@ impl Binder { Ok(func_call.into()) } + /// The optimization check for the following case-when expression pattern + /// e.g., select case 1 when (...) then (...) else (...) end; + fn check_constant_case_when_optimization( + &mut self, + conditions: Vec, + results_expr: Vec, + operand: Option>, + fallback: Option, + constant_case_when_eval_inputs: &mut Vec, + ) -> bool { + // The operand value to be compared later + let operand_value; + + if let Some(operand) = operand { + let Ok(operand) = self.bind_expr_inner(*operand) else { + return false; + }; + if !operand.is_const() { + return false; + } + operand_value = operand; + } else { + return false; + } + + for (condition, result) in zip_eq_fast(conditions, results_expr) { + if let Expr::Value(_) = condition.clone() { + let Ok(res) = self.bind_expr_inner(condition.clone()) else { + return false; + }; + // Found a match + if res == operand_value { + constant_case_when_eval_inputs.push(result); + return true; + } + } else { + return false; + } + } + + // Otherwise this will eventually go through fallback arm + debug_assert!( + constant_case_when_eval_inputs.is_empty(), + "expect `inputs` to be empty" + ); + + let Some(fallback) = fallback else { + return false; + }; + + constant_case_when_eval_inputs.push(fallback); + true + } + /// The helper function to check if the current case-when /// expression in `bind_case` could be optimized /// into `ConstantLookupExpression` @@ -493,6 +547,12 @@ impl Binder { let Ok(operand) = self.bind_expr_inner(*operand) else { return false; }; + // This optimization should be done in subsequent optimization phase + // if the operand is const + // e.g., select case 1 when 1 then 114514 else 1919810 end; + if operand.is_const() { + return false; + } constant_lookup_inputs.push(operand); } else { return false; @@ -506,7 +566,7 @@ impl Binder { constant_lookup_inputs.push(input); } else { // If at least one condition is not in the simple form / not constant, - // we can NOT do the subsequent optimization then + // we can NOT do the subsequent optimization pass return false; } @@ -538,6 +598,27 @@ impl Binder { .transpose()?; let mut constant_lookup_inputs = Vec::new(); + let mut constant_case_when_eval_inputs = Vec::new(); + + let constant_case_when_flag = self.check_constant_case_when_optimization( + conditions.clone(), + results_expr.clone(), + operand.clone(), + else_result_expr.clone(), + &mut constant_case_when_eval_inputs, + ); + + if constant_case_when_flag { + // Sanity check + if constant_case_when_eval_inputs.len() != 1 { + return Err(ErrorCode::BindError( + "expect `constant_case_when_eval_inputs` only contains a single bound expression".to_string() + ) + .into()); + } + // Directly return the first element of the vector + return Ok(constant_case_when_eval_inputs[0].take()); + } // See if the case-when expression can be optimized let optimize_flag = self.check_bind_case_optimization( diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs index ce97aeaac552c..f1bcc3f46f62c 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/mod.rs @@ -30,6 +30,7 @@ mod pg_index; mod pg_indexes; mod pg_inherits; mod pg_keywords; +mod pg_language; mod pg_locks; mod pg_matviews; mod pg_namespace; @@ -37,6 +38,7 @@ mod pg_opclass; mod pg_operator; mod pg_partitioned_table; mod pg_proc; +mod pg_rewrite; mod pg_roles; mod pg_settings; mod pg_shadow; @@ -44,6 +46,7 @@ mod pg_shdescription; mod pg_stat_activity; mod pg_tables; mod pg_tablespace; +mod pg_trigger; mod pg_type; mod pg_user; mod pg_views; diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs index 0d2dc8c8a41a7..d2954f033b3b3 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_class.rs @@ -29,7 +29,8 @@ use risingwave_frontend_macro::system_catalog; END relkind, 0 AS relam, 0 AS reltablespace, - ARRAY[]::varchar[] AS reloptions + ARRAY[]::varchar[] AS reloptions, + null AS relpartbound FROM rw_catalog.rw_relations ")] #[derive(Fields)] @@ -46,4 +47,6 @@ struct PgClass { relam: i32, reltablespace: i32, reloptions: Vec, + // PG uses pg_node_tree type but RW doesn't support it + relpartbound: Option, } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs index d69a4b881570e..c25755f4c3c9c 100644 --- a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_constraint.rs @@ -15,12 +15,17 @@ use risingwave_common::types::Fields; use risingwave_frontend_macro::system_catalog; +use crate::catalog::schema_catalog::SchemaCatalog; +use crate::catalog::system_catalog::{SysCatalogReaderImpl, SystemTableCatalog}; +use crate::error::Result; +use crate::TableCatalog; + /// The catalog `pg_constraint` records information about table and index inheritance hierarchies. /// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-constraint.html`] /// This is introduced only for pg compatibility and is not used in our system. -#[system_catalog(view, "pg_catalog.pg_constraint")] #[derive(Fields)] struct PgConstraint { + #[primary_key] oid: i32, conname: String, connamespace: i32, @@ -38,12 +43,105 @@ struct PgConstraint { conislocal: bool, coninhcount: i32, connoinherit: bool, - conkey: Vec, - confkey: Vec, - conpfeqop: Vec, - conppeqop: Vec, - conffeqop: Vec, - confdelsetcols: Vec, - conexclop: Vec, - conbin: String, + conkey: Option>, + confkey: Option>, + conpfeqop: Option>, + conppeqop: Option>, + conffeqop: Option>, + confdelsetcols: Option>, + conexclop: Option>, + conbin: Option, +} + +impl PgConstraint { + fn from_system_table(schema: &SchemaCatalog, table: &SystemTableCatalog) -> PgConstraint { + // List of the constrained columns. First column starts from 1. + let conkey: Vec<_> = table.pk.iter().map(|i| (*i + 1) as i16).collect(); + PgConstraint { + oid: table.id.table_id() as i32, // Use table_id as a mock oid of constraint here. + conname: format!("{}_pkey", &table.name), + connamespace: schema.id() as i32, + contype: "p".to_owned(), // p = primary key constraint + condeferrable: false, + convalidated: true, + conrelid: table.id.table_id() as i32, + contypid: 0, + // Use table_id as a mock index oid of constraint here. + conindid: table.id.table_id() as i32, + conparentid: 0, + confrelid: 0, + confupdtype: " ".to_owned(), + confdeltype: " ".to_owned(), + confmatchtype: " ".to_owned(), + conislocal: true, + coninhcount: 0, + connoinherit: true, + conkey: Some(conkey), + confkey: None, + conpfeqop: None, + conppeqop: None, + conffeqop: None, + confdelsetcols: None, + conexclop: None, + conbin: None, + } + } + + fn from_table(schema: &SchemaCatalog, table: &TableCatalog) -> PgConstraint { + // List of the constrained columns. First column starts from 1. + let conkey: Vec<_> = table + .pk + .iter() + .map(|i| (i.column_index + 1) as i16) + .collect(); + PgConstraint { + oid: table.id.table_id() as i32, // Use table_id as a mock oid of constraint here. + conname: format!("{}_pkey", &table.name), + connamespace: schema.id() as i32, + contype: "p".to_owned(), // p = primary key constraint + condeferrable: false, + convalidated: true, + conrelid: table.id.table_id() as i32, + contypid: 0, + // Use table_id as a mock index oid of constraint here. + conindid: table.id.table_id() as i32, + conparentid: 0, + confrelid: 0, + confupdtype: " ".to_owned(), + confdeltype: " ".to_owned(), + confmatchtype: " ".to_owned(), + conislocal: true, + coninhcount: 0, + connoinherit: true, + conkey: Some(conkey), + confkey: None, + conpfeqop: None, + conppeqop: None, + conffeqop: None, + confdelsetcols: None, + conexclop: None, + conbin: None, + } + } +} + +#[system_catalog(table, "pg_catalog.pg_constraint")] +fn read_pg_constraint(reader: &SysCatalogReaderImpl) -> Result> { + let catalog_reader = reader.catalog_reader.read_guard(); + let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; + + Ok(schemas.flat_map(read_pg_constraint_in_schema).collect()) +} + +fn read_pg_constraint_in_schema(schema: &SchemaCatalog) -> Vec { + // Note: We only support primary key constraints now. + let system_table_rows = schema + .iter_system_tables() + .map(|table| PgConstraint::from_system_table(schema, table.as_ref())); + + let table_rows = schema + .iter_valid_table() + .map(|table| PgConstraint::from_table(schema, table.as_ref())); + + system_table_rows.chain(table_rows).collect() } diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_language.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_language.rs new file mode 100644 index 0000000000000..3f29502bca3e8 --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_language.rs @@ -0,0 +1,34 @@ +// 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 risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; + +/// The catalog `pg_language` registers languages in which you can write functions or stored procedures. +/// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-language.html`] +/// This is introduced only for pg compatibility and is not used in our system. +#[system_catalog(view, "pg_catalog.pg_language")] +#[derive(Fields)] +struct PgLanguage { + #[primary_key] + oid: i32, + lanname: String, + lanowner: i32, + lanispl: bool, + lanpltrusted: bool, + lanplcallfoid: i32, + laninline: i32, + lanvalidator: i32, + lanacl: Vec, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_rewrite.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_rewrite.rs new file mode 100644 index 0000000000000..3a048d56ec414 --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_rewrite.rs @@ -0,0 +1,33 @@ +// 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 risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; + +/// The catalog `pg_rewrite` stores rewrite rules for tables and views. +/// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-rewrite.html`] +/// This is introduced only for pg compatibility and is not used in our system. +#[system_catalog(view, "pg_catalog.pg_rewrite")] +#[derive(Fields)] +struct PgRewrite { + #[primary_key] + oid: i32, + rulename: String, + ev_class: i32, + ev_type: String, + ev_enabled: String, + is_instead: bool, + ev_qual: String, + ev_action: String, +} diff --git a/src/frontend/src/catalog/system_catalog/pg_catalog/pg_trigger.rs b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_trigger.rs new file mode 100644 index 0000000000000..223363f48b4a8 --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/pg_catalog/pg_trigger.rs @@ -0,0 +1,44 @@ +// 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 risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; + +/// The catalog `pg_trigger` stores triggers on tables and views. +/// Ref: [`https://www.postgresql.org/docs/current/catalog-pg-trigger.html`] +/// This is introduced only for pg compatibility and is not used in our system. +#[system_catalog(view, "pg_catalog.pg_trigger")] +#[derive(Fields)] +struct PgTrigger { + #[primary_key] + oid: i32, + tgrelid: i32, + tgparentid: i32, + tgname: String, + tgfoid: i32, + tgtype: i16, + tgenabled: String, + tgisinternal: bool, + tgconstrrelid: i32, + tgconstrindid: i32, + tgconstraint: i32, + tgdeferrable: bool, + tginitdeferred: bool, + tgnargs: i16, + tgattr: Vec, + tgargs: Vec, + tgqual: Option, + tgoldtable: Option, + tgnewtable: Option, +} diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs index 8491da7062711..7a5f48a190a0d 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_columns.rs @@ -15,6 +15,7 @@ use risingwave_common::types::Fields; use risingwave_frontend_macro::system_catalog; +use crate::catalog::schema_catalog::SchemaCatalog; use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; use crate::expr::{ExprDisplay, ExprImpl}; @@ -22,9 +23,12 @@ use crate::expr::{ExprDisplay, ExprImpl}; #[derive(Fields)] #[primary_key(relation_id, name)] struct RwColumn { - relation_id: i32, // belonged relation id - name: String, // column name - position: i32, // 1-indexed position + relation_id: i32, + // belonged relation id + name: String, + // column name + position: i32, + // 1-indexed position is_hidden: bool, is_primary_key: bool, is_distribution_key: bool, @@ -41,131 +45,125 @@ fn read_rw_columns(reader: &SysCatalogReaderImpl) -> Result> { let catalog_reader = reader.catalog_reader.read_guard(); let schemas = catalog_reader.iter_schemas(&reader.auth_context.database)?; - Ok(schemas - .flat_map(|schema| { - let view_rows = schema.iter_view().flat_map(|view| { - view.columns - .iter() - .enumerate() - .map(|(index, column)| RwColumn { - relation_id: view.id as i32, - name: column.name.clone(), - position: index as i32 + 1, - is_hidden: false, - is_primary_key: false, - is_distribution_key: false, - is_generated: false, - generation_expression: None, - data_type: column.data_type().to_string(), - type_oid: column.data_type().to_oid(), - type_len: column.data_type().type_len(), - udt_type: column.data_type().pg_name().into(), - }) - }); + Ok(schemas.flat_map(read_rw_columns_in_schema).collect()) +} + +fn read_rw_columns_in_schema(schema: &SchemaCatalog) -> Vec { + let view_rows = schema.iter_view().flat_map(|view| { + view.columns + .iter() + .enumerate() + .map(|(index, column)| RwColumn { + relation_id: view.id as i32, + name: column.name.clone(), + position: index as i32 + 1, + is_hidden: false, + is_primary_key: false, + is_distribution_key: false, + is_generated: false, + generation_expression: None, + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }); + + let sink_rows = schema.iter_sink().flat_map(|sink| { + sink.full_columns() + .iter() + .enumerate() + .map(|(index, column)| RwColumn { + relation_id: sink.id.sink_id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: sink.downstream_pk.contains(&index), + is_distribution_key: sink.distribution_key.contains(&index), + is_generated: false, + generation_expression: None, + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }); - let sink_rows = schema - .iter_sink() - .flat_map(|sink| { - sink.full_columns() - .iter() - .enumerate() - .map(|(index, column)| RwColumn { - relation_id: sink.id.sink_id as i32, - name: column.name().into(), - position: index as i32 + 1, - is_hidden: column.is_hidden, - is_primary_key: sink.downstream_pk.contains(&index), - is_distribution_key: sink.distribution_key.contains(&index), - is_generated: false, - generation_expression: None, - data_type: column.data_type().to_string(), - type_oid: column.data_type().to_oid(), - type_len: column.data_type().type_len(), - udt_type: column.data_type().pg_name().into(), - }) - }) - .chain(view_rows); + let catalog_rows = schema.iter_system_tables().flat_map(|table| { + table + .columns + .iter() + .enumerate() + .map(move |(index, column)| RwColumn { + relation_id: table.id.table_id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: table.pk.contains(&index), + is_distribution_key: false, + is_generated: false, + generation_expression: None, + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }); - let catalog_rows = schema - .iter_system_tables() - .flat_map(|table| { - table - .columns - .iter() - .enumerate() - .map(move |(index, column)| RwColumn { - relation_id: table.id.table_id as i32, - name: column.name().into(), - position: index as i32 + 1, - is_hidden: column.is_hidden, - is_primary_key: table.pk.contains(&index), - is_distribution_key: false, - is_generated: false, - generation_expression: None, - data_type: column.data_type().to_string(), - type_oid: column.data_type().to_oid(), - type_len: column.data_type().type_len(), - udt_type: column.data_type().pg_name().into(), - }) - }) - .chain(sink_rows); + let table_rows = schema.iter_valid_table().flat_map(|table| { + let schema = table.column_schema(); + table + .columns + .iter() + .enumerate() + .map(move |(index, column)| RwColumn { + relation_id: table.id.table_id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: table.pk().iter().any(|idx| idx.column_index == index), + is_distribution_key: table.distribution_key.contains(&index), + is_generated: column.is_generated(), + generation_expression: column.generated_expr().map(|expr_node| { + let expr = ExprImpl::from_expr_proto(expr_node).unwrap(); + let expr_display = ExprDisplay { + expr: &expr, + input_schema: &schema, + }; + expr_display.to_string() + }), + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }); - let table_rows = schema - .iter_valid_table() - .flat_map(|table| { - let schema = table.column_schema(); - table - .columns - .iter() - .enumerate() - .map(move |(index, column)| RwColumn { - relation_id: table.id.table_id as i32, - name: column.name().into(), - position: index as i32 + 1, - is_hidden: column.is_hidden, - is_primary_key: table.pk().iter().any(|idx| idx.column_index == index), - is_distribution_key: table.distribution_key.contains(&index), - is_generated: column.is_generated(), - generation_expression: column.generated_expr().map(|expr_node| { - let expr = ExprImpl::from_expr_proto(expr_node).unwrap(); - let expr_display = ExprDisplay { - expr: &expr, - input_schema: &schema, - }; - expr_display.to_string() - }), - data_type: column.data_type().to_string(), - type_oid: column.data_type().to_oid(), - type_len: column.data_type().type_len(), - udt_type: column.data_type().pg_name().into(), - }) - }) - .chain(catalog_rows); + let schema_rows = schema.iter_source().flat_map(|source| { + source + .columns + .iter() + .enumerate() + .map(move |(index, column)| RwColumn { + relation_id: source.id as i32, + name: column.name().into(), + position: index as i32 + 1, + is_hidden: column.is_hidden, + is_primary_key: source.pk_col_ids.contains(&column.column_id()), + is_distribution_key: false, + is_generated: false, + generation_expression: None, + data_type: column.data_type().to_string(), + type_oid: column.data_type().to_oid(), + type_len: column.data_type().type_len(), + udt_type: column.data_type().pg_name().into(), + }) + }); - // source columns - schema - .iter_source() - .flat_map(|source| { - source - .columns - .iter() - .enumerate() - .map(move |(index, column)| RwColumn { - relation_id: source.id as i32, - name: column.name().into(), - position: index as i32 + 1, - is_hidden: column.is_hidden, - is_primary_key: source.pk_col_ids.contains(&column.column_id()), - is_distribution_key: false, - is_generated: false, - generation_expression: None, - data_type: column.data_type().to_string(), - type_oid: column.data_type().to_oid(), - type_len: column.data_type().type_len(), - udt_type: column.data_type().pg_name().into(), - }) - }) - .chain(table_rows) - }) - .collect()) + view_rows + .chain(sink_rows) + .chain(catalog_rows) + .chain(table_rows) + .chain(schema_rows) + .collect() } diff --git a/src/frontend/src/error.rs b/src/frontend/src/error.rs index 5c1daa024afb1..9898e26a79ba0 100644 --- a/src/frontend/src/error.rs +++ b/src/frontend/src/error.rs @@ -17,6 +17,7 @@ use risingwave_common::array::ArrayError; use risingwave_common::error::{BoxedError, NoFunction, NotImplemented}; use risingwave_common::session_config::SessionConfigError; use risingwave_common::util::value_encoding::error::ValueEncodingError; +use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; use risingwave_expr::ExprError; use risingwave_pb::PbFieldNotFound; @@ -208,6 +209,12 @@ impl From for RwError { } } +impl From for RwError { + fn from(e: ConnectorError) -> Self { + ErrorCode::ConnectorError(e.into()).into() + } +} + impl From for RwError { fn from(err: PbFieldNotFound) -> Self { ErrorCode::InternalError(format!( diff --git a/src/frontend/src/handler/create_index.rs b/src/frontend/src/handler/create_index.rs index af9d9f52b1752..c687f56fad8e8 100644 --- a/src/frontend/src/handler/create_index.rs +++ b/src/frontend/src/handler/create_index.rs @@ -45,15 +45,11 @@ use crate::session::SessionImpl; use crate::stream_fragmenter::build_graph; use crate::TableCatalog; -pub(crate) fn gen_create_index_plan( +pub(crate) fn resolve_index_schema( session: &SessionImpl, - context: OptimizerContextRef, index_name: ObjectName, table_name: ObjectName, - columns: Vec, - include: Vec, - distributed_by: Vec, -) -> Result<(PlanRef, PbTable, PbIndex)> { +) -> Result<(String, Arc, String)> { let db_name = session.database(); let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, table_name)?; let search_path = session.config().search_path(); @@ -63,12 +59,22 @@ pub(crate) fn gen_create_index_plan( let index_table_name = Binder::resolve_index_name(index_name)?; let catalog_reader = session.env().catalog_reader(); - let (table, schema_name) = { - let read_guard = catalog_reader.read_guard(); - let (table, schema_name) = - read_guard.get_table_by_name(db_name, schema_path, &table_name)?; - (table.clone(), schema_name.to_string()) - }; + let read_guard = catalog_reader.read_guard(); + let (table, schema_name) = read_guard.get_table_by_name(db_name, schema_path, &table_name)?; + Ok((schema_name.to_string(), table.clone(), index_table_name)) +} + +pub(crate) fn gen_create_index_plan( + session: &SessionImpl, + context: OptimizerContextRef, + schema_name: String, + table: Arc, + index_table_name: String, + columns: Vec, + include: Vec, + distributed_by: Vec, +) -> Result<(PlanRef, PbTable, PbIndex)> { + let table_name = table.name.clone(); if table.is_index() { return Err( @@ -404,22 +410,27 @@ pub async fn handle_create_index( let session = handler_args.session.clone(); let (graph, index_table, index) = { - { - if let Either::Right(resp) = session.check_relation_name_duplicated( - index_name.clone(), - StatementType::CREATE_INDEX, - if_not_exists, - )? { - return Ok(resp); - } + let (schema_name, table, index_table_name) = + resolve_index_schema(&session, index_name, table_name)?; + let qualified_index_name = ObjectName(vec![ + Ident::with_quote_unchecked('"', &schema_name), + Ident::with_quote_unchecked('"', &index_table_name), + ]); + if let Either::Right(resp) = session.check_relation_name_duplicated( + qualified_index_name, + StatementType::CREATE_INDEX, + if_not_exists, + )? { + return Ok(resp); } let context = OptimizerContext::from_handler_args(handler_args); let (plan, index_table, index) = gen_create_index_plan( &session, context.into(), - index_name.clone(), - table_name, + schema_name, + table, + index_table_name, columns, include, distributed_by, @@ -437,7 +448,7 @@ pub async fn handle_create_index( tracing::trace!( "name={}, graph=\n{}", - index_name, + index.name, serde_json::to_string_pretty(&graph).unwrap() ); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index bbb2d93b21790..c8cfd938c23a2 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -62,6 +62,7 @@ use risingwave_sqlparser::ast::{ ProtobufSchema, SourceWatermark, }; use risingwave_sqlparser::parser::IncludeOption; +use thiserror_ext::AsReport; use super::RwPgResponse; use crate::binder::Binder; @@ -1081,7 +1082,7 @@ pub(super) async fn check_source_schema( } else if connector == ICEBERG_CONNECTOR { Ok(check_iceberg_source(props, columns) .await - .map_err(|err| ProtocolError(err.to_string()))?) + .map_err(|err| ProtocolError(err.to_report_string()))?) } else { Ok(()) } diff --git a/src/frontend/src/handler/explain.rs b/src/frontend/src/handler/explain.rs index b966cca8f50cf..b46882156a24c 100644 --- a/src/frontend/src/handler/explain.rs +++ b/src/frontend/src/handler/explain.rs @@ -18,7 +18,7 @@ use risingwave_common::types::Fields; use risingwave_sqlparser::ast::{ExplainOptions, ExplainType, Statement}; use thiserror_ext::AsReport; -use super::create_index::gen_create_index_plan; +use super::create_index::{gen_create_index_plan, resolve_index_schema}; use super::create_mv::gen_create_mv_plan; use super::create_sink::{gen_sink_plan, get_partition_compute_info}; use super::create_table::ColumnIdGenerator; @@ -133,15 +133,20 @@ async fn do_handle_explain( include, distributed_by, .. - } => gen_create_index_plan( - &session, - context.clone(), - name, - table_name, - columns, - include, - distributed_by, - ) + } => { + let (schema_name, table, index_table_name) = + resolve_index_schema(&session, name, table_name)?; + gen_create_index_plan( + &session, + context.clone(), + schema_name, + table, + index_table_name, + columns, + include, + distributed_by, + ) + } .map(|x| x.0), // -- Batch Queries -- diff --git a/src/frontend/src/handler/util.rs b/src/frontend/src/handler/util.rs index ab9d4fe415b33..6c9a9bb45f2ac 100644 --- a/src/frontend/src/handler/util.rs +++ b/src/frontend/src/handler/util.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use std::task::{Context, Poll}; use anyhow::Context as _; -use bytes::Bytes; +use bytes::{Bytes, BytesMut}; use futures::Stream; use itertools::Itertools; use pgwire::pg_field_descriptor::PgFieldDescriptor; @@ -29,7 +29,7 @@ use pin_project_lite::pin_project; use risingwave_common::array::DataChunk; use risingwave_common::catalog::Field; use risingwave_common::row::Row as _; -use risingwave_common::types::{DataType, ScalarRefImpl, Timestamptz}; +use risingwave_common::types::{write_date_time_tz, DataType, ScalarRefImpl, Timestamptz}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR; use risingwave_connector::source::KAFKA_CONNECTOR; @@ -139,10 +139,9 @@ fn timestamptz_to_string_with_session_data( let tz = d.into_timestamptz(); let time_zone = Timestamptz::lookup_time_zone(&session_data.timezone).unwrap(); let instant_local = tz.to_datetime_in_zone(time_zone); - instant_local - .format("%Y-%m-%d %H:%M:%S%.f%:z") - .to_string() - .into() + let mut result_string = BytesMut::new(); + write_date_time_tz(instant_local, &mut result_string).unwrap(); + result_string.into() } fn to_pg_rows( diff --git a/src/frontend/src/scheduler/error.rs b/src/frontend/src/scheduler/error.rs index f68f72b8727e7..590c235e13901 100644 --- a/src/frontend/src/scheduler/error.rs +++ b/src/frontend/src/scheduler/error.rs @@ -14,6 +14,7 @@ use risingwave_batch::error::BatchError; use risingwave_common::session_config::QueryMode; +use risingwave_connector::error::ConnectorError; use risingwave_rpc_client::error::RpcError; use thiserror::Error; use tonic::{Code, Status}; @@ -63,6 +64,13 @@ pub enum SchedulerError { BatchError, ), + #[error(transparent)] + Connector( + #[from] + #[backtrace] + ConnectorError, + ), + #[error(transparent)] Internal( #[from] diff --git a/src/meta/model_v2/src/table.rs b/src/meta/model_v2/src/table.rs index 06710d42d9d25..0039b9cccc2e0 100644 --- a/src/meta/model_v2/src/table.rs +++ b/src/meta/model_v2/src/table.rs @@ -24,7 +24,7 @@ use crate::{ TableId, TableVersion, }; -#[derive(Clone, Debug, PartialEq, Copy, Eq, EnumIter, DeriveActiveEnum)] +#[derive(Clone, Debug, PartialEq, Hash, Copy, Eq, EnumIter, DeriveActiveEnum)] #[sea_orm(rs_type = "String", db_type = "String(None)")] pub enum TableType { #[sea_orm(string_value = "TABLE")] diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 4792293863237..3397317fe1939 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -478,20 +478,13 @@ pub async fn start_service_as_election_leader( prometheus_http_query::Client::from_str(x).unwrap() }); let prometheus_selector = opts.prometheus_selector.unwrap_or_default(); - let diagnose_command = match &metadata_manager { - MetadataManager::V1(mgr) => Some(Arc::new( - risingwave_meta::manager::diagnose::DiagnoseCommand::new( - mgr.cluster_manager.clone(), - mgr.catalog_manager.clone(), - mgr.fragment_manager.clone(), - hummock_manager.clone(), - env.event_log_manager_ref(), - prometheus_client.clone(), - prometheus_selector.clone(), - ), - )), - MetadataManager::V2(_) => None, - }; + let diagnose_command = Arc::new(risingwave_meta::manager::diagnose::DiagnoseCommand::new( + metadata_manager.clone(), + hummock_manager.clone(), + env.event_log_manager_ref(), + prometheus_client.clone(), + prometheus_selector.clone(), + )); let trace_state = otlp_embedded::State::new(otlp_embedded::Config { max_length: opts.cached_traces_num, diff --git a/src/meta/service/src/cloud_service.rs b/src/meta/service/src/cloud_service.rs index 2ee28b1427edc..9c213bd7cb9e2 100644 --- a/src/meta/service/src/cloud_service.rs +++ b/src/meta/service/src/cloud_service.rs @@ -18,6 +18,7 @@ use std::sync::LazyLock; use async_trait::async_trait; use regex::Regex; use risingwave_connector::dispatch_source_prop; +use risingwave_connector::error::ConnectorResult; use risingwave_connector::source::kafka::private_link::insert_privatelink_broker_rewrite_map; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, @@ -135,7 +136,7 @@ impl CloudService for CloudServiceImpl { { return Ok(new_rwc_validate_fail_response( ErrorType::PrivatelinkResolveErr, - e.to_string(), + e.to_report_string(), )); } } else { @@ -151,13 +152,13 @@ impl CloudService for CloudServiceImpl { if let Err(e) = props { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaInvalidProperties, - e.to_string(), + e.to_report_string(), )); }; async fn new_enumerator( props: P, - ) -> Result { + ) -> ConnectorResult { P::SplitEnumerator::new(props, SourceEnumeratorContext::default().into()).await } @@ -166,15 +167,15 @@ impl CloudService for CloudServiceImpl { if let Err(e) = enumerator { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaInvalidProperties, - e.to_string(), + e.to_report_string(), )); } if let Err(e) = enumerator.unwrap().list_splits().await { - let error_message = e.to_string(); + let error_message = e.to_report_string(); if error_message.contains("BrokerTransportFailure") { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaBrokerUnreachable, - e.to_string(), + e.to_report_string(), )); } static TOPIC_NOT_FOUND: LazyLock = @@ -182,12 +183,12 @@ impl CloudService for CloudServiceImpl { if TOPIC_NOT_FOUND.is_match(error_message.as_str()) { return Ok(new_rwc_validate_fail_response( ErrorType::KafkaTopicNotFound, - e.to_string(), + e.to_report_string(), )); } return Ok(new_rwc_validate_fail_response( ErrorType::KafkaOther, - e.to_string(), + e.to_report_string(), )); } }); diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index bd2f24f1baf46..b64829c9005c5 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -608,7 +608,9 @@ impl GlobalBarrierManager { }; // Tracing related stuff - tracing::info!(target: "rw_tracing", parent: prev_epoch.span(), epoch = curr_epoch.value().0, "new barrier enqueued"); + prev_epoch.span().in_scope(|| { + tracing::info!(target: "rw_tracing", epoch = curr_epoch.value().0, "new barrier enqueued"); + }); span.record("epoch", curr_epoch.value().0); let command_ctx = Arc::new(CommandContext::new( diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index e26e1af0f0cff..fae19874ab407 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -845,7 +845,7 @@ impl CatalogController { let ret = src_manager.register_source(&pb_source).await; if let Err(e) = ret { txn.rollback().await?; - return Err(e.into()); + return Err(e); } } txn.commit().await?; @@ -2446,6 +2446,18 @@ impl CatalogController { } } +/// `CatalogStats` is a struct to store the statistics of all catalogs. +pub struct CatalogStats { + pub table_num: u64, + pub mview_num: u64, + pub index_num: u64, + pub source_num: u64, + pub sink_num: u64, + pub function_num: u64, + pub streaming_job_num: u64, + pub actor_num: u64, +} + impl CatalogControllerInner { pub async fn snapshot(&self) -> MetaResult<(Catalog, Vec)> { let databases = self.list_databases().await?; @@ -2476,6 +2488,40 @@ impl CatalogControllerInner { )) } + pub async fn stats(&self) -> MetaResult { + let mut table_num_map: HashMap<_, _> = Table::find() + .select_only() + .column(table::Column::TableType) + .column_as(table::Column::TableId.count(), "num") + .group_by(table::Column::TableType) + .having(table::Column::TableType.ne(TableType::Internal)) + .into_tuple::<(TableType, i64)>() + .all(&self.db) + .await? + .into_iter() + .map(|(table_type, num)| (table_type, num as u64)) + .collect(); + + let source_num = Source::find().count(&self.db).await?; + let sink_num = Sink::find().count(&self.db).await?; + let function_num = Function::find().count(&self.db).await?; + let streaming_job_num = StreamingJob::find().count(&self.db).await?; + let actor_num = Actor::find().count(&self.db).await?; + + Ok(CatalogStats { + table_num: table_num_map.remove(&TableType::Table).unwrap_or(0), + mview_num: table_num_map + .remove(&TableType::MaterializedView) + .unwrap_or(0), + index_num: table_num_map.remove(&TableType::Index).unwrap_or(0), + source_num, + sink_num, + function_num, + streaming_job_num, + actor_num, + }) + } + async fn list_databases(&self) -> MetaResult> { let db_objs = Database::find() .find_also_related(Object) diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 833e642a83e74..c9beada284d5d 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -702,6 +702,23 @@ impl CatalogController { Ok(count > 0) } + pub async fn worker_actor_count(&self) -> MetaResult> { + let inner = self.inner.read().await; + let actor_cnt: Vec<(WorkerId, i64)> = Actor::find() + .select_only() + .column(actor::Column::WorkerId) + .column_as(actor::Column::ActorId.count(), "count") + .group_by(actor::Column::WorkerId) + .into_tuple() + .all(&inner.db) + .await?; + + Ok(actor_cnt + .into_iter() + .map(|(worker_id, count)| (worker_id, count as usize)) + .collect()) + } + // TODO: This function is too heavy, we should avoid using it and implement others on demand. pub async fn table_fragments(&self) -> MetaResult> { let inner = self.inner.read().await; diff --git a/src/meta/src/dashboard/mod.rs b/src/meta/src/dashboard/mod.rs index 814e14d42af9e..b4ae8e2c04b36 100644 --- a/src/meta/src/dashboard/mod.rs +++ b/src/meta/src/dashboard/mod.rs @@ -30,6 +30,7 @@ use axum::Router; use hyper::Request; use parking_lot::Mutex; use risingwave_rpc_client::ComputeClientPool; +use thiserror_ext::AsReport; use tower::{ServiceBuilder, ServiceExt}; use tower_http::add_extension::AddExtensionLayer; use tower_http::cors::{self, CorsLayer}; @@ -46,7 +47,7 @@ pub struct DashboardService { pub metadata_manager: MetadataManager, pub compute_clients: ComputeClientPool, pub ui_path: Option, - pub diagnose_command: Option, + pub diagnose_command: DiagnoseCommandRef, pub trace_state: otlp_embedded::StateRef, } @@ -354,13 +355,7 @@ pub(super) mod handlers { } pub async fn diagnose(Extension(srv): Extension) -> Result { - let report = if let Some(cmd) = &srv.diagnose_command { - cmd.report().await - } else { - "Not supported in sql-backend".to_string() - }; - - Ok(report) + Ok(srv.diagnose_command.report().await) } pub async fn get_back_pressure( @@ -455,7 +450,7 @@ impl DashboardService { proxy::proxy(req, cache).await.or_else(|err| { Ok(( StatusCode::INTERNAL_SERVER_ERROR, - format!("Unhandled internal error: {}", err), + err.context("Unhandled internal error").to_report_string(), ) .into_response()) }) diff --git a/src/meta/src/error.rs b/src/meta/src/error.rs index 9833a51cc1934..18230bf74c213 100644 --- a/src/meta/src/error.rs +++ b/src/meta/src/error.rs @@ -14,6 +14,7 @@ use aws_sdk_ec2::error::DisplayErrorContext; use risingwave_common::error::BoxedError; +use risingwave_connector::error::ConnectorError; use risingwave_connector::sink::SinkError; use risingwave_pb::PbFieldNotFound; use risingwave_rpc_client::error::{RpcError, ToTonicStatus}; @@ -88,6 +89,13 @@ pub enum MetaErrorInner { #[error("SystemParams error: {0}")] SystemParams(String), + #[error(transparent)] + Connector( + #[from] + #[backtrace] + ConnectorError, + ), + #[error("Sink error: {0}")] Sink( #[from] diff --git a/src/meta/src/hummock/compaction/overlap_strategy.rs b/src/meta/src/hummock/compaction/overlap_strategy.rs index 001b480b24255..1a713cc401175 100644 --- a/src/meta/src/hummock/compaction/overlap_strategy.rs +++ b/src/meta/src/hummock/compaction/overlap_strategy.rs @@ -95,15 +95,10 @@ impl OverlapInfo for RangeOverlapInfo { if overlap_begin >= others.len() { return overlap_begin..overlap_begin; } - let mut overlap_end = overlap_begin; - for table in &others[overlap_begin..] { - if key_range.compare_right_with(&table.key_range.as_ref().unwrap().left) - == cmp::Ordering::Less - { - break; - } - overlap_end += 1; - } + let overlap_end = others.partition_point(|table_status| { + key_range.compare_right_with(&table_status.key_range.as_ref().unwrap().left) + != cmp::Ordering::Less + }); overlap_begin..overlap_end } None => others.len()..others.len(), diff --git a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs index 849cacd694113..6617b9496fe10 100644 --- a/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/base_level_compaction_picker.rs @@ -620,6 +620,7 @@ pub mod tests { ); let mut picker = LevelCompactionPicker::new(1, config, Arc::new(CompactionDeveloperConfig::default())); + let ret = picker .pick_compaction(&levels, &levels_handler, &mut local_stats) .unwrap(); diff --git a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs index e34963ab48f9f..01c3c050e574c 100644 --- a/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs +++ b/src/meta/src/hummock/compaction/picker/min_overlap_compaction_picker.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::BTreeSet; use std::sync::Arc; use risingwave_hummock_sdk::append_sstable_info_to_string; @@ -209,11 +208,15 @@ impl NonOverlapSubLevelPicker { } } + // Use the incoming sst as the basic range and select as many sub levels as possible + // 1. Build basic range based on sst + // 2. Add a new sub level in each round + // 3. Expand sst according to the basic range from new to old sub levels. + // 4. According to the size and count restrictions, select the plan that contains the most sub levels as much as possible fn pick_sub_level( &self, levels: &[Level], level_handler: &LevelHandler, - sst_index: usize, sst: &SstableInfo, ) -> SubLevelSstables { let mut ret = SubLevelSstables { @@ -222,127 +225,100 @@ impl NonOverlapSubLevelPicker { sstable_infos: vec![vec![]; levels.len()], }; ret.sstable_infos[0].extend(vec![sst.clone()]); - let mut overlap_info = self.overlap_strategy.create_overlap_info(); - let mut select_sst_id_set = BTreeSet::default(); - #[allow(clippy::single_range_in_vec_init)] - let mut overlap_len_and_begins = vec![(sst_index..(sst_index + 1))]; - for sst in &ret.sstable_infos[0] { - overlap_info.update(sst); - select_sst_id_set.insert(sst.sst_id); - } - for (target_index, target_level) in levels.iter().enumerate().skip(1) { + let mut pick_levels_range = Vec::default(); + let mut max_select_level_count = 0; + + // Pay attention to the order here: Make sure to select the lowest sub_level to meet the requirements of base compaction. If you break the assumption of this order, you need to redesign it. + // TODO: Use binary selection to replace the step algorithm to optimize algorithm complexity + 'expand_new_level: for (target_index, target_level) in levels.iter().enumerate().skip(1) { if target_level.level_type() != LevelType::Nonoverlapping { break; } - // more than 1 sub_level - if ret.total_file_count > 1 && ret.total_file_size >= self.max_compaction_bytes - || ret.total_file_count >= self.max_file_count as usize + if ret + .sstable_infos + .iter() + .filter(|ssts| !ssts.is_empty()) + .count() + > MAX_COMPACT_LEVEL_COUNT { break; } + // reset the `basic_overlap_info` with basic sst + let mut basic_overlap_info = self.overlap_strategy.create_overlap_info(); + basic_overlap_info.update(sst); + let mut overlap_files_range = - overlap_info.check_multiple_include(&target_level.table_infos); + basic_overlap_info.check_multiple_include(&target_level.table_infos); if overlap_files_range.is_empty() { overlap_files_range = - overlap_info.check_multiple_overlap(&target_level.table_infos); + basic_overlap_info.check_multiple_overlap(&target_level.table_infos); } - // We allow a layer in the middle without overlap, so we need to continue to - // the next layer to search for overlap - let mut pending_compact = false; - let mut current_level_size = 0; - for index in overlap_files_range.start..overlap_files_range.end { - let other = &target_level.table_infos[index]; - if level_handler.is_pending_compact(&other.sst_id) { - pending_compact = true; - break; - } - overlap_info.update(other); - select_sst_id_set.insert(other.sst_id); - current_level_size += other.file_size; + if overlap_files_range.is_empty() { + continue; } - if pending_compact { - break; - } + let mut overlap_levels = vec![]; - let mut extra_overlap_levels = vec![]; + let mut add_files_size: u64 = 0; + let mut add_files_count: usize = 0; - let mut add_files_size = 0; - // check reverse overlap - for (reverse_index, old_overlap_range) in - overlap_len_and_begins.iter_mut().enumerate().rev() - { + let mut select_level_count = 0; + for reverse_index in (0..=target_index).rev() { let target_tables = &levels[reverse_index].table_infos; - // It has select all files in this sub-level, so it can not overlap with more files. - if ret.sstable_infos[reverse_index].len() == target_tables.len() { + + overlap_files_range = if target_index == reverse_index { + overlap_files_range + } else { + basic_overlap_info.check_multiple_overlap(target_tables) + }; + + // We allow a layer in the middle without overlap, so we need to continue to + // the next layer to search for overlap + if overlap_files_range.is_empty() { + // empty level continue; } - let new_overlap_range = overlap_info.check_multiple_overlap(target_tables); - let mut extra_overlap_sst = Vec::with_capacity(new_overlap_range.len()); - for new_overlap_index in new_overlap_range.clone() { - if old_overlap_range.contains(&new_overlap_index) { - // Since some of the files have already been selected when selecting - // upwards, we filter here to avoid adding sst repeatedly - continue; - } - let other = &target_tables[new_overlap_index]; + for other in &target_tables[overlap_files_range.clone()] { if level_handler.is_pending_compact(&other.sst_id) { - pending_compact = true; - break; + break 'expand_new_level; } - debug_assert!(!select_sst_id_set.contains(&other.sst_id)); - add_files_size += other.file_size; - overlap_info.update(other); - select_sst_id_set.insert(other.sst_id); - extra_overlap_sst.push(other.clone()); + basic_overlap_info.update(other); + + add_files_size += other.get_file_size(); + add_files_count += 1; } - if pending_compact { - break; + // When size / file count has exceeded the limit, we need to abandon this plan, it cannot be expanded to the last sub_level + if max_select_level_count > 1 + && (add_files_size >= self.max_compaction_bytes + || add_files_count >= self.max_file_count as usize) + { + break 'expand_new_level; } - extra_overlap_levels.push((reverse_index, extra_overlap_sst)); - *old_overlap_range = new_overlap_range; + overlap_levels.push((reverse_index, overlap_files_range.clone())); + select_level_count += 1; } - // check reverse overlap - if pending_compact { - // encountering a pending file means we don't need to continue processing this - // interval - break; + if select_level_count > max_select_level_count { + max_select_level_count = select_level_count; + pick_levels_range = overlap_levels; } + } - let add_files_count = overlap_files_range.len() - + extra_overlap_levels - .iter() - .map(|(_, files)| files.len()) - .sum::(); - - if ret - .sstable_infos + for (reverse_index, sst_range) in pick_levels_range { + let level_ssts = &levels[reverse_index].table_infos; + ret.sstable_infos[reverse_index] = level_ssts[sst_range].to_vec(); + ret.total_file_count += ret.sstable_infos[reverse_index].len(); + ret.total_file_size += ret.sstable_infos[reverse_index] .iter() - .filter(|ssts| !ssts.is_empty()) - .count() - > MAX_COMPACT_LEVEL_COUNT - { - break; - } - - ret.total_file_count += add_files_count; - ret.total_file_size += add_files_size + current_level_size; - if !overlap_files_range.is_empty() { - ret.sstable_infos[target_index] - .extend_from_slice(&target_level.table_infos[overlap_files_range.clone()]); - } - overlap_len_and_begins.push(overlap_files_range); - for (reverse_index, files) in extra_overlap_levels { - ret.sstable_infos[reverse_index].extend(files); - } + .map(|sst| sst.file_size) + .sum::(); } // sort sst per level due to reverse expand @@ -355,76 +331,7 @@ impl NonOverlapSubLevelPicker { }); if self.enable_check_task_level_overlap { - use std::fmt::Write; - - use itertools::Itertools; - - use crate::hummock::compaction::overlap_strategy::OverlapInfo; - let mut overlap_info: Option> = None; - for (level_idx, ssts) in ret.sstable_infos.iter().enumerate().rev() { - if let Some(overlap_info) = overlap_info.as_mut() { - // skip the check if `overlap_info` is not initialized (i.e. the first non-empty level is not met) - let level = levels.get(level_idx).unwrap(); - let overlap_sst_range = overlap_info.check_multiple_overlap(&level.table_infos); - if !overlap_sst_range.is_empty() { - let expected_sst_ids = level.table_infos[overlap_sst_range.clone()] - .iter() - .map(|s| s.object_id) - .collect_vec(); - let actual_sst_ids = ssts.iter().map(|s| s.object_id).collect_vec(); - // `actual_sst_ids` can be larger than `expected_sst_ids` because we may use a larger key range to select SSTs. - // `expected_sst_ids` must be a sub-range of `actual_sst_ids` to ensure correctness. - let start_idx = actual_sst_ids - .iter() - .position(|sst_id| sst_id == expected_sst_ids.first().unwrap()); - if start_idx.map_or(true, |idx| { - actual_sst_ids[idx..idx + expected_sst_ids.len()] != expected_sst_ids - }) { - // Print SstableInfo for `actual_sst_ids` - let mut actual_sst_infos = String::new(); - ssts.iter().for_each(|s| { - append_sstable_info_to_string(&mut actual_sst_infos, s) - }); - - // Print SstableInfo for `expected_sst_ids` - let mut expected_sst_infos = String::new(); - level.table_infos[overlap_sst_range.clone()] - .iter() - .for_each(|s| { - append_sstable_info_to_string(&mut expected_sst_infos, s) - }); - - // Print SstableInfo for selected ssts in all sub-levels - let mut ret_sst_infos = String::new(); - ret.sstable_infos - .iter() - .enumerate() - .for_each(|(idx, ssts)| { - writeln!( - ret_sst_infos, - "sub level {}", - levels.get(idx).unwrap().sub_level_id - ) - .unwrap(); - ssts.iter().for_each(|s| { - append_sstable_info_to_string(&mut ret_sst_infos, s) - }); - }); - panic!( - "Compact task overlap check fails. Actual: {} Expected: {} Ret {}", - actual_sst_infos, expected_sst_infos, ret_sst_infos - ); - } - } - } else if !ssts.is_empty() { - // init the `overlap_info` when meeting the first non-empty level. - overlap_info = Some(self.overlap_strategy.create_overlap_info()); - } - - for sst in ssts { - overlap_info.as_mut().unwrap().update(sst); - } - } + self.verify_task_level_overlap(&ret, levels); } ret.sstable_infos.retain(|ssts| !ssts.is_empty()); @@ -441,13 +348,12 @@ impl NonOverlapSubLevelPicker { } let mut scores = vec![]; - let select_tables = &l0[0].table_infos; - for (sst_index, sst) in select_tables.iter().enumerate() { + for sst in &l0[0].table_infos { if level_handler.is_pending_compact(&sst.sst_id) { continue; } - let ret = self.pick_sub_level(l0, level_handler, sst_index, sst); + let ret = self.pick_sub_level(l0, level_handler, sst); if ret.sstable_infos.len() < self.min_depth && ret.total_file_size < self.min_compaction_bytes { @@ -493,10 +399,84 @@ impl NonOverlapSubLevelPicker { expected } + + fn verify_task_level_overlap(&self, ret: &SubLevelSstables, levels: &[Level]) { + use std::fmt::Write; + + use itertools::Itertools; + + use crate::hummock::compaction::overlap_strategy::OverlapInfo; + let mut overlap_info: Option> = None; + for (level_idx, ssts) in ret.sstable_infos.iter().enumerate().rev() { + if let Some(overlap_info) = overlap_info.as_mut() { + // skip the check if `overlap_info` is not initialized (i.e. the first non-empty level is not met) + let level = levels.get(level_idx).unwrap(); + let overlap_sst_range = overlap_info.check_multiple_overlap(&level.table_infos); + if !overlap_sst_range.is_empty() { + let expected_sst_ids = level.table_infos[overlap_sst_range.clone()] + .iter() + .map(|s| s.object_id) + .collect_vec(); + let actual_sst_ids = ssts.iter().map(|s| s.object_id).collect_vec(); + // `actual_sst_ids` can be larger than `expected_sst_ids` because we may use a larger key range to select SSTs. + // `expected_sst_ids` must be a sub-range of `actual_sst_ids` to ensure correctness. + let start_idx = actual_sst_ids + .iter() + .position(|sst_id| sst_id == expected_sst_ids.first().unwrap()); + if start_idx.map_or(true, |idx| { + actual_sst_ids[idx..idx + expected_sst_ids.len()] != expected_sst_ids + }) { + // Print SstableInfo for `actual_sst_ids` + let mut actual_sst_infos = String::new(); + ssts.iter() + .for_each(|s| append_sstable_info_to_string(&mut actual_sst_infos, s)); + + // Print SstableInfo for `expected_sst_ids` + let mut expected_sst_infos = String::new(); + level.table_infos[overlap_sst_range.clone()] + .iter() + .for_each(|s| { + append_sstable_info_to_string(&mut expected_sst_infos, s) + }); + + // Print SstableInfo for selected ssts in all sub-levels + let mut ret_sst_infos = String::new(); + ret.sstable_infos + .iter() + .enumerate() + .for_each(|(idx, ssts)| { + writeln!( + ret_sst_infos, + "sub level {}", + levels.get(idx).unwrap().sub_level_id + ) + .unwrap(); + ssts.iter().for_each(|s| { + append_sstable_info_to_string(&mut ret_sst_infos, s) + }); + }); + panic!( + "Compact task overlap check fails. Actual: {} Expected: {} Ret {}", + actual_sst_infos, expected_sst_infos, ret_sst_infos + ); + } + } + } else if !ssts.is_empty() { + // init the `overlap_info` when meeting the first non-empty level. + overlap_info = Some(self.overlap_strategy.create_overlap_info()); + } + + for sst in ssts { + overlap_info.as_mut().unwrap().update(sst); + } + } + } } #[cfg(test)] pub mod tests { + use std::collections::BTreeSet; + pub use risingwave_pb::hummock::{Level, LevelType}; use super::*; @@ -655,12 +635,12 @@ pub mod tests { level_type: LevelType::Nonoverlapping as i32, table_infos: vec![ generate_table(4, 1, 50, 199, 1), - generate_table(5, 1, 200, 399, 1), + generate_table(5, 1, 200, 249, 1), generate_table(9, 1, 250, 300, 2), generate_table(10, 1, 350, 400, 2), generate_table(11, 1, 450, 500, 2), ], - total_file_size: 250, + total_file_size: 350, ..Default::default() }, Level { diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index fdae16efe5a7b..ecfac68833d0e 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -858,6 +858,22 @@ impl FragmentManager { actor_maps } + pub async fn node_actor_count(&self) -> HashMap { + let mut actor_count = HashMap::new(); + + let map = &self.core.read().await.table_fragments; + for fragments in map.values() { + for actor_status in fragments.actor_status.values() { + if let Some(pu) = &actor_status.parallel_unit { + let e = actor_count.entry(pu.worker_node_id).or_insert(0); + *e += 1; + } + } + } + + actor_count + } + // edit the `rate_limit` of the `Source` node in given `source_id`'s fragments // return the actor_ids to be applied pub async fn update_source_rate_limit_by_source_id( diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs index a8c1d050ab748..76184de9eb7ce 100644 --- a/src/meta/src/manager/diagnose.rs +++ b/src/meta/src/manager/diagnose.rs @@ -13,7 +13,7 @@ // limitations under the License. use std::cmp::{Ordering, Reverse}; -use std::collections::{BinaryHeap, HashMap}; +use std::collections::BinaryHeap; use std::fmt::Write; use std::sync::Arc; @@ -27,17 +27,16 @@ use risingwave_pb::meta::EventLog; use risingwave_pb::monitor_service::StackTraceResponse; use risingwave_rpc_client::ComputeClientPool; use serde_json::json; +use thiserror_ext::AsReport; use crate::hummock::HummockManagerRef; use crate::manager::event_log::EventLogMangerRef; -use crate::manager::{CatalogManagerRef, ClusterManagerRef, FragmentManagerRef}; +use crate::manager::MetadataManager; pub type DiagnoseCommandRef = Arc; pub struct DiagnoseCommand { - cluster_manager: ClusterManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, hummock_manger: HummockManagerRef, event_log_manager: EventLogMangerRef, prometheus_client: Option, @@ -46,18 +45,14 @@ pub struct DiagnoseCommand { impl DiagnoseCommand { pub fn new( - cluster_manager: ClusterManagerRef, - catalog_manager: CatalogManagerRef, - fragment_manager: FragmentManagerRef, + metadata_manager: MetadataManager, hummock_manger: HummockManagerRef, event_log_manager: EventLogMangerRef, prometheus_client: Option, prometheus_selector: String, ) -> Self { Self { - cluster_manager, - catalog_manager, - fragment_manager, + metadata_manager, hummock_manger, event_log_manager, prometheus_client, @@ -90,49 +85,60 @@ impl DiagnoseCommand { #[cfg_attr(coverage, coverage(off))] async fn write_catalog(&self, s: &mut String) { + match &self.metadata_manager { + MetadataManager::V1(_) => self.write_catalog_v1(s).await, + MetadataManager::V2(_) => self.write_catalog_v2(s).await, + } + } + + #[cfg_attr(coverage, coverage(off))] + async fn write_catalog_v1(&self, s: &mut String) { + let mgr = self.metadata_manager.as_v1_ref(); let _ = writeln!(s, "number of fragment: {}", self.fragment_num().await); let _ = writeln!(s, "number of actor: {}", self.actor_num().await); let _ = writeln!( s, "number of source: {}", - self.catalog_manager.source_count().await + mgr.catalog_manager.source_count().await ); let _ = writeln!( s, "number of table: {}", - self.catalog_manager.table_count().await + mgr.catalog_manager.table_count().await ); let _ = writeln!( s, "number of materialized view: {}", - self.catalog_manager.materialized_view_count().await + mgr.catalog_manager.materialized_view_count().await ); let _ = writeln!( s, "number of sink: {}", - self.catalog_manager.sink_count().await + mgr.catalog_manager.sink_count().await ); let _ = writeln!( s, "number of index: {}", - self.catalog_manager.index_count().await + mgr.catalog_manager.index_count().await ); let _ = writeln!( s, "number of function: {}", - self.catalog_manager.function_count().await + mgr.catalog_manager.function_count().await ); } #[cfg_attr(coverage, coverage(off))] async fn fragment_num(&self) -> usize { - let core = self.fragment_manager.get_fragment_read_guard().await; + let mgr = self.metadata_manager.as_v1_ref(); + let core = mgr.fragment_manager.get_fragment_read_guard().await; core.table_fragments().len() } #[cfg_attr(coverage, coverage(off))] async fn actor_num(&self) -> usize { - let core = self.fragment_manager.get_fragment_read_guard().await; + let mgr = self.metadata_manager.as_v1_ref(); + let core = mgr.fragment_manager.get_fragment_read_guard().await; core.table_fragments() .values() .map(|t| t.actor_status.len()) @@ -140,25 +146,38 @@ impl DiagnoseCommand { } #[cfg_attr(coverage, coverage(off))] - async fn write_worker_nodes(&self, s: &mut String) { - let mut worker_actor_count: HashMap = HashMap::new(); - for f in self - .fragment_manager - .get_fragment_read_guard() - .await - .table_fragments() - .values() - { - for a in f.actor_status.values() { - if let Some(pu) = &a.parallel_unit { - let e = worker_actor_count.entry(pu.worker_node_id).or_insert(0); - *e += 1; - } + async fn write_catalog_v2(&self, s: &mut String) { + let mgr = self.metadata_manager.as_v2_ref(); + let guard = mgr.catalog_controller.get_inner_read_guard().await; + let stat = match guard.stats().await { + Ok(stat) => stat, + Err(err) => { + tracing::warn!(error=?err.as_report(), "failed to get catalog stats"); + return; } - } + }; + let _ = writeln!(s, "number of fragment: {}", stat.streaming_job_num); + let _ = writeln!(s, "number of actor: {}", stat.actor_num); + let _ = writeln!(s, "number of source: {}", stat.source_num); + let _ = writeln!(s, "number of table: {}", stat.table_num); + let _ = writeln!(s, "number of materialized view: {}", stat.mview_num); + let _ = writeln!(s, "number of sink: {}", stat.sink_num); + let _ = writeln!(s, "number of index: {}", stat.index_num); + let _ = writeln!(s, "number of function: {}", stat.function_num); + } + + #[cfg_attr(coverage, coverage(off))] + async fn write_worker_nodes(&self, s: &mut String) { + let Ok(worker_actor_count) = self.metadata_manager.worker_actor_count().await else { + tracing::warn!("failed to get worker actor count"); + return; + }; use comfy_table::{Row, Table}; - let worker_nodes = self.cluster_manager.list_worker_node(None, None).await; + let Ok(worker_nodes) = self.metadata_manager.list_worker_node(None, None).await else { + tracing::warn!("failed to get worker nodes"); + return; + }; let mut table = Table::new(); table.set_header({ let mut row = Row::new(); @@ -636,10 +655,14 @@ impl DiagnoseCommand { #[cfg_attr(coverage, coverage(off))] async fn write_await_tree(&self, s: &mut String) { // Most lines of code are copied from dashboard::handlers::dump_await_tree_all, because the latter cannot be called directly from here. - let worker_nodes = self - .cluster_manager + let Ok(worker_nodes) = self + .metadata_manager .list_worker_node(Some(WorkerType::ComputeNode), None) - .await; + .await + else { + tracing::warn!("failed to get worker nodes"); + return; + }; let mut all = Default::default(); diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index eef1c0c101256..fd9b51fa8ba09 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -637,6 +637,19 @@ impl MetadataManager { } } + pub async fn worker_actor_count(&self) -> MetaResult> { + match &self { + MetadataManager::V1(mgr) => Ok(mgr.fragment_manager.node_actor_count().await), + MetadataManager::V2(mgr) => { + let actor_cnt = mgr.catalog_controller.worker_actor_count().await?; + Ok(actor_cnt + .into_iter() + .map(|(id, cnt)| (id as WorkerId, cnt)) + .collect()) + } + } + } + pub async fn count_streaming_job(&self) -> MetaResult { match self { MetadataManager::V1(mgr) => Ok(mgr.fragment_manager.count_streaming_job().await), diff --git a/src/meta/src/manager/sink_coordination/coordinator_worker.rs b/src/meta/src/manager/sink_coordination/coordinator_worker.rs index bebef2d307dcc..e1c096aa3cf98 100644 --- a/src/meta/src/manager/sink_coordination/coordinator_worker.rs +++ b/src/meta/src/manager/sink_coordination/coordinator_worker.rs @@ -110,7 +110,7 @@ impl CoordinatorWorker { .wait_for_writers(first_writer_request.vnode_bitmap) .await { - error!("failed to wait for all writers: {:?}", e); + error!(error = %e.as_report(), "failed to wait for all writers"); worker .send_to_all_sink_writers(|| { Err(Status::cancelled("failed to wait for all writers")) diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index 7cff9dc4a9b7a..908c62ebdffe6 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -31,6 +31,7 @@ use risingwave_common::util::stream_graph_visitor::{ }; use risingwave_common::{bail, current_cluster_version}; use risingwave_connector::dispatch_source_prop; +use risingwave_connector::error::ConnectorError; use risingwave_connector::source::cdc::CdcSourceType; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceProperties, SplitEnumerator, @@ -436,7 +437,7 @@ impl DdlController { mgr.catalog_manager .cancel_create_source_procedure(&source) .await?; - return Err(e.into()); + return Err(e); } mgr.catalog_manager @@ -804,7 +805,7 @@ impl DdlController { pub(crate) async fn validate_cdc_table( table: &Table, table_fragments: &TableFragments, - ) -> anyhow::Result<()> { + ) -> MetaResult<()> { let stream_scan_fragment = table_fragments .fragments .values() @@ -820,7 +821,7 @@ impl DdlController { async fn new_enumerator_for_validate( source_props: P, - ) -> Result { + ) -> Result { P::SplitEnumerator::new(source_props, SourceEnumeratorContext::default().into()).await } diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index 8af470ce7df65..eb3d6b3205c4c 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -20,10 +20,11 @@ use std::ops::Deref; use std::sync::Arc; use std::time::Duration; -use anyhow::{anyhow, Context}; +use anyhow::Context; use risingwave_common::catalog::TableId; use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_connector::dispatch_source_prop; +use risingwave_connector::error::ConnectorResult; use risingwave_connector::source::{ ConnectorProperties, SourceEnumeratorContext, SourceEnumeratorInfo, SourceProperties, SplitEnumerator, SplitId, SplitImpl, SplitMetaData, @@ -81,12 +82,12 @@ struct ConnectorSourceWorker { source_is_up: LabelGuardedIntGauge<2>, } -fn extract_prop_from_existing_source(source: &Source) -> anyhow::Result { +fn extract_prop_from_existing_source(source: &Source) -> ConnectorResult { let mut properties = ConnectorProperties::extract(source.with_properties.clone(), false)?; properties.init_from_pb_source(source); Ok(properties) } -fn extract_prop_from_new_source(source: &Source) -> anyhow::Result { +fn extract_prop_from_new_source(source: &Source) -> ConnectorResult { let mut properties = ConnectorProperties::extract(source.with_properties.clone(), true)?; properties.init_from_pb_source(source); Ok(properties) @@ -96,7 +97,7 @@ const DEFAULT_SOURCE_WORKER_TICK_INTERVAL: Duration = Duration::from_secs(30); impl ConnectorSourceWorker

{ /// Recreate the `SplitEnumerator` to establish a new connection to the external source service. - async fn refresh(&mut self) -> anyhow::Result<()> { + async fn refresh(&mut self) -> MetaResult<()> { let enumerator = P::SplitEnumerator::new( self.connector_properties.clone(), Arc::new(SourceEnumeratorContext { @@ -124,7 +125,7 @@ impl ConnectorSourceWorker

{ period: Duration, splits: Arc>, metrics: Arc, - ) -> anyhow::Result { + ) -> MetaResult { let enumerator = P::SplitEnumerator::new( connector_properties.clone(), Arc::new(SourceEnumeratorContext { @@ -711,7 +712,7 @@ impl SourceManager { let handle = core .managed_sources .get(&source_id) - .ok_or_else(|| anyhow!("could not found source {}", source_id))?; + .with_context(|| format!("could not find source {}", source_id))?; if handle.splits.lock().await.splits.is_none() { // force refresh source @@ -758,7 +759,7 @@ impl SourceManager { } /// register connector worker for source. - pub async fn register_source(&self, source: &Source) -> anyhow::Result<()> { + pub async fn register_source(&self, source: &Source) -> MetaResult<()> { let mut core = self.core.lock().await; if core.managed_sources.contains_key(&source.get_id()) { tracing::warn!("source {} already registered", source.get_id()); @@ -823,7 +824,7 @@ impl SourceManager { break worker; } Err(e) => { - tracing::warn!("failed to create source worker: {}", e); + tracing::warn!(error = %e.as_report(), "failed to create source worker"); } } }; @@ -852,7 +853,7 @@ impl SourceManager { source: &Source, managed_sources: &mut HashMap, metrics: Arc, - ) -> anyhow::Result<()> { + ) -> MetaResult<()> { tracing::info!("spawning new watcher for source {}", source.id); let splits = Arc::new(Mutex::new(SharedSplitMap { splits: None })); @@ -879,11 +880,12 @@ impl SourceManager { // in kafka tokio::time::timeout(Self::DEFAULT_SOURCE_TICK_TIMEOUT, worker.tick()) .await - .map_err(|_e| { - anyhow!( - "failed to fetch meta info for source {}, error: timeout {}", + .ok() + .with_context(|| { + format!( + "failed to fetch meta info for source {}, timeout {:?}", source.id, - Self::DEFAULT_SOURCE_TICK_TIMEOUT.as_secs() + Self::DEFAULT_SOURCE_TICK_TIMEOUT ) })??; @@ -984,8 +986,8 @@ pub fn build_actor_split_impls( mod tests { use std::collections::{BTreeMap, HashMap, HashSet}; - use anyhow::anyhow; use risingwave_common::types::JsonbVal; + use risingwave_connector::error::ConnectorResult; use risingwave_connector::source::{SplitId, SplitMetaData}; use serde::{Deserialize, Serialize}; @@ -1006,11 +1008,11 @@ mod tests { serde_json::to_value(*self).unwrap().into() } - fn restore_from_json(value: JsonbVal) -> anyhow::Result { - serde_json::from_value(value.take()).map_err(|e| anyhow!(e)) + fn restore_from_json(value: JsonbVal) -> ConnectorResult { + serde_json::from_value(value.take()).map_err(Into::into) } - fn update_with_offset(&mut self, _start_offset: String) -> anyhow::Result<()> { + fn update_with_offset(&mut self, _start_offset: String) -> ConnectorResult<()> { Ok(()) } } diff --git a/src/risedevtool/src/bin/risedev-dev.rs b/src/risedevtool/src/bin/risedev-dev.rs index ea9099eae319c..9723ee89fbd51 100644 --- a/src/risedevtool/src/bin/risedev-dev.rs +++ b/src/risedevtool/src/bin/risedev-dev.rs @@ -31,6 +31,7 @@ use risedev::{ RISEDEV_SESSION_NAME, }; use tempfile::tempdir; +use thiserror_ext::AsReport; use yaml_rust::YamlEmitter; #[derive(Default)] @@ -444,9 +445,9 @@ fn main() -> Result<()> { } Err(err) => { println!( - "{} - Failed to start: {:?}", // with `Caused by` + "{} - Failed to start: {:#}", // pretty with `Caused by` style("ERROR").red().bold(), - err, + err.as_report(), ); println!(); println!( diff --git a/src/risedevtool/src/preflight_check.rs b/src/risedevtool/src/preflight_check.rs index 17dc48884fea8..9b25d39423566 100644 --- a/src/risedevtool/src/preflight_check.rs +++ b/src/risedevtool/src/preflight_check.rs @@ -17,6 +17,7 @@ use std::process::Command; use anyhow::Result; use console::style; +use thiserror_ext::AsReport; fn preflight_check_proxy() -> Result<()> { if env::var("http_proxy").is_ok() @@ -72,7 +73,7 @@ pub fn preflight_check() -> Result<()> { "[{}] {} - failed to run proxy preflight check: {}", style("risedev-preflight-check").bold(), style("WARN").yellow().bold(), - e + e.as_report() ); } @@ -81,7 +82,7 @@ pub fn preflight_check() -> Result<()> { "[{}] {} - failed to run ulimit preflight check: {}", style("risedev-preflight-check").bold(), style("WARN").yellow().bold(), - e + e.as_report() ); } diff --git a/src/storage/src/hummock/compactor/compactor_runner.rs b/src/storage/src/hummock/compactor/compactor_runner.rs index 710a6fb3c4a65..441480d66a6b7 100644 --- a/src/storage/src/hummock/compactor/compactor_runner.rs +++ b/src/storage/src/hummock/compactor/compactor_runner.rs @@ -766,6 +766,7 @@ where let mut is_new_user_key = full_key_tracker.observe(iter.key()).is_some(); let mut drop = false; + // CRITICAL WARN: Because of memtable spill, there may be several versions of the same user-key share the same `pure_epoch`. Do not change this code unless necessary. let epoch = iter_key.epoch_with_gap.pure_epoch(); let value = iter.value(); if is_new_user_key { @@ -818,7 +819,9 @@ where // in our design, frontend avoid to access keys which had be deleted, so we dont // need to consider the epoch when the compaction_filter match (it // means that mv had drop) - if (epoch <= task_config.watermark && task_config.gc_delete_keys && value.is_delete()) + // Because of memtable spill, there may be a PUT key share the same `pure_epoch` with DELETE key. + // Do not assume that "the epoch of keys behind must be smaller than the current key." + if (epoch < task_config.watermark && task_config.gc_delete_keys && value.is_delete()) || (epoch < task_config.watermark && (watermark_can_see_last_key || earliest_range_delete_which_can_see_iter_key <= task_config.watermark)) diff --git a/src/storage/src/hummock/compactor/fast_compactor_runner.rs b/src/storage/src/hummock/compactor/fast_compactor_runner.rs index 935bfa27c4666..ab4ddc216b1a8 100644 --- a/src/storage/src/hummock/compactor/fast_compactor_runner.rs +++ b/src/storage/src/hummock/compactor/fast_compactor_runner.rs @@ -600,7 +600,9 @@ impl CompactTaskExecutor { self.watermark_can_see_last_key = false; self.last_key_is_delete = false; } - if epoch <= self.task_config.watermark + + // See note in `compactor_runner.rs`. + if epoch < self.task_config.watermark && self.task_config.gc_delete_keys && value.is_delete() { diff --git a/src/stream/benches/stream_hash_agg.rs b/src/stream/benches/stream_hash_agg.rs index 90ac7ef7725e7..2a0d1a93cc6b7 100644 --- a/src/stream/benches/stream_hash_agg.rs +++ b/src/stream/benches/stream_hash_agg.rs @@ -26,7 +26,7 @@ use risingwave_storage::memory::MemoryStateStore; use risingwave_storage::StateStore; use risingwave_stream::executor::test_utils::agg_executor::new_boxed_hash_agg_executor; use risingwave_stream::executor::test_utils::*; -use risingwave_stream::executor::{BoxedExecutor, PkIndices}; +use risingwave_stream::executor::{Executor, PkIndices}; use tokio::runtime::Runtime; risingwave_expr_impl::enable!(); @@ -47,7 +47,7 @@ fn bench_hash_agg(c: &mut Criterion) { /// This aims to mirror `q17`'s aggregator. /// We can include more executor patterns as needed. -fn setup_bench_hash_agg(store: S) -> BoxedExecutor { +fn setup_bench_hash_agg(store: S) -> Executor { // ---- Define hash agg executor parameters ---- let input_data_types = vec![ // to_char(date_time) @@ -119,7 +119,8 @@ fn setup_bench_hash_agg(store: S) -> BoxedExecutor { ); // ---- Create MockSourceExecutor ---- - let (mut tx, source) = MockSource::channel(schema, PkIndices::new()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); tx.push_barrier(1, false); for chunk in chunks { tx.push_chunk(chunk); @@ -134,7 +135,7 @@ fn setup_bench_hash_agg(store: S) -> BoxedExecutor { block_on(new_boxed_hash_agg_executor( store, - Box::new(source), + source, false, agg_calls, row_count_index, @@ -146,7 +147,7 @@ fn setup_bench_hash_agg(store: S) -> BoxedExecutor { )) } -pub async fn execute_executor(executor: BoxedExecutor) { +pub async fn execute_executor(executor: Executor) { let mut stream = executor.execute(); while let Some(ret) = stream.next().await { _ = black_box(ret.unwrap()); diff --git a/src/stream/src/executor/agg_common.rs b/src/stream/src/executor/agg_common.rs index 603cb7f244f87..91c414877a76b 100644 --- a/src/stream/src/executor/agg_common.rs +++ b/src/stream/src/executor/agg_common.rs @@ -29,7 +29,7 @@ pub struct AggExecutorArgs { pub version: PbAggNodeVersion, // basic - pub input: Box, + pub input: Executor, pub actor_ctx: ActorContextRef, pub info: ExecutorInfo, diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 0afea6720cad2..2dd277449be13 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -22,7 +22,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{DataChunk, Op, StreamChunk}; use risingwave_common::bail; -use risingwave_common::catalog::Schema; use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::OwnedRow; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -40,8 +39,8 @@ use crate::executor::backfill::utils::{ }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, - HashMap, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, HashMap, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; @@ -57,7 +56,7 @@ pub struct ArrangementBackfillExecutor { upstream_table: ReplicatedStateTable, /// Upstream with the same schema with the upstream table. - upstream: BoxedExecutor, + upstream: Executor, /// Internal state table for persisting state of backfill state. state_table: StateTable, @@ -69,8 +68,6 @@ pub struct ArrangementBackfillExecutor { actor_id: ActorId, - info: ExecutorInfo, - metrics: Arc, chunk_size: usize, @@ -86,9 +83,8 @@ where #[allow(clippy::too_many_arguments)] #[allow(dead_code)] pub fn new( - info: ExecutorInfo, upstream_table: ReplicatedStateTable, - upstream: BoxedExecutor, + upstream: Executor, state_table: StateTable, output_indices: Vec, progress: CreateMviewProgress, @@ -97,7 +93,6 @@ where rate_limit: Option, ) -> Self { Self { - info, upstream_table, upstream, state_table, @@ -710,7 +705,7 @@ where } } -impl Executor for ArrangementBackfillExecutor +impl Execute for ArrangementBackfillExecutor where S: StateStore, SD: ValueRowSerde, @@ -718,16 +713,4 @@ where fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index bb99e2ecab369..e4b83622951f6 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -45,8 +45,8 @@ use crate::executor::backfill::utils::{ }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::task::CreateMviewProgress; @@ -55,13 +55,12 @@ const METADATA_STATE_LEN: usize = 4; pub struct CdcBackfillExecutor { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// The external table to be backfilled external_table: ExternalStorageTable, /// Upstream changelog stream which may contain metadata columns, e.g. `_rw_offset` - upstream: BoxedExecutor, + upstream: Executor, /// The column indices need to be forwarded to the downstream from the upstream and table scan. /// User may select a subset of columns from the upstream table. @@ -83,9 +82,8 @@ impl CdcBackfillExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, external_table: ExternalStorageTable, - upstream: BoxedExecutor, + upstream: Executor, output_indices: Vec, progress: Option, metrics: Arc, @@ -95,7 +93,6 @@ impl CdcBackfillExecutor { ) -> Self { Self { actor_ctx, - info, external_table, upstream, output_indices, @@ -609,22 +606,10 @@ fn get_rw_columns(schema: &Schema) -> Vec { .collect_vec() } -impl Executor for CdcBackfillExecutor { +impl Execute for CdcBackfillExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -639,7 +624,6 @@ mod tests { use crate::executor::backfill::cdc::cdc_backfill::transform_upstream; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[tokio::test] async fn test_transform_upstream_chunk() { @@ -649,7 +633,8 @@ mod tests { Field::unnamed(DataType::Varchar), // _rw_table_name ]); let pk_indices = vec![1]; - let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema.clone(), pk_indices.clone()); // let payload = r#"{"before": null,"after":{"O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" },"source":{"version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null},"op":"r","ts_ms":1695277757017,"transaction":null}"#.to_string(); let payload = r#"{ "payload": { "before": null, "after": { "O_ORDERKEY": 5, "O_CUSTKEY": 44485, "O_ORDERSTATUS": "F", "O_TOTALPRICE": "144659.20", "O_ORDERDATE": "1994-07-30" }, "source": { "version": "1.9.7.Final", "connector": "mysql", "name": "RW_CDC_1002", "ts_ms": 1695277757000, "snapshot": "last", "db": "mydb", "sequence": null, "table": "orders_new", "server_id": 0, "gtid": null, "file": "binlog.000008", "pos": 3693, "row": 0, "thread": null, "query": null }, "op": "r", "ts_ms": 1695277757017, "transaction": null } }"#; diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index 824bc51e13b00..4aea8f63b26ea 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -20,7 +20,6 @@ use futures::stream::select_with_strategy; use futures::{stream, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{DataChunk, Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::Datum; @@ -39,8 +38,8 @@ use crate::executor::backfill::utils::{ }; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, Barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, - Message, Mutation, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, Barrier, BoxedMessageStream, Execute, Executor, Message, Mutation, + StreamExecutorError, StreamExecutorResult, }; use crate::task::{ActorId, CreateMviewProgress}; @@ -76,12 +75,10 @@ pub struct BackfillState { /// in the same worker, so that we can read uncommitted data from the upstream table without /// waiting. pub struct BackfillExecutor { - info: ExecutorInfo, - /// Upstream table upstream_table: StorageTable, /// Upstream with the same schema with the upstream table. - upstream: BoxedExecutor, + upstream: Executor, /// Internal state table for persisting state of backfill state. state_table: Option>, @@ -110,9 +107,8 @@ where { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, upstream_table: StorageTable, - upstream: BoxedExecutor, + upstream: Executor, state_table: Option>, output_indices: Vec, progress: CreateMviewProgress, @@ -122,7 +118,6 @@ where ) -> Self { let actor_id = progress.actor_id(); Self { - info, upstream_table, upstream, state_table, @@ -742,23 +737,11 @@ where } } -impl Executor for BackfillExecutor +impl Execute for BackfillExecutor where S: StateStore, { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/barrier_recv.rs b/src/stream/src/executor/barrier_recv.rs index 66480ef1cb591..4ca07a80382a5 100644 --- a/src/stream/src/executor/barrier_recv.rs +++ b/src/stream/src/executor/barrier_recv.rs @@ -13,52 +13,37 @@ // limitations under the License. use futures::StreamExt; -use risingwave_common::catalog::Schema; use tokio::sync::mpsc::UnboundedReceiver; use tokio_stream::wrappers::UnboundedReceiverStream; use super::{ - ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, - PkIndices, PkIndicesRef, StreamExecutorError, + ActorContext, ActorContextRef, Barrier, BoxedMessageStream, Execute, Message, + StreamExecutorError, }; /// The executor only for receiving barrier from the meta service. It always resides in the leaves /// of the streaming graph. pub struct BarrierRecvExecutor { _ctx: ActorContextRef, - info: ExecutorInfo, /// The barrier receiver registered in the local barrier manager. barrier_receiver: UnboundedReceiver, } impl BarrierRecvExecutor { - pub fn new( - ctx: ActorContextRef, - info: ExecutorInfo, - barrier_receiver: UnboundedReceiver, - ) -> Self { + pub fn new(ctx: ActorContextRef, barrier_receiver: UnboundedReceiver) -> Self { Self { _ctx: ctx, - info, barrier_receiver, } } pub fn for_test(barrier_receiver: UnboundedReceiver) -> Self { - Self::new( - ActorContext::for_test(0), - ExecutorInfo { - schema: Schema::empty().clone(), - pk_indices: PkIndices::new(), - identity: "BarrierRecvExecutor".to_string(), - }, - barrier_receiver, - ) + Self::new(ActorContext::for_test(0), barrier_receiver) } } -impl Executor for BarrierRecvExecutor { +impl Execute for BarrierRecvExecutor { fn execute(self: Box) -> BoxedMessageStream { UnboundedReceiverStream::new(self.barrier_receiver) .map(|barrier| Ok(Message::Barrier(barrier))) @@ -69,18 +54,6 @@ impl Executor for BarrierRecvExecutor { })) .boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] diff --git a/src/stream/src/executor/batch_query.rs b/src/stream/src/executor/batch_query.rs index 533a327432f43..7c92bcd732423 100644 --- a/src/stream/src/executor/batch_query.rs +++ b/src/stream/src/executor/batch_query.rs @@ -24,7 +24,7 @@ use risingwave_storage::table::collect_data_chunk; use risingwave_storage::StateStore; use super::error::StreamExecutorError; -use super::{Executor, ExecutorInfo, Message}; +use super::{Execute, Message}; use crate::executor::BoxedMessageStream; pub struct BatchQueryExecutor { @@ -34,18 +34,18 @@ pub struct BatchQueryExecutor { /// The number of tuples in one [`StreamChunk`] batch_size: usize, - info: ExecutorInfo, + schema: Schema, } impl BatchQueryExecutor where S: StateStore, { - pub fn new(table: StorageTable, batch_size: usize, info: ExecutorInfo) -> Self { + pub fn new(table: StorageTable, batch_size: usize, schema: Schema) -> Self { Self { table, batch_size, - info, + schema, } } @@ -62,7 +62,7 @@ where pin_mut!(iter); while let Some(data_chunk) = - collect_data_chunk(&mut iter, self.schema(), Some(self.batch_size)) + collect_data_chunk(&mut iter, &self.schema, Some(self.batch_size)) .instrument_await("batch_query_executor_collect_chunk") .await? { @@ -73,7 +73,7 @@ where } } -impl Executor for BatchQueryExecutor +impl Execute for BatchQueryExecutor where S: StateStore, { @@ -81,18 +81,6 @@ where unreachable!("should call `execute_with_epoch`") } - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - fn execute_with_epoch(self: Box, epoch: u64) -> BoxedMessageStream { self.execute_inner(epoch).boxed() } @@ -100,9 +88,6 @@ where #[cfg(test)] mod test { - - use std::vec; - use futures_async_stream::for_await; use super::*; @@ -114,15 +99,10 @@ mod test { let test_batch_count = 5; let table = gen_basic_table(test_batch_count * test_batch_size).await; - let info = ExecutorInfo { - schema: table.schema().clone(), - pk_indices: vec![0, 1], - identity: "BatchQuery".to_owned(), - }; - - let executor = Box::new(BatchQueryExecutor::new(table, test_batch_size, info)); - - let stream = executor.execute_with_epoch(u64::MAX); + let schema = table.schema().clone(); + let stream = BatchQueryExecutor::new(table, test_batch_size, schema) + .boxed() + .execute_with_epoch(u64::MAX); let mut batch_cnt = 0; #[for_await] diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs index 0228f826a4bac..3e65bf878b552 100644 --- a/src/stream/src/executor/chain.rs +++ b/src/stream/src/executor/chain.rs @@ -14,10 +14,9 @@ use futures::StreamExt; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; -use super::{expect_first_barrier, BoxedExecutor, Executor, ExecutorInfo, Message}; +use super::{expect_first_barrier, Execute, Executor, Message}; use crate::task::{ActorId, CreateMviewProgress}; /// [`ChainExecutor`] is an executor that enables synchronization between the existing stream and @@ -25,11 +24,9 @@ use crate::task::{ActorId, CreateMviewProgress}; /// feature. It pipes new data of existing MVs to newly created MV only all of the old data in the /// existing MVs are dispatched. pub struct ChainExecutor { - info: ExecutorInfo, + snapshot: Executor, - snapshot: BoxedExecutor, - - upstream: BoxedExecutor, + upstream: Executor, progress: CreateMviewProgress, @@ -41,14 +38,12 @@ pub struct ChainExecutor { impl ChainExecutor { pub fn new( - info: ExecutorInfo, - snapshot: BoxedExecutor, - upstream: BoxedExecutor, + snapshot: Executor, + upstream: Executor, progress: CreateMviewProgress, upstream_only: bool, ) -> Self { Self { - info, snapshot, upstream, actor_id: progress.actor_id(), @@ -104,22 +99,10 @@ impl ChainExecutor { } } -impl Executor for ChainExecutor { +impl Execute for ChainExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -135,9 +118,7 @@ mod test { use super::ChainExecutor; use crate::executor::test_utils::MockSource; - use crate::executor::{ - AddMutation, Barrier, Executor, ExecutorInfo, Message, Mutation, PkIndices, - }; + use crate::executor::{AddMutation, Barrier, Execute, Message, Mutation, PkIndices}; use crate::task::{CreateMviewProgress, LocalBarrierManager}; #[tokio::test] @@ -147,48 +128,35 @@ mod test { let actor_id = progress.actor_id(); let schema = Schema::new(vec![Field::unnamed(DataType::Int64)]); - let first = Box::new( - MockSource::with_chunks( - schema.clone(), - PkIndices::new(), - vec![ - StreamChunk::from_pretty("I\n + 1"), - StreamChunk::from_pretty("I\n + 2"), - ], - ) - .stop_on_finish(false), - ); - - let second = Box::new(MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1).with_mutation(Mutation::Add( - AddMutation { - adds: maplit::hashmap! { - 0 => vec![Dispatcher { - downstream_actor_id: vec![actor_id], - ..Default::default() - }], - }, - added_actors: maplit::hashset! { actor_id }, - splits: Default::default(), - pause: false, + let first = MockSource::with_chunks(vec![ + StreamChunk::from_pretty("I\n + 1"), + StreamChunk::from_pretty("I\n + 2"), + ]) + .stop_on_finish(false) + .into_executor(schema.clone(), PkIndices::new()); + + let second = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1).with_mutation(Mutation::Add( + AddMutation { + adds: maplit::hashmap! { + 0 => vec![Dispatcher { + downstream_actor_id: vec![actor_id], + ..Default::default() + }], }, - ))), - Message::Chunk(StreamChunk::from_pretty("I\n + 3")), - Message::Chunk(StreamChunk::from_pretty("I\n + 4")), - ], - )); - - let info = ExecutorInfo { - schema, - pk_indices: PkIndices::new(), - identity: "ChainExecutor".to_string(), - }; - let chain = ChainExecutor::new(info, first, second, progress, false); - - let mut chain = Box::new(chain).execute(); + added_actors: maplit::hashset! { actor_id }, + splits: Default::default(), + pause: false, + }, + ))), + Message::Chunk(StreamChunk::from_pretty("I\n + 3")), + Message::Chunk(StreamChunk::from_pretty("I\n + 4")), + ]) + .into_executor(schema.clone(), PkIndices::new()); + + let chain = ChainExecutor::new(first, second, progress, false); + + let mut chain = chain.boxed().execute(); chain.next().await; let mut count = 0; diff --git a/src/stream/src/executor/dedup/append_only_dedup.rs b/src/stream/src/executor/dedup/append_only_dedup.rs index 6049b963359ab..34b102ff1e030 100644 --- a/src/stream/src/executor/dedup/append_only_dedup.rs +++ b/src/stream/src/executor/dedup/append_only_dedup.rs @@ -19,7 +19,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; -use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_storage::StateStore; @@ -29,39 +28,39 @@ use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorResult, }; use crate::task::AtomicU64Ref; /// [`AppendOnlyDedupExecutor`] drops any message that has duplicate pk columns with previous /// messages. It only accepts append-only input, and its output will be append-only as well. pub struct AppendOnlyDedupExecutor { - input: Option, + ctx: ActorContextRef, + + input: Option, + dedup_cols: Vec, state_table: StateTable, cache: DedupCache, - - info: ExecutorInfo, - ctx: ActorContextRef, } impl AppendOnlyDedupExecutor { pub fn new( - input: BoxedExecutor, - state_table: StateTable, - info: ExecutorInfo, ctx: ActorContextRef, + input: Executor, + dedup_cols: Vec, + state_table: StateTable, watermark_epoch: AtomicU64Ref, metrics: Arc, ) -> Self { let metrics_info = MetricsInfo::new(metrics, state_table.table_id(), ctx.id, "AppendOnly Dedup"); Self { + ctx, input: Some(input), + dedup_cols, state_table, cache: DedupCache::new(watermark_epoch, metrics_info), - info, - ctx, } } @@ -90,7 +89,7 @@ impl AppendOnlyDedupExecutor { .data_chunk() .rows_with_holes() .map(|row_ref| { - row_ref.map(|row| row.project(self.pk_indices()).to_owned_row()) + row_ref.map(|row| row.project(&self.dedup_cols).to_owned_row()) }) .collect_vec(); @@ -185,22 +184,10 @@ impl AppendOnlyDedupExecutor { } } -impl Executor for AppendOnlyDedupExecutor { +impl Execute for AppendOnlyDedupExecutor { fn execute(self: Box) -> BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -230,7 +217,8 @@ mod tests { Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), ]); - let pk_indices = vec![0]; + let dedup_col_indices = vec![0]; + let pk_indices = dedup_col_indices.clone(); let order_types = vec![OrderType::ascending()]; let state_store = MemoryStateStore::new(); @@ -243,20 +231,17 @@ mod tests { ) .await; - let (mut tx, input) = MockSource::channel(schema.clone(), pk_indices.clone()); - let info = ExecutorInfo { - schema, - pk_indices, - identity: "AppendOnlyDedupExecutor".to_string(), - }; - let mut dedup_executor = Box::new(AppendOnlyDedupExecutor::new( - Box::new(input), - state_table, - info, + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, pk_indices); + let mut dedup_executor = AppendOnlyDedupExecutor::new( ActorContext::for_test(123), + source, + dedup_col_indices, + state_table, Arc::new(AtomicU64::new(0)), Arc::new(StreamingMetrics::unused()), - )) + ) + .boxed() .execute(); tx.push_barrier(1, false); diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index a21012c5f1664..ee347ee489258 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -35,17 +35,17 @@ use tokio::time::Instant; use tracing::{event, Instrument}; use super::exchange::output::{new_output, BoxedOutput}; -use super::{AddMutation, UpdateMutation, Watermark}; +use super::{AddMutation, Executor, UpdateMutation, Watermark}; use crate::error::StreamResult; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{Barrier, BoxedExecutor, Message, Mutation, StreamConsumer}; +use crate::executor::{Barrier, Message, Mutation, StreamConsumer}; use crate::task::{ActorId, DispatcherId, SharedContext}; /// [`DispatchExecutor`] consumes messages and send them into downstream actors. Usually, /// data chunks will be dispatched with some specified policy, while control message /// such as barriers will be distributed to all receivers. pub struct DispatchExecutor { - input: BoxedExecutor, + input: Executor, inner: DispatchExecutorInner, } @@ -341,7 +341,7 @@ impl DispatchExecutorInner { impl DispatchExecutor { pub fn new( - input: BoxedExecutor, + input: Executor, dispatchers: Vec, actor_id: u32, fragment_id: u32, @@ -1042,6 +1042,7 @@ mod tests { use crate::executor::exchange::output::Output; use crate::executor::exchange::permit::channel_for_test; use crate::executor::receiver::ReceiverExecutor; + use crate::executor::Execute; use crate::task::test_utils::helper_make_local_actor; #[derive(Debug)] @@ -1152,7 +1153,7 @@ mod tests { let (tx, rx) = channel_for_test(); let actor_id = 233; let fragment_id = 666; - let input = Box::new(ReceiverExecutor::for_test(rx)); + let input = Executor::new(Default::default(), ReceiverExecutor::for_test(rx).boxed()); let ctx = Arc::new(SharedContext::for_test()); let metrics = Arc::new(StreamingMetrics::unused()); diff --git a/src/stream/src/executor/dml.rs b/src/stream/src/executor/dml.rs index 8e0a186e9c81c..0463c695639d7 100644 --- a/src/stream/src/executor/dml.rs +++ b/src/stream/src/executor/dml.rs @@ -19,25 +19,20 @@ use either::Either; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::{ColumnDesc, Schema, TableId, TableVersionId}; +use risingwave_common::catalog::{ColumnDesc, TableId, TableVersionId}; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::transaction::transaction_message::TxnMsg; use risingwave_dml::dml_manager::DmlManagerRef; use super::error::StreamExecutorError; -use super::{ - expect_first_barrier, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, - Mutation, PkIndicesRef, -}; +use super::{expect_first_barrier, BoxedMessageStream, Execute, Executor, Message, Mutation}; use crate::common::StreamChunkBuilder; use crate::executor::stream_reader::StreamReaderWithPause; /// [`DmlExecutor`] accepts both stream data and batch data for data manipulation on a specific /// table. The two streams will be merged into one and then sent to downstream. pub struct DmlExecutor { - info: ExecutorInfo, - - upstream: BoxedExecutor, + upstream: Executor, /// Stores the information of batch data channels. dml_manager: DmlManagerRef, @@ -71,10 +66,8 @@ struct TxnBuffer { } impl DmlExecutor { - #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, - upstream: BoxedExecutor, + upstream: Executor, dml_manager: DmlManagerRef, table_id: TableId, table_version_id: TableVersionId, @@ -82,7 +75,6 @@ impl DmlExecutor { chunk_size: usize, ) -> Self { Self { - info, upstream, dml_manager, table_id, @@ -277,22 +269,10 @@ impl DmlExecutor { } } -impl Executor for DmlExecutor { +impl Execute for DmlExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -300,7 +280,7 @@ mod tests { use std::sync::Arc; use risingwave_common::array::StreamChunk; - use risingwave_common::catalog::{ColumnId, Field, INITIAL_TABLE_VERSION_ID}; + use risingwave_common::catalog::{ColumnId, Field, Schema, INITIAL_TABLE_VERSION_ID}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::transaction::transaction_id::TxnId; use risingwave_common::types::DataType; @@ -326,23 +306,18 @@ mod tests { let pk_indices = vec![0]; let dml_manager = Arc::new(DmlManager::for_test()); - let (mut tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); - let info = ExecutorInfo { - schema, - pk_indices, - identity: "DmlExecutor".to_string(), - }; + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, pk_indices); - let dml_executor = Box::new(DmlExecutor::new( - info, - Box::new(source), + let dml_executor = DmlExecutor::new( + source, dml_manager.clone(), table_id, INITIAL_TABLE_VERSION_ID, column_descs, 1024, - )); - let mut dml_executor = dml_executor.execute(); + ); + let mut dml_executor = dml_executor.boxed().execute(); let stream_chunk1 = StreamChunk::from_pretty( " I I diff --git a/src/stream/src/executor/dynamic_filter.rs b/src/stream/src/executor/dynamic_filter.rs index d3299d99e8a33..6bcd3e97ed563 100644 --- a/src/stream/src/executor/dynamic_filter.rs +++ b/src/stream/src/executor/dynamic_filter.rs @@ -38,10 +38,7 @@ use risingwave_storage::StateStore; use super::barrier_align::*; use super::error::StreamExecutorError; use super::monitor::StreamingMetrics; -use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, - PkIndicesRef, -}; +use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message}; use crate::common::table::state_table::{StateTable, WatermarkCacheParameterizedStateTable}; use crate::common::StreamChunkBuilder; use crate::executor::expect_first_barrier_from_aligned_stream; @@ -49,10 +46,12 @@ use crate::task::ActorEvalErrorReport; pub struct DynamicFilterExecutor { ctx: ActorContextRef, - info: ExecutorInfo, - source_l: Option, - source_r: Option, + eval_error_report: ActorEvalErrorReport, + + schema: Schema, + source_l: Option, + source_r: Option, key_l: usize, comparator: ExprNodeType, left_table: WatermarkCacheParameterizedStateTable, @@ -71,9 +70,9 @@ impl DynamicFilterExecutor, @@ -83,9 +82,14 @@ impl DynamicFilterExecutor Self { + let eval_error_report = ActorEvalErrorReport { + actor_context: ctx.clone(), + identity: Arc::from(info.identity.as_str()), + }; Self { ctx, - info, + eval_error_report, + schema: info.schema.clone(), source_l: Some(source_l), source_r: Some(source_r), key_l, @@ -263,7 +267,7 @@ impl DynamicFilterExecutor DynamicFilterExecutor DynamicFilterExecutor DynamicFilterExecutor Executor +impl Execute for DynamicFilterExecutor { fn execute(self: Box) -> BoxedMessageStream { - self.into_stream().boxed() - } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity + self.execute_inner().boxed() } } @@ -561,21 +550,20 @@ mod tests { let schema = Schema { fields: vec![Field::unnamed(DataType::Int64)], }; - let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![0]); - let (tx_r, source_r) = MockSource::channel(schema, vec![]); - - let schema = source_l.schema().clone(); - let info = ExecutorInfo { - schema, - pk_indices: vec![0], - identity: "DynamicFilterExecutor".to_string(), - }; + let (tx_l, source_l) = MockSource::channel(); + let source_l = source_l.into_executor(schema.clone(), vec![0]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r.into_executor(schema, vec![]); let executor = DynamicFilterExecutor::::new( ActorContext::for_test(123), - info, - Box::new(source_l), - Box::new(source_r), + &ExecutorInfo { + schema: source_l.schema().clone(), + pk_indices: vec![0], + identity: "DynamicFilterExecutor".to_string(), + }, + source_l, + source_r, 0, comparator, mem_state_l, @@ -585,7 +573,7 @@ mod tests { always_relax, false, ); - (tx_l, tx_r, Box::new(executor).execute()) + (tx_l, tx_r, executor.boxed().execute()) } #[tokio::test] diff --git a/src/stream/src/executor/expand.rs b/src/stream/src/executor/expand.rs index cb8c45732d17f..2375ebca3d0df 100644 --- a/src/stream/src/executor/expand.rs +++ b/src/stream/src/executor/expand.rs @@ -16,26 +16,19 @@ use std::fmt::Debug; use futures::StreamExt; use futures_async_stream::try_stream; -use risingwave_common::array::{Array, I64Array}; -use risingwave_common::catalog::Schema; +use risingwave_common::array::{Array, I64Array, StreamChunk}; use super::error::StreamExecutorError; -use super::*; +use super::{BoxedMessageStream, Execute, Executor, Message}; pub struct ExpandExecutor { - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, column_subsets: Vec>, } impl ExpandExecutor { - pub fn new( - info: ExecutorInfo, - input: Box, - column_subsets: Vec>, - ) -> Self { + pub fn new(input: Executor, column_subsets: Vec>) -> Self { Self { - info, input, column_subsets, } @@ -73,19 +66,7 @@ impl Debug for ExpandExecutor { } } -impl Executor for ExpandExecutor { - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - +impl Execute for ExpandExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } @@ -100,7 +81,7 @@ mod tests { use super::ExpandExecutor; use crate::executor::test_utils::MockSource; - use crate::executor::{Executor, ExecutorInfo, PkIndices}; + use crate::executor::{Execute, PkIndices}; #[tokio::test] async fn test_expand() { @@ -111,29 +92,19 @@ mod tests { + 6 6 3 - 7 5 4", ); - let input_schema = Schema { - fields: vec![ + let source = MockSource::with_chunks(vec![chunk1]).into_executor( + Schema::new(vec![ Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), Field::unnamed(DataType::Int64), - ], - }; - let source = MockSource::with_chunks(input_schema.clone(), PkIndices::new(), vec![chunk1]); - let schema = { - let mut fields = input_schema.into_fields(); - fields.extend(fields.clone()); - fields.push(Field::with_name(DataType::Int64, "flag")); - Schema::new(fields) - }; - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "ExpandExecutor".to_string(), - }; + ]), + PkIndices::new(), + ); let column_subsets = vec![vec![0, 1], vec![1, 2]]; - let expand = Box::new(ExpandExecutor::new(info, Box::new(source), column_subsets)); - let mut expand = expand.execute(); + let mut expand = ExpandExecutor::new(source, column_subsets) + .boxed() + .execute(); let chunk = expand.next().await.unwrap().unwrap().into_chunk().unwrap(); assert_eq!( diff --git a/src/stream/src/executor/filter.rs b/src/stream/src/executor/filter.rs index e51307b2ffee0..ffc8847c42abf 100644 --- a/src/stream/src/executor/filter.rs +++ b/src/stream/src/executor/filter.rs @@ -15,13 +15,17 @@ use std::fmt::{Debug, Formatter}; use std::sync::Arc; +use futures::StreamExt; +use futures_async_stream::try_stream; use risingwave_common::array::{Array, ArrayImpl, Op, StreamChunk}; use risingwave_common::buffer::BitmapBuilder; -use risingwave_common::catalog::Schema; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; -use super::*; +use super::{ + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError, + StreamExecutorResult, +}; /// `FilterExecutor` filters data with the `expr`. The `expr` takes a chunk of data, /// and returns a boolean array on whether each item should be retained. And then, @@ -29,8 +33,7 @@ use super::*; /// to the result of the expression. pub struct FilterExecutor { _ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, /// Expression of the current filter, note that the filter must always have the same output for /// the same input. @@ -38,15 +41,9 @@ pub struct FilterExecutor { } impl FilterExecutor { - pub fn new( - ctx: ActorContextRef, - info: ExecutorInfo, - input: Box, - expr: NonStrictExpression, - ) -> Self { + pub fn new(ctx: ActorContextRef, input: Executor, expr: NonStrictExpression) -> Self { Self { _ctx: ctx, - info, input, expr, } @@ -135,19 +132,7 @@ impl Debug for FilterExecutor { } } -impl Executor for FilterExecutor { - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - +impl Execute for FilterExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } @@ -218,23 +203,14 @@ mod tests { ], }; let pk_indices = PkIndices::new(); - let source = - MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk1, chunk2]); - let info = ExecutorInfo { - schema, - pk_indices, - identity: "FilterExecutor".to_string(), - }; + let source = MockSource::with_chunks(vec![chunk1, chunk2]) + .into_executor(schema.clone(), pk_indices.clone()); let test_expr = build_from_pretty("(greater_than:boolean $0:int8 $1:int8)"); - let filter = Box::new(FilterExecutor::new( - ActorContext::for_test(123), - info, - Box::new(source), - test_expr, - )); - let mut filter = filter.execute(); + let mut filter = FilterExecutor::new(ActorContext::for_test(123), source, test_expr) + .boxed() + .execute(); let chunk = filter.next().await.unwrap().unwrap().into_chunk().unwrap(); assert_eq!( diff --git a/src/stream/src/executor/flow_control.rs b/src/stream/src/executor/flow_control.rs index 53d0f15a602ae..4e4212a1e246e 100644 --- a/src/stream/src/executor/flow_control.rs +++ b/src/stream/src/executor/flow_control.rs @@ -15,11 +15,14 @@ use std::fmt::{Debug, Formatter}; use std::num::NonZeroU32; +use futures::StreamExt; +use futures_async_stream::try_stream; use governor::clock::MonotonicClock; use governor::{Quota, RateLimiter}; -use risingwave_common::catalog::Schema; -use super::*; +use super::{ + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError, +}; /// Flow Control Executor is used to control the rate of the input executor. /// @@ -31,27 +34,16 @@ use super::*; /// /// It is used to throttle problematic MVs that are consuming too much resources. pub struct FlowControlExecutor { - input: BoxedExecutor, + input: Executor, actor_ctx: ActorContextRef, - identity: String, rate_limit: Option, } impl FlowControlExecutor { - pub fn new( - input: Box, - actor_ctx: ActorContextRef, - rate_limit: Option, - ) -> Self { - let identity = if rate_limit.is_some() { - format!("{} (flow controlled)", input.identity()) - } else { - input.identity().to_owned() - }; + pub fn new(input: Executor, actor_ctx: ActorContextRef, rate_limit: Option) -> Self { Self { input, actor_ctx, - identity, rate_limit, } } @@ -128,20 +120,8 @@ impl Debug for FlowControlExecutor { } } -impl Executor for FlowControlExecutor { +impl Execute for FlowControlExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - self.input.schema() - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - self.input.pk_indices() - } - - fn identity(&self) -> &str { - &self.identity - } } diff --git a/src/stream/src/executor/hash_agg.rs b/src/stream/src/executor/hash_agg.rs index 5078fd0bc5154..ccaa4d98ce11d 100644 --- a/src/stream/src/executor/hash_agg.rs +++ b/src/stream/src/executor/hash_agg.rs @@ -39,8 +39,7 @@ use super::aggregation::{ }; use super::sort_buffer::SortBuffer; use super::{ - expect_first_barrier, ActorContextRef, ExecutorInfo, PkIndicesRef, StreamExecutorResult, - Watermark, + expect_first_barrier, ActorContextRef, Executor, ExecutorInfo, StreamExecutorResult, Watermark, }; use crate::cache::{cache_may_stale, new_with_hasher, ManagedLruCache}; use crate::common::metrics::MetricsInfo; @@ -49,7 +48,7 @@ use crate::common::StreamChunkBuilder; use crate::error::StreamResult; use crate::executor::aggregation::AggGroup as GenericAggGroup; use crate::executor::error::StreamExecutorError; -use crate::executor::{BoxedMessageStream, Executor, Message}; +use crate::executor::{BoxedMessageStream, Execute, Message}; use crate::task::AtomicU64Ref; type AggGroup = GenericAggGroup; @@ -74,7 +73,7 @@ type AggGroupCache = ManagedLruCache>, Precompu /// all modifications will be flushed to the storage backend. Meanwhile, the executor will go /// through `group_change_set`, and produce a stream chunk based on the state changes. pub struct HashAggExecutor { - input: Box, + input: Executor, inner: ExecutorInner, } @@ -193,27 +192,15 @@ impl ExecutionStats { } } -impl Executor for HashAggExecutor { +impl Execute for HashAggExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl HashAggExecutor { pub fn new(args: AggExecutorArgs) -> StreamResult { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); let group_key_len = args.extra.group_key_indices.len(); // NOTE: we assume the prefix of table pk is exactly the group key diff --git a/src/stream/src/executor/hash_join.rs b/src/stream/src/executor/hash_join.rs index 471d382a763d0..3727da397d01f 100644 --- a/src/stream/src/executor/hash_join.rs +++ b/src/stream/src/executor/hash_join.rs @@ -22,7 +22,6 @@ use futures_async_stream::try_stream; use itertools::Itertools; use multimap::MultiMap; use risingwave_common::array::{Op, RowRef, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::hash::{HashKey, NullBitmap}; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, DefaultOrd, ToOwnedDatum}; @@ -42,8 +41,7 @@ use super::join::{JoinTypePrimitive, SideTypePrimitive, *}; use super::monitor::StreamingMetrics; use super::watermark::*; use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, Message, - PkIndicesRef, Watermark, + ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message, Watermark, }; use crate::common::table::state_table::StateTable; use crate::executor::expect_first_barrier_from_aligned_stream; @@ -143,9 +141,9 @@ pub struct HashJoinExecutor, + input_l: Option, /// Right input executor - input_r: Option, + input_r: Option, /// The data types of the formed new columns actual_output_data_types: Vec, /// The parameters of the left join executor @@ -191,22 +189,10 @@ impl std::fmt::Debug } } -impl Executor for HashJoinExecutor { +impl Execute for HashJoinExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } struct EqJoinArgs<'a, K: HashKey, S: StateStore> { @@ -227,8 +213,8 @@ impl HashJoinExecutor, @@ -1156,8 +1142,10 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![1]); - let (tx_r, source_r) = MockSource::channel(schema, vec![1]); + let (tx_l, source_l) = MockSource::channel(); + let source_l = source_l.into_executor(schema.clone(), vec![1]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r.into_executor(schema, vec![1]); let params_l = JoinParams::new(vec![0], vec![1]); let params_r = JoinParams::new(vec![0], vec![1]); let cond = with_condition.then(|| create_cond(condition_text)); @@ -1200,8 +1188,8 @@ mod tests { let executor = HashJoinExecutor::::new( ActorContext::for_test(123), info, - Box::new(source_l), - Box::new(source_r), + source_l, + source_r, params_l, params_r, vec![null_safe], @@ -1217,7 +1205,7 @@ mod tests { Arc::new(StreamingMetrics::unused()), 1024, ); - (tx_l, tx_r, Box::new(executor).execute()) + (tx_l, tx_r, executor.boxed().execute()) } async fn create_classical_executor( @@ -1238,8 +1226,10 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (tx_l, source_l) = MockSource::channel(schema.clone(), vec![0]); - let (tx_r, source_r) = MockSource::channel(schema, vec![0]); + let (tx_l, source_l) = MockSource::channel(); + let source_l = source_l.into_executor(schema.clone(), vec![0]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r.into_executor(schema, vec![0]); let params_l = JoinParams::new(vec![0, 1], vec![]); let params_r = JoinParams::new(vec![0, 1], vec![]); let cond = with_condition.then(|| create_cond(None)); @@ -1290,8 +1280,8 @@ mod tests { let executor = HashJoinExecutor::::new( ActorContext::for_test(123), info, - Box::new(source_l), - Box::new(source_r), + source_l, + source_r, params_l, params_r, vec![false], @@ -1307,7 +1297,7 @@ mod tests { Arc::new(StreamingMetrics::unused()), 1024, ); - (tx_l, tx_r, Box::new(executor).execute()) + (tx_l, tx_r, executor.boxed().execute()) } #[tokio::test] diff --git a/src/stream/src/executor/hop_window.rs b/src/stream/src/executor/hop_window.rs index 4bcca4d593072..801f3daa9f53a 100644 --- a/src/stream/src/executor/hop_window.rs +++ b/src/stream/src/executor/hop_window.rs @@ -23,13 +23,12 @@ use risingwave_expr::expr::NonStrictExpression; use risingwave_expr::ExprError; use super::error::StreamExecutorError; -use super::{ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message}; +use super::{ActorContextRef, Execute, Executor, Message}; use crate::common::StreamChunkBuilder; pub struct HopWindowExecutor { _ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, pub time_col_idx: usize, pub window_slide: Interval, pub window_size: Interval, @@ -43,8 +42,7 @@ impl HopWindowExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, time_col_idx: usize, window_slide: Interval, window_size: Interval, @@ -55,7 +53,6 @@ impl HopWindowExecutor { ) -> Self { HopWindowExecutor { _ctx: ctx, - info, input, time_col_idx, window_slide, @@ -68,22 +65,10 @@ impl HopWindowExecutor { } } -impl Executor for HopWindowExecutor { +impl Execute for HopWindowExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl HopWindowExecutor { @@ -253,12 +238,13 @@ mod tests { use risingwave_expr::expr::test_utils::make_hop_window_expression; use risingwave_expr::expr::NonStrictExpression; + use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Executor, ExecutorInfo, StreamChunk}; + use crate::executor::{ActorContext, Execute, StreamChunk}; const CHUNK_SIZE: usize = 256; - fn create_executor(output_indices: Vec) -> Box { + fn create_executor(output_indices: Vec) -> Box { let field1 = Field::unnamed(DataType::Int64); let field2 = Field::unnamed(DataType::Int64); let field3 = Field::with_name(DataType::Timestamp, "created_at"); @@ -278,7 +264,7 @@ mod tests { .replace('^', "2022-02-02T"), ); let input = - MockSource::with_chunks(schema.clone(), pk_indices.clone(), vec![chunk]).boxed(); + MockSource::with_chunks(vec![chunk]).into_executor(schema.clone(), pk_indices.clone()); let window_slide = Interval::from_minutes(15); let window_size = Interval::from_minutes(30); let window_offset = Interval::from_minutes(0); @@ -291,14 +277,8 @@ mod tests { ) .unwrap(); - super::HopWindowExecutor::new( + HopWindowExecutor::new( ActorContext::for_test(123), - ExecutorInfo { - // TODO: the schema is incorrect, but it seems useless here. - schema, - pk_indices, - identity: "HopWindowExecutor".to_string(), - }, input, 2, window_slide, diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index 0759f4bd2d813..691ba0f46f341 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -35,7 +35,7 @@ use crate::executor::receiver::ReceiverExecutor; use crate::executor::test_utils::agg_executor::{ generate_agg_schema, new_boxed_simple_agg_executor, }; -use crate::executor::{Executor, MergeExecutor, ProjectExecutor, StatelessSimpleAggExecutor}; +use crate::executor::{Execute, MergeExecutor, ProjectExecutor, StatelessSimpleAggExecutor}; use crate::task::{LocalBarrierManager, SharedContext}; /// This test creates a merger-dispatcher pair, and run a sum. Each chunk @@ -50,27 +50,25 @@ async fn test_merger_sum_aggr() { let actor_ctx = ActorContext::for_test(0); // `make_actor` build an actor to do local aggregation let make_actor = |input_rx| { - let _schema = Schema { + let input_schema = Schema { fields: vec![Field::unnamed(DataType::Int64)], }; - let input = ReceiverExecutor::for_test(input_rx); + let input = Executor::new( + ExecutorInfo { + schema: input_schema, + pk_indices: PkIndices::new(), + identity: "ReceiverExecutor".to_string(), + }, + ReceiverExecutor::for_test(input_rx).boxed(), + ); let agg_calls = vec![ AggCall::from_pretty("(count:int8)"), AggCall::from_pretty("(sum:int8 $0:int8)"), ]; let schema = generate_agg_schema(&input, &agg_calls, None); // for the local aggregator, we need two states: row count and sum - let aggregator = StatelessSimpleAggExecutor::new( - actor_ctx.clone(), - ExecutorInfo { - schema, - pk_indices: vec![], - identity: format!("StatelessSimpleAggExecutor {:X}", 1), - }, - input.boxed(), - agg_calls, - ) - .unwrap(); + let aggregator = + StatelessSimpleAggExecutor::new(actor_ctx.clone(), input, schema, agg_calls).unwrap(); let (tx, rx) = channel_for_test(); let consumer = SenderConsumer { input: aggregator.boxed(), @@ -108,13 +106,15 @@ async fn test_merger_sum_aggr() { // create a round robin dispatcher, which dispatches messages to the actors let (input, rx) = channel_for_test(); - let schema = Schema { - fields: vec![ - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - ], - }; - let receiver_op = Box::new(ReceiverExecutor::for_test(rx)); + let receiver_op = Executor::new( + ExecutorInfo { + // input schema of local simple agg + schema: Schema::new(vec![Field::unnamed(DataType::Int64)]), + pk_indices: PkIndices::new(), + identity: "ReceiverExecutor".to_string(), + }, + ReceiverExecutor::for_test(rx).boxed(), + ); let dispatcher = DispatchExecutor::new( receiver_op, vec![DispatcherImpl::RoundRobin(RoundRobinDataDispatcher::new( @@ -138,14 +138,25 @@ async fn test_merger_sum_aggr() { handles.push(tokio::spawn(actor.run())); // use a merge operator to collect data from dispatchers before sending them to aggregator - let merger = MergeExecutor::for_test(outputs, schema); + let merger = Executor::new( + ExecutorInfo { + // output schema of local simple agg + schema: Schema::new(vec![ + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int64), + ]), + pk_indices: PkIndices::new(), + identity: "MergeExecutor".to_string(), + }, + MergeExecutor::for_test(outputs).boxed(), + ); // for global aggregator, we need to sum data and sum row count let is_append_only = false; let aggregator = new_boxed_simple_agg_executor( actor_ctx.clone(), MemoryStateStore::new(), - merger.boxed(), + merger, is_append_only, vec![ AggCall::from_pretty("(sum0:int8 $0:int8)"), @@ -160,13 +171,6 @@ async fn test_merger_sum_aggr() { let projection = ProjectExecutor::new( actor_ctx.clone(), - ExecutorInfo { - schema: Schema { - fields: vec![Field::unnamed(DataType::Int64)], - }, - pk_indices: vec![], - identity: format!("ProjectExecutor {:X}", 3), - }, aggregator, vec![ // TODO: use the new streaming_if_null expression here, and add `None` tests @@ -232,7 +236,7 @@ async fn test_merger_sum_aggr() { } struct MockConsumer { - input: BoxedExecutor, + input: Box, data: Arc>>, } @@ -259,7 +263,7 @@ impl StreamConsumer for MockConsumer { /// `SenderConsumer` consumes data from input executor and send it into a channel. pub struct SenderConsumer { - input: BoxedExecutor, + input: Box, channel: BoxedOutput, } diff --git a/src/stream/src/executor/lookup.rs b/src/stream/src/executor/lookup.rs index 87cc163ea3dff..2c1de3170a801 100644 --- a/src/stream/src/executor/lookup.rs +++ b/src/stream/src/executor/lookup.rs @@ -14,11 +14,10 @@ use async_trait::async_trait; use futures::StreamExt; -use risingwave_common::catalog::Schema; use risingwave_common::types::DataType; use risingwave_storage::StateStore; -use crate::executor::{Barrier, BoxedMessageStream, Executor, PkIndicesRef}; +use crate::executor::{Barrier, BoxedMessageStream, Execute}; mod cache; mod sides; @@ -28,7 +27,7 @@ mod impl_; pub use impl_::LookupExecutorParams; -use super::{ActorContextRef, ExecutorInfo}; +use super::{ActorContextRef, Executor}; #[cfg(test)] mod tests; @@ -42,8 +41,6 @@ mod tests; pub struct LookupExecutor { ctx: ActorContextRef, - info: ExecutorInfo, - /// the data types of the produced data chunk inside lookup (before reordering) chunk_data_types: Vec, @@ -54,10 +51,10 @@ pub struct LookupExecutor { stream: StreamJoinSide, /// The executor for arrangement. - arrangement_executor: Option>, + arrangement_executor: Option, /// The executor for stream. - stream_executor: Option>, + stream_executor: Option, /// The last received barrier. last_barrier: Option, @@ -83,20 +80,8 @@ pub struct LookupExecutor { } #[async_trait] -impl Executor for LookupExecutor { +impl Execute for LookupExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/lookup/impl_.rs b/src/stream/src/executor/lookup/impl_.rs index a503a73a0c8a0..db2511e19e7bd 100644 --- a/src/stream/src/executor/lookup/impl_.rs +++ b/src/stream/src/executor/lookup/impl_.rs @@ -46,11 +46,11 @@ pub struct LookupExecutorParams { /// The side for arrangement. Currently, it should be a /// `MaterializeExecutor`. - pub arrangement: Box, + pub arrangement: Executor, /// The side for stream. It can be any stream, but it will generally be a /// `MaterializeExecutor`. - pub stream: Box, + pub stream: Executor, /// Should be the same as [`ColumnDesc`] in the arrangement. /// @@ -198,7 +198,6 @@ impl LookupExecutor { Self { ctx, - info, chunk_data_types, last_barrier: None, stream_executor: Some(stream), diff --git a/src/stream/src/executor/lookup/sides.rs b/src/stream/src/executor/lookup/sides.rs index 26bfdd39ec401..6e64519503a93 100644 --- a/src/stream/src/executor/lookup/sides.rs +++ b/src/stream/src/executor/lookup/sides.rs @@ -211,10 +211,7 @@ pub async fn align_barrier(mut left: BoxedMessageStream, mut right: BoxedMessage /// * Barrier (prev = `[2`], current = `[3`]) /// * `[Msg`] Arrangement (batch) #[try_stream(ok = ArrangeMessage, error = StreamExecutorError)] -pub async fn stream_lookup_arrange_prev_epoch( - stream: Box, - arrangement: Box, -) { +pub async fn stream_lookup_arrange_prev_epoch(stream: Executor, arrangement: Executor) { let mut input = pin!(align_barrier(stream.execute(), arrangement.execute())); let mut arrange_buf = vec![]; let mut stream_side_end = false; @@ -295,10 +292,7 @@ pub async fn stream_lookup_arrange_prev_epoch( /// * `[Do`] lookup `a` in arrangement of epoch `[2`] (current epoch) /// * Barrier (prev = `[2`], current = `[3`]) #[try_stream(ok = ArrangeMessage, error = StreamExecutorError)] -pub async fn stream_lookup_arrange_this_epoch( - stream: Box, - arrangement: Box, -) { +pub async fn stream_lookup_arrange_this_epoch(stream: Executor, arrangement: Executor) { let mut input = pin!(align_barrier(stream.execute(), arrangement.execute())); let mut stream_buf = vec![]; let mut arrange_buf = vec![]; @@ -432,14 +426,16 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (mut tx_l, source_l) = MockSource::channel(schema.clone(), vec![1]); - let (tx_r, source_r) = MockSource::channel(schema, vec![1]); - - let mut stream = stream_lookup_arrange_this_epoch( - Box::new(source_l.stop_on_finish(false)), - Box::new(source_r.stop_on_finish(false)), - ) - .boxed(); + let (mut tx_l, source_l) = MockSource::channel(); + let source_l = source_l + .stop_on_finish(false) + .into_executor(schema.clone(), vec![1]); + let (tx_r, source_r) = MockSource::channel(); + let source_r = source_r + .stop_on_finish(false) + .into_executor(schema, vec![1]); + + let mut stream = stream_lookup_arrange_this_epoch(source_l, source_r).boxed(); // Simulate recovery test drop(tx_r); diff --git a/src/stream/src/executor/lookup/tests.rs b/src/stream/src/executor/lookup/tests.rs index 7d152e8a5f4e7..9b335fcca6428 100644 --- a/src/stream/src/executor/lookup/tests.rs +++ b/src/stream/src/executor/lookup/tests.rs @@ -30,8 +30,8 @@ use crate::executor::lookup::impl_::LookupExecutorParams; use crate::executor::lookup::LookupExecutor; use crate::executor::test_utils::*; use crate::executor::{ - ActorContext, Barrier, BoxedMessageStream, Executor, ExecutorInfo, MaterializeExecutor, - Message, PkIndices, + ActorContext, Barrier, BoxedMessageStream, Execute, Executor, ExecutorInfo, + MaterializeExecutor, Message, PkIndices, }; fn arrangement_col_descs() -> Vec { @@ -68,10 +68,7 @@ fn arrangement_col_arrange_rules_join_key() -> Vec { /// | + | 2337 | 8 | 3 | /// | - | 2333 | 6 | 3 | /// | b | | | 3 -> 4 | -async fn create_arrangement( - table_id: TableId, - memory_state_store: MemoryStateStore, -) -> Box { +async fn create_arrangement(table_id: TableId, memory_state_store: MemoryStateStore) -> Executor { // Two columns of int32 type, the second column is arrange key. let columns = arrangement_col_descs(); @@ -101,30 +98,32 @@ async fn create_arrangement( .collect_vec(), ); - let source = MockSource::with_messages( - schema, - vec![0], - vec![ - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, vec![0]); - Box::new( + Executor::new( + ExecutorInfo { + schema: source.schema().clone(), + pk_indices: source.pk_indices().to_vec(), + identity: "MaterializeExecutor".to_string(), + }, MaterializeExecutor::for_test( - Box::new(source), + source, memory_state_store, table_id, arrangement_col_arrange_rules(), column_ids, - 1, Arc::new(AtomicU64::new(0)), ConflictBehavior::NoCheck, ) - .await, + .await + .boxed(), ) } @@ -139,7 +138,7 @@ async fn create_arrangement( /// | b | | | 2 -> 3 | /// | - | 6 | 1 | 3 | /// | b | | | 3 -> 4 | -fn create_source() -> Box { +fn create_source() -> Executor { let columns = vec![ ColumnDesc::new_atomic(DataType::Int64, "join_column", 1), ColumnDesc::new_atomic(DataType::Int64, "rowid_column", 2), @@ -163,19 +162,14 @@ fn create_source() -> Box { .collect_vec(), ); - let source = MockSource::with_messages( - schema, - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); - - Box::new(source) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, PkIndices::new()) } async fn next_msg(buffer: &mut Vec, executor: &mut BoxedMessageStream) { diff --git a/src/stream/src/executor/lookup_union.rs b/src/stream/src/executor/lookup_union.rs index c52d562277459..2ffddbda45b46 100644 --- a/src/stream/src/executor/lookup_union.rs +++ b/src/stream/src/executor/lookup_union.rs @@ -18,35 +18,28 @@ use futures::future::{join_all, select, Either}; use futures::{FutureExt, SinkExt, StreamExt}; use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; -use super::*; -use crate::executor::{BoxedMessageStream, ExecutorInfo}; +use super::{Barrier, BoxedMessageStream, Execute, Executor, Message}; /// Merges data from multiple inputs with order. If `order = [2, 1, 0]`, then /// it will first pipe data from the third input; after the third input gets a barrier, it will then /// pipe the second, and finally the first. In the future we could have more efficient /// implementation. pub struct LookupUnionExecutor { - info: ExecutorInfo, - inputs: Vec, + inputs: Vec, order: Vec, } impl std::fmt::Debug for LookupUnionExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("LookupUnionExecutor") - .field("schema", &self.info.schema) - .field("pk_indices", &self.info.pk_indices) - .finish() + f.debug_struct("LookupUnionExecutor").finish() } } impl LookupUnionExecutor { - pub fn new(info: ExecutorInfo, inputs: Vec, order: Vec) -> Self { + pub fn new(inputs: Vec, order: Vec) -> Self { Self { - info, inputs, order: order.iter().map(|x| *x as _).collect(), } @@ -54,22 +47,10 @@ impl LookupUnionExecutor { } #[async_trait] -impl Executor for LookupUnionExecutor { +impl Execute for LookupUnionExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl LookupUnionExecutor { @@ -154,52 +135,36 @@ mod tests { let schema = Schema { fields: vec![Field::unnamed(DataType::Int64)], }; - let source0 = MockSource::with_messages( - schema.clone(), - vec![0], - vec![ - Message::Chunk(StreamChunk::from_pretty("I\n + 1")), - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(StreamChunk::from_pretty("I\n + 2")), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(StreamChunk::from_pretty("I\n + 3")), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ) - .stop_on_finish(false); - let source1 = MockSource::with_messages( - schema.clone(), - vec![0], - vec![ - Message::Chunk(StreamChunk::from_pretty("I\n + 11")), - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(StreamChunk::from_pretty("I\n + 12")), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - ) - .stop_on_finish(false); - let source2 = MockSource::with_messages( - schema, - vec![0], - vec![ - Message::Chunk(StreamChunk::from_pretty("I\n + 21")), - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(StreamChunk::from_pretty("I\n + 22")), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - ) - .stop_on_finish(false); - - let executor = Box::new(LookupUnionExecutor::new( - ExecutorInfo { - schema: source0.schema().clone(), - pk_indices: vec![0], - identity: "LookupUnionExecutor".to_string(), - }, - vec![Box::new(source0), Box::new(source1), Box::new(source2)], - vec![2, 1, 0], - )) - .execute(); + let source0 = MockSource::with_messages(vec![ + Message::Chunk(StreamChunk::from_pretty("I\n + 1")), + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(StreamChunk::from_pretty("I\n + 2")), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(StreamChunk::from_pretty("I\n + 3")), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .stop_on_finish(false) + .into_executor(schema.clone(), vec![0]); + let source1 = MockSource::with_messages(vec![ + Message::Chunk(StreamChunk::from_pretty("I\n + 11")), + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(StreamChunk::from_pretty("I\n + 12")), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .stop_on_finish(false) + .into_executor(schema.clone(), vec![0]); + let source2 = MockSource::with_messages(vec![ + Message::Chunk(StreamChunk::from_pretty("I\n + 21")), + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(StreamChunk::from_pretty("I\n + 22")), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .stop_on_finish(false) + .into_executor(schema, vec![0]); + + let executor = LookupUnionExecutor::new(vec![source0, source1, source2], vec![2, 1, 0]) + .boxed() + .execute(); let outputs: Vec<_> = executor.try_collect().await.unwrap(); assert_eq!( diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs index 145a937fcdb2e..e5b88dd274e1e 100644 --- a/src/stream/src/executor/merge.rs +++ b/src/stream/src/executor/merge.rs @@ -20,7 +20,6 @@ use anyhow::Context as _; use futures::stream::{FusedStream, FuturesUnordered, StreamFuture}; use futures::{pin_mut, Stream, StreamExt}; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use tokio::time::Instant; use super::error::StreamExecutorError; @@ -38,9 +37,6 @@ pub struct MergeExecutor { /// The context of the actor. actor_context: ActorContextRef, - /// Logical Operator Info - info: ExecutorInfo, - /// Upstream channels. upstreams: Vec, @@ -61,7 +57,6 @@ impl MergeExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, fragment_id: FragmentId, upstream_fragment_id: FragmentId, inputs: Vec, @@ -71,7 +66,6 @@ impl MergeExecutor { ) -> Self { Self { actor_context: ctx, - info, upstreams: inputs, fragment_id, upstream_fragment_id, @@ -81,17 +75,12 @@ impl MergeExecutor { } #[cfg(test)] - pub fn for_test(inputs: Vec, schema: Schema) -> Self { + pub fn for_test(inputs: Vec) -> Self { use super::exchange::input::LocalInput; use crate::executor::exchange::input::Input; Self::new( ActorContext::for_test(114), - ExecutorInfo { - schema, - pk_indices: vec![], - identity: "MergeExecutor".to_string(), - }, 514, 1919, inputs @@ -245,22 +234,10 @@ impl MergeExecutor { } } -impl Executor for MergeExecutor { +impl Execute for MergeExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } /// A stream for merging messages from multiple upstreams. @@ -464,7 +441,7 @@ mod tests { use super::*; use crate::executor::exchange::input::RemoteInput; use crate::executor::exchange::permit::channel_for_test; - use crate::executor::{Barrier, Executor, Mutation}; + use crate::executor::{Barrier, Execute, Mutation}; use crate::task::test_utils::helper_make_local_actor; fn build_test_chunk(epoch: u64) -> StreamChunk { @@ -483,7 +460,7 @@ mod tests { txs.push(tx); rxs.push(rx); } - let merger = MergeExecutor::for_test(rxs, Schema::default()); + let merger = MergeExecutor::for_test(rxs); let mut handles = Vec::with_capacity(CHANNEL_NUMBER); let epochs = (10..1000u64).step_by(10).collect_vec(); @@ -556,8 +533,6 @@ mod tests { #[tokio::test] async fn test_configuration_change() { - let schema = Schema { fields: vec![] }; - let actor_id = 233; let (untouched, old, new) = (234, 235, 238); // upstream actors let ctx = Arc::new(SharedContext::for_test()); @@ -592,13 +567,8 @@ mod tests { .try_collect() .unwrap(); - let merge = MergeExecutor::new( + let mut merge = MergeExecutor::new( ActorContext::for_test(actor_id), - ExecutorInfo { - schema, - pk_indices: vec![], - identity: "MergeExecutor".to_string(), - }, fragment_id, upstream_fragment_id, inputs, @@ -609,8 +579,6 @@ mod tests { .boxed() .execute(); - pin_mut!(merge); - // 2. Take downstream receivers. let txs = [untouched, old, new] .into_iter() diff --git a/src/stream/src/executor/mod.rs b/src/stream/src/executor/mod.rs index 40ae252c03cf0..1e82768096f36 100644 --- a/src/stream/src/executor/mod.rs +++ b/src/stream/src/executor/mod.rs @@ -20,7 +20,6 @@ use await_tree::InstrumentAwait; use enum_as_inner::EnumAsInner; use futures::stream::BoxStream; use futures::{Stream, StreamExt}; -use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::StreamChunk; use risingwave_common::buffer::Bitmap; @@ -154,7 +153,6 @@ pub use wrapper::WrapperExecutor; use self::barrier_align::AlignedMessageStream; -pub type BoxedExecutor = Box; pub type MessageStreamItem = StreamExecutorResult; pub type BoxedMessageStream = BoxStream<'static, MessageStreamItem>; @@ -166,48 +164,27 @@ pub trait MessageStream = futures::Stream + Send; /// Static information of an executor. #[derive(Debug, Default, Clone)] pub struct ExecutorInfo { - /// See [`Executor::schema`]. + /// The schema of the OUTPUT of the executor. pub schema: Schema, - /// See [`Executor::pk_indices`]. + /// The primary key indices of the OUTPUT of the executor. + /// Schema is used by both OLAP and streaming, therefore + /// pk indices are maintained independently. pub pk_indices: PkIndices, - /// See [`Executor::identity`]. + /// Identity of the executor. pub identity: String, } -/// `Executor` supports handling of control messages. -pub trait Executor: Send + 'static { +/// [`Execute`] describes the methods an executor should implement to handle control messages. +pub trait Execute: Send + 'static { fn execute(self: Box) -> BoxedMessageStream; - /// Return the schema of the OUTPUT of the executor. - fn schema(&self) -> &Schema; - - /// Return the primary key indices of the OUTPUT of the executor. - /// Schema is used by both OLAP and streaming, therefore - /// pk indices are maintained independently. - fn pk_indices(&self) -> PkIndicesRef<'_>; - - /// Identity of the executor. - fn identity(&self) -> &str; - fn execute_with_epoch(self: Box, _epoch: u64) -> BoxedMessageStream { self.execute() } - #[inline(always)] - fn info(&self) -> ExecutorInfo { - let schema = self.schema().to_owned(); - let pk_indices = self.pk_indices().to_owned(); - let identity = self.identity().to_owned(); - ExecutorInfo { - schema, - pk_indices, - identity, - } - } - - fn boxed(self) -> BoxedExecutor + fn boxed(self) -> Box where Self: Sized + Send + 'static, { @@ -215,12 +192,64 @@ pub trait Executor: Send + 'static { } } -impl std::fmt::Debug for BoxedExecutor { +/// [`Executor`] combines the static information ([`ExecutorInfo`]) and the executable object to +/// handle messages ([`Execute`]). +pub struct Executor { + info: ExecutorInfo, + execute: Box, +} + +impl Executor { + pub fn new(info: ExecutorInfo, execute: Box) -> Self { + Self { info, execute } + } + + pub fn info(&self) -> &ExecutorInfo { + &self.info + } + + pub fn schema(&self) -> &Schema { + &self.info.schema + } + + pub fn pk_indices(&self) -> PkIndicesRef<'_> { + &self.info.pk_indices + } + + pub fn identity(&self) -> &str { + &self.info.identity + } + + pub fn execute(self) -> BoxedMessageStream { + self.execute.execute() + } + + pub fn execute_with_epoch(self, epoch: u64) -> BoxedMessageStream { + self.execute.execute_with_epoch(epoch) + } +} + +impl std::fmt::Debug for Executor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.write_str(self.identity()) } } +impl From<(ExecutorInfo, Box)> for Executor { + fn from((info, execute): (ExecutorInfo, Box)) -> Self { + Self::new(info, execute) + } +} + +impl From<(ExecutorInfo, E)> for Executor +where + E: Execute, +{ + fn from((info, execute): (ExecutorInfo, E)) -> Self { + Self::new(info, execute.boxed()) + } +} + pub const INVALID_EPOCH: u64 = 0; type UpstreamFragmentId = FragmentId; diff --git a/src/stream/src/executor/mview/materialize.rs b/src/stream/src/executor/mview/materialize.rs index 69e238a476f74..c96f875765e8e 100644 --- a/src/stream/src/executor/mview/materialize.rs +++ b/src/stream/src/executor/mview/materialize.rs @@ -42,16 +42,16 @@ use crate::common::table::state_table::StateTableInner; use crate::executor::error::StreamExecutorError; use crate::executor::monitor::StreamingMetrics; use crate::executor::{ - expect_first_barrier, ActorContext, ActorContextRef, AddMutation, BoxedExecutor, - BoxedMessageStream, Executor, ExecutorInfo, Message, Mutation, PkIndicesRef, - StreamExecutorResult, UpdateMutation, + expect_first_barrier, ActorContext, ActorContextRef, AddMutation, BoxedMessageStream, Execute, + Executor, Message, Mutation, StreamExecutorResult, UpdateMutation, }; use crate::task::{ActorId, AtomicU64Ref}; /// `MaterializeExecutor` materializes changes in stream into a materialized view on storage. pub struct MaterializeExecutor { - input: BoxedExecutor, - info: ExecutorInfo, + input: Executor, + + schema: Schema, state_table: StateTableInner, @@ -71,8 +71,8 @@ impl MaterializeExecutor { /// should be `None`. #[allow(clippy::too_many_arguments)] pub async fn new( - input: BoxedExecutor, - info: ExecutorInfo, + input: Executor, + schema: Schema, store: S, arrange_key: Vec, actor_context: ActorContextRef, @@ -99,7 +99,7 @@ impl MaterializeExecutor { Self { input, - info, + schema, state_table, arrange_key_indices, actor_context, @@ -115,7 +115,7 @@ impl MaterializeExecutor { let actor_id_str = self.actor_context.id.to_string(); let fragment_id_str = self.actor_context.fragment_id.to_string(); - let data_types = self.schema().data_types().clone(); + let data_types = self.schema.data_types(); let mut input = self.input.execute(); let barrier = expect_first_barrier(&mut input).await?; @@ -264,12 +264,11 @@ impl MaterializeExecutor { /// Create a new `MaterializeExecutor` without distribution info for test purpose. #[allow(clippy::too_many_arguments)] pub async fn for_test( - input: BoxedExecutor, + input: Executor, store: S, table_id: TableId, keys: Vec, column_ids: Vec, - executor_id: u64, watermark_epoch: AtomicU64Ref, conflict_behavior: ConflictBehavior, ) -> Self { @@ -293,14 +292,10 @@ impl MaterializeExecutor { Self { input, + schema, state_table, arrange_key_indices: arrange_columns.clone(), actor_context: ActorContext::for_test(0), - info: ExecutorInfo { - schema, - pk_indices: arrange_columns, - identity: format!("MaterializeExecutor {:X}", executor_id), - }, materialize_cache: MaterializeCache::new(watermark_epoch, MetricsInfo::for_test()), conflict_behavior, } @@ -429,32 +424,15 @@ impl MaterializeBuffer { self.buffer } } -impl Executor for MaterializeExecutor { +impl Execute for MaterializeExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } impl std::fmt::Debug for MaterializeExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { f.debug_struct("MaterializeExecutor") - .field("info", &self.info()) .field("arrange_key_indices", &self.arrange_key_indices) .finish() } @@ -676,17 +654,14 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -703,19 +678,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::NoCheck, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::NoCheck, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -784,17 +757,14 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -811,19 +781,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::Overwrite, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::Overwrite, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -880,18 +848,15 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -908,19 +873,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::Overwrite, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::Overwrite, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1012,19 +975,16 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1041,19 +1001,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::Overwrite, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::Overwrite, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1196,18 +1154,15 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1224,19 +1179,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::IgnoreConflict, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1307,15 +1260,12 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1332,19 +1282,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::IgnoreConflict, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, ) + .await + .boxed() .execute(); let _msg1 = materialize_executor .next() @@ -1426,19 +1374,16 @@ mod tests { ); // Prepare stream executors. - let source = MockSource::with_messages( - schema.clone(), - PkIndices::new(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(chunk1), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(chunk2), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(chunk3), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(chunk1), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(chunk2), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(chunk3), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema.clone(), PkIndices::new()); let order_types = vec![OrderType::ascending()]; let column_descs = vec![ @@ -1455,19 +1400,17 @@ mod tests { vec![0, 1], ); - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store, - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - ConflictBehavior::IgnoreConflict, - ) - .await, + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store, + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + ConflictBehavior::IgnoreConflict, ) + .await + .boxed() .execute(); materialize_executor.next().await.transpose().unwrap(); @@ -1635,21 +1578,20 @@ mod tests { .chain(iter::once(Message::Barrier(Barrier::new_test_barrier(2)))) .collect(); // Prepare stream executors. - let source = MockSource::with_messages(schema.clone(), PkIndices::new(), messages); - - let mut materialize_executor = Box::new( - MaterializeExecutor::for_test( - Box::new(source), - memory_state_store.clone(), - table_id, - vec![ColumnOrder::new(0, OrderType::ascending())], - column_ids, - 1, - Arc::new(AtomicU64::new(0)), - conflict_behavior, - ) - .await, + let source = + MockSource::with_messages(messages).into_executor(schema.clone(), PkIndices::new()); + + let mut materialize_executor = MaterializeExecutor::for_test( + source, + memory_state_store.clone(), + table_id, + vec![ColumnOrder::new(0, OrderType::ascending())], + column_ids, + Arc::new(AtomicU64::new(0)), + conflict_behavior, ) + .await + .boxed() .execute(); materialize_executor.expect_barrier().await; diff --git a/src/stream/src/executor/no_op.rs b/src/stream/src/executor/no_op.rs index f116eced8864d..ac12bf99d5d7f 100644 --- a/src/stream/src/executor/no_op.rs +++ b/src/stream/src/executor/no_op.rs @@ -12,44 +12,23 @@ // See the License for the specific language governing permissions and // limitations under the License. -use risingwave_common::catalog::Schema; - -use super::{ - ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef, -}; +use super::{ActorContextRef, BoxedMessageStream, Execute, Executor}; /// No-op executor directly forwards the input stream. Currently used to break the multiple edges in /// the fragment graph. pub struct NoOpExecutor { _ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, } impl NoOpExecutor { - pub fn new(ctx: ActorContextRef, info: ExecutorInfo, input: BoxedExecutor) -> Self { - Self { - _ctx: ctx, - info, - input, - } + pub fn new(ctx: ActorContextRef, input: Executor) -> Self { + Self { _ctx: ctx, input } } } -impl Executor for NoOpExecutor { +impl Execute for NoOpExecutor { fn execute(self: Box) -> BoxedMessageStream { self.input.execute() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/now.rs b/src/stream/src/executor/now.rs index 9d00646a9bcc1..755e48f325965 100644 --- a/src/stream/src/executor/now.rs +++ b/src/stream/src/executor/now.rs @@ -18,7 +18,6 @@ use std::ops::Bound::Unbounded; use futures::{pin_mut, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::row::{self, OwnedRow}; use risingwave_common::types::{DataType, Datum}; use risingwave_storage::StateStore; @@ -26,13 +25,12 @@ use tokio::sync::mpsc::UnboundedReceiver; use tokio_stream::wrappers::UnboundedReceiverStream; use super::{ - Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, Mutation, PkIndicesRef, - StreamExecutorError, Watermark, + Barrier, BoxedMessageStream, Execute, Message, Mutation, StreamExecutorError, Watermark, }; use crate::common::table::state_table::StateTable; pub struct NowExecutor { - info: ExecutorInfo, + data_types: Vec, /// Receiver of barrier channel. barrier_receiver: UnboundedReceiver, @@ -42,24 +40,23 @@ pub struct NowExecutor { impl NowExecutor { pub fn new( - info: ExecutorInfo, + data_types: Vec, barrier_receiver: UnboundedReceiver, state_table: StateTable, ) -> Self { Self { - info, + data_types, barrier_receiver, state_table, } } #[try_stream(ok = Message, error = StreamExecutorError)] - async fn into_stream(self) { + async fn execute_inner(self) { let Self { + data_types, barrier_receiver, mut state_table, - info, - .. } = self; // Whether the executor is paused. @@ -131,15 +128,12 @@ impl NowExecutor { let row = row::once(×tamp); state_table.update(last_row, row); - StreamChunk::from_rows( - &[(Op::Delete, last_row), (Op::Insert, row)], - &info.schema.data_types(), - ) + StreamChunk::from_rows(&[(Op::Delete, last_row), (Op::Insert, row)], &data_types) } else { let row = row::once(×tamp); state_table.insert(row); - StreamChunk::from_rows(&[(Op::Insert, row)], &info.schema.data_types()) + StreamChunk::from_rows(&[(Op::Insert, row)], &data_types) }; yield Message::Chunk(stream_chunk); @@ -155,28 +149,16 @@ impl NowExecutor { } } -impl Executor for NowExecutor { +impl Execute for NowExecutor { fn execute(self: Box) -> BoxedMessageStream { - self.into_stream().boxed() - } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity + self.execute_inner().boxed() } } #[cfg(test)] mod tests { use risingwave_common::array::StreamChunk; - use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema, TableId}; + use risingwave_common::catalog::{ColumnDesc, ColumnId, TableId}; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_storage::memory::MemoryStateStore; @@ -186,8 +168,7 @@ mod tests { use crate::common::table::state_table::StateTable; use crate::executor::test_utils::StreamExecutorTestExt; use crate::executor::{ - Barrier, BoxedMessageStream, Executor, ExecutorInfo, Mutation, StreamExecutorResult, - Watermark, + Barrier, BoxedMessageStream, Execute, Mutation, StreamExecutorResult, Watermark, }; #[tokio::test] @@ -410,22 +391,8 @@ mod tests { let (sender, barrier_receiver) = unbounded_channel(); - let schema = Schema::new(vec![Field { - data_type: DataType::Timestamptz, - name: String::from("now"), - sub_fields: vec![], - type_name: String::default(), - }]); - - let now_executor = NowExecutor::new( - ExecutorInfo { - schema, - pk_indices: vec![], - identity: "NowExecutor".to_string(), - }, - barrier_receiver, - state_table, - ); - (sender, Box::new(now_executor).execute()) + let now_executor = + NowExecutor::new(vec![DataType::Timestamptz], barrier_receiver, state_table); + (sender, now_executor.boxed().execute()) } } diff --git a/src/stream/src/executor/over_window/eowc.rs b/src/stream/src/executor/over_window/eowc.rs index 601a2f536aa00..32f1fafe9b8db 100644 --- a/src/stream/src/executor/over_window/eowc.rs +++ b/src/stream/src/executor/over_window/eowc.rs @@ -39,8 +39,8 @@ use crate::cache::{new_unbounded, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::task::AtomicU64Ref; @@ -95,14 +95,14 @@ type PartitionCache = ManagedLruCache; // TODO(rc): us /// - `WindowState` should output agg result for `curr output row`. /// - Recover: iterate through state table, push rows to `WindowState`, ignore ready windows. pub struct EowcOverWindowExecutor { - input: Box, + input: Executor, inner: ExecutorInner, } struct ExecutorInner { actor_ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, calls: Vec, input_pk_indices: Vec, partition_key_indices: Vec, @@ -117,30 +117,18 @@ struct ExecutionVars { _phantom: PhantomData, } -impl Executor for EowcOverWindowExecutor { +impl Execute for EowcOverWindowExecutor { fn execute(self: Box) -> BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } pub struct EowcOverWindowExecutorArgs { pub actor_ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, + pub schema: Schema, pub calls: Vec, pub partition_key_indices: Vec, pub order_key_index: usize, @@ -150,13 +138,13 @@ pub struct EowcOverWindowExecutorArgs { impl EowcOverWindowExecutor { pub fn new(args: EowcOverWindowExecutorArgs) -> Self { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); Self { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: args.info, + schema: args.schema, calls: args.calls, input_pk_indices: input_info.pk_indices, partition_key_indices: args.partition_key_indices, @@ -237,7 +225,7 @@ impl EowcOverWindowExecutor { vars: &mut ExecutionVars, chunk: StreamChunk, ) -> StreamExecutorResult> { - let mut builders = this.info.schema.create_array_builders(chunk.capacity()); // just an estimate + let mut builders = this.schema.create_array_builders(chunk.capacity()); // just an estimate // We assume that the input is sorted by order key. for record in chunk.records() { @@ -318,7 +306,7 @@ impl EowcOverWindowExecutor { for key in keys_to_evict { let order_key = memcmp_encoding::decode_row( &key.order_key, - &[this.info.schema[this.order_key_index].data_type()], + &[this.schema[this.order_key_index].data_type()], &[OrderType::ascending()], )?; let state_row_pk = (&partition_key).chain(order_key).chain(key.pk); diff --git a/src/stream/src/executor/over_window/general.rs b/src/stream/src/executor/over_window/general.rs index df4e3ac5c84c2..a7245c57f368c 100644 --- a/src/stream/src/executor/over_window/general.rs +++ b/src/stream/src/executor/over_window/general.rs @@ -23,6 +23,7 @@ use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::stream_record::Record; use risingwave_common::array::{Op, RowRef, StreamChunk}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::session_config::OverWindowCachePolicy as CachePolicy; use risingwave_common::types::{DataType, DefaultOrdered}; @@ -46,8 +47,8 @@ use crate::common::StreamChunkBuilder; use crate::executor::monitor::StreamingMetrics; use crate::executor::over_window::over_partition::AffectedRange; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, - StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, Execute, Executor, Message, StreamExecutorError, + StreamExecutorResult, }; use crate::task::AtomicU64Ref; @@ -57,14 +58,14 @@ use crate::task::AtomicU64Ref; /// - State table schema = output schema, state table pk = `partition key | order key | input pk`. /// - Output schema = input schema + window function results. pub struct OverWindowExecutor { - input: Box, + input: Executor, inner: ExecutorInner, } struct ExecutorInner { actor_ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, calls: Vec, partition_key_indices: Vec, order_key_indices: Vec, @@ -98,22 +99,10 @@ struct ExecutionStats { cache_lookup: u64, } -impl Executor for OverWindowExecutor { +impl Execute for OverWindowExecutor { fn execute(self: Box) -> crate::executor::BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> crate::executor::PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl ExecutorInner { @@ -145,10 +134,10 @@ impl ExecutorInner { pub struct OverWindowExecutorArgs { pub actor_ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, + pub schema: Schema, pub calls: Vec, pub partition_key_indices: Vec, pub order_key_indices: Vec, @@ -164,7 +153,7 @@ pub struct OverWindowExecutorArgs { impl OverWindowExecutor { pub fn new(args: OverWindowExecutorArgs) -> Self { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); let input_schema = &input_info.schema; let has_unbounded_frame = args @@ -195,7 +184,7 @@ impl OverWindowExecutor { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: args.info, + schema: args.schema, calls: args.calls, partition_key_indices: args.partition_key_indices, order_key_indices: args.order_key_indices, @@ -328,8 +317,7 @@ impl OverWindowExecutor { // `input pk` => `Record` let mut key_change_update_buffer: BTreeMap, Record> = BTreeMap::new(); - let mut chunk_builder = - StreamChunkBuilder::new(this.chunk_size, this.info.schema.data_types()); + let mut chunk_builder = StreamChunkBuilder::new(this.chunk_size, this.schema.data_types()); // Prepare things needed by metrics. let actor_id = this.actor_ctx.id.to_string(); diff --git a/src/stream/src/executor/project.rs b/src/stream/src/executor/project.rs index efc381560e75d..c44d93e22639a 100644 --- a/src/stream/src/executor/project.rs +++ b/src/stream/src/executor/project.rs @@ -14,27 +14,30 @@ use std::fmt::{Debug, Formatter}; +use futures::StreamExt; +use futures_async_stream::try_stream; use multimap::MultiMap; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; -use risingwave_common::types::ToOwnedDatum; +use risingwave_common::types::{ScalarImpl, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; -use super::*; +use super::{ + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, StreamExecutorError, + StreamExecutorResult, Watermark, +}; /// `ProjectExecutor` project data with the `expr`. The `expr` takes a chunk of data, /// and returns a new data chunk. And then, `ProjectExecutor` will insert, delete /// or update element into next operator according to the result of the expression. pub struct ProjectExecutor { - input: BoxedExecutor, + input: Executor, inner: Inner, } struct Inner { _ctx: ActorContextRef, - info: ExecutorInfo, /// Expressions of the current projection. exprs: Vec, @@ -55,8 +58,7 @@ impl ProjectExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: Box, + input: Executor, exprs: Vec, watermark_derivations: MultiMap, nondecreasing_expr_indices: Vec, @@ -67,7 +69,6 @@ impl ProjectExecutor { input, inner: Inner { _ctx: ctx, - info, exprs, watermark_derivations, nondecreasing_expr_indices, @@ -86,19 +87,7 @@ impl Debug for ProjectExecutor { } } -impl Executor for ProjectExecutor { - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } - +impl Execute for ProjectExecutor { fn execute(self: Box) -> BoxedMessageStream { self.inner.execute(self.input).boxed() } @@ -142,8 +131,8 @@ impl Inner { ret.push(derived_watermark); } else { warn!( - "{} derive a NULL watermark with the expression {}!", - self.info.identity, out_col_idx + "a NULL watermark is derived with the expression {}!", + out_col_idx ); } } @@ -151,7 +140,7 @@ impl Inner { } #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute(mut self, input: BoxedExecutor) { + async fn execute(mut self, input: Executor) { #[for_await] for msg in input.execute() { let msg = msg?; @@ -241,28 +230,20 @@ mod tests { ], }; let pk_indices = vec![0]; - let (mut tx, source) = MockSource::channel(schema, pk_indices); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, pk_indices); let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)"); - let info = ExecutorInfo { - schema: Schema { - fields: vec![Field::unnamed(DataType::Int64)], - }, - pk_indices: vec![], - identity: "ProjectExecutor".to_string(), - }; - - let project = Box::new(ProjectExecutor::new( + let project = ProjectExecutor::new( ActorContext::for_test(123), - info, - Box::new(source), + source, vec![test_expr], MultiMap::new(), vec![], 0.0, - )); - let mut project = project.execute(); + ); + let mut project = project.boxed().execute(); tx.push_barrier(1, false); let barrier = project.next().await.unwrap().unwrap(); @@ -329,34 +310,22 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (mut tx, source) = MockSource::channel(schema, PkIndices::new()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); let a_expr = build_from_pretty("(add:int8 $0:int8 1:int8)"); let b_expr = build_from_pretty("(subtract:int8 $0:int8 1:int8)"); let c_expr = NonStrictExpression::for_test(DummyNondecreasingExpr); - let info = ExecutorInfo { - schema: Schema { - fields: vec![ - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - ], - }, - pk_indices: vec![], - identity: "ProjectExecutor".to_string(), - }; - - let project = Box::new(ProjectExecutor::new( + let project = ProjectExecutor::new( ActorContext::for_test(123), - info, - Box::new(source), + source, vec![a_expr, b_expr, c_expr], MultiMap::from_iter(vec![(0, 0), (0, 1)].into_iter()), vec![2], 0.0, - )); - let mut project = project.execute(); + ); + let mut project = project.boxed().execute(); tx.push_barrier(1, false); tx.push_int64_watermark(0, 100); diff --git a/src/stream/src/executor/project_set.rs b/src/stream/src/executor/project_set.rs index 5167a71db2cb8..9fadb5949dac2 100644 --- a/src/stream/src/executor/project_set.rs +++ b/src/stream/src/executor/project_set.rs @@ -20,7 +20,6 @@ use futures_async_stream::try_stream; use multimap::MultiMap; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bail; -use risingwave_common::catalog::Schema; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::{DataType, Datum, DatumRef, ToOwnedDatum}; use risingwave_common::util::iter_util::ZipEqFast; @@ -28,10 +27,7 @@ use risingwave_expr::expr::{LogReport, NonStrictExpression}; use risingwave_expr::table_function::ProjectSetSelectItem; use super::error::StreamExecutorError; -use super::{ - ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, PkIndicesRef, - StreamExecutorResult, Watermark, -}; +use super::{ActorContextRef, Execute, Executor, Message, StreamExecutorResult, Watermark}; use crate::common::StreamChunkBuilder; const PROJ_ROW_ID_OFFSET: usize = 1; @@ -40,13 +36,12 @@ const PROJ_ROW_ID_OFFSET: usize = 1; /// and returns a new data chunk. And then, `ProjectSetExecutor` will insert, delete /// or update element into next operator according to the result of the expression. pub struct ProjectSetExecutor { - input: BoxedExecutor, + input: Executor, inner: Inner, } struct Inner { _ctx: ActorContextRef, - info: ExecutorInfo, /// Expressions of the current project_section. select_list: Vec, @@ -62,8 +57,7 @@ impl ProjectSetExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: Box, + input: Executor, select_list: Vec, chunk_size: usize, watermark_derivations: MultiMap, @@ -71,7 +65,6 @@ impl ProjectSetExecutor { ) -> Self { let inner = Inner { _ctx: ctx, - info, select_list, chunk_size, watermark_derivations, @@ -90,27 +83,15 @@ impl Debug for ProjectSetExecutor { } } -impl Executor for ProjectSetExecutor { +impl Execute for ProjectSetExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.inner.execute(self.input).boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl Inner { #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute(self, input: BoxedExecutor) { + async fn execute(self, input: Executor) { assert!(!self.select_list.is_empty()); // First column will be `projected_row_id`, which represents the index in the // output table @@ -270,8 +251,8 @@ impl Inner { ret.push(derived_watermark); } else { warn!( - "{} derive a NULL watermark with the expression {}!", - self.info.identity, expr_idx + "a NULL watermark is derived with the expression {}!", + expr_idx ); } } diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs index 1bb83009a203e..0b51bab665d41 100644 --- a/src/stream/src/executor/rearranged_chain.rs +++ b/src/stream/src/executor/rearranged_chain.rs @@ -19,12 +19,9 @@ use futures::stream::select_with_strategy; use futures::{stream, StreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::Schema; use super::error::StreamExecutorError; -use super::{ - expect_first_barrier, Barrier, BoxedExecutor, Executor, ExecutorInfo, Message, MessageStream, -}; +use super::{expect_first_barrier, Barrier, Execute, Executor, Message, MessageStream}; use crate::task::{ActorId, CreateMviewProgress}; /// `ChainExecutor` is an executor that enables synchronization between the existing stream and @@ -35,11 +32,9 @@ use crate::task::{ActorId, CreateMviewProgress}; /// [`RearrangedChainExecutor`] resolves the latency problem when creating MV with a huge amount of /// existing data, by rearranging the barrier from the upstream. Check the design doc for details. pub struct RearrangedChainExecutor { - info: ExecutorInfo, + snapshot: Executor, - snapshot: BoxedExecutor, - - upstream: BoxedExecutor, + upstream: Executor, progress: CreateMviewProgress, @@ -84,14 +79,8 @@ impl RearrangedMessage { } impl RearrangedChainExecutor { - pub fn new( - info: ExecutorInfo, - snapshot: BoxedExecutor, - upstream: BoxedExecutor, - progress: CreateMviewProgress, - ) -> Self { + pub fn new(snapshot: Executor, upstream: Executor, progress: CreateMviewProgress) -> Self { Self { - info, snapshot, upstream, actor_id: progress.actor_id(), @@ -288,26 +277,10 @@ impl RearrangedChainExecutor { } } -impl Executor for RearrangedChainExecutor { +impl Execute for RearrangedChainExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } // TODO: add new unit tests for rearranged chain diff --git a/src/stream/src/executor/receiver.rs b/src/stream/src/executor/receiver.rs index 273a3fcf1b339..62f2dd694f910 100644 --- a/src/stream/src/executor/receiver.rs +++ b/src/stream/src/executor/receiver.rs @@ -17,7 +17,6 @@ use anyhow::Context; use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; -use risingwave_common::catalog::Schema; use tokio::time::Instant; use super::exchange::input::BoxedInput; @@ -25,17 +24,12 @@ use super::ActorContextRef; use crate::executor::exchange::input::new_input; use crate::executor::monitor::StreamingMetrics; use crate::executor::utils::ActorInputMetrics; -use crate::executor::{ - expect_first_barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef, -}; +use crate::executor::{expect_first_barrier, BoxedMessageStream, Execute, Message}; use crate::task::{FragmentId, SharedContext}; /// `ReceiverExecutor` is used along with a channel. After creating a mpsc channel, /// there should be a `ReceiverExecutor` running in the background, so as to push /// messages down to the executors. pub struct ReceiverExecutor { - /// Logical Operator Info - info: ExecutorInfo, - /// Input from upstream. input: BoxedInput, @@ -57,10 +51,7 @@ pub struct ReceiverExecutor { impl std::fmt::Debug for ReceiverExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("ReceiverExecutor") - .field("schema", &self.info.schema) - .field("pk_indices", &self.info.pk_indices) - .finish() + f.debug_struct("ReceiverExecutor").finish() } } @@ -68,7 +59,6 @@ impl ReceiverExecutor { #[allow(clippy::too_many_arguments)] pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, fragment_id: FragmentId, upstream_fragment_id: FragmentId, input: BoxedInput, @@ -78,7 +68,6 @@ impl ReceiverExecutor { ) -> Self { Self { input, - info, actor_context: ctx, upstream_fragment_id, metrics, @@ -95,11 +84,6 @@ impl ReceiverExecutor { Self::new( ActorContext::for_test(114), - ExecutorInfo { - schema: Schema::default(), - pk_indices: vec![], - identity: "ReceiverExecutor".to_string(), - }, 514, 1919, LocalInput::new(input, 0).boxed_input(), @@ -110,7 +94,7 @@ impl ReceiverExecutor { } } -impl Executor for ReceiverExecutor { +impl Execute for ReceiverExecutor { fn execute(mut self: Box) -> BoxedMessageStream { let actor_id = self.actor_context.id; @@ -207,22 +191,6 @@ impl Executor for ReceiverExecutor { stream.boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } #[cfg(test)] @@ -235,13 +203,11 @@ mod tests { use risingwave_pb::stream_plan::update_mutation::MergeUpdate; use super::*; - use crate::executor::{ActorContext, Barrier, Executor, Mutation, UpdateMutation}; + use crate::executor::{ActorContext, Barrier, Execute, Mutation, UpdateMutation}; use crate::task::test_utils::helper_make_local_actor; #[tokio::test] async fn test_configuration_change() { - let schema = Schema { fields: vec![] }; - let actor_id = 233; let (old, new) = (114, 514); // old and new upstream actor id @@ -271,15 +237,8 @@ mod tests { ) .unwrap(); - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "ReceiverExecutor".to_string(), - }; - let receiver = ReceiverExecutor::new( ActorContext::for_test(actor_id), - info, fragment_id, upstream_fragment_id, input, diff --git a/src/stream/src/executor/row_id_gen.rs b/src/stream/src/executor/row_id_gen.rs index fe0ed6d908925..0a0fe1ffe84e1 100644 --- a/src/stream/src/executor/row_id_gen.rs +++ b/src/stream/src/executor/row_id_gen.rs @@ -19,23 +19,19 @@ use risingwave_common::array::{ Array, ArrayBuilder, ArrayRef, Op, SerialArrayBuilder, StreamChunk, }; use risingwave_common::buffer::Bitmap; -use risingwave_common::catalog::Schema; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::types::Serial; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_common::util::row_id::RowIdGenerator; -use super::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, PkIndicesRef, -}; +use super::{expect_first_barrier, ActorContextRef, Execute, Executor}; use crate::executor::{Message, StreamExecutorError}; /// [`RowIdGenExecutor`] generates row id for data, where the user has not specified a pk. pub struct RowIdGenExecutor { ctx: ActorContextRef, - info: ExecutorInfo, - upstream: Option, + upstream: Option, row_id_index: usize, @@ -45,14 +41,12 @@ pub struct RowIdGenExecutor { impl RowIdGenExecutor { pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - upstream: BoxedExecutor, + upstream: Executor, row_id_index: usize, vnodes: Bitmap, ) -> Self { Self { ctx, - info, upstream: Some(upstream), row_id_index, row_id_generator: Self::new_generator(&vnodes), @@ -126,22 +120,10 @@ impl RowIdGenExecutor { } } -impl Executor for RowIdGenExecutor { +impl Execute for RowIdGenExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -154,7 +136,7 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Executor}; + use crate::executor::{ActorContext, Execute}; #[tokio::test] async fn test_row_id_gen_executor() { @@ -165,20 +147,16 @@ mod tests { let pk_indices = vec![0]; let row_id_index = 0; let row_id_generator = Bitmap::ones(VirtualNode::COUNT); - let (mut tx, upstream) = MockSource::channel(schema.clone(), pk_indices.clone()); - let row_id_gen_executor = Box::new(RowIdGenExecutor::new( + let (mut tx, upstream) = MockSource::channel(); + let upstream = upstream.into_executor(schema.clone(), pk_indices.clone()); + + let row_id_gen_executor = RowIdGenExecutor::new( ActorContext::for_test(233), - ExecutorInfo { - schema, - pk_indices, - identity: "RowIdGenExecutor".to_string(), - }, - Box::new(upstream), + upstream, row_id_index, row_id_generator, - )); - - let mut row_id_gen_executor = row_id_gen_executor.execute(); + ); + let mut row_id_gen_executor = row_id_gen_executor.boxed().execute(); // Init barrier tx.push_barrier(1, false); diff --git a/src/stream/src/executor/simple_agg.rs b/src/stream/src/executor/simple_agg.rs index b8915a070dbc5..a863fc0432876 100644 --- a/src/stream/src/executor/simple_agg.rs +++ b/src/stream/src/executor/simple_agg.rs @@ -47,7 +47,7 @@ use crate::task::AtomicU64Ref; /// Therefore, we "automatically" implemented a window function inside /// `SimpleAggExecutor`. pub struct SimpleAggExecutor { - input: Box, + input: Executor, inner: ExecutorInner, } @@ -111,27 +111,15 @@ struct ExecutionVars { state_changed: bool, } -impl Executor for SimpleAggExecutor { +impl Execute for SimpleAggExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl SimpleAggExecutor { pub fn new(args: AggExecutorArgs) -> StreamResult { - let input_info = args.input.info(); + let input_info = args.input.info().clone(); Ok(Self { input: args.input, inner: ExecutorInner { @@ -335,7 +323,8 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); tx.push_barrier(1, false); tx.push_barrier(2, false); tx.push_chunk(StreamChunk::from_pretty( @@ -364,7 +353,7 @@ mod tests { let simple_agg = new_boxed_simple_agg_executor( ActorContext::for_test(123), store, - Box::new(source), + source, false, agg_calls, 0, diff --git a/src/stream/src/executor/sink.rs b/src/stream/src/executor/sink.rs index 4e64c3456bc8f..7928308c3f94a 100644 --- a/src/stream/src/executor/sink.rs +++ b/src/stream/src/executor/sink.rs @@ -34,7 +34,7 @@ use risingwave_connector::sink::{ use thiserror_ext::AsReport; use super::error::{StreamExecutorError, StreamExecutorResult}; -use super::{BoxedExecutor, Executor, ExecutorInfo, Message, PkIndices}; +use super::{Execute, Executor, ExecutorInfo, Message, PkIndices}; use crate::executor::{ expect_first_barrier, ActorContextRef, BoxedMessageStream, MessageStream, Mutation, }; @@ -43,7 +43,7 @@ use crate::task::ActorId; pub struct SinkExecutor { actor_context: ActorContextRef, info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, sink: SinkImpl, input_columns: Vec, sink_param: SinkParam, @@ -83,7 +83,7 @@ impl SinkExecutor { pub async fn new( actor_context: ActorContextRef, info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, sink_writer_param: SinkWriterParam, sink_param: SinkParam, columns: Vec, @@ -412,22 +412,10 @@ impl SinkExecutor { } } -impl Executor for SinkExecutor { +impl Execute for SinkExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } #[cfg(test)] @@ -475,28 +463,25 @@ mod test { .collect(); let pk_indices = vec![0]; - let mock = MockSource::with_messages( - schema.clone(), - pk_indices.clone(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 3 2 1", - ))), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I U- 3 2 1 U+ 3 4 1 + 5 6 7", - ))), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I - 5 6 7", - ))), - ], - ); + ))), + ]) + .into_executor(schema.clone(), pk_indices.clone()); let sink_param = SinkParam { sink_id: 0.into(), @@ -522,7 +507,7 @@ mod test { let sink_executor = SinkExecutor::new( ActorContext::for_test(0), info, - Box::new(mock), + source, SinkWriterParam::for_test(), sink_param, columns.clone(), @@ -531,7 +516,7 @@ mod test { .await .unwrap(); - let mut executor = SinkExecutor::execute(Box::new(sink_executor)); + let mut executor = sink_executor.boxed().execute(); // Barrier message. executor.next().await.unwrap().unwrap(); @@ -598,32 +583,29 @@ mod test { .map(|column| Field::from(column.column_desc.clone())) .collect(); - let mock = MockSource::with_messages( - schema.clone(), - vec![0, 1], - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 1 1 10", - ))), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 1 3 30", - ))), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I + 1 2 20 - 1 2 20", - ))), - Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( - " I I I + ))), + Message::Chunk(std::mem::take(&mut StreamChunk::from_pretty( + " I I I - 1 1 10", - ))), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + ))), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), vec![0, 1]); let sink_param = SinkParam { sink_id: 0.into(), @@ -649,7 +631,7 @@ mod test { let sink_executor = SinkExecutor::new( ActorContext::for_test(0), info, - Box::new(mock), + source, SinkWriterParam::for_test(), sink_param, columns.clone(), @@ -658,7 +640,7 @@ mod test { .await .unwrap(); - let mut executor = SinkExecutor::execute(Box::new(sink_executor)); + let mut executor = sink_executor.boxed().execute(); // Barrier message. executor.next().await.unwrap().unwrap(); @@ -739,15 +721,12 @@ mod test { .collect(); let pk_indices = vec![0]; - let mock = MockSource::with_messages( - schema.clone(), - pk_indices.clone(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Barrier(Barrier::new_test_barrier(3)), - ], - ); + let source = MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Barrier(Barrier::new_test_barrier(3)), + ]) + .into_executor(schema.clone(), pk_indices.clone()); let sink_param = SinkParam { sink_id: 0.into(), @@ -773,7 +752,7 @@ mod test { let sink_executor = SinkExecutor::new( ActorContext::for_test(0), info, - Box::new(mock), + source, SinkWriterParam::for_test(), sink_param, columns, @@ -782,7 +761,7 @@ mod test { .await .unwrap(); - let mut executor = SinkExecutor::execute(Box::new(sink_executor)); + let mut executor = sink_executor.boxed().execute(); // Barrier message. assert_eq!( diff --git a/src/stream/src/executor/sort.rs b/src/stream/src/executor/sort.rs index f07967ce45299..72474eea32977 100644 --- a/src/stream/src/executor/sort.rs +++ b/src/stream/src/executor/sort.rs @@ -20,23 +20,23 @@ use risingwave_storage::StateStore; use super::sort_buffer::SortBuffer; use super::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, Watermark, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, Watermark, }; use crate::common::table::state_table::StateTable; use crate::common::StreamChunkBuilder; pub struct SortExecutor { - input: BoxedExecutor, + input: Executor, inner: ExecutorInner, } pub struct SortExecutorArgs { pub actor_ctx: ActorContextRef, - pub info: ExecutorInfo, - pub input: BoxedExecutor, + pub input: Executor, + pub schema: Schema, pub buffer_table: StateTable, pub chunk_size: usize, pub sort_column_index: usize, @@ -44,8 +44,8 @@ pub struct SortExecutorArgs { struct ExecutorInner { actor_ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, buffer_table: StateTable, chunk_size: usize, sort_column_index: usize, @@ -55,22 +55,10 @@ struct ExecutionVars { buffer: SortBuffer, } -impl Executor for SortExecutor { +impl Execute for SortExecutor { fn execute(self: Box) -> BoxedMessageStream { self.executor_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info.pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info.identity - } } impl SortExecutor { @@ -79,7 +67,7 @@ impl SortExecutor { input: args.input, inner: ExecutorInner { actor_ctx: args.actor_ctx, - info: args.info, + schema: args.schema, buffer_table: args.buffer_table, chunk_size: args.chunk_size, sort_column_index: args.sort_column_index, @@ -114,7 +102,7 @@ impl SortExecutor { if col_idx == this.sort_column_index => { let mut chunk_builder = - StreamChunkBuilder::new(this.chunk_size, this.info.schema.data_types()); + StreamChunkBuilder::new(this.chunk_size, this.schema.data_types()); #[for_await] for row in vars @@ -172,7 +160,7 @@ mod tests { use super::*; use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; - use crate::executor::{ActorContext, BoxedMessageStream, Executor}; + use crate::executor::{ActorContext, BoxedMessageStream, Execute}; async fn create_executor( sort_column_index: usize, @@ -202,15 +190,12 @@ mod tests { ) .await; - let (tx, source) = MockSource::channel(input_schema, input_pk_indices); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema, input_pk_indices); let sort_executor = SortExecutor::new(SortExecutorArgs { actor_ctx: ActorContext::for_test(123), - info: ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "SortExecutor".to_string(), - }, - input: source.boxed(), + schema: source.schema().clone(), + input: source, buffer_table, chunk_size: 1024, sort_column_index, diff --git a/src/stream/src/executor/source/fetch_executor.rs b/src/stream/src/executor/source/fetch_executor.rs index 36513fc4aba36..73d1b5b42f1c6 100644 --- a/src/stream/src/executor/source/fetch_executor.rs +++ b/src/stream/src/executor/source/fetch_executor.rs @@ -22,7 +22,7 @@ use futures::stream::{self, StreamExt}; use futures::{pin_mut, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::StreamChunk; -use risingwave_common::catalog::{ColumnId, Schema, TableId}; +use risingwave_common::catalog::{ColumnId, TableId}; use risingwave_common::hash::VnodeBitmapExt; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{ScalarRef, ScalarRefImpl}; @@ -39,8 +39,13 @@ use risingwave_storage::store::PrefetchOptions; use risingwave_storage::StateStore; use thiserror_ext::AsReport; +use super::{get_split_offset_col_idx, SourceStateTableHandler}; use crate::executor::stream_reader::StreamReaderWithPause; -use crate::executor::*; +use crate::executor::{ + expect_first_barrier, get_split_offset_mapping_from_chunk, prune_additional_cols, + ActorContextRef, BoxedMessageStream, Execute, Executor, Message, Mutation, StreamExecutorError, + StreamExecutorResult, StreamSourceCore, +}; const SPLIT_BATCH_SIZE: usize = 1000; @@ -48,13 +53,12 @@ type SplitBatch = Option>; pub struct FsFetchExecutor { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: Option>, /// Upstream list executor. - upstream: Option, + upstream: Option, // control options for connector level source_ctrl_opts: SourceCtrlOpts, @@ -69,15 +73,13 @@ impl FsFetchExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: StreamSourceCore, - upstream: BoxedExecutor, + upstream: Executor, source_ctrl_opts: SourceCtrlOpts, connector_params: ConnectorParams, ) -> Self { Self { actor_ctx, - info, stream_source_core: Some(stream_source_core), upstream: Some(upstream), source_ctrl_opts, @@ -354,22 +356,10 @@ impl FsFetchExecutor { } } -impl Executor for FsFetchExecutor { +impl Execute for FsFetchExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl Debug for FsFetchExecutor { @@ -378,7 +368,6 @@ impl Debug for FsFetchExecutor { f.debug_struct("FsFetchExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("FsFetchExecutor").finish() diff --git a/src/stream/src/executor/source/fs_source_executor.rs b/src/stream/src/executor/source/fs_source_executor.rs index a2478cdb6bb0d..3f2d13b8be76f 100644 --- a/src/stream/src/executor/source/fs_source_executor.rs +++ b/src/stream/src/executor/source/fs_source_executor.rs @@ -22,9 +22,9 @@ use anyhow::anyhow; use either::Either; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_connector::error::ConnectorError; use risingwave_connector::source::reader::desc::{FsSourceDesc, SourceDescBuilder}; use risingwave_connector::source::{ BoxChunkSourceStream, ConnectorState, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, @@ -49,7 +49,6 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; /// such as s3. pub struct FsSourceExecutor { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: StreamSourceCore, @@ -70,7 +69,6 @@ impl FsSourceExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: StreamSourceCore, metrics: Arc, barrier_receiver: UnboundedReceiver, @@ -79,7 +77,6 @@ impl FsSourceExecutor { ) -> StreamResult { Ok(Self { actor_ctx, - info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -442,7 +439,7 @@ impl FsSourceExecutor { self.stream_source_core.latest_split_info.get_mut(id).map( |origin_split| { origin_split.update_in_place(offset.clone())?; - Ok::<_, anyhow::Error>((id.clone(), origin_split.clone())) + Ok::<_, ConnectorError>((id.clone(), origin_split.clone())) }, ) }) @@ -480,22 +477,10 @@ impl FsSourceExecutor { } } -impl Executor for FsSourceExecutor { +impl Execute for FsSourceExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl Debug for FsSourceExecutor { @@ -503,7 +488,6 @@ impl Debug for FsSourceExecutor { f.debug_struct("FsSourceExecutor") .field("source_id", &self.stream_source_core.source_id) .field("column_ids", &self.stream_source_core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } } diff --git a/src/stream/src/executor/source/list_executor.rs b/src/stream/src/executor/source/list_executor.rs index f2a6e47ae8641..7996848a749e3 100644 --- a/src/stream/src/executor/source/list_executor.rs +++ b/src/stream/src/executor/source/list_executor.rs @@ -20,7 +20,6 @@ use either::Either; use futures::{StreamExt, TryStreamExt}; use futures_async_stream::try_stream; use risingwave_common::array::Op; -use risingwave_common::catalog::Schema; use risingwave_common::system_param::local_manager::SystemParamsReaderRef; use risingwave_connector::source::reader::desc::{SourceDesc, SourceDescBuilder}; use risingwave_connector::source::SourceCtrlOpts; @@ -39,7 +38,6 @@ const CHUNK_SIZE: usize = 1024; #[allow(dead_code)] pub struct FsListExecutor { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: Option>, @@ -64,7 +62,6 @@ impl FsListExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: Option>, metrics: Arc, barrier_receiver: UnboundedReceiver, @@ -74,7 +71,6 @@ impl FsListExecutor { ) -> Self { Self { actor_ctx, - info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -193,22 +189,10 @@ impl FsListExecutor { } } -impl Executor for FsListExecutor { +impl Execute for FsListExecutor { fn execute(self: Box) -> BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl Debug for FsListExecutor { @@ -217,7 +201,6 @@ impl Debug for FsListExecutor { f.debug_struct("FsListExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("FsListExecutor").finish() diff --git a/src/stream/src/executor/source/source_executor.rs b/src/stream/src/executor/source/source_executor.rs index e2567bb141492..a695b3c8b6b4b 100644 --- a/src/stream/src/executor/source/source_executor.rs +++ b/src/stream/src/executor/source/source_executor.rs @@ -43,7 +43,6 @@ const WAIT_BARRIER_MULTIPLE_TIMES: u128 = 5; pub struct SourceExecutor { actor_ctx: ActorContextRef, - info: ExecutorInfo, /// Streaming source for external stream_source_core: Option>, @@ -68,7 +67,6 @@ impl SourceExecutor { #[allow(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, - info: ExecutorInfo, stream_source_core: Option>, metrics: Arc, barrier_receiver: UnboundedReceiver, @@ -78,7 +76,6 @@ impl SourceExecutor { ) -> Self { Self { actor_ctx, - info, stream_source_core, metrics, barrier_receiver: Some(barrier_receiver), @@ -527,8 +524,7 @@ impl SourceExecutor { // chunks. self_paused = true; tracing::warn!( - "source {} paused, wait barrier for {:?}", - self.info.identity, + "source paused, wait barrier for {:?}", last_barrier_time.elapsed() ); stream.pause_stream(); @@ -616,7 +612,7 @@ impl SourceExecutor { } } -impl Executor for SourceExecutor { +impl Execute for SourceExecutor { fn execute(self: Box) -> BoxedMessageStream { if self.stream_source_core.is_some() { self.execute_with_stream_source().boxed() @@ -624,18 +620,6 @@ impl Executor for SourceExecutor { self.execute_without_stream_source().boxed() } } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl Debug for SourceExecutor { @@ -644,7 +628,6 @@ impl Debug for SourceExecutor { f.debug_struct("SourceExecutor") .field("source_id", &core.source_id) .field("column_ids", &core.column_ids) - .field("pk_indices", &self.info.pk_indices) .finish() } else { f.debug_struct("SourceExecutor").finish() @@ -683,7 +666,6 @@ mod tests { fields: vec![Field::with_name(DataType::Int32, "sequence_int")], }; let row_id_index = None; - let pk_indices = vec![0]; let source_info = StreamSourceInfo { row_format: PbRowFormatType::Native as i32, ..Default::default() @@ -720,11 +702,6 @@ mod tests { let executor = SourceExecutor::new( ActorContext::for_test(0), - ExecutorInfo { - schema, - pk_indices, - identity: "SourceExecutor".to_string(), - }, Some(core), Arc::new(StreamingMetrics::unused()), barrier_rx, @@ -732,7 +709,7 @@ mod tests { SourceCtrlOpts::default(), ConnectorParams::default(), ); - let mut executor = Box::new(executor).execute(); + let mut executor = executor.boxed().execute(); let init_barrier = Barrier::new_test_barrier(1).with_mutation(Mutation::Add(AddMutation { adds: HashMap::new(), @@ -776,7 +753,6 @@ mod tests { fields: vec![Field::with_name(DataType::Int32, "v1")], }; let row_id_index = None; - let pk_indices = vec![0_usize]; let source_info = StreamSourceInfo { row_format: PbRowFormatType::Native as i32, ..Default::default() @@ -814,11 +790,6 @@ mod tests { let executor = SourceExecutor::new( ActorContext::for_test(0), - ExecutorInfo { - schema, - pk_indices, - identity: "SourceExecutor".to_string(), - }, Some(core), Arc::new(StreamingMetrics::unused()), barrier_rx, @@ -826,7 +797,7 @@ mod tests { SourceCtrlOpts::default(), ConnectorParams::default(), ); - let mut handler = Box::new(executor).execute(); + let mut handler = executor.boxed().execute(); let init_barrier = Barrier::new_test_barrier(1).with_mutation(Mutation::Add(AddMutation { adds: HashMap::new(), diff --git a/src/stream/src/executor/stateless_simple_agg.rs b/src/stream/src/executor/stateless_simple_agg.rs index 8a6334b7743b5..1ad25fe5c7f6c 100644 --- a/src/stream/src/executor/stateless_simple_agg.rs +++ b/src/stream/src/executor/stateless_simple_agg.rs @@ -16,7 +16,6 @@ use futures::StreamExt; use futures_async_stream::try_stream; use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::aggregate::{ build_retractable, AggCall, AggregateState, BoxedAggregateFunction, @@ -29,28 +28,16 @@ use crate::error::StreamResult; pub struct StatelessSimpleAggExecutor { _ctx: ActorContextRef, - pub(super) info: ExecutorInfo, - pub(super) input: Box, + pub(super) input: Executor, + pub(super) schema: Schema, pub(super) aggs: Vec, pub(super) agg_calls: Vec, } -impl Executor for StatelessSimpleAggExecutor { +impl Execute for StatelessSimpleAggExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } impl StatelessSimpleAggExecutor { @@ -73,7 +60,7 @@ impl StatelessSimpleAggExecutor { let StatelessSimpleAggExecutor { _ctx, input, - info, + schema, aggs, agg_calls, } = self; @@ -94,7 +81,7 @@ impl StatelessSimpleAggExecutor { if is_dirty { is_dirty = false; - let mut builders = info.schema.create_array_builders(1); + let mut builders = schema.create_array_builders(1); for ((agg, state), builder) in aggs .iter() .zip_eq_fast(states.iter_mut()) @@ -124,15 +111,15 @@ impl StatelessSimpleAggExecutor { impl StatelessSimpleAggExecutor { pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: Box, + input: Executor, + schema: Schema, agg_calls: Vec, ) -> StreamResult { let aggs = agg_calls.iter().map(build_retractable).try_collect()?; Ok(StatelessSimpleAggExecutor { _ctx: ctx, - info, input, + schema, aggs, agg_calls, }) @@ -150,34 +137,24 @@ mod tests { use super::*; use crate::executor::test_utils::agg_executor::generate_agg_schema; use crate::executor::test_utils::MockSource; - use crate::executor::{Executor, StatelessSimpleAggExecutor}; + use crate::executor::{Execute, StatelessSimpleAggExecutor}; #[tokio::test] async fn test_no_chunk() { let schema = schema_test_utils::ii(); - let (mut tx, source) = MockSource::channel(schema, vec![2]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); tx.push_barrier(1, false); tx.push_barrier(2, false); tx.push_barrier(3, false); let agg_calls = vec![AggCall::from_pretty("(count:int8)")]; let schema = generate_agg_schema(&source, &agg_calls, None); - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "StatelessSimpleAggExecutor".to_string(), - }; - let simple_agg = Box::new( - StatelessSimpleAggExecutor::new( - ActorContext::for_test(123), - info, - Box::new(source), - agg_calls, - ) - .unwrap(), - ); - let mut simple_agg = simple_agg.execute(); + let simple_agg = + StatelessSimpleAggExecutor::new(ActorContext::for_test(123), source, schema, agg_calls) + .unwrap(); + let mut simple_agg = simple_agg.boxed().execute(); assert_matches!( simple_agg.next().await.unwrap().unwrap(), @@ -196,7 +173,8 @@ mod tests { #[tokio::test] async fn test_local_simple_agg() { let schema = schema_test_utils::iii(); - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk\ + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); tx.push_barrier(1, false); tx.push_chunk(StreamChunk::from_pretty( " I I I @@ -220,22 +198,11 @@ mod tests { AggCall::from_pretty("(sum:int8 $1:int8)"), ]; let schema = generate_agg_schema(&source, &agg_calls, None); - let info = ExecutorInfo { - schema, - pk_indices: vec![], - identity: "StatelessSimpleAggExecutor".to_string(), - }; - let simple_agg = Box::new( - StatelessSimpleAggExecutor::new( - ActorContext::for_test(123), - info, - Box::new(source), - agg_calls, - ) - .unwrap(), - ); - let mut simple_agg = simple_agg.execute(); + let simple_agg = + StatelessSimpleAggExecutor::new(ActorContext::for_test(123), source, schema, agg_calls) + .unwrap(); + let mut simple_agg = simple_agg.boxed().execute(); // Consume the init barrier simple_agg.next().await.unwrap().unwrap(); diff --git a/src/stream/src/executor/subscription.rs b/src/stream/src/executor/subscription.rs index f1a2923c9085c..dfe352909fd8e 100644 --- a/src/stream/src/executor/subscription.rs +++ b/src/stream/src/executor/subscription.rs @@ -16,15 +16,14 @@ use core::time::Duration; use futures::prelude::stream::StreamExt; use futures_async_stream::try_stream; -use risingwave_common::catalog::Schema; use risingwave_common::types::Timestamptz; use risingwave_common::util::epoch::Epoch; use risingwave_storage::store::LocalStateStore; use tokio::time::Instant; use super::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, StreamExecutorError, StreamExecutorResult, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + StreamExecutorError, StreamExecutorResult, }; use crate::common::log_store_impl::kv_log_store::ReaderTruncationOffsetType; use crate::common::log_store_impl::subscription_log_store::SubscriptionLogStoreWriter; @@ -33,8 +32,7 @@ const EXECUTE_GC_INTERVAL: u64 = 3600; pub struct SubscriptionExecutor { actor_context: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, log_store: SubscriptionLogStoreWriter, retention_seconds: u64, } @@ -44,14 +42,12 @@ impl SubscriptionExecutor { #[expect(clippy::unused_async)] pub async fn new( actor_context: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, log_store: SubscriptionLogStoreWriter, retention_seconds: u64, ) -> StreamExecutorResult { Ok(Self { actor_context, - info, input, log_store, retention_seconds, @@ -115,24 +111,8 @@ impl SubscriptionExecutor { } } } -impl Executor for SubscriptionExecutor { +impl Execute for SubscriptionExecutor { fn execute(self: Box) -> BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } diff --git a/src/stream/src/executor/subtask.rs b/src/stream/src/executor/subtask.rs index 0130a73fc411a..fea9644b151f3 100644 --- a/src/stream/src/executor/subtask.rs +++ b/src/stream/src/executor/subtask.rs @@ -20,7 +20,7 @@ use tokio::sync::mpsc::error::SendError; use tokio_stream::wrappers::ReceiverStream; use super::actor::spawn_blocking_drop_stream; -use super::{BoxedExecutor, Executor, ExecutorInfo, Message, MessageStreamItem}; +use super::{Execute, Executor, Message, MessageStreamItem}; use crate::task::ActorId; /// Handle used to drive the subtask. @@ -29,31 +29,13 @@ pub type SubtaskHandle = impl Future + Send + 'static; /// The thin wrapper for subtask-wrapped executor, containing a channel to receive the messages from /// the subtask. pub struct SubtaskRxExecutor { - info: ExecutorInfo, - rx: mpsc::Receiver, } -impl Executor for SubtaskRxExecutor { +impl Execute for SubtaskRxExecutor { fn execute(self: Box) -> super::BoxedMessageStream { ReceiverStream::new(self.rx).boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } /// Wrap an executor into a subtask and a thin receiver executor, connected by a channel with a @@ -62,15 +44,9 @@ impl Executor for SubtaskRxExecutor { /// Used when there're multiple stateful executors in an actor. These subtasks can be concurrently /// executed to improve the I/O performance, while the computing resource can be still bounded to a /// single thread. -pub fn wrap(input: BoxedExecutor, actor_id: ActorId) -> (SubtaskHandle, SubtaskRxExecutor) { +pub fn wrap(input: Executor, actor_id: ActorId) -> (SubtaskHandle, SubtaskRxExecutor) { let (tx, rx) = mpsc::channel(1); - let rx_executor = SubtaskRxExecutor { - info: ExecutorInfo { - identity: "SubtaskRxExecutor".to_owned(), - ..input.info() - }, - rx, - }; + let rx_executor = SubtaskRxExecutor { rx }; let handle = async move { let mut input = input.execute(); diff --git a/src/stream/src/executor/temporal_join.rs b/src/stream/src/executor/temporal_join.rs index da0ac7b45dbdc..f6c614f41aef4 100644 --- a/src/stream/src/executor/temporal_join.rs +++ b/src/stream/src/executor/temporal_join.rs @@ -25,7 +25,6 @@ use futures_async_stream::{for_await, try_stream}; use local_stats_alloc::{SharedStatsAlloc, StatsAlloc}; use lru::DefaultHasher; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::Schema; use risingwave_common::estimate_size::{EstimateSize, KvSize}; use risingwave_common::hash::{HashKey, NullBitmap}; use risingwave_common::row::{OwnedRow, Row, RowExt}; @@ -40,21 +39,21 @@ use risingwave_storage::StateStore; use super::join::{JoinType, JoinTypePrimitive}; use super::{ - Barrier, Executor, ExecutorInfo, Message, MessageStream, StreamExecutorError, + Barrier, Execute, ExecutorInfo, Message, MessageStream, StreamExecutorError, StreamExecutorResult, }; use crate::cache::{cache_may_stale, new_with_hasher_in, ManagedLruCache}; use crate::common::metrics::MetricsInfo; use crate::executor::join::builder::JoinStreamChunkBuilder; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, BoxedExecutor, Watermark}; +use crate::executor::{ActorContextRef, Executor, Watermark}; use crate::task::AtomicU64Ref; pub struct TemporalJoinExecutor { ctx: ActorContextRef, info: ExecutorInfo, - left: BoxedExecutor, - right: BoxedExecutor, + left: Executor, + right: Executor, right_table: TemporalSide, left_join_keys: Vec, right_join_keys: Vec, @@ -261,7 +260,7 @@ async fn internal_messages_until_barrier(stream: impl MessageStream, expected_ba // any number of `InternalMessage::Chunk(left_chunk)` and followed by // `InternalMessage::Barrier(right_chunks, barrier)`. #[try_stream(ok = InternalMessage, error = StreamExecutorError)] -async fn align_input(left: Box, right: Box) { +async fn align_input(left: Executor, right: Executor) { let mut left = pin!(left.execute()); let mut right = pin!(right.execute()); // Keep producing intervals until stream exhaustion or errors. @@ -313,8 +312,8 @@ impl TemporalJoinExecutor pub fn new( ctx: ActorContextRef, info: ExecutorInfo, - left: BoxedExecutor, - right: BoxedExecutor, + left: Executor, + right: Executor, table: StorageTable, left_join_keys: Vec, right_join_keys: Vec, @@ -475,22 +474,10 @@ impl TemporalJoinExecutor } } -impl Executor +impl Execute for TemporalJoinExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.into_stream().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } diff --git a/src/stream/src/executor/test_utils.rs b/src/stream/src/executor/test_utils.rs index cc7505164d154..0be0c9b5648b1 100644 --- a/src/stream/src/executor/test_utils.rs +++ b/src/stream/src/executor/test_utils.rs @@ -21,8 +21,8 @@ use tokio::sync::mpsc; use super::error::StreamExecutorError; use super::{ - Barrier, BoxedMessageStream, Executor, Message, MessageStream, PkIndices, StreamChunk, - StreamExecutorResult, Watermark, + Barrier, BoxedMessageStream, Execute, Executor, ExecutorInfo, Message, MessageStream, + StreamChunk, StreamExecutorResult, Watermark, }; pub mod prelude { @@ -40,12 +40,10 @@ pub mod prelude { pub use crate::common::table::state_table::StateTable; pub use crate::executor::test_utils::expr::build_from_pretty; pub use crate::executor::test_utils::{MessageSender, MockSource, StreamExecutorTestExt}; - pub use crate::executor::{ActorContext, BoxedMessageStream, Executor, PkIndices}; + pub use crate::executor::{ActorContext, BoxedMessageStream, Execute, PkIndices}; } pub struct MockSource { - schema: Schema, - pk_indices: PkIndices, rx: mpsc::UnboundedReceiver, /// Whether to send a `Stop` barrier on stream finish. @@ -107,20 +105,15 @@ impl MessageSender { impl std::fmt::Debug for MockSource { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("MockSource") - .field("schema", &self.schema) - .field("pk_indices", &self.pk_indices) - .finish() + f.debug_struct("MockSource").finish() } } impl MockSource { #[allow(dead_code)] - pub fn channel(schema: Schema, pk_indices: PkIndices) -> (MessageSender, Self) { + pub fn channel() -> (MessageSender, Self) { let (tx, rx) = mpsc::unbounded_channel(); let source = Self { - schema, - pk_indices, rx, stop_on_finish: true, }; @@ -128,16 +121,16 @@ impl MockSource { } #[allow(dead_code)] - pub fn with_messages(schema: Schema, pk_indices: PkIndices, msgs: Vec) -> Self { - let (tx, source) = Self::channel(schema, pk_indices); + pub fn with_messages(msgs: Vec) -> Self { + let (tx, source) = Self::channel(); for msg in msgs { tx.0.send(msg).unwrap(); } source } - pub fn with_chunks(schema: Schema, pk_indices: PkIndices, chunks: Vec) -> Self { - let (tx, source) = Self::channel(schema, pk_indices); + pub fn with_chunks(chunks: Vec) -> Self { + let (tx, source) = Self::channel(); for chunk in chunks { tx.0.send(Message::Chunk(chunk)).unwrap(); } @@ -153,6 +146,17 @@ impl MockSource { } } + pub fn into_executor(self, schema: Schema, pk_indices: Vec) -> Executor { + Executor::new( + ExecutorInfo { + schema, + pk_indices, + identity: "MockSource".to_string(), + }, + self.boxed(), + ) + } + #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self: Box) { let mut epoch = 1; @@ -168,22 +172,10 @@ impl MockSource { } } -impl Executor for MockSource { +impl Execute for MockSource { fn execute(self: Box) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &Schema { - &self.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.pk_indices - } - - fn identity(&self) -> &str { - "MockSource" - } } /// `row_nonnull` builds a `OwnedRow` with concrete values. @@ -295,14 +287,14 @@ pub mod agg_executor { }; use crate::executor::aggregation::AggStateStorage; use crate::executor::{ - ActorContext, ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, HashAggExecutor, - PkIndices, SimpleAggExecutor, + ActorContext, ActorContextRef, Executor, ExecutorInfo, HashAggExecutor, PkIndices, + SimpleAggExecutor, }; - /// Generate agg executor's schema from `input`, `agg_calls` and `group_key_indices`. + /// Generate aggExecuter's schema from `input`, `agg_calls` and `group_key_indices`. /// For [`crate::executor::HashAggExecutor`], the group key indices should be provided. pub fn generate_agg_schema( - input: &dyn Executor, + input_ref: &Executor, agg_calls: &[AggCall], group_key_indices: Option<&[usize]>, ) -> Schema { @@ -313,7 +305,7 @@ pub mod agg_executor { let fields = if let Some(key_indices) = group_key_indices { let keys = key_indices .iter() - .map(|idx| input.schema().fields[*idx].clone()); + .map(|idx| input_ref.schema().fields[*idx].clone()); keys.chain(aggs).collect() } else { @@ -331,7 +323,7 @@ pub mod agg_executor { agg_call: &AggCall, group_key_indices: &[usize], pk_indices: &[usize], - input_ref: &dyn Executor, + input_ref: &Executor, is_append_only: bool, ) -> AggStateStorage { match agg_call.kind { @@ -402,7 +394,7 @@ pub mod agg_executor { table_id: TableId, agg_calls: &[AggCall], group_key_indices: &[usize], - input_ref: &dyn Executor, + input_ref: &Executor, ) -> StateTable { let input_fields = input_ref.schema().fields(); @@ -441,7 +433,7 @@ pub mod agg_executor { #[allow(clippy::too_many_arguments)] pub async fn new_boxed_hash_agg_executor( store: S, - input: Box, + input: Executor, is_append_only: bool, agg_calls: Vec, row_count_index: usize, @@ -450,7 +442,7 @@ pub mod agg_executor { extreme_cache_size: usize, emit_on_window_close: bool, executor_id: u64, - ) -> Box { + ) -> Executor { let mut storages = Vec::with_capacity(agg_calls.iter().len()); for (idx, agg_call) in agg_calls.iter().enumerate() { storages.push( @@ -460,7 +452,7 @@ pub mod agg_executor { agg_call, &group_key_indices, &pk_indices, - input.as_ref(), + &input, is_append_only, ) .await, @@ -472,23 +464,23 @@ pub mod agg_executor { TableId::new(agg_calls.len() as u32), &agg_calls, &group_key_indices, - input.as_ref(), + &input, ) .await; - let schema = generate_agg_schema(input.as_ref(), &agg_calls, Some(&group_key_indices)); + let schema = generate_agg_schema(&input, &agg_calls, Some(&group_key_indices)); let info = ExecutorInfo { schema, pk_indices, identity: format!("HashAggExecutor {:X}", executor_id), }; - HashAggExecutor::::new(AggExecutorArgs { + let exec = HashAggExecutor::::new(AggExecutorArgs { version: PbAggNodeVersion::Max, input, actor_ctx: ActorContext::for_test(123), - info, + info: info.clone(), extreme_cache_size, @@ -506,21 +498,21 @@ pub mod agg_executor { emit_on_window_close, }, }) - .unwrap() - .boxed() + .unwrap(); + (info, exec).into() } #[allow(clippy::too_many_arguments)] pub async fn new_boxed_simple_agg_executor( actor_ctx: ActorContextRef, store: S, - input: BoxedExecutor, + input: Executor, is_append_only: bool, agg_calls: Vec, row_count_index: usize, pk_indices: PkIndices, executor_id: u64, - ) -> Box { + ) -> Executor { let storages = future::join_all(agg_calls.iter().enumerate().map(|(idx, agg_call)| { create_agg_state_storage( store.clone(), @@ -528,7 +520,7 @@ pub mod agg_executor { agg_call, &[], &pk_indices, - input.as_ref(), + &input, is_append_only, ) })) @@ -539,23 +531,23 @@ pub mod agg_executor { TableId::new(agg_calls.len() as u32), &agg_calls, &[], - input.as_ref(), + &input, ) .await; - let schema = generate_agg_schema(input.as_ref(), &agg_calls, None); + let schema = generate_agg_schema(&input, &agg_calls, None); let info = ExecutorInfo { schema, pk_indices, identity: format!("SimpleAggExecutor {:X}", executor_id), }; - SimpleAggExecutor::new(AggExecutorArgs { + let exec = SimpleAggExecutor::new(AggExecutorArgs { version: PbAggNodeVersion::Max, input, actor_ctx, - info, + info: info.clone(), extreme_cache_size: 1024, @@ -567,8 +559,8 @@ pub mod agg_executor { watermark_epoch: Arc::new(AtomicU64::new(0)), extra: SimpleAggExecutorExtraArgs {}, }) - .unwrap() - .boxed() + .unwrap(); + (info, exec).into() } } diff --git a/src/stream/src/executor/top_n/group_top_n.rs b/src/stream/src/executor/top_n/group_top_n.rs index ed0f64d098a7a..d4c0fa029341e 100644 --- a/src/stream/src/executor/top_n/group_top_n.rs +++ b/src/stream/src/executor/top_n/group_top_n.rs @@ -17,6 +17,7 @@ use std::sync::Arc; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; use risingwave_common::hash::HashKey; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; @@ -32,7 +33,7 @@ use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; use crate::task::AtomicU64Ref; pub type GroupTopNExecutor = @@ -41,9 +42,9 @@ pub type GroupTopNExecutor = impl GroupTopNExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -55,7 +56,7 @@ impl GroupTopNExecutor GroupTopNExecutor { - info: ExecutorInfo, + schema: Schema, /// `LIMIT XXX`. None means no limit. limit: usize, @@ -97,7 +98,7 @@ pub struct InnerGroupTopNExecutor InnerGroupTopNExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -113,12 +114,11 @@ impl InnerGroupTopNExecutor::new(state_table, cache_key_serde.clone()); Ok(Self { - info, + schema, offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, @@ -191,7 +191,7 @@ where .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .inc(); let mut topn_cache = - TopNCache::new(self.offset, self.limit, self.info().schema.data_types()); + TopNCache::new(self.offset, self.limit, self.schema.data_types()); self.managed_state .init_topn_cache(Some(group_key), &mut topn_cache) .await?; @@ -227,7 +227,7 @@ where .group_top_n_cached_entry_count .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.caches.len() as i64); - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -238,10 +238,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - fn update_epoch(&mut self, epoch: u64) { self.caches.update_epoch(epoch); } @@ -289,7 +285,7 @@ mod tests { use super::*; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Barrier, Message}; + use crate::executor::{ActorContext, Barrier, Execute, Message}; fn create_schema() -> Schema { Schema { @@ -355,24 +351,21 @@ mod tests { vec![chunk0, chunk1, chunk2, chunk3] } - fn create_source() -> Box { + fn create_source() -> Executor { let mut chunks = create_stream_chunks(); let schema = create_schema(); - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Barrier(Barrier::new_test_barrier(4)), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(5)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Barrier(Barrier::new_test_barrier(4)), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(5)), + ]) + .into_executor(schema, pk_indices()) } #[tokio::test] @@ -388,26 +381,20 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix - identity: "GroupTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - GroupTopNExecutor::::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (0, 2), - order_by_1(), - vec![1], - state_table, - Arc::new(AtomicU64::new(0)), - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = GroupTopNExecutor::::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 2), + order_by_1(), + vec![1], + state_table, + Arc::new(AtomicU64::new(0)), + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -490,26 +477,20 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix - identity: "GroupTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - GroupTopNExecutor::::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (1, 2), - order_by_1(), - vec![1], - state_table, - Arc::new(AtomicU64::new(0)), - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = GroupTopNExecutor::::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 2), + order_by_1(), + vec![1], + state_table, + Arc::new(AtomicU64::new(0)), + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -585,26 +566,20 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), // this includes group key as prefix - identity: "GroupTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - GroupTopNExecutor::::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (0, 2), - order_by_2(), - vec![1, 2], - state_table, - Arc::new(AtomicU64::new(0)), - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = GroupTopNExecutor::::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 2), + order_by_2(), + vec![1, 2], + state_table, + Arc::new(AtomicU64::new(0)), + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); diff --git a/src/stream/src/executor/top_n/group_top_n_appendonly.rs b/src/stream/src/executor/top_n/group_top_n_appendonly.rs index 44522ddfb31ff..94f1df73770ab 100644 --- a/src/stream/src/executor/top_n/group_top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/group_top_n_appendonly.rs @@ -16,6 +16,7 @@ use std::sync::Arc; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::buffer::Bitmap; +use risingwave_common::catalog::Schema; use risingwave_common::hash::HashKey; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; @@ -31,7 +32,7 @@ use crate::common::metrics::MetricsInfo; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; use crate::task::AtomicU64Ref; /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need @@ -45,9 +46,9 @@ impl { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -59,7 +60,7 @@ impl input, ctx: ctx.clone(), inner: InnerAppendOnlyGroupTopNExecutor::new( - info, + schema, storage_key, offset_and_limit, order_by, @@ -73,7 +74,7 @@ impl } pub struct InnerAppendOnlyGroupTopNExecutor { - info: ExecutorInfo, + schema: Schema, /// `LIMIT XXX`. None means no limit. limit: usize, @@ -103,7 +104,7 @@ impl { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -119,12 +120,11 @@ impl "GroupTopN", ); - let cache_key_serde = - create_cache_key_serde(&storage_key, &info.schema, &order_by, &group_by); + let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &group_by); let managed_state = ManagedTopNState::::new(state_table, cache_key_serde.clone()); Ok(Self { - info, + schema, offset: offset_and_limit.0, limit: offset_and_limit.1, managed_state, @@ -147,7 +147,7 @@ where let mut res_rows = Vec::with_capacity(self.limit); let keys = K::build(&self.group_by, chunk.data_chunk())?; - let data_types = self.info().schema.data_types(); + let data_types = self.schema.data_types(); let row_deserializer = RowDeserializer::new(data_types.clone()); let table_id_str = self.managed_state.table().table_id().to_string(); let actor_id_str = self.ctx.id.to_string(); @@ -197,7 +197,7 @@ where .group_top_n_appendonly_cached_entry_count .with_label_values(&[&table_id_str, &actor_id_str, &fragment_id_str]) .set(self.caches.len() as i64); - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -208,10 +208,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - fn update_vnode_bitmap(&mut self, vnode_bitmap: Arc) { let cache_may_stale = self.managed_state.update_vnode_bitmap(vnode_bitmap); if cache_may_stale { diff --git a/src/stream/src/executor/top_n/top_n_appendonly.rs b/src/stream/src/executor/top_n/top_n_appendonly.rs index 8da016fd8c7ac..095222939a6e8 100644 --- a/src/stream/src/executor/top_n/top_n_appendonly.rs +++ b/src/stream/src/executor/top_n/top_n_appendonly.rs @@ -13,6 +13,7 @@ // limitations under the License. use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::catalog::Schema; use risingwave_common::row::{RowDeserializer, RowExt}; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::ColumnOrder; @@ -24,7 +25,7 @@ use super::{ManagedTopNState, TopNCache, NO_GROUP_KEY}; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; /// If the input is append-only, `AppendOnlyGroupTopNExecutor` does not need /// to keep all the rows seen. As long as a record @@ -38,9 +39,9 @@ pub type AppendOnlyTopNExecutor = impl AppendOnlyTopNExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -50,7 +51,7 @@ impl AppendOnlyTopNExecutor input, ctx, inner: InnerAppendOnlyTopNExecutor::new( - info, + schema, storage_key, offset_and_limit, order_by, @@ -61,7 +62,7 @@ impl AppendOnlyTopNExecutor } pub struct InnerAppendOnlyTopNExecutor { - info: ExecutorInfo, + schema: Schema, /// The storage key indices of the `TopNExecutor` storage_key_indices: PkIndices, @@ -80,7 +81,7 @@ pub struct InnerAppendOnlyTopNExecutor { impl InnerAppendOnlyTopNExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -89,12 +90,12 @@ impl InnerAppendOnlyTopNExecutor::new(state_table, cache_key_serde.clone()); - let data_types = info.schema.data_types(); + let data_types = schema.data_types(); Ok(Self { - info, + schema, managed_state, storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), cache: TopNCache::new(num_offset, num_limit, data_types), @@ -111,7 +112,7 @@ where async fn apply_chunk(&mut self, chunk: StreamChunk) -> StreamExecutorResult { let mut res_ops = Vec::with_capacity(self.cache.limit); let mut res_rows = Vec::with_capacity(self.cache.limit); - let data_types = self.info().schema.data_types(); + let data_types = self.schema.data_types(); let row_deserializer = RowDeserializer::new(data_types); // apply the chunk to state table for (op, row_ref) in chunk.rows() { @@ -128,7 +129,7 @@ where )?; } - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -139,10 +140,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - async fn init(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { self.managed_state.init_epoch(epoch); self.managed_state @@ -170,7 +167,7 @@ mod tests { use super::AppendOnlyTopNExecutor; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table; use crate::executor::test_utils::MockSource; - use crate::executor::{ActorContext, Barrier, Executor, ExecutorInfo, Message, PkIndices}; + use crate::executor::{ActorContext, Barrier, Execute, Executor, Message, PkIndices}; fn create_stream_chunks() -> Vec { let chunk1 = StreamChunk::from_pretty( @@ -223,21 +220,17 @@ mod tests { vec![0, 1] } - fn create_source() -> Box { + fn create_source() -> Executor { let mut chunks = create_stream_chunks(); - let schema = create_schema(); - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[2])), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[2])), + ]) + .into_executor(create_schema(), pk_indices()) } #[tokio::test] @@ -251,24 +244,18 @@ mod tests { ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "AppendOnlyTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - AppendOnlyTopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key, - (0, 5), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = AppendOnlyTopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key, + (0, 5), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init epoch top_n_executor.next().await.unwrap().unwrap(); @@ -338,24 +325,18 @@ mod tests { ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "AppendOnlyTopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - AppendOnlyTopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (3, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = AppendOnlyTopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (3, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init epoch top_n_executor.next().await.unwrap().unwrap(); diff --git a/src/stream/src/executor/top_n/top_n_plain.rs b/src/stream/src/executor/top_n/top_n_plain.rs index 5df41cfca3a05..b09f13764ee79 100644 --- a/src/stream/src/executor/top_n/top_n_plain.rs +++ b/src/stream/src/executor/top_n/top_n_plain.rs @@ -13,6 +13,7 @@ // limitations under the License. use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::catalog::Schema; use risingwave_common::row::RowExt; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::sort_util::ColumnOrder; @@ -23,7 +24,7 @@ use super::{ManagedTopNState, TopNCache, TopNCacheTrait}; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; use crate::executor::error::StreamExecutorResult; -use crate::executor::{ActorContextRef, Executor, ExecutorInfo, PkIndices, Watermark}; +use crate::executor::{ActorContextRef, Executor, PkIndices, Watermark}; /// `TopNExecutor` works with input with modification, it keeps all the data /// records/rows that have been seen, and returns topN records overall. @@ -33,9 +34,9 @@ pub type TopNExecutor = impl TopNExecutor { #[allow(clippy::too_many_arguments)] pub fn new( - input: Box, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -45,7 +46,7 @@ impl TopNExecutor { input, ctx, inner: InnerTopNExecutor::new( - info, + schema, storage_key, offset_and_limit, order_by, @@ -61,16 +62,16 @@ impl TopNExecutor { #[allow(clippy::too_many_arguments)] #[cfg(test)] pub fn new_with_ties_for_test( - input: Box, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, state_table: StateTable, ) -> StreamResult { let mut inner = - InnerTopNExecutor::new(info, storage_key, offset_and_limit, order_by, state_table)?; + InnerTopNExecutor::new(schema, storage_key, offset_and_limit, order_by, state_table)?; inner.cache.high_capacity = 2; @@ -79,7 +80,7 @@ impl TopNExecutor { } pub struct InnerTopNExecutor { - info: ExecutorInfo, + schema: Schema, /// The storage key indices of the `TopNExecutor` storage_key_indices: PkIndices, @@ -103,7 +104,7 @@ impl InnerTopNExecutor { /// into `CacheKey`. #[allow(clippy::too_many_arguments)] pub fn new( - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -112,12 +113,12 @@ impl InnerTopNExecutor { let num_offset = offset_and_limit.0; let num_limit = offset_and_limit.1; - let cache_key_serde = create_cache_key_serde(&storage_key, &info.schema, &order_by, &[]); + let cache_key_serde = create_cache_key_serde(&storage_key, &schema, &order_by, &[]); let managed_state = ManagedTopNState::::new(state_table, cache_key_serde.clone()); - let data_types = info.schema.data_types(); + let data_types = schema.data_types(); Ok(Self { - info, + schema, managed_state, storage_key_indices: storage_key.into_iter().map(|op| op.column_index).collect(), cache: TopNCache::new(num_offset, num_limit, data_types), @@ -162,7 +163,7 @@ where } } } - generate_output(res_rows, res_ops, &self.info().schema) + generate_output(res_rows, res_ops, &self.schema) } async fn flush_data(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { @@ -173,10 +174,6 @@ where self.managed_state.try_flush().await } - fn info(&self) -> &ExecutorInfo { - &self.info - } - async fn init(&mut self, epoch: EpochPair) -> StreamExecutorResult<()> { self.managed_state.init_epoch(epoch); self.managed_state @@ -206,7 +203,7 @@ mod tests { mod test1 { use super::*; - use crate::executor::ActorContext; + use crate::executor::{ActorContext, Execute}; fn create_stream_chunks() -> Vec { let chunk1 = StreamChunk::from_pretty( " I I @@ -265,24 +262,21 @@ mod tests { vec![0, 1] } - fn create_source() -> Box { + fn create_source() -> Executor { let mut chunks = create_stream_chunks(); let schema = create_schema(); - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Barrier(Barrier::new_test_barrier(2)), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Barrier(Barrier::new_test_barrier(4)), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(5)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Barrier(Barrier::new_test_barrier(2)), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Barrier(Barrier::new_test_barrier(4)), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(5)), + ]) + .into_executor(schema, pk_indices()) } #[tokio::test] @@ -295,24 +289,18 @@ mod tests { ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (3, 1000), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (3, 1000), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -396,24 +384,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (0, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -509,24 +491,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, true>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (0, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, true>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -621,24 +597,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (3, 4), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (3, 4), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -715,9 +685,9 @@ mod tests { use super::*; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table_from_state_store; - use crate::executor::ActorContext; + use crate::executor::{ActorContext, Execute}; - fn create_source_new() -> Box { + fn create_source_new() -> Executor { let mut chunks = vec![ StreamChunk::from_pretty( " I I I I @@ -746,21 +716,18 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } - fn create_source_new_before_recovery() -> Box { + fn create_source_new_before_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I I I @@ -779,19 +746,16 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } - fn create_source_new_after_recovery() -> Box { + fn create_source_new_after_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I I I @@ -812,16 +776,13 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, pk_indices()) } fn storage_key() -> Vec { @@ -853,24 +814,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (1, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -937,24 +892,18 @@ mod tests { ) .await; let source = create_source_new_before_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (1, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -997,24 +946,18 @@ mod tests { // recovery let source = create_source_new_after_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor_after_recovery = Box::new( - TopNExecutor::<_, false>::new( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (1, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor_after_recovery.execute(); + let schema = source.schema().clone(); + let top_n_executor_after_recovery = TopNExecutor::<_, false>::new( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (1, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor_after_recovery.boxed().execute(); // barrier assert_matches!( @@ -1057,9 +1000,9 @@ mod tests { use super::*; use crate::executor::test_utils::top_n_executor::create_in_memory_state_table_from_state_store; - use crate::executor::ActorContext; + use crate::executor::{ActorContext, Execute}; - fn create_source() -> Box { + fn create_source() -> Executor { let mut chunks = vec![ StreamChunk::from_pretty( " I I @@ -1094,18 +1037,15 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Chunk(std::mem::take(&mut chunks[2])), - Message::Chunk(std::mem::take(&mut chunks[3])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Chunk(std::mem::take(&mut chunks[2])), + Message::Chunk(std::mem::take(&mut chunks[3])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } fn storage_key() -> Vec { @@ -1131,24 +1071,18 @@ mod tests { &pk_indices(), ) .await; - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::new_with_ties_for_test( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (0, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::new_with_ties_for_test( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -1208,7 +1142,7 @@ mod tests { ); } - fn create_source_before_recovery() -> Box { + fn create_source_before_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I @@ -1235,19 +1169,16 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(1)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(2)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(1)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(2)), + ]) + .into_executor(schema, pk_indices()) } - fn create_source_after_recovery() -> Box { + fn create_source_after_recovery() -> Executor { let mut chunks = [ StreamChunk::from_pretty( " I I @@ -1264,16 +1195,13 @@ mod tests { Field::unnamed(DataType::Int64), ], }; - Box::new(MockSource::with_messages( - schema, - pk_indices(), - vec![ - Message::Barrier(Barrier::new_test_barrier(3)), - Message::Chunk(std::mem::take(&mut chunks[0])), - Message::Chunk(std::mem::take(&mut chunks[1])), - Message::Barrier(Barrier::new_test_barrier(4)), - ], - )) + MockSource::with_messages(vec![ + Message::Barrier(Barrier::new_test_barrier(3)), + Message::Chunk(std::mem::take(&mut chunks[0])), + Message::Chunk(std::mem::take(&mut chunks[1])), + Message::Barrier(Barrier::new_test_barrier(4)), + ]) + .into_executor(schema, pk_indices()) } #[tokio::test] @@ -1287,24 +1215,18 @@ mod tests { ) .await; let source = create_source_before_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor = Box::new( - TopNExecutor::new_with_ties_for_test( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (0, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor.execute(); + let schema = source.schema().clone(); + let top_n_executor = TopNExecutor::new_with_ties_for_test( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor.boxed().execute(); // consume the init barrier top_n_executor.next().await.unwrap().unwrap(); @@ -1350,24 +1272,18 @@ mod tests { // recovery let source = create_source_after_recovery(); - let info = ExecutorInfo { - schema: source.schema().clone(), - pk_indices: source.pk_indices().to_vec(), - identity: "TopNExecutor 1".to_string(), - }; - let top_n_executor_after_recovery = Box::new( - TopNExecutor::new_with_ties_for_test( - source as Box, - ActorContext::for_test(0), - info, - storage_key(), - (0, 3), - order_by(), - state_table, - ) - .unwrap(), - ); - let mut top_n_executor = top_n_executor_after_recovery.execute(); + let schema = source.schema().clone(); + let top_n_executor_after_recovery = TopNExecutor::new_with_ties_for_test( + source, + ActorContext::for_test(0), + schema, + storage_key(), + (0, 3), + order_by(), + state_table, + ) + .unwrap(); + let mut top_n_executor = top_n_executor_after_recovery.boxed().execute(); // barrier assert_matches!( diff --git a/src/stream/src/executor/top_n/utils.rs b/src/stream/src/executor/top_n/utils.rs index b2310f0d352b1..bbd956cde2168 100644 --- a/src/stream/src/executor/top_n/utils.rs +++ b/src/stream/src/executor/top_n/utils.rs @@ -30,8 +30,8 @@ use risingwave_common::util::sort_util::ColumnOrder; use super::CacheKey; use crate::executor::error::{StreamExecutorError, StreamExecutorResult}; use crate::executor::{ - expect_first_barrier, ActorContextRef, BoxedExecutor, BoxedMessageStream, Executor, - ExecutorInfo, Message, PkIndicesRef, Watermark, + expect_first_barrier, ActorContextRef, BoxedMessageStream, Execute, Executor, Message, + Watermark, }; pub trait TopNExecutorBase: Send + 'static { @@ -50,8 +50,6 @@ pub trait TopNExecutorBase: Send + 'static { /// Flush the buffered chunk to the storage backend. fn try_flush_data(&mut self) -> impl Future> + Send; - fn info(&self) -> &ExecutorInfo; - /// Update the vnode bitmap for the state table and manipulate the cache if necessary, only used /// by Group Top-N since it's distributed. fn update_vnode_bitmap(&mut self, _vnode_bitmap: Arc) { @@ -72,34 +70,18 @@ pub trait TopNExecutorBase: Send + 'static { /// The struct wraps a [`TopNExecutorBase`] pub struct TopNExecutorWrapper { - pub(super) input: BoxedExecutor, + pub(super) input: Executor, pub(super) ctx: ActorContextRef, pub(super) inner: E, } -impl Executor for TopNExecutorWrapper +impl Execute for TopNExecutorWrapper where E: TopNExecutorBase, { fn execute(self: Box) -> BoxedMessageStream { self.top_n_executor_execute().boxed() } - - fn schema(&self) -> &Schema { - &self.inner.info().schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.inner.info().pk_indices - } - - fn identity(&self) -> &str { - &self.inner.info().identity - } - - fn info(&self) -> ExecutorInfo { - self.inner.info().clone() - } } impl TopNExecutorWrapper diff --git a/src/stream/src/executor/union.rs b/src/stream/src/executor/union.rs index a3b3df6873e6b..acd35ce080cda 100644 --- a/src/stream/src/executor/union.rs +++ b/src/stream/src/executor/union.rs @@ -17,57 +17,37 @@ use std::pin::Pin; use std::task::{Context, Poll}; use futures::stream::{FusedStream, FuturesUnordered}; -use futures::StreamExt; +use futures::{Stream, StreamExt}; use futures_async_stream::try_stream; use pin_project::pin_project; -use risingwave_common::catalog::Schema; use super::watermark::BufferedWatermarks; -use super::*; -use crate::executor::{BoxedMessageStream, ExecutorInfo}; +use super::{ + Barrier, BoxedMessageStream, Execute, Executor, Message, MessageStreamItem, StreamExecutorError, +}; /// `UnionExecutor` merges data from multiple inputs. pub struct UnionExecutor { - info: ExecutorInfo, - inputs: Vec, + inputs: Vec, } impl std::fmt::Debug for UnionExecutor { fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result { - f.debug_struct("UnionExecutor") - .field("schema", &self.info.schema) - .field("pk_indices", &self.info.pk_indices) - .finish() + f.debug_struct("UnionExecutor").finish() } } impl UnionExecutor { - pub fn new(info: ExecutorInfo, inputs: Vec) -> Self { - Self { info, inputs } + pub fn new(inputs: Vec) -> Self { + Self { inputs } } } -impl Executor for UnionExecutor { +impl Execute for UnionExecutor { fn execute(self: Box) -> BoxedMessageStream { let streams = self.inputs.into_iter().map(|e| e.execute()).collect(); merge(streams).boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } #[pin_project] @@ -170,8 +150,10 @@ mod tests { use async_stream::try_stream; use risingwave_common::array::stream_chunk::StreamChunkTestExt; use risingwave_common::array::StreamChunk; + use risingwave_common::types::{DataType, ScalarImpl}; use super::*; + use crate::executor::Watermark; #[tokio::test] async fn union() { diff --git a/src/stream/src/executor/utils.rs b/src/stream/src/executor/utils.rs index 8ff9f8f29576d..18b64633a3e12 100644 --- a/src/stream/src/executor/utils.rs +++ b/src/stream/src/executor/utils.rs @@ -13,40 +13,19 @@ // limitations under the License. use futures::StreamExt; -use risingwave_common::catalog::Schema; use risingwave_common::metrics::LabelGuardedIntCounter; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{BoxedMessageStream, Executor, ExecutorInfo, PkIndicesRef}; +use crate::executor::{BoxedMessageStream, Execute}; use crate::task::{ActorId, FragmentId}; #[derive(Default)] -pub struct DummyExecutor { - pub info: ExecutorInfo, -} +pub struct DummyExecutor; -impl DummyExecutor { - pub fn new(info: ExecutorInfo) -> Self { - Self { info } - } -} - -impl Executor for DummyExecutor { +impl Execute for DummyExecutor { fn execute(self: Box) -> BoxedMessageStream { futures::stream::pending().boxed() } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } } pub(crate) struct ActorInputMetrics { diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index 2a927f1a3a780..d7deee71f1443 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -24,10 +24,7 @@ use risingwave_common::util::iter_util::ZipEqFast; use risingwave_expr::expr::NonStrictExpression; use tokio::sync::mpsc::UnboundedReceiver; -use super::{ - ActorContextRef, Barrier, BoxedMessageStream, Executor, ExecutorInfo, Message, PkIndicesRef, - StreamExecutorError, -}; +use super::{ActorContextRef, Barrier, BoxedMessageStream, Execute, Message, StreamExecutorError}; use crate::task::CreateMviewProgress; const DEFAULT_CHUNK_SIZE: usize = 1024; @@ -36,8 +33,8 @@ const DEFAULT_CHUNK_SIZE: usize = 1024; /// May refractor with `BarrierRecvExecutor` in the near future. pub struct ValuesExecutor { ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, // Receiver of barrier channel. barrier_receiver: UnboundedReceiver, progress: CreateMviewProgress, @@ -49,14 +46,14 @@ impl ValuesExecutor { /// Currently hard-code the `pk_indices` as the last column. pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, progress: CreateMviewProgress, rows: Vec>, barrier_receiver: UnboundedReceiver, ) -> Self { Self { ctx, - info, + schema, progress, barrier_receiver, rows: rows.into_iter(), @@ -64,9 +61,9 @@ impl ValuesExecutor { } #[try_stream(ok = Message, error = StreamExecutorError)] - async fn into_stream(self) { + async fn execute_inner(self) { let Self { - info, + schema, mut progress, mut barrier_receiver, mut rows, @@ -96,7 +93,7 @@ impl ValuesExecutor { } } - let cardinality = info.schema.len(); + let cardinality = schema.len(); ensure!(cardinality > 0); while !rows.is_empty() { // We need a one row chunk rather than an empty chunk because constant @@ -105,7 +102,7 @@ impl ValuesExecutor { let one_row_chunk = DataChunk::new_dummy(1); let chunk_size = DEFAULT_CHUNK_SIZE.min(rows.len()); - let mut array_builders = info.schema.create_array_builders(chunk_size); + let mut array_builders = schema.create_array_builders(chunk_size); for row in rows.by_ref().take(chunk_size) { for (expr, builder) in row.into_iter().zip_eq_fast(&mut array_builders) { let out = expr.eval_infallible(&one_row_chunk).await; @@ -135,21 +132,9 @@ impl ValuesExecutor { } } -impl Executor for ValuesExecutor { +impl Execute for ValuesExecutor { fn execute(self: Box) -> BoxedMessageStream { - self.into_stream().boxed() - } - - fn schema(&self) -> &Schema { - &self.info.schema - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity + self.execute_inner().boxed() } } @@ -167,7 +152,7 @@ mod tests { use super::ValuesExecutor; use crate::executor::test_utils::StreamExecutorTestExt; - use crate::executor::{ActorContext, AddMutation, Barrier, Executor, ExecutorInfo, Mutation}; + use crate::executor::{ActorContext, AddMutation, Barrier, Execute, Mutation}; use crate::task::{CreateMviewProgress, LocalBarrierManager}; #[tokio::test] @@ -206,15 +191,9 @@ mod tests { .iter() // for each column .map(|col| Field::unnamed(col.return_type())) .collect::(); - let pk_indices = vec![schema.len() - 1]; - let info = ExecutorInfo { - schema, - pk_indices, - identity: "ValuesExecutor".to_string(), - }; let values_executor_struct = ValuesExecutor::new( ActorContext::for_test(actor_id), - info, + schema, progress, vec![exprs .into_iter() diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index 1aa671b4f671c..4875d8a53c2d1 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -36,9 +36,7 @@ use risingwave_storage::StateStore; use super::error::StreamExecutorError; use super::filter::FilterExecutor; -use super::{ - ActorContextRef, BoxedExecutor, Executor, ExecutorInfo, Message, StreamExecutorResult, -}; +use super::{ActorContextRef, Execute, Executor, ExecutorInfo, Message, StreamExecutorResult}; use crate::common::table::state_table::StateTable; use crate::executor::{expect_first_barrier, Watermark}; use crate::task::ActorEvalErrorReport; @@ -48,59 +46,49 @@ use crate::task::ActorEvalErrorReport; /// filtered. pub struct WatermarkFilterExecutor { ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + input: Executor, /// The expression used to calculate the watermark value. watermark_expr: NonStrictExpression, /// The column we should generate watermark and filter on. event_time_col_idx: usize, table: StateTable, global_watermark_table: StorageTable, + + eval_error_report: ActorEvalErrorReport, } impl WatermarkFilterExecutor { pub fn new( ctx: ActorContextRef, - info: ExecutorInfo, - input: BoxedExecutor, + info: &ExecutorInfo, + input: Executor, watermark_expr: NonStrictExpression, event_time_col_idx: usize, table: StateTable, global_watermark_table: StorageTable, ) -> Self { + let eval_error_report = ActorEvalErrorReport { + actor_context: ctx.clone(), + identity: Arc::from(info.identity.as_ref()), + }; + Self { ctx, - info, input, watermark_expr, event_time_col_idx, table, global_watermark_table, + eval_error_report, } } } -impl Executor for WatermarkFilterExecutor { +impl Execute for WatermarkFilterExecutor { fn execute(self: Box) -> super::BoxedMessageStream { self.execute_inner().boxed() } - - fn schema(&self) -> &risingwave_common::catalog::Schema { - &self.info.schema - } - - fn pk_indices(&self) -> super::PkIndicesRef<'_> { - &self.info.pk_indices - } - - fn identity(&self) -> &str { - &self.info.identity - } - - fn info(&self) -> ExecutorInfo { - self.info.clone() - } } impl WatermarkFilterExecutor { @@ -111,16 +99,11 @@ impl WatermarkFilterExecutor { event_time_col_idx, watermark_expr, ctx, - info, mut table, mut global_watermark_table, + eval_error_report, } = *self; - let eval_error_report = ActorEvalErrorReport { - actor_context: ctx.clone(), - identity: info.identity.into(), - }; - let watermark_type = watermark_expr.return_type(); assert_eq!( watermark_type, @@ -456,7 +439,7 @@ mod tests { async fn create_watermark_filter_executor( mem_state: MemoryStateStore, - ) -> (BoxedExecutor, MessageSender) { + ) -> (Box, MessageSender) { let schema = Schema { fields: vec![ Field::unnamed(DataType::Int16), // pk @@ -476,7 +459,8 @@ mod tests { ) .await; - let (tx, source) = MockSource::channel(schema, vec![0]); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![0]); let info = ExecutorInfo { schema: source.schema().clone(), @@ -487,8 +471,8 @@ mod tests { ( WatermarkFilterExecutor::new( ActorContext::for_test(123), - info, - source.boxed(), + &info, + source, watermark_expr, 1, table, diff --git a/src/stream/src/executor/wrapper.rs b/src/stream/src/executor/wrapper.rs index ebd3f6d9bfa73..dddd94da5ab73 100644 --- a/src/stream/src/executor/wrapper.rs +++ b/src/stream/src/executor/wrapper.rs @@ -15,9 +15,8 @@ use std::sync::Arc; use futures::StreamExt; -use risingwave_common::catalog::Schema; -use super::*; +use super::{ActorContextRef, BoxedMessageStream, Execute, Executor, ExecutorInfo, MessageStream}; mod epoch_check; mod epoch_provide; @@ -27,7 +26,7 @@ mod update_check; /// [`WrapperExecutor`] will do some sanity checks and logging for the wrapped executor. pub struct WrapperExecutor { - input: BoxedExecutor, + input: Executor, actor_ctx: ActorContextRef, @@ -36,7 +35,7 @@ pub struct WrapperExecutor { impl WrapperExecutor { pub fn new( - input: BoxedExecutor, + input: Executor, actor_ctx: ActorContextRef, enable_executor_row_count: bool, ) -> Self { @@ -88,9 +87,9 @@ impl WrapperExecutor { } } -impl Executor for WrapperExecutor { +impl Execute for WrapperExecutor { fn execute(self: Box) -> BoxedMessageStream { - let info = Arc::new(self.input.info()); + let info = Arc::new(self.input.info().clone()); Self::wrap( self.enable_executor_row_count, info, @@ -101,7 +100,7 @@ impl Executor for WrapperExecutor { } fn execute_with_epoch(self: Box, epoch: u64) -> BoxedMessageStream { - let info = Arc::new(self.input.info()); + let info = Arc::new(self.input.info().clone()); Self::wrap( self.enable_executor_row_count, info, @@ -110,16 +109,4 @@ impl Executor for WrapperExecutor { ) .boxed() } - - fn schema(&self) -> &Schema { - self.input.schema() - } - - fn pk_indices(&self) -> PkIndicesRef<'_> { - self.input.pk_indices() - } - - fn identity(&self) -> &str { - self.input.identity() - } } diff --git a/src/stream/src/executor/wrapper/epoch_check.rs b/src/stream/src/executor/wrapper/epoch_check.rs index 732c67630a345..13d755dcbc60d 100644 --- a/src/stream/src/executor/wrapper/epoch_check.rs +++ b/src/stream/src/executor/wrapper/epoch_check.rs @@ -80,18 +80,18 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[tokio::test] async fn test_epoch_ok() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_barrier(1, false); tx.push_chunk(StreamChunk::default()); tx.push_barrier(2, false); tx.push_barrier(3, false); tx.push_barrier(4, false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert_matches!(checked.next().await.unwrap().unwrap(), Message::Barrier(b) if b.epoch.curr == 1); @@ -104,14 +104,15 @@ mod tests { #[should_panic] #[tokio::test] async fn test_epoch_bad() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_barrier(100, false); tx.push_chunk(StreamChunk::default()); tx.push_barrier(514, false); tx.push_barrier(514, false); tx.push_barrier(114, false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert_matches!(checked.next().await.unwrap().unwrap(), Message::Barrier(b) if b.epoch.curr == 100); @@ -125,11 +126,12 @@ mod tests { #[should_panic] #[tokio::test] async fn test_epoch_first_not_barrier() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::default()); tx.push_barrier(114, false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -137,9 +139,11 @@ mod tests { #[tokio::test] async fn test_empty() { - let (_, mut source) = MockSource::channel(Default::default(), vec![]); - source = source.stop_on_finish(false); - let checked = epoch_check(source.info().into(), source.boxed().execute()); + let (_, source) = MockSource::channel(); + let source = source + .stop_on_finish(false) + .into_executor(Default::default(), vec![]); + let checked = epoch_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert!(checked.next().await.transpose().unwrap().is_none()); diff --git a/src/stream/src/executor/wrapper/schema_check.rs b/src/stream/src/executor/wrapper/schema_check.rs index 462ef5962042e..ad5f0acd3dfaa 100644 --- a/src/stream/src/executor/wrapper/schema_check.rs +++ b/src/stream/src/executor/wrapper/schema_check.rs @@ -62,7 +62,6 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[tokio::test] async fn test_schema_ok() { @@ -73,7 +72,8 @@ mod tests { ], }; - let (mut tx, source) = MockSource::channel(schema, vec![1]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![1]); tx.push_chunk(StreamChunk::from_pretty( " I F + 100 200.0 @@ -82,7 +82,7 @@ mod tests { )); tx.push_barrier(1, false); - let checked = schema_check(source.info().into(), source.boxed().execute()); + let checked = schema_check(source.info().clone().into(), source.execute()); pin_mut!(checked); assert_matches!(checked.next().await.unwrap().unwrap(), Message::Chunk(_)); @@ -99,7 +99,8 @@ mod tests { ], }; - let (mut tx, source) = MockSource::channel(schema, vec![1]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![1]); tx.push_chunk(StreamChunk::from_pretty( " I I + 100 200 @@ -108,7 +109,7 @@ mod tests { )); tx.push_barrier(1, false); - let checked = schema_check(source.info().into(), source.boxed().execute()); + let checked = schema_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); } diff --git a/src/stream/src/executor/wrapper/update_check.rs b/src/stream/src/executor/wrapper/update_check.rs index 4049960845282..37a2ca4f2a942 100644 --- a/src/stream/src/executor/wrapper/update_check.rs +++ b/src/stream/src/executor/wrapper/update_check.rs @@ -62,12 +62,12 @@ mod tests { use super::*; use crate::executor::test_utils::MockSource; - use crate::executor::Executor; #[should_panic] #[tokio::test] async fn test_not_next_to_each_other() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::from_pretty( " I U- 114 @@ -76,7 +76,7 @@ mod tests { U+ 810", )); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -85,13 +85,14 @@ mod tests { #[should_panic] #[tokio::test] async fn test_first_one_update_insert() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::from_pretty( " I U+ 114", )); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -100,7 +101,8 @@ mod tests { #[should_panic] #[tokio::test] async fn test_last_one_update_delete() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::from_pretty( " I U- 114 @@ -108,7 +110,7 @@ mod tests { U- 1919810", )); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); // should panic @@ -116,10 +118,11 @@ mod tests { #[tokio::test] async fn test_empty_chunk() { - let (mut tx, source) = MockSource::channel(Default::default(), vec![]); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(Default::default(), vec![]); tx.push_chunk(StreamChunk::default()); - let checked = update_check(source.info().into(), source.boxed().execute()); + let checked = update_check(source.info().clone().into(), source.execute()); pin_mut!(checked); checked.next().await.unwrap().unwrap(); diff --git a/src/stream/src/from_proto/append_only_dedup.rs b/src/stream/src/from_proto/append_only_dedup.rs index 4ae2ee24ce798..020d13f3ff96b 100644 --- a/src/stream/src/from_proto/append_only_dedup.rs +++ b/src/stream/src/from_proto/append_only_dedup.rs @@ -20,7 +20,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{AppendOnlyDedupExecutor, BoxedExecutor}; +use crate::executor::{AppendOnlyDedupExecutor, Executor}; use crate::task::ExecutorParams; pub struct AppendOnlyDedupExecutorBuilder; @@ -32,18 +32,19 @@ impl ExecutorBuilder for AppendOnlyDedupExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let table = node.get_state_table()?; let vnodes = params.vnode_bitmap.map(Arc::new); let state_table = StateTable::from_table_catalog(table, store, vnodes).await; - Ok(Box::new(AppendOnlyDedupExecutor::new( + let exec = AppendOnlyDedupExecutor::new( + params.actor_context, input, + params.info.pk_indices.clone(), /* TODO(rc): should change to use `dedup_column_indices`, but need to check backward compatibility */ state_table, - params.info, - params.actor_context, params.watermark_epoch, params.executor_stats.clone(), - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/barrier_recv.rs b/src/stream/src/from_proto/barrier_recv.rs index 032afde2ee1dc..b8c38f0f53d32 100644 --- a/src/stream/src/from_proto/barrier_recv.rs +++ b/src/stream/src/from_proto/barrier_recv.rs @@ -27,7 +27,7 @@ impl ExecutorBuilder for BarrierRecvExecutorBuilder { params: ExecutorParams, _node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { assert!( params.input.is_empty(), "barrier receiver should not have input" @@ -38,6 +38,7 @@ impl ExecutorBuilder for BarrierRecvExecutorBuilder { .local_barrier_manager .register_sender(params.actor_context.id, sender); - Ok(BarrierRecvExecutor::new(params.actor_context, params.info, barrier_receiver).boxed()) + let exec = BarrierRecvExecutor::new(params.actor_context, barrier_receiver); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/batch_query.rs b/src/stream/src/from_proto/batch_query.rs index 4fc492f7c984d..5e86d7e6d5b30 100644 --- a/src/stream/src/from_proto/batch_query.rs +++ b/src/stream/src/from_proto/batch_query.rs @@ -30,12 +30,12 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { if node.table_desc.is_none() { // used in sharing cdc source backfill as a dummy batch plan node let mut info = params.info; info.identity = "DummyBatchQueryExecutor".to_string(); - return Ok(Box::new(DummyExecutor::new(info))); + return Ok((info, DummyExecutor).into()); } let table_desc: &StorageTableDesc = node.get_table_desc()?; @@ -55,9 +55,11 @@ impl ExecutorBuilder for BatchQueryExecutorBuilder { ); assert_eq!(table.schema().data_types(), params.info.schema.data_types()); - let executor = - BatchQueryExecutor::new(table, params.env.config().developer.chunk_size, params.info); - - Ok(executor.boxed()) + let exec = BatchQueryExecutor::new( + table, + params.env.config().developer.chunk_size, + params.info.schema.clone(), + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/cdc_filter.rs b/src/stream/src/from_proto/cdc_filter.rs index 5747e1894d4a8..673f6a1c5e712 100644 --- a/src/stream/src/from_proto/cdc_filter.rs +++ b/src/stream/src/from_proto/cdc_filter.rs @@ -28,11 +28,12 @@ impl ExecutorBuilder for CdcFilterExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let search_condition = build_non_strict_from_prost(node.get_search_condition()?, params.eval_error_report)?; - Ok(FilterExecutor::new(params.actor_context, params.info, input, search_condition).boxed()) + let exec = FilterExecutor::new(params.actor_context, input, search_condition); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/dml.rs b/src/stream/src/from_proto/dml.rs index d7f21fda7bf2d..cc8836890ce7f 100644 --- a/src/stream/src/from_proto/dml.rs +++ b/src/stream/src/from_proto/dml.rs @@ -20,7 +20,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::error::StreamResult; use crate::executor::dml::DmlExecutor; -use crate::executor::BoxedExecutor; +use crate::executor::Executor; use crate::task::ExecutorParams; pub struct DmlExecutorBuilder; @@ -32,19 +32,19 @@ impl ExecutorBuilder for DmlExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [upstream]: [_; 1] = params.input.try_into().unwrap(); let table_id = TableId::new(node.table_id); let column_descs = node.column_descs.iter().map(Into::into).collect_vec(); - Ok(Box::new(DmlExecutor::new( - params.info, + let exec = DmlExecutor::new( upstream, params.env.dml_manager_ref(), table_id, node.table_version_id, column_descs, params.env.config().developer.chunk_size, - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/dynamic_filter.rs b/src/stream/src/from_proto/dynamic_filter.rs index babb2680a9575..c09fd767ad9ef 100644 --- a/src/stream/src/from_proto/dynamic_filter.rs +++ b/src/stream/src/from_proto/dynamic_filter.rs @@ -32,7 +32,7 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [source_l, source_r]: [_; 2] = params.input.try_into().unwrap(); let key_l = node.get_left_key() as usize; @@ -62,7 +62,7 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { let left_table = node.get_left_table()?; let cleaned_by_watermark = left_table.get_cleaned_by_watermark(); - if cleaned_by_watermark { + let exec = if cleaned_by_watermark { let state_table_l = WatermarkCacheStateTable::from_table_catalog( node.get_left_table()?, store, @@ -70,9 +70,9 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { ) .await; - Ok(Box::new(DynamicFilterExecutor::new( + DynamicFilterExecutor::new( params.actor_context, - params.info, + ¶ms.info, source_l, source_r, key_l, @@ -83,14 +83,15 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { params.env.config().developer.chunk_size, condition_always_relax, cleaned_by_watermark, - ))) + ) + .boxed() } else { let state_table_l = StateTable::from_table_catalog(node.get_left_table()?, store, Some(vnodes)).await; - Ok(Box::new(DynamicFilterExecutor::new( + DynamicFilterExecutor::new( params.actor_context, - params.info, + ¶ms.info, source_l, source_r, key_l, @@ -101,7 +102,10 @@ impl ExecutorBuilder for DynamicFilterExecutorBuilder { params.env.config().developer.chunk_size, condition_always_relax, cleaned_by_watermark, - ))) - } + ) + .boxed() + }; + + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/eowc_over_window.rs b/src/stream/src/from_proto/eowc_over_window.rs index b2136f51c3970..4f6e873d7bcf3 100644 --- a/src/stream/src/from_proto/eowc_over_window.rs +++ b/src/stream/src/from_proto/eowc_over_window.rs @@ -21,9 +21,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{ - BoxedExecutor, EowcOverWindowExecutor, EowcOverWindowExecutorArgs, Executor, -}; +use crate::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs, Executor}; use crate::task::ExecutorParams; pub struct EowcOverWindowExecutorBuilder; @@ -35,7 +33,7 @@ impl ExecutorBuilder for EowcOverWindowExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let calls: Vec<_> = node .get_calls() @@ -56,18 +54,18 @@ impl ExecutorBuilder for EowcOverWindowExecutorBuilder { let state_table = StateTable::from_table_catalog_inconsistent_op(node.get_state_table()?, store, vnodes) .await; - Ok(EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { + let exec = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { actor_ctx: params.actor_context, - info: params.info, input, + schema: params.info.schema.clone(), calls, partition_key_indices, order_key_index, state_table, watermark_epoch: params.watermark_epoch, - }) - .boxed()) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/expand.rs b/src/stream/src/from_proto/expand.rs index 40bb972cdbd33..34d4313e24cb7 100644 --- a/src/stream/src/from_proto/expand.rs +++ b/src/stream/src/from_proto/expand.rs @@ -26,7 +26,7 @@ impl ExecutorBuilder for ExpandExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let column_subsets = node .column_subsets @@ -39,6 +39,6 @@ impl ExecutorBuilder for ExpandExecutorBuilder { .collect_vec() }) .collect_vec(); - Ok(ExpandExecutor::new(params.info, input, column_subsets).boxed()) + Ok((params.info, ExpandExecutor::new(input, column_subsets)).into()) } } diff --git a/src/stream/src/from_proto/filter.rs b/src/stream/src/from_proto/filter.rs index a82b9cc65dea7..a4efb953064b8 100644 --- a/src/stream/src/from_proto/filter.rs +++ b/src/stream/src/from_proto/filter.rs @@ -27,11 +27,12 @@ impl ExecutorBuilder for FilterExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let search_condition = build_non_strict_from_prost(node.get_search_condition()?, params.eval_error_report)?; - Ok(FilterExecutor::new(params.actor_context, params.info, input, search_condition).boxed()) + let exec = FilterExecutor::new(params.actor_context, input, search_condition); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/group_top_n.rs b/src/stream/src/from_proto/group_top_n.rs index a90e13d3f496b..51512cb9d94d3 100644 --- a/src/stream/src/from_proto/group_top_n.rs +++ b/src/stream/src/from_proto/group_top_n.rs @@ -14,6 +14,7 @@ use std::sync::Arc; +use risingwave_common::catalog::Schema; use risingwave_common::hash::{HashKey, HashKeyDispatcher}; use risingwave_common::types::DataType; use risingwave_common::util::sort_util::ColumnOrder; @@ -21,7 +22,7 @@ use risingwave_pb::stream_plan::GroupTopNNode; use super::*; use crate::common::table::state_table::StateTable; -use crate::executor::{ActorContextRef, AppendOnlyGroupTopNExecutor, GroupTopNExecutor}; +use crate::executor::{ActorContextRef, AppendOnlyGroupTopNExecutor, Executor, GroupTopNExecutor}; use crate::task::AtomicU64Ref; pub struct GroupTopNExecutorBuilder; @@ -33,7 +34,7 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder StreamResult { + ) -> StreamResult { let group_by: Vec = node .get_group_key() .iter() @@ -61,7 +62,7 @@ impl ExecutorBuilder for GroupTopNExecutorBuilder ExecutorBuilder for GroupTopNExecutorBuilder { - input: BoxedExecutor, + input: Executor, ctx: ActorContextRef, - info: ExecutorInfo, + schema: Schema, storage_key: Vec, offset_and_limit: (usize, usize), order_by: Vec, @@ -94,7 +95,7 @@ struct GroupTopNExecutorDispatcherArgs { } impl HashKeyDispatcher for GroupTopNExecutorDispatcherArgs { - type Output = StreamResult; + type Output = StreamResult>; fn dispatch_impl(self) -> Self::Output { macro_rules! build { @@ -102,7 +103,7 @@ impl HashKeyDispatcher for GroupTopNExecutorDispatcherArgs { Ok($excutor::::new( self.input, self.ctx, - self.info, + self.schema, self.storage_key, self.offset_and_limit, self.order_by, diff --git a/src/stream/src/from_proto/hash_agg.rs b/src/stream/src/from_proto/hash_agg.rs index 52390b6806d0e..3331640519617 100644 --- a/src/stream/src/from_proto/hash_agg.rs +++ b/src/stream/src/from_proto/hash_agg.rs @@ -35,7 +35,7 @@ pub struct HashAggExecutorDispatcherArgs { } impl HashKeyDispatcher for HashAggExecutorDispatcherArgs { - type Output = StreamResult; + type Output = StreamResult>; fn dispatch_impl(self) -> Self::Output { Ok(HashAggExecutor::::new(self.args)?.boxed()) @@ -55,7 +55,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let group_key_indices = node .get_group_key() .iter() @@ -93,13 +93,13 @@ impl ExecutorBuilder for HashAggExecutorBuilder { build_distinct_dedup_table_from_proto(node.get_distinct_dedup_tables(), store, vnodes) .await; - HashAggExecutorDispatcherArgs { + let exec = HashAggExecutorDispatcherArgs { args: AggExecutorArgs { version: node.version(), input, actor_ctx: params.actor_context, - info: params.info, + info: params.info.clone(), extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size, @@ -122,6 +122,7 @@ impl ExecutorBuilder for HashAggExecutorBuilder { }, group_key_types, } - .dispatch() + .dispatch()?; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/hash_join.rs b/src/stream/src/from_proto/hash_join.rs index 0c50f26f941b6..ad69fd1ecd5ac 100644 --- a/src/stream/src/from_proto/hash_join.rs +++ b/src/stream/src/from_proto/hash_join.rs @@ -27,7 +27,7 @@ use super::*; use crate::common::table::state_table::StateTable; use crate::executor::hash_join::*; use crate::executor::monitor::StreamingMetrics; -use crate::executor::{ActorContextRef, JoinType}; +use crate::executor::{ActorContextRef, Executor, JoinType}; use crate::task::AtomicU64Ref; pub struct HashJoinExecutorBuilder; @@ -39,7 +39,7 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let is_append_only = node.is_append_only; let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for hash join")); @@ -136,7 +136,7 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { let args = HashJoinExecutorDispatcherArgs { ctx: params.actor_context, - info: params.info, + info: params.info.clone(), source_l, source_r, params_l, @@ -157,15 +157,16 @@ impl ExecutorBuilder for HashJoinExecutorBuilder { chunk_size: params.env.config().developer.chunk_size, }; - args.dispatch() + let exec = args.dispatch()?; + Ok((params.info, exec).into()) } } struct HashJoinExecutorDispatcherArgs { ctx: ActorContextRef, info: ExecutorInfo, - source_l: Box, - source_r: Box, + source_l: Executor, + source_r: Executor, params_l: JoinParams, params_r: JoinParams, null_safe: Vec, @@ -185,34 +186,33 @@ struct HashJoinExecutorDispatcherArgs { } impl HashKeyDispatcher for HashJoinExecutorDispatcherArgs { - type Output = StreamResult; + type Output = StreamResult>; fn dispatch_impl(self) -> Self::Output { /// This macro helps to fill the const generic type parameter. macro_rules! build { ($join_type:ident) => { - Ok(Box::new( - HashJoinExecutor::::new( - self.ctx, - self.info, - self.source_l, - self.source_r, - self.params_l, - self.params_r, - self.null_safe, - self.output_indices, - self.cond, - self.inequality_pairs, - self.state_table_l, - self.degree_state_table_l, - self.state_table_r, - self.degree_state_table_r, - self.lru_manager, - self.is_append_only, - self.metrics, - self.chunk_size, - ), - )) + Ok(HashJoinExecutor::::new( + self.ctx, + self.info, + self.source_l, + self.source_r, + self.params_l, + self.params_r, + self.null_safe, + self.output_indices, + self.cond, + self.inequality_pairs, + self.state_table_l, + self.degree_state_table_l, + self.state_table_r, + self.degree_state_table_r, + self.lru_manager, + self.is_append_only, + self.metrics, + self.chunk_size, + ) + .boxed()) }; } match self.join_type_proto { diff --git a/src/stream/src/from_proto/hop_window.rs b/src/stream/src/from_proto/hop_window.rs index a56cbc2d8c976..2598ae927608e 100644 --- a/src/stream/src/from_proto/hop_window.rs +++ b/src/stream/src/from_proto/hop_window.rs @@ -27,16 +27,8 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { - let ExecutorParams { - actor_context, - info, - input, - env, - .. - } = params; - - let input = input.into_iter().next().unwrap(); + ) -> StreamResult { + let input = params.input.into_iter().next().unwrap(); // TODO: reuse the schema derivation with frontend. let output_indices = node .get_output_indices() @@ -59,11 +51,10 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { let window_slide = node.get_window_slide()?.into(); let window_size = node.get_window_size()?.into(); - let chunk_size = env.config().developer.chunk_size; + let chunk_size = params.env.config().developer.chunk_size; - Ok(HopWindowExecutor::new( - actor_context, - info, + let exec = HopWindowExecutor::new( + params.actor_context, input, time_col, window_slide, @@ -72,7 +63,7 @@ impl ExecutorBuilder for HopWindowExecutorBuilder { window_end_exprs, output_indices, chunk_size, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/lookup.rs b/src/stream/src/from_proto/lookup.rs index 1f873f31127c5..dc7f7e3c49dfe 100644 --- a/src/stream/src/from_proto/lookup.rs +++ b/src/stream/src/from_proto/lookup.rs @@ -30,7 +30,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let lookup = node; let [stream, arrangement]: [_; 2] = params.input.try_into().unwrap(); @@ -69,9 +69,9 @@ impl ExecutorBuilder for LookupExecutorBuilder { table_desc, ); - Ok(Box::new(LookupExecutor::new(LookupExecutorParams { + let exec = LookupExecutor::new(LookupExecutorParams { ctx: params.actor_context, - info: params.info, + info: params.info.clone(), arrangement, stream, arrangement_col_descs, @@ -83,6 +83,7 @@ impl ExecutorBuilder for LookupExecutorBuilder { storage_table, watermark_epoch: params.watermark_epoch, chunk_size: params.env.config().developer.chunk_size, - }))) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/lookup_union.rs b/src/stream/src/from_proto/lookup_union.rs index e9cc0ed33311a..93e847609cae2 100644 --- a/src/stream/src/from_proto/lookup_union.rs +++ b/src/stream/src/from_proto/lookup_union.rs @@ -26,7 +26,8 @@ impl ExecutorBuilder for LookupUnionExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { - Ok(LookupUnionExecutor::new(params.info, params.input, node.order.clone()).boxed()) + ) -> StreamResult { + let exec = LookupUnionExecutor::new(params.input, node.order.clone()); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/merge.rs b/src/stream/src/from_proto/merge.rs index aa0a6a9d2bb92..eded1f59d294e 100644 --- a/src/stream/src/from_proto/merge.rs +++ b/src/stream/src/from_proto/merge.rs @@ -27,7 +27,7 @@ impl ExecutorBuilder for MergeExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let upstreams = node.get_upstream_actor_id(); let upstream_fragment_id = node.get_upstream_fragment_id(); @@ -56,10 +56,9 @@ impl ExecutorBuilder for MergeExecutorBuilder { DispatcherType::NoShuffle => true, }; - if always_single_input { - Ok(ReceiverExecutor::new( + let exec = if always_single_input { + ReceiverExecutor::new( params.actor_context, - params.info, params.fragment_id, upstream_fragment_id, inputs.into_iter().exactly_one().unwrap(), @@ -67,11 +66,10 @@ impl ExecutorBuilder for MergeExecutorBuilder { params.operator_id, params.executor_stats.clone(), ) - .boxed()) + .boxed() } else { - Ok(MergeExecutor::new( + MergeExecutor::new( params.actor_context, - params.info, params.fragment_id, upstream_fragment_id, inputs, @@ -79,7 +77,8 @@ impl ExecutorBuilder for MergeExecutorBuilder { params.operator_id, params.executor_stats.clone(), ) - .boxed()) - } + .boxed() + }; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/mod.rs b/src/stream/src/from_proto/mod.rs index 46ec1a6cc8e59..c00a9fd844df7 100644 --- a/src/stream/src/from_proto/mod.rs +++ b/src/stream/src/from_proto/mod.rs @@ -93,7 +93,7 @@ use self::top_n::*; use self::union::*; use self::watermark_filter::WatermarkFilterBuilder; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, Executor, ExecutorInfo}; +use crate::executor::{Execute, Executor, ExecutorInfo}; use crate::from_proto::subscription::SubscriptionExecutorBuilder; use crate::from_proto::values::ValuesExecutorBuilder; use crate::task::ExecutorParams; @@ -101,12 +101,12 @@ use crate::task::ExecutorParams; trait ExecutorBuilder { type Node; - /// Create a [`BoxedExecutor`] from [`StreamNode`]. + /// Create an [`Executor`] from [`StreamNode`]. fn new_boxed_executor( params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> impl std::future::Future> + Send; + ) -> impl std::future::Future> + Send; } macro_rules! build_executor { @@ -127,7 +127,7 @@ pub async fn create_executor( params: ExecutorParams, node: &StreamNode, store: impl StateStore, -) -> StreamResult { +) -> StreamResult { build_executor! { params, node, diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index ecc54b20d8535..80aab576435ca 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -32,7 +32,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let order_key = node @@ -51,7 +51,7 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { ($SD:ident) => { MaterializeExecutor::<_, $SD>::new( input, - params.info, + params.info.schema.clone(), store, order_key, params.actor_context, @@ -66,13 +66,13 @@ impl ExecutorBuilder for MaterializeExecutorBuilder { }; } - let executor = if versioned { + let exec = if versioned { new_executor!(ColumnAwareSerde) } else { new_executor!(BasicSerde) }; - Ok(executor) + Ok((params.info, exec).into()) } } @@ -85,7 +85,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let keys = node @@ -102,9 +102,9 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { let vnodes = params.vnode_bitmap.map(Arc::new); let conflict_behavior = ConflictBehavior::from_protobuf(&table.handle_pk_conflict_behavior()); - let executor = MaterializeExecutor::<_, BasicSerde>::new( + let exec = MaterializeExecutor::<_, BasicSerde>::new( input, - params.info, + params.info.schema.clone(), store, keys, params.actor_context, @@ -116,6 +116,6 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { ) .await; - Ok(executor.boxed()) + Ok((params.info, exec.boxed()).into()) } } diff --git a/src/stream/src/from_proto/no_op.rs b/src/stream/src/from_proto/no_op.rs index 606b584d9109c..31494d771b903 100644 --- a/src/stream/src/from_proto/no_op.rs +++ b/src/stream/src/from_proto/no_op.rs @@ -17,7 +17,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, Executor, NoOpExecutor}; +use crate::executor::{Executor, NoOpExecutor}; use crate::task::ExecutorParams; pub struct NoOpExecutorBuilder; @@ -29,8 +29,8 @@ impl ExecutorBuilder for NoOpExecutorBuilder { params: ExecutorParams, _node: &NoOpNode, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); - Ok(NoOpExecutor::new(params.actor_context, params.info, input).boxed()) + Ok((params.info, NoOpExecutor::new(params.actor_context, input)).into()) } } diff --git a/src/stream/src/from_proto/now.rs b/src/stream/src/from_proto/now.rs index f622cdef1343e..06de0cacb7197 100644 --- a/src/stream/src/from_proto/now.rs +++ b/src/stream/src/from_proto/now.rs @@ -19,7 +19,7 @@ use tokio::sync::mpsc::unbounded_channel; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, NowExecutor}; +use crate::executor::{Executor, NowExecutor}; use crate::task::ExecutorParams; pub struct NowExecutorBuilder; @@ -31,7 +31,7 @@ impl ExecutorBuilder for NowExecutorBuilder { params: ExecutorParams, node: &NowNode, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let (sender, barrier_receiver) = unbounded_channel(); params .local_barrier_manager @@ -40,10 +40,11 @@ impl ExecutorBuilder for NowExecutorBuilder { let state_table = StateTable::from_table_catalog(node.get_state_table()?, store, None).await; - Ok(Box::new(NowExecutor::new( - params.info, + let exec = NowExecutor::new( + params.info.schema.data_types(), barrier_receiver, state_table, - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/over_window.rs b/src/stream/src/from_proto/over_window.rs index d2fc2893b6135..f7ca73c183a81 100644 --- a/src/stream/src/from_proto/over_window.rs +++ b/src/stream/src/from_proto/over_window.rs @@ -23,7 +23,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::common::table::state_table::StateTable; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, Executor, OverWindowExecutor, OverWindowExecutorArgs}; +use crate::executor::{Executor, OverWindowExecutor, OverWindowExecutorArgs}; use crate::task::ExecutorParams; pub struct OverWindowExecutorBuilder; @@ -35,7 +35,7 @@ impl ExecutorBuilder for OverWindowExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let calls: Vec<_> = node .get_calls() @@ -60,12 +60,12 @@ impl ExecutorBuilder for OverWindowExecutorBuilder { )); let state_table = StateTable::from_table_catalog(node.get_state_table()?, store, vnodes).await; - Ok(OverWindowExecutor::new(OverWindowExecutorArgs { + let exec = OverWindowExecutor::new(OverWindowExecutorArgs { actor_ctx: params.actor_context, - info: params.info, input, + schema: params.info.schema.clone(), calls, partition_key_indices, order_key_indices, @@ -79,7 +79,7 @@ impl ExecutorBuilder for OverWindowExecutorBuilder { cache_policy: OverWindowCachePolicy::from_protobuf( node.get_cache_policy().unwrap_or_default(), ), - }) - .boxed()) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/project.rs b/src/stream/src/from_proto/project.rs index 6fc9ada39aed8..d7f96c4dffcbf 100644 --- a/src/stream/src/from_proto/project.rs +++ b/src/stream/src/from_proto/project.rs @@ -30,7 +30,7 @@ impl ExecutorBuilder for ProjectExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let project_exprs: Vec<_> = node .get_select_list() @@ -60,15 +60,14 @@ impl ExecutorBuilder for ProjectExecutorBuilder { ) }); let materialize_selectivity_threshold = if extremely_light { 0.0 } else { 0.5 }; - Ok(ProjectExecutor::new( + let exec = ProjectExecutor::new( params.actor_context, - params.info, input, project_exprs, watermark_derivations, nondecreasing_expr_indices, materialize_selectivity_threshold, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/project_set.rs b/src/stream/src/from_proto/project_set.rs index b98e9e467ccf2..c2338394b33ef 100644 --- a/src/stream/src/from_proto/project_set.rs +++ b/src/stream/src/from_proto/project_set.rs @@ -29,7 +29,7 @@ impl ExecutorBuilder for ProjectSetExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let select_list: Vec<_> = node .get_select_list() @@ -55,15 +55,14 @@ impl ExecutorBuilder for ProjectSetExecutorBuilder { .collect(); let chunk_size = params.env.config().developer.chunk_size; - Ok(ProjectSetExecutor::new( + let exec = ProjectSetExecutor::new( params.actor_context, - params.info, input, select_list, chunk_size, watermark_derivations, nondecreasing_expr_indices, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/row_id_gen.rs b/src/stream/src/from_proto/row_id_gen.rs index 1333a99aebfa5..ab87566547ac6 100644 --- a/src/stream/src/from_proto/row_id_gen.rs +++ b/src/stream/src/from_proto/row_id_gen.rs @@ -18,7 +18,7 @@ use risingwave_storage::StateStore; use super::ExecutorBuilder; use crate::error::StreamResult; use crate::executor::row_id_gen::RowIdGenExecutor; -use crate::executor::BoxedExecutor; +use crate::executor::Executor; use crate::task::ExecutorParams; pub struct RowIdGenExecutorBuilder; @@ -30,19 +30,18 @@ impl ExecutorBuilder for RowIdGenExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [upstream]: [_; 1] = params.input.try_into().unwrap(); tracing::debug!("row id gen executor: {:?}", params.vnode_bitmap); let vnodes = params .vnode_bitmap .expect("vnodes not set for row id gen executor"); - let executor = RowIdGenExecutor::new( + let exec = RowIdGenExecutor::new( params.actor_context, - params.info, upstream, node.row_id_index as _, vnodes, ); - Ok(Box::new(executor)) + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/simple_agg.rs b/src/stream/src/from_proto/simple_agg.rs index 209a08daf6a22..16809edb8bcaf 100644 --- a/src/stream/src/from_proto/simple_agg.rs +++ b/src/stream/src/from_proto/simple_agg.rs @@ -34,7 +34,7 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let agg_calls: Vec = node .get_agg_calls() @@ -55,12 +55,12 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { build_distinct_dedup_table_from_proto(node.get_distinct_dedup_tables(), store, None) .await; - Ok(SimpleAggExecutor::new(AggExecutorArgs { + let exec = SimpleAggExecutor::new(AggExecutorArgs { version: node.version(), input, actor_ctx: params.actor_context, - info: params.info, + info: params.info.clone(), extreme_cache_size: params.env.config().developer.unsafe_extreme_cache_size, @@ -71,7 +71,8 @@ impl ExecutorBuilder for SimpleAggExecutorBuilder { distinct_dedup_tables, watermark_epoch: params.watermark_epoch, extra: SimpleAggExecutorExtraArgs {}, - })? - .boxed()) + })?; + + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/sink.rs b/src/stream/src/from_proto/sink.rs index 87d63cd80f599..884925d4a35ab 100644 --- a/src/stream/src/from_proto/sink.rs +++ b/src/stream/src/from_proto/sink.rs @@ -103,7 +103,7 @@ impl ExecutorBuilder for SinkExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input_executor]: [_; 1] = params.input.try_into().unwrap(); let sink_desc = node.sink_desc.as_ref().unwrap(); @@ -189,23 +189,22 @@ impl ExecutorBuilder for SinkExecutorBuilder { connector, sink_id.sink_id, params.executor_id ); - match node.log_store_type() { + let exec = match node.log_store_type() { // Default value is the normal in memory log store to be backward compatible with the // previously unset value SinkLogStoreType::InMemoryLogStore | SinkLogStoreType::Unspecified => { let factory = BoundedInMemLogStoreFactory::new(1); - Ok(Box::new( - SinkExecutor::new( - params.actor_context, - params.info, - input_executor, - sink_write_param, - sink_param, - columns, - factory, - ) - .await?, - )) + SinkExecutor::new( + params.actor_context, + params.info.clone(), + input_executor, + sink_write_param, + sink_param, + columns, + factory, + ) + .await? + .boxed() } SinkLogStoreType::KvLogStore => { let metrics = KvLogStoreMetrics::new( @@ -230,19 +229,20 @@ impl ExecutorBuilder for SinkExecutorBuilder { pk_info, ); - Ok(Box::new( - SinkExecutor::new( - params.actor_context, - params.info, - input_executor, - sink_write_param, - sink_param, - columns, - factory, - ) - .await?, - )) + SinkExecutor::new( + params.actor_context, + params.info.clone(), + input_executor, + sink_write_param, + sink_param, + columns, + factory, + ) + .await? + .boxed() } - } + }; + + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/sort.rs b/src/stream/src/from_proto/sort.rs index dcb59f77f9549..e901d66e5d8ff 100644 --- a/src/stream/src/from_proto/sort.rs +++ b/src/stream/src/from_proto/sort.rs @@ -29,18 +29,19 @@ impl ExecutorBuilder for SortExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let vnodes = Arc::new(params.vnode_bitmap.expect("vnodes not set for sort")); let state_table = StateTable::from_table_catalog(node.get_state_table()?, store, Some(vnodes)).await; - Ok(Box::new(SortExecutor::new(SortExecutorArgs { + let exec = SortExecutor::new(SortExecutorArgs { actor_ctx: params.actor_context, - info: params.info, + schema: params.info.schema.clone(), input, buffer_table: state_table, chunk_size: params.env.config().developer.chunk_size, sort_column_index: node.sort_column_index as _, - }))) + }); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/source/fs_fetch.rs b/src/stream/src/from_proto/source/fs_fetch.rs index 97cf26129d8d5..355f7d874dec8 100644 --- a/src/stream/src/from_proto/source/fs_fetch.rs +++ b/src/stream/src/from_proto/source/fs_fetch.rs @@ -25,7 +25,7 @@ use risingwave_storage::StateStore; use crate::error::StreamResult; use crate::executor::{ - BoxedExecutor, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler, + Execute, Executor, FlowControlExecutor, FsFetchExecutor, SourceStateTableHandler, StreamSourceCore, }; use crate::from_proto::ExecutorBuilder; @@ -40,7 +40,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [upstream]: [_; 1] = params.input.try_into().unwrap(); let source = node.node_inner.as_ref().unwrap(); @@ -61,6 +61,7 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { ); let source_ctrl_opts = SourceCtrlOpts { chunk_size: params.env.config().developer.chunk_size, + rate_limit: source.rate_limit.map(|x| x as _), }; let source_column_ids: Vec<_> = source_desc_builder @@ -88,11 +89,10 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { state_table_handler, ); - let executor = match properties { + let exec = match properties { risingwave_connector::source::ConnectorProperties::Gcs(_) => { FsFetchExecutor::<_, OpendalGcs>::new( params.actor_context.clone(), - params.info, stream_source_core, upstream, source_ctrl_opts, @@ -103,7 +103,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { risingwave_connector::source::ConnectorProperties::OpendalS3(_) => { FsFetchExecutor::<_, OpendalS3>::new( params.actor_context.clone(), - params.info, stream_source_core, upstream, source_ctrl_opts, @@ -114,7 +113,6 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { risingwave_connector::source::ConnectorProperties::PosixFs(_) => { FsFetchExecutor::<_, OpendalPosixFs>::new( params.actor_context.clone(), - params.info, stream_source_core, upstream, source_ctrl_opts, @@ -124,7 +122,11 @@ impl ExecutorBuilder for FsFetchExecutorBuilder { } _ => unreachable!(), }; + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); + let rate_limit = source.rate_limit.map(|x| x as _); - Ok(FlowControlExecutor::new(executor, params.actor_context, rate_limit).boxed()) + let exec = FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/source/trad_source.rs b/src/stream/src/from_proto/source/trad_source.rs index 28d923ffb69cc..7870bab10151b 100644 --- a/src/stream/src/from_proto/source/trad_source.rs +++ b/src/stream/src/from_proto/source/trad_source.rs @@ -22,8 +22,9 @@ use risingwave_connector::source::{ use risingwave_pb::data::data_type::TypeName as PbTypeName; use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType; use risingwave_pb::plan_common::{ - AdditionalColumn, AdditionalColumnKey, AdditionalColumnTimestamp, ColumnDescVersion, - FormatType, PbEncodeType, + AdditionalColumn, AdditionalColumnKey, AdditionalColumnTimestamp, + AdditionalColumnType as LegacyAdditionalColumnType, ColumnDescVersion, FormatType, + PbEncodeType, }; use risingwave_pb::stream_plan::SourceNode; use risingwave_storage::panic_store::PanicStateStore; @@ -45,7 +46,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let (sender, barrier_receiver) = unbounded_channel(); params .local_barrier_manager @@ -53,7 +54,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { let system_params = params.env.system_params_manager_ref().get_params(); if let Some(source) = &node.source_inner { - let executor = { + let exec = { let source_id = TableId::new(source.source_id); let source_name = source.source_name.clone(); let mut source_info = source.get_info()?.clone(); @@ -75,16 +76,16 @@ impl ExecutorBuilder for SourceExecutorBuilder { } let mut source_columns = source.columns.clone(); - { // compatible code: introduced in https://github.com/risingwavelabs/risingwave/pull/13707 // for upsert and (avro | protobuf) overwrite the `_rw_key` column's ColumnDesc.additional_column_type to Key if source_info.format() == FormatType::Upsert && (source_info.row_encode() == PbEncodeType::Avro - || source_info.row_encode() == PbEncodeType::Protobuf) + || source_info.row_encode() == PbEncodeType::Protobuf + || source_info.row_encode() == PbEncodeType::Json) { - let _ = source_columns.iter_mut().map(|c| { - let _ = c.column_desc.as_mut().map(|desc| { + for c in &mut source_columns { + if let Some(desc) = c.column_desc.as_mut() { let is_bytea = desc .get_column_type() .map(|col_type| col_type.type_name == PbTypeName::Bytea as i32) @@ -93,7 +94,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { &desc.version() ) && is_bytea - // the column is from a legacy version + // the column is from a legacy version (before v1.5.x) && desc.version == ColumnDescVersion::Unspecified as i32 { desc.additional_column = Some(AdditionalColumn { @@ -102,8 +103,20 @@ impl ExecutorBuilder for SourceExecutorBuilder { )), }); } - }); - }); + + // the column is from a legacy version (v1.6.x) + // introduced in https://github.com/risingwavelabs/risingwave/pull/15226 + if desc.additional_column_type + == LegacyAdditionalColumnType::Key as i32 + { + desc.additional_column = Some(AdditionalColumn { + column_type: Some(AdditionalColumnType::Key( + AdditionalColumnKey {}, + )), + }); + } + } + } } } @@ -157,6 +170,7 @@ impl ExecutorBuilder for SourceExecutorBuilder { let source_ctrl_opts = SourceCtrlOpts { chunk_size: params.env.config().developer.chunk_size, + rate_limit: source.rate_limit.map(|x| x as _), }; let source_column_ids: Vec<_> = source_desc_builder @@ -191,7 +205,6 @@ impl ExecutorBuilder for SourceExecutorBuilder { #[expect(deprecated)] crate::executor::FsSourceExecutor::new( params.actor_context.clone(), - params.info, stream_source_core, params.executor_stats, barrier_receiver, @@ -202,7 +215,6 @@ impl ExecutorBuilder for SourceExecutorBuilder { } else if is_fs_v2_connector { FsListExecutor::new( params.actor_context.clone(), - params.info.clone(), Some(stream_source_core), params.executor_stats.clone(), barrier_receiver, @@ -214,7 +226,6 @@ impl ExecutorBuilder for SourceExecutorBuilder { } else { SourceExecutor::new( params.actor_context.clone(), - params.info.clone(), Some(stream_source_core), params.executor_stats.clone(), barrier_receiver, @@ -225,14 +236,18 @@ impl ExecutorBuilder for SourceExecutorBuilder { .boxed() } }; + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); + let rate_limit = source.rate_limit.map(|x| x as _); - Ok(FlowControlExecutor::new(executor, params.actor_context, rate_limit).boxed()) + let exec = + FlowControlExecutor::new((info, exec).into(), params.actor_context, rate_limit); + Ok((params.info, exec).into()) } else { // If there is no external stream source, then no data should be persisted. We pass a // `PanicStateStore` type here for indication. - Ok(SourceExecutor::::new( + let exec = SourceExecutor::::new( params.actor_context, - params.info, None, params.executor_stats, barrier_receiver, @@ -240,8 +255,8 @@ impl ExecutorBuilder for SourceExecutorBuilder { // we don't expect any data in, so no need to set chunk_sizes SourceCtrlOpts::default(), params.env.connector_params(), - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } } diff --git a/src/stream/src/from_proto/stateless_simple_agg.rs b/src/stream/src/from_proto/stateless_simple_agg.rs index 87617b42f83e4..2ce5e612737e8 100644 --- a/src/stream/src/from_proto/stateless_simple_agg.rs +++ b/src/stream/src/from_proto/stateless_simple_agg.rs @@ -27,7 +27,7 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder { params: ExecutorParams, node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let agg_calls: Vec = node .get_agg_calls() @@ -35,9 +35,12 @@ impl ExecutorBuilder for StatelessSimpleAggExecutorBuilder { .map(AggCall::from_protobuf) .try_collect()?; - Ok( - StatelessSimpleAggExecutor::new(params.actor_context, params.info, input, agg_calls)? - .boxed(), - ) + let exec = StatelessSimpleAggExecutor::new( + params.actor_context, + input, + params.info.schema.clone(), + agg_calls, + )?; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/stream_cdc_scan.rs b/src/stream/src/from_proto/stream_cdc_scan.rs index 66e08c2bb4695..2736fdd712cb7 100644 --- a/src/stream/src/from_proto/stream_cdc_scan.rs +++ b/src/stream/src/from_proto/stream_cdc_scan.rs @@ -23,7 +23,7 @@ use risingwave_pb::stream_plan::StreamCdcScanNode; use super::*; use crate::common::table::state_table::StateTable; -use crate::executor::{CdcBackfillExecutor, ExternalStorageTable, FlowControlExecutor}; +use crate::executor::{CdcBackfillExecutor, Executor, ExternalStorageTable, FlowControlExecutor}; pub struct StreamCdcScanExecutorBuilder; @@ -34,7 +34,7 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [upstream]: [_; 1] = params.input.try_into().unwrap(); let output_indices = node @@ -95,9 +95,8 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { .map(|x| std::cmp::min(x as usize, chunk_size)) .unwrap_or(chunk_size); - let executor = CdcBackfillExecutor::new( + let exec = CdcBackfillExecutor::new( params.actor_context.clone(), - params.info, external_table, upstream, output_indices, @@ -106,14 +105,15 @@ impl ExecutorBuilder for StreamCdcScanExecutorBuilder { state_table, backfill_chunk_size, disable_backfill, - ) - .boxed(); + ); + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); - Ok(FlowControlExecutor::new( - executor, + let exec = FlowControlExecutor::new( + (info, exec).into(), params.actor_context, node.rate_limit.map(|x| x as _), - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/stream_scan.rs b/src/stream/src/from_proto/stream_scan.rs index c3f15802afc8d..5f0ae484e04c8 100644 --- a/src/stream/src/from_proto/stream_scan.rs +++ b/src/stream/src/from_proto/stream_scan.rs @@ -37,7 +37,7 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { params: ExecutorParams, node: &Self::Node, state_store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [upstream, snapshot]: [_; 2] = params.input.try_into().unwrap(); // For reporting the progress. let progress = params @@ -50,13 +50,13 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { .map(|&i| i as usize) .collect_vec(); - let executor = match node.stream_scan_type() { + let exec = match node.stream_scan_type() { StreamScanType::Chain | StreamScanType::UpstreamOnly => { let upstream_only = matches!(node.stream_scan_type(), StreamScanType::UpstreamOnly); - ChainExecutor::new(params.info, snapshot, upstream, progress, upstream_only).boxed() + ChainExecutor::new(snapshot, upstream, progress, upstream_only).boxed() } StreamScanType::Rearrange => { - RearrangedChainExecutor::new(params.info, snapshot, upstream, progress).boxed() + RearrangedChainExecutor::new(snapshot, upstream, progress).boxed() } StreamScanType::Backfill => { @@ -83,7 +83,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { StorageTable::new_partial(state_store.clone(), column_ids, vnodes, table_desc); BackfillExecutor::new( - params.info, upstream_table, upstream, state_table, @@ -126,7 +125,6 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { ) .await; ArrangementBackfillExecutor::<_, $SD>::new( - params.info, upstream_table, upstream, state_table, @@ -147,11 +145,14 @@ impl ExecutorBuilder for StreamScanExecutorBuilder { } StreamScanType::Unspecified => unreachable!(), }; - Ok(FlowControlExecutor::new( - executor, + let mut info = params.info.clone(); + info.identity = format!("{} (flow controlled)", info.identity); + + let exec = FlowControlExecutor::new( + (info, exec).into(), params.actor_context, node.rate_limit.map(|x| x as _), - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/subscription.rs b/src/stream/src/from_proto/subscription.rs index 4d5e6eb6855b1..ff0c1f8c1d084 100644 --- a/src/stream/src/from_proto/subscription.rs +++ b/src/stream/src/from_proto/subscription.rs @@ -21,7 +21,7 @@ use crate::common::log_store_impl::kv_log_store::serde::LogStoreRowSerde; use crate::common::log_store_impl::kv_log_store::KV_LOG_STORE_V2_INFO; use crate::common::log_store_impl::subscription_log_store::SubscriptionLogStoreWriter; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, SubscriptionExecutor}; +use crate::executor::{Executor, SubscriptionExecutor}; pub struct SubscriptionExecutorBuilder; @@ -32,7 +32,7 @@ impl ExecutorBuilder for SubscriptionExecutorBuilder { params: crate::task::ExecutorParams, node: &Self::Node, state_store: impl risingwave_storage::StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let table_id = TableId::new(node.log_store_table.as_ref().unwrap().id); let local_state_store = state_store @@ -61,15 +61,13 @@ impl ExecutorBuilder for SubscriptionExecutorBuilder { let log_store_identity = format!("subscription-executor[{}]", params.executor_id); let log_store = SubscriptionLogStoreWriter::new(table_id, local_state_store, serde, log_store_identity); - Ok(Box::new( - SubscriptionExecutor::new( - params.actor_context, - params.info, - input, - log_store, - node.retention_seconds, - ) - .await?, - )) + let exec = SubscriptionExecutor::new( + params.actor_context, + input, + log_store, + node.retention_seconds, + ) + .await?; + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/temporal_join.rs b/src/stream/src/from_proto/temporal_join.rs index 7c91b8c1d261d..15badec97e5cc 100644 --- a/src/stream/src/from_proto/temporal_join.rs +++ b/src/stream/src/from_proto/temporal_join.rs @@ -35,7 +35,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let table_desc: &StorageTableDesc = node.get_table_desc()?; let table = { let column_ids = table_desc @@ -101,7 +101,7 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { let dispatcher_args = TemporalJoinExecutorDispatcherArgs { ctx: params.actor_context, - info: params.info, + info: params.info.clone(), left: source_l, right: source_r, right_table: table, @@ -119,15 +119,15 @@ impl ExecutorBuilder for TemporalJoinExecutorBuilder { join_key_data_types, }; - dispatcher_args.dispatch() + Ok((params.info, dispatcher_args.dispatch()?).into()) } } struct TemporalJoinExecutorDispatcherArgs { ctx: ActorContextRef, info: ExecutorInfo, - left: BoxedExecutor, - right: BoxedExecutor, + left: Executor, + right: Executor, right_table: StorageTable, left_join_keys: Vec, right_join_keys: Vec, @@ -144,7 +144,7 @@ struct TemporalJoinExecutorDispatcherArgs { } impl HashKeyDispatcher for TemporalJoinExecutorDispatcherArgs { - type Output = StreamResult; + type Output = StreamResult>; fn dispatch_impl(self) -> Self::Output { /// This macro helps to fill the const generic type parameter. diff --git a/src/stream/src/from_proto/top_n.rs b/src/stream/src/from_proto/top_n.rs index 5c74f9f97e0eb..10071534e57df 100644 --- a/src/stream/src/from_proto/top_n.rs +++ b/src/stream/src/from_proto/top_n.rs @@ -30,7 +30,7 @@ impl ExecutorBuilder for TopNExecutorBuilder StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let table = node.get_table()?; @@ -52,7 +52,7 @@ impl ExecutorBuilder for TopNExecutorBuilder::new( input, params.actor_context, - params.info, + params.info.schema.clone(), storage_key, (node.offset as usize, node.limit as usize), order_by, @@ -62,11 +62,12 @@ impl ExecutorBuilder for TopNExecutorBuilder> = match (APPEND_ONLY, node.with_ties) { (true, true) => build!(AppendOnlyTopNExecutor, true), (true, false) => build!(AppendOnlyTopNExecutor, false), (false, true) => build!(TopNExecutor, true), (false, false) => build!(TopNExecutor, false), - } + }; + Ok((params.info, exec?).into()) } } diff --git a/src/stream/src/from_proto/union.rs b/src/stream/src/from_proto/union.rs index e11ea8f3f2d21..1360f92c1c83b 100644 --- a/src/stream/src/from_proto/union.rs +++ b/src/stream/src/from_proto/union.rs @@ -26,7 +26,7 @@ impl ExecutorBuilder for UnionExecutorBuilder { params: ExecutorParams, _node: &Self::Node, _store: impl StateStore, - ) -> StreamResult { - Ok(UnionExecutor::new(params.info, params.input).boxed()) + ) -> StreamResult { + Ok((params.info, UnionExecutor::new(params.input)).into()) } } diff --git a/src/stream/src/from_proto/values.rs b/src/stream/src/from_proto/values.rs index b3ed75e4f0903..28e9ca086414b 100644 --- a/src/stream/src/from_proto/values.rs +++ b/src/stream/src/from_proto/values.rs @@ -20,7 +20,7 @@ use tokio::sync::mpsc::unbounded_channel; use super::ExecutorBuilder; use crate::error::StreamResult; -use crate::executor::{BoxedExecutor, ValuesExecutor}; +use crate::executor::{Executor, ValuesExecutor}; use crate::task::ExecutorParams; /// Build a `ValuesExecutor` for stream. As is a leaf, current workaround registers a `sender` for @@ -34,7 +34,7 @@ impl ExecutorBuilder for ValuesExecutorBuilder { params: ExecutorParams, node: &ValuesNode, _store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let (sender, barrier_receiver) = unbounded_channel(); params .local_barrier_manager @@ -55,12 +55,13 @@ impl ExecutorBuilder for ValuesExecutorBuilder { .collect_vec() }) .collect_vec(); - Ok(Box::new(ValuesExecutor::new( + let exec = ValuesExecutor::new( params.actor_context, - params.info, + params.info.schema.clone(), progress, rows, barrier_receiver, - ))) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/from_proto/watermark_filter.rs b/src/stream/src/from_proto/watermark_filter.rs index cc3382bcc65fc..f01695e991487 100644 --- a/src/stream/src/from_proto/watermark_filter.rs +++ b/src/stream/src/from_proto/watermark_filter.rs @@ -34,7 +34,7 @@ impl ExecutorBuilder for WatermarkFilterBuilder { params: ExecutorParams, node: &Self::Node, store: impl StateStore, - ) -> StreamResult { + ) -> StreamResult { let [input]: [_; 1] = params.input.try_into().unwrap(); let watermark_descs = node.get_watermark_descs().clone(); let [watermark_desc]: [_; 1] = watermark_descs.try_into().unwrap(); @@ -63,15 +63,15 @@ impl ExecutorBuilder for WatermarkFilterBuilder { let table = StateTable::from_table_catalog_inconsistent_op(&table, store, Some(vnodes)).await; - Ok(WatermarkFilterExecutor::new( + let exec = WatermarkFilterExecutor::new( params.actor_context, - params.info, + ¶ms.info, input, watermark_expr, event_time_col_idx, table, global_watermark_table, - ) - .boxed()) + ); + Ok((params.info, exec).into()) } } diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index a0ad2a19c42ca..974375ebfa9fc 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -46,7 +46,10 @@ use crate::error::StreamResult; use crate::executor::exchange::permit::Receiver; use crate::executor::monitor::StreamingMetrics; use crate::executor::subtask::SubtaskHandle; -use crate::executor::*; +use crate::executor::{ + Actor, ActorContext, ActorContextRef, Barrier, DispatchExecutor, DispatcherImpl, Executor, + ExecutorInfo, WrapperExecutor, +}; use crate::from_proto::create_executor; use crate::task::barrier_manager::{EventSender, LocalActorOperation, LocalBarrierWorker}; use crate::task::{ @@ -104,7 +107,7 @@ pub struct ExecutorParams { pub op_info: String, /// The input executor. - pub input: Vec, + pub input: Vec, /// FragmentId of the actor pub fragment_id: FragmentId, @@ -370,7 +373,7 @@ impl StreamActorManager { /// Create dispatchers with downstream information registered before fn create_dispatcher( &self, - input: BoxedExecutor, + input: Executor, dispatchers: &[stream_plan::Dispatcher], actor_id: ActorId, fragment_id: FragmentId, @@ -405,7 +408,7 @@ impl StreamActorManager { has_stateful: bool, subtasks: &mut Vec, shared_context: &Arc, - ) -> StreamResult { + ) -> StreamResult { // The "stateful" here means that the executor may issue read operations to the state store // massively and continuously. Used to decide whether to apply the optimization of subtasks. fn is_stateful_executor(stream_node: &StreamNode) -> bool { @@ -457,21 +460,22 @@ impl StreamActorManager { let schema: Schema = node.fields.iter().map(Field::from).collect(); let identity = format!("{} {:X}", node.get_node_body().unwrap(), executor_id); + let info = ExecutorInfo { + schema, + pk_indices, + identity, + }; + let eval_error_report = ActorEvalErrorReport { actor_context: actor_context.clone(), - identity: identity.clone().into(), + identity: info.identity.clone().into(), }; // Build the executor with params. let executor_params = ExecutorParams { env: env.clone(), - info: ExecutorInfo { - schema: schema.clone(), - pk_indices: pk_indices.clone(), - identity: identity.clone(), - }, - + info: info.clone(), executor_id, operator_id, op_info, @@ -487,26 +491,14 @@ impl StreamActorManager { }; let executor = create_executor(executor_params, node, store).await?; - assert_eq!( - executor.pk_indices(), - &pk_indices, - "`pk_indices` of {} not consistent with what derived by optimizer", - executor.identity() - ); - assert_eq!( - executor.schema(), - &schema, - "`schema` of {} not consistent with what derived by optimizer", - executor.identity() - ); // Wrap the executor for debug purpose. - let executor = WrapperExecutor::new( + let wrapped = WrapperExecutor::new( executor, actor_context.clone(), env.config().developer.enable_executor_row_count, - ) - .boxed(); + ); + let executor = (info, wrapped).into(); // If there're multiple stateful executors in this actor, we will wrap it into a subtask. let executor = if has_stateful && is_stateful { @@ -533,7 +525,7 @@ impl StreamActorManager { actor_context: &ActorContextRef, vnode_bitmap: Option, shared_context: &Arc, - ) -> StreamResult<(BoxedExecutor, Vec)> { + ) -> StreamResult<(Executor, Vec)> { let mut subtasks = vec![]; let executor = dispatch_state_store!(env.state_store(), store, { diff --git a/src/stream/tests/integration_tests/eowc_over_window.rs b/src/stream/tests/integration_tests/eowc_over_window.rs index 6941d86f3a759..cc674e556ab5e 100644 --- a/src/stream/tests/integration_tests/eowc_over_window.rs +++ b/src/stream/tests/integration_tests/eowc_over_window.rs @@ -14,9 +14,7 @@ use risingwave_expr::aggregate::{AggArgs, AggKind}; use risingwave_expr::window_function::{Frame, FrameBound, WindowFuncCall, WindowFuncKind}; -use risingwave_stream::executor::{ - EowcOverWindowExecutor, EowcOverWindowExecutorArgs, ExecutorInfo, -}; +use risingwave_stream::executor::{EowcOverWindowExecutor, EowcOverWindowExecutorArgs}; use crate::prelude::*; @@ -54,7 +52,6 @@ async fn create_executor( }); Schema { fields } }; - let output_pk_indices = vec![2]; let state_table = StateTable::new_without_distribution_inconsistent_op( store, @@ -65,17 +62,14 @@ async fn create_executor( ) .await; - let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema, input_pk_indices.clone()); let executor = EowcOverWindowExecutor::new(EowcOverWindowExecutorArgs { actor_ctx: ActorContext::for_test(123), - info: ExecutorInfo { - schema: output_schema, - pk_indices: output_pk_indices, - identity: "EowcOverWindowExecutor".to_string(), - }, - input: source.boxed(), + input: source, + schema: output_schema, calls, partition_key_indices, order_key_index, diff --git a/src/stream/tests/integration_tests/hash_agg.rs b/src/stream/tests/integration_tests/hash_agg.rs index bf82a2986bf7d..aecf0ad4d5251 100644 --- a/src/stream/tests/integration_tests/hash_agg.rs +++ b/src/stream/tests/integration_tests/hash_agg.rs @@ -38,10 +38,11 @@ async fn test_hash_agg_count_sum() { AggCall::from_pretty("(sum:int8 $2:int8)"), ]; - let (mut tx, source) = MockSource::channel(schema, PkIndices::new()); + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); let hash_agg = new_boxed_hash_agg_executor( store, - Box::new(source), + source, false, agg_calls, 0, @@ -116,10 +117,11 @@ async fn test_hash_agg_min() { AggCall::from_pretty("(min:int8 $1:int8)"), ]; - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); let hash_agg = new_boxed_hash_agg_executor( store, - Box::new(source), + source, false, agg_calls, 0, @@ -191,10 +193,11 @@ async fn test_hash_agg_min_append_only() { AggCall::from_pretty("(min:int8 $1:int8)"), ]; - let (mut tx, source) = MockSource::channel(schema, vec![2]); // pk + let (mut tx, source) = MockSource::channel(); + let source = source.into_executor(schema, vec![2]); let hash_agg = new_boxed_hash_agg_executor( store, - Box::new(source), + source, true, // is append only agg_calls, 0, @@ -266,10 +269,11 @@ async fn test_hash_agg_emit_on_window_close() { let agg_calls = vec![AggCall::from_pretty("(count:int8)")]; let create_executor = || async { - let (tx, source) = MockSource::channel(input_schema.clone(), PkIndices::new()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema.clone(), PkIndices::new()); let hash_agg = new_boxed_hash_agg_executor( store.clone(), - Box::new(source), + source, false, agg_calls.clone(), 0, diff --git a/src/stream/tests/integration_tests/hop_window.rs b/src/stream/tests/integration_tests/hop_window.rs index f0bd65c84b69e..97e2886dbecda 100644 --- a/src/stream/tests/integration_tests/hop_window.rs +++ b/src/stream/tests/integration_tests/hop_window.rs @@ -16,7 +16,7 @@ use risingwave_common::types::test_utils::IntervalTestExt; use risingwave_common::types::{Interval, Timestamp}; use risingwave_expr::expr::test_utils::make_hop_window_expression; use risingwave_expr::expr::NonStrictExpression; -use risingwave_stream::executor::{ExecutorInfo, HopWindowExecutor}; +use risingwave_stream::executor::HopWindowExecutor; use crate::prelude::*; @@ -24,12 +24,15 @@ const TIME_COL_IDX: usize = 2; const CHUNK_SIZE: usize = 256; fn create_executor(output_indices: Vec) -> (MessageSender, BoxedMessageStream) { - let field1 = Field::unnamed(DataType::Int64); - let field2 = Field::unnamed(DataType::Int64); - let field3 = Field::with_name(DataType::Timestamp, "created_at"); - let schema = Schema::new(vec![field1, field2, field3]); - let pk_indices = vec![0]; - let (tx, source) = MockSource::channel(schema.clone(), pk_indices.clone()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor( + Schema::new(vec![ + Field::unnamed(DataType::Int64), + Field::unnamed(DataType::Int64), + Field::with_name(DataType::Timestamp, "created_at"), + ]), + vec![0], + ); let window_slide = Interval::from_minutes(15); let window_size = Interval::from_minutes(30); @@ -47,12 +50,7 @@ fn create_executor(output_indices: Vec) -> (MessageSender, BoxedMessageSt tx, HopWindowExecutor::new( ActorContext::for_test(123), - ExecutorInfo { - schema, - pk_indices, - identity: "HopWindowExecutor".to_string(), - }, - Box::new(source), + source, TIME_COL_IDX, window_slide, window_size, diff --git a/src/stream/tests/integration_tests/over_window.rs b/src/stream/tests/integration_tests/over_window.rs index 78203d79a87ac..0be8e1848e9cd 100644 --- a/src/stream/tests/integration_tests/over_window.rs +++ b/src/stream/tests/integration_tests/over_window.rs @@ -18,7 +18,7 @@ use risingwave_expr::window_function::{ Frame, FrameBound, FrameExclusion, WindowFuncCall, WindowFuncKind, }; use risingwave_stream::executor::monitor::StreamingMetrics; -use risingwave_stream::executor::{ExecutorInfo, OverWindowExecutor, OverWindowExecutorArgs}; +use risingwave_stream::executor::{OverWindowExecutor, OverWindowExecutorArgs}; use crate::prelude::*; @@ -63,7 +63,6 @@ async fn create_executor( }); Schema { fields } }; - let output_pk_indices = vec![2]; let state_table = StateTable::new_without_distribution( store, @@ -74,17 +73,14 @@ async fn create_executor( ) .await; - let (tx, source) = MockSource::channel(input_schema, input_pk_indices.clone()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(input_schema, input_pk_indices.clone()); let executor = OverWindowExecutor::new(OverWindowExecutorArgs { actor_ctx: ActorContext::for_test(123), - info: ExecutorInfo { - schema: output_schema, - pk_indices: output_pk_indices, - identity: "OverWindowExecutor".to_string(), - }, - input: source.boxed(), + input: source, + schema: output_schema, calls, partition_key_indices, order_key_indices, diff --git a/src/stream/tests/integration_tests/project_set.rs b/src/stream/tests/integration_tests/project_set.rs index 79fcbfc0d48db..5cff03284f4b6 100644 --- a/src/stream/tests/integration_tests/project_set.rs +++ b/src/stream/tests/integration_tests/project_set.rs @@ -14,7 +14,7 @@ use multimap::MultiMap; use risingwave_expr::table_function::repeat; -use risingwave_stream::executor::{ExecutorInfo, ProjectSetExecutor}; +use risingwave_stream::executor::ProjectSetExecutor; use crate::prelude::*; @@ -27,30 +27,17 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) { Field::unnamed(DataType::Int64), ], }; - let (tx, source) = MockSource::channel(schema, PkIndices::new()); + let (tx, source) = MockSource::channel(); + let source = source.into_executor(schema, PkIndices::new()); let test_expr = build_from_pretty("(add:int8 $0:int8 $1:int8)").into_inner(); let test_expr_watermark = build_from_pretty("(add:int8 $0:int8 1:int8)").into_inner(); let tf1 = repeat(build_from_pretty("1:int4").into_inner(), 1); let tf2 = repeat(build_from_pretty("2:int4").into_inner(), 2); - let info = ExecutorInfo { - schema: Schema { - fields: vec![ - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int64), - Field::unnamed(DataType::Int32), - Field::unnamed(DataType::Int32), - ], - }, - pk_indices: vec![], - identity: "ProjectSetExecutor".to_string(), - }; - - let project_set = Box::new(ProjectSetExecutor::new( + let project_set = ProjectSetExecutor::new( ActorContext::for_test(123), - info, - Box::new(source), + source, vec![ test_expr.into(), test_expr_watermark.into(), @@ -60,8 +47,8 @@ fn create_executor() -> (MessageSender, BoxedMessageStream) { CHUNK_SIZE, MultiMap::from_iter(std::iter::once((0, 1))), vec![], - )); - (tx, project_set.execute()) + ); + (tx, project_set.boxed().execute()) } #[tokio::test] diff --git a/src/tests/regress/src/lib.rs b/src/tests/regress/src/lib.rs index efdd0f1422c00..a18afdebc843f 100644 --- a/src/tests/regress/src/lib.rs +++ b/src/tests/regress/src/lib.rs @@ -27,6 +27,9 @@ #![deny(rustdoc::broken_intra_doc_links)] #![feature(path_file_prefix)] #![feature(let_chains)] +#![feature(register_tool)] +#![register_tool(rw)] +#![allow(rw::format_error)] mod opts; diff --git a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs index 4907a2adb2a7c..4c6f743f08f8b 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/background_ddl.rs @@ -15,6 +15,7 @@ use std::time::Duration; use anyhow::{anyhow, Result}; +use risingwave_common::error::v2::AsReport; use risingwave_simulation::cluster::{Cluster, Configuration, Session}; use tokio::time::sleep; @@ -348,7 +349,7 @@ async fn test_high_barrier_latency_cancel(config: Configuration) -> Result<()> { .run("CREATE MATERIALIZED VIEW mv1 as values(1)") .await { - tracing::info!("Recreate mv failed with {e:?}"); + tracing::info!(error = %e.as_report(), "Recreate mv failed"); continue; } else { tracing::info!("recreated mv");