diff --git a/.gitignore b/.gitignore index 5f54a467b21b7..81e8609f3334e 100644 --- a/.gitignore +++ b/.gitignore @@ -83,7 +83,7 @@ e2e_test/iceberg/metastore_db **/*.sqlite **/*.sqlite-journal -*.slt.temp +*.slt*.temp .direnv/ diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index b86de52c6014c..b55575d6eb1a8 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -38,7 +38,8 @@ apt-get -y install jq echo "--- e2e, inline test" RUST_LOG="debug,risingwave_stream=info,risingwave_batch=info,risingwave_storage=info" \ risedev ci-start ci-inline-source-test -risedev slt './e2e_test/source_inline/**/*.slt' +risedev slt './e2e_test/source_inline/**/*.slt' -j16 +risedev slt './e2e_test/source_inline/**/*.slt.serial' echo "--- Kill cluster" risedev ci-kill diff --git a/ci/scripts/run-backfill-tests.sh b/ci/scripts/run-backfill-tests.sh index b589b148dd945..46655c9633910 100755 --- a/ci/scripts/run-backfill-tests.sh +++ b/ci/scripts/run-backfill-tests.sh @@ -27,7 +27,7 @@ CLUSTER_PROFILE='ci-1cn-1fe-user-kafka-with-recovery' echo "--- Configuring cluster profiles" if [[ -n "${BUILDKITE:-}" ]]; then echo "Running in buildkite" - RUNTIME_CLUSTER_PROFILE='ci-3cn-1fe' + RUNTIME_CLUSTER_PROFILE='ci-backfill-3cn-1fe' MINIO_RATE_LIMIT_CLUSTER_PROFILE='ci-backfill-3cn-1fe-with-minio-rate-limit' else echo "Running locally" diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 278436a884c70..73e84f371c35e 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -33,7 +33,6 @@ user create_compaction_group_for_mv user datestyle user enable_join_ordering user enable_share_plan -user enable_shared_source user enable_two_phase_agg user extra_float_digits user force_split_distinct_agg @@ -61,6 +60,7 @@ user streaming_max_parallelism user streaming_over_window_cache_policy user streaming_parallelism user streaming_use_arrangement_backfill +user streaming_use_shared_source user streaming_use_snapshot_backfill user synchronize_seqscans user timezone diff --git a/e2e_test/source_inline/README.md b/e2e_test/source_inline/README.md index 8f766c5637fb6..fa8cf25c56b0f 100644 --- a/e2e_test/source_inline/README.md +++ b/e2e_test/source_inline/README.md @@ -5,6 +5,19 @@ Compared with prior source tests ( `e2e_test/source` ), tests in this directory See the [connector development guide](http://risingwavelabs.github.io/risingwave/connector/intro.html#end-to-end-tests) for more information about how to set up the test environment, run tests, and write tests. +## Serial Tests + +Tests ending with `.slt.serial` can only be run in serial, e.g., it contains `RECOVER` statement which will break other connections, or it has some special `system` commands. + +Other tests can be run in parallel. + +```bash +# run all parallel tests +risedev slt './e2e_test/source_inline/**/*.slt' -j16 +# run all serial tests +risedev slt './e2e_test/source_inline/**/*.slt.serial' +``` + ## Install Dependencies Some additional tools are needed to run the `system` commands in tests. diff --git a/e2e_test/source_inline/cdc/mysql/mysql_create_drop.slt b/e2e_test/source_inline/cdc/mysql/mysql_create_drop.slt.serial similarity index 100% rename from e2e_test/source_inline/cdc/mysql/mysql_create_drop.slt rename to e2e_test/source_inline/cdc/mysql/mysql_create_drop.slt.serial diff --git a/e2e_test/source_inline/kafka/alter/add_column.slt b/e2e_test/source_inline/kafka/alter/add_column.slt index cde818f6baf90..9d0bbe671079d 100644 --- a/e2e_test/source_inline/kafka/alter/add_column.slt +++ b/e2e_test/source_inline/kafka/alter/add_column.slt @@ -1,5 +1,8 @@ control substitution on +statement ok +SET streaming_use_shared_source TO false; + system ok rpk topic delete kafka_alter || true @@ -269,3 +272,6 @@ select * from t statement ok drop table t; + +statement ok +SET streaming_use_shared_source TO true; diff --git a/e2e_test/source_inline/kafka/alter/rate_limit_source_kafka.slt b/e2e_test/source_inline/kafka/alter/rate_limit_source_kafka.slt index 9bfa7238e3476..96fd016c5812d 100644 --- a/e2e_test/source_inline/kafka/alter/rate_limit_source_kafka.slt +++ b/e2e_test/source_inline/kafka/alter/rate_limit_source_kafka.slt @@ -1,5 +1,8 @@ control substitution on +statement ok +SET streaming_use_shared_source TO false; + ############## Create kafka seed data statement ok @@ -127,3 +130,6 @@ drop sink kafka_sink; statement ok drop table kafka_seed_data; + +statement ok +SET streaming_use_shared_source TO true; diff --git a/e2e_test/source_inline/kafka/avro/alter_source.slt b/e2e_test/source_inline/kafka/avro/alter_source.slt index 57677af57cd92..5a73941dc7897 100644 --- a/e2e_test/source_inline/kafka/avro/alter_source.slt +++ b/e2e_test/source_inline/kafka/avro/alter_source.slt @@ -1,5 +1,8 @@ control substitution on +statement ok +SET streaming_use_shared_source TO false; + # https://github.com/risingwavelabs/risingwave/issues/16486 # cleanup @@ -66,3 +69,6 @@ ABC 1 statement ok drop source s cascade; + +statement ok +SET streaming_use_shared_source TO true; diff --git a/e2e_test/source_inline/kafka/avro/glue.slt b/e2e_test/source_inline/kafka/avro/glue.slt index 4be9378b530df..a4359e9a7c124 100644 --- a/e2e_test/source_inline/kafka/avro/glue.slt +++ b/e2e_test/source_inline/kafka/avro/glue.slt @@ -1,5 +1,8 @@ control substitution on +statement ok +SET streaming_use_shared_source TO false; + system ok rpk topic delete 'glue-sample-my-event' @@ -14,6 +17,8 @@ EOF statement ok ALTER SYSTEM SET license_key TO ''; +sleep 1s + statement error create source t with ( connector = 'kafka', @@ -146,3 +151,6 @@ drop source t; system ok rpk topic delete 'glue-sample-my-event' + +statement ok +SET streaming_use_shared_source TO true; diff --git a/e2e_test/source_inline/kafka/consumer_group.slt b/e2e_test/source_inline/kafka/consumer_group.slt.serial similarity index 100% rename from e2e_test/source_inline/kafka/consumer_group.slt rename to e2e_test/source_inline/kafka/consumer_group.slt.serial diff --git a/e2e_test/source_inline/kafka/protobuf/alter_source.slt b/e2e_test/source_inline/kafka/protobuf/alter_source.slt index e94ffc9541504..7f1b431a06bac 100644 --- a/e2e_test/source_inline/kafka/protobuf/alter_source.slt +++ b/e2e_test/source_inline/kafka/protobuf/alter_source.slt @@ -1,18 +1,21 @@ control substitution on +statement ok +SET streaming_use_shared_source TO false; + system ok -rpk topic delete sr_pb_test || true; \ -(rpk sr subject delete 'sr_pb_test-value' && rpk sr subject delete 'sr_pb_test-value' --permanent) || true; +rpk topic delete pb_alter_source_test || true; \ +(rpk sr subject delete 'pb_alter_source_test-value' && rpk sr subject delete 'pb_alter_source_test-value' --permanent) || true; system ok -python3 e2e_test/source_inline/kafka/protobuf/pb.py "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" "${RISEDEV_SCHEMA_REGISTRY_URL}" "sr_pb_test" 20 user +python3 e2e_test/source_inline/kafka/protobuf/pb.py "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" "${RISEDEV_SCHEMA_REGISTRY_URL}" "pb_alter_source_test" 20 user statement ok CREATE SOURCE src_user INCLUDE timestamp -- include explicitly here to test a bug found in https://github.com/risingwavelabs/risingwave/pull/17293 WITH ( ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, - topic = 'sr_pb_test', + topic = 'pb_alter_source_test', scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE PROTOBUF( @@ -26,7 +29,7 @@ CREATE MATERIALIZED VIEW mv_user AS SELECT * FROM src_user; statement ok CREATE TABLE t_user WITH ( ${RISEDEV_KAFKA_WITH_OPTIONS_COMMON}, - topic = 'sr_pb_test', + topic = 'pb_alter_source_test', scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE PROTOBUF( @@ -42,7 +45,7 @@ SELECT age FROM t_user; # Push more events with extended fields system ok -python3 e2e_test/source_inline/kafka/protobuf/pb.py "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" "${RISEDEV_SCHEMA_REGISTRY_URL}" "sr_pb_test" 5 user_with_more_fields +python3 e2e_test/source_inline/kafka/protobuf/pb.py "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" "${RISEDEV_SCHEMA_REGISTRY_URL}" "pb_alter_source_test" 5 user_with_more_fields sleep 5s @@ -69,7 +72,7 @@ SELECT COUNT(*), MAX(age), MIN(age), SUM(age) FROM t_user; # Push more events with extended fields system ok -python3 e2e_test/source_inline/kafka/protobuf/pb.py "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" "${RISEDEV_SCHEMA_REGISTRY_URL}" "sr_pb_test" 5 user_with_more_fields +python3 e2e_test/source_inline/kafka/protobuf/pb.py "${RISEDEV_KAFKA_BOOTSTRAP_SERVERS}" "${RISEDEV_SCHEMA_REGISTRY_URL}" "pb_alter_source_test" 5 user_with_more_fields sleep 5s @@ -89,3 +92,6 @@ DROP MATERIALIZED VIEW mv_user; statement ok DROP SOURCE src_user; + +statement ok +SET streaming_use_shared_source TO true; diff --git a/e2e_test/source_inline/kafka/protobuf/recover.slt b/e2e_test/source_inline/kafka/protobuf/recover.slt.serial similarity index 100% rename from e2e_test/source_inline/kafka/protobuf/recover.slt rename to e2e_test/source_inline/kafka/protobuf/recover.slt.serial diff --git a/e2e_test/source_inline/kafka/shared_source.slt b/e2e_test/source_inline/kafka/shared_source.slt.serial similarity index 79% rename from e2e_test/source_inline/kafka/shared_source.slt rename to e2e_test/source_inline/kafka/shared_source.slt.serial index ca429781604e8..3397f90f081da 100644 --- a/e2e_test/source_inline/kafka/shared_source.slt +++ b/e2e_test/source_inline/kafka/shared_source.slt.serial @@ -1,7 +1,7 @@ control substitution on statement ok -SET enable_shared_source TO true; +SET streaming_use_shared_source TO true; system ok rpk topic create shared_source -p 4 @@ -14,6 +14,11 @@ create source s_before_produce (v1 int, v2 varchar) with ( scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE JSON; +query T +select connector, is_shared from rw_sources where name = 's_before_produce'; +---- +KAFKA t + statement ok create materialized view mv_before_produce as select * from s_before_produce; @@ -23,10 +28,10 @@ sleep 2s system ok internal_table.mjs --name mv_before_produce --type sourcebackfill ---- -0,"""Finished""" -1,"""Finished""" -2,"""Finished""" -3,"""Finished""" +0,"{""num_consumed_rows"": 0, ""state"": ""Finished"", ""target_offset"": null}" +1,"{""num_consumed_rows"": 0, ""state"": ""Finished"", ""target_offset"": null}" +2,"{""num_consumed_rows"": 0, ""state"": ""Finished"", ""target_offset"": null}" +3,"{""num_consumed_rows"": 0, ""state"": ""Finished"", ""target_offset"": null}" system ok @@ -34,6 +39,9 @@ cat << EOF | rpk topic produce shared_source -f "%p %v\n" -p 0 0 {"v1": 1, "v2": "a"} 1 {"v1": 2, "v2": "b"} 2 {"v1": 3, "v2": "c"} +2 {"v1": 3, "v2": "c"} +3 {"v1": 4, "v2": "d"} +3 {"v1": 4, "v2": "d"} 3 {"v1": 4, "v2": "d"} EOF @@ -78,15 +86,15 @@ internal_table.mjs --name s0 --type source system ok internal_table.mjs --name mv_1 --type sourcebackfill ---- -0,"{""SourceCachingUp"": ""0""}" -1,"{""SourceCachingUp"": ""0""}" -2,"{""SourceCachingUp"": ""0""}" -3,"{""SourceCachingUp"": ""0""}" +0,"{""num_consumed_rows"": 1, ""state"": {""SourceCachingUp"": ""0""}, ""target_offset"": ""0""}" +1,"{""num_consumed_rows"": 1, ""state"": {""SourceCachingUp"": ""0""}, ""target_offset"": ""0""}" +2,"{""num_consumed_rows"": 2, ""state"": {""SourceCachingUp"": ""1""}, ""target_offset"": ""1""}" +3,"{""num_consumed_rows"": 3, ""state"": {""SourceCachingUp"": ""2""}, ""target_offset"": ""2""}" # This does not affect the behavior for CREATE MATERIALIZED VIEW below. It also uses the shared source, and creates SourceBackfillExecutor. statement ok -SET enable_shared_source TO false; +SET streaming_use_shared_source TO false; statement ok create materialized view mv_2 as select * from s0; @@ -96,26 +104,35 @@ sleep 2s query ?? rowsort select v1, v2 from s0; ---- -1 a -2 b -3 c -4 d +1 a +2 b +3 c +3 c +4 d +4 d +4 d query ?? rowsort select v1, v2 from mv_1; ---- -1 a -2 b -3 c -4 d +1 a +2 b +3 c +3 c +4 d +4 d +4 d query ?? rowsort select v1, v2 from mv_2; ---- -1 a -2 b -3 c -4 d +1 a +2 b +3 c +3 c +4 d +4 d +4 d system ok cat << EOF | rpk topic produce shared_source -f "%p %v\n" -p 0 @@ -133,33 +150,39 @@ internal_table.mjs --name s0 --type source ---- 0,"{""split_info"": {""partition"": 0, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" 1,"{""split_info"": {""partition"": 1, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" -2,"{""split_info"": {""partition"": 2, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" -3,"{""split_info"": {""partition"": 3, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" +2,"{""split_info"": {""partition"": 2, ""start_offset"": 2, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" +3,"{""split_info"": {""partition"": 3, ""start_offset"": 3, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" query ?? rowsort select v1, v2 from s0; ---- -1 a -1 aa -2 b -2 bb -3 c -3 cc -4 d -4 dd +1 a +1 aa +2 b +2 bb +3 c +3 c +3 cc +4 d +4 d +4 d +4 dd query ?? rowsort select v1, v2 from mv_1; ---- -1 a -1 aa -2 b -2 bb -3 c -3 cc -4 d -4 dd +1 a +1 aa +2 b +2 bb +3 c +3 c +3 cc +4 d +4 d +4 d +4 dd # start_offset changed to 1 @@ -168,18 +191,18 @@ internal_table.mjs --name s0 --type source ---- 0,"{""split_info"": {""partition"": 0, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" 1,"{""split_info"": {""partition"": 1, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" -2,"{""split_info"": {""partition"": 2, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" -3,"{""split_info"": {""partition"": 3, ""start_offset"": 1, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" +2,"{""split_info"": {""partition"": 2, ""start_offset"": 2, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" +3,"{""split_info"": {""partition"": 3, ""start_offset"": 3, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" # Transition from SourceCachingUp to Finished after consuming one upstream message. system ok internal_table.mjs --name mv_1 --type sourcebackfill ---- -0,"""Finished""" -1,"""Finished""" -2,"""Finished""" -3,"""Finished""" +0,"{""num_consumed_rows"": 2, ""state"": ""Finished"", ""target_offset"": ""0""}" +1,"{""num_consumed_rows"": 2, ""state"": ""Finished"", ""target_offset"": ""0""}" +2,"{""num_consumed_rows"": 3, ""state"": ""Finished"", ""target_offset"": ""1""}" +3,"{""num_consumed_rows"": 4, ""state"": ""Finished"", ""target_offset"": ""2""}" system ok @@ -198,26 +221,26 @@ sleep 3s query ?? rowsort select v1, count(*) from s0 group by v1; ---- -1 12 -2 12 -3 12 -4 12 +1 12 +2 12 +3 13 +4 14 query ?? rowsort select v1, count(*) from mv_1 group by v1; ---- -1 12 -2 12 -3 12 -4 12 +1 12 +2 12 +3 13 +4 14 query ?? rowsort select v1, count(*) from mv_before_produce group by v1; ---- -1 12 -2 12 -3 12 -4 12 +1 12 +2 12 +3 13 +4 14 # start_offset changed to 11 @@ -226,8 +249,8 @@ internal_table.mjs --name s0 --type source ---- 0,"{""split_info"": {""partition"": 0, ""start_offset"": 11, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" 1,"{""split_info"": {""partition"": 1, ""start_offset"": 11, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" -2,"{""split_info"": {""partition"": 2, ""start_offset"": 11, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" -3,"{""split_info"": {""partition"": 3, ""start_offset"": 11, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" +2,"{""split_info"": {""partition"": 2, ""start_offset"": 12, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" +3,"{""split_info"": {""partition"": 3, ""start_offset"": 13, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" # # Note: the parallelism depends on the risedev profile. diff --git a/e2e_test/source_inline/kafka/temporary_kafka_batch.slt b/e2e_test/source_inline/kafka/temporary_kafka_batch.slt index 69144a884486f..47c6124e595d0 100644 --- a/e2e_test/source_inline/kafka/temporary_kafka_batch.slt +++ b/e2e_test/source_inline/kafka/temporary_kafka_batch.slt @@ -71,6 +71,12 @@ t t t +# ensure now() is larger +sleep 1s + +statement ok +flush; + query B select _rw_kafka_timestamp < now() from s1 ---- diff --git a/e2e_test/source_inline/pubsub/pubsub.slt b/e2e_test/source_inline/pubsub/pubsub.slt.serial similarity index 100% rename from e2e_test/source_inline/pubsub/pubsub.slt rename to e2e_test/source_inline/pubsub/pubsub.slt.serial diff --git a/e2e_test/source_legacy/basic/kafka.slt b/e2e_test/source_legacy/basic/kafka.slt index 227c0aa46bac1..4e597a98801df 100644 --- a/e2e_test/source_legacy/basic/kafka.slt +++ b/e2e_test/source_legacy/basic/kafka.slt @@ -1,3 +1,6 @@ +statement ok +SET streaming_use_shared_source TO false; + # We don't support CSV header for Kafka statement error CSV HEADER is not supported when creating table with Kafka connector create table s0 (v1 int, v2 varchar) with ( @@ -496,6 +499,16 @@ WITH ( scan.startup.mode = 'earliest' ) FORMAT PLAIN ENCODE JSON +statement ok +CREATE TABLE test_include_payload_only +INCLUDE payload +WITH ( + connector = 'kafka', + topic = 'kafka_1_partition_topic', + properties.bootstrap.server = 'message_queue:29092', + scan.startup.mode = 'earliest' +) FORMAT PLAIN ENCODE JSON + statement ok flush; @@ -908,5 +921,11 @@ drop table source_with_rdkafka_props; statement ok drop table debezium_ignore_key; +statement ok +drop table test_include_payload_only; + statement ok drop table test_include_payload; + +statement ok +SET streaming_use_shared_source TO true; diff --git a/e2e_test/time_travel/basic.slt b/e2e_test/time_travel/basic.slt index 962fd8e096a59..6dd65cf7125d3 100644 --- a/e2e_test/time_travel/basic.slt +++ b/e2e_test/time_travel/basic.slt @@ -1,6 +1,9 @@ statement ok SET RW_IMPLICIT_FLUSH TO true; +statement error time_travel_retention_ms cannot be less than 600000 +ALTER SYSTEM SET time_travel_retention_ms to 10; + statement ok CREATE TABLE t (k INT); diff --git a/proto/hummock.proto b/proto/hummock.proto index 6d3d7b0d32d7d..ec0e20f6013ce 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -63,6 +63,10 @@ message InputLevel { repeated SstableInfo table_infos = 3; } +message NewL0SubLevel { + repeated SstableInfo inserted_table_infos = 1; +} + message IntraLevelDelta { uint32 level_idx = 1; uint64 l0_sub_level_id = 2; @@ -112,6 +116,7 @@ message GroupDelta { GroupConstruct group_construct = 2; GroupDestroy group_destroy = 3; GroupMerge group_merge = 6; + NewL0SubLevel new_l0_sub_level = 7; } } @@ -528,7 +533,6 @@ message ReportCompactionTaskResponse { message ValidationTask { repeated SstableInfo sst_infos = 1; map sst_id_to_worker_id = 2; - uint64 epoch = 3; } // Delete SSTs in object store diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 45703554c2367..95cf407f8a56f 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -28,6 +28,7 @@ message BarrierCompleteResponse { bool done = 2; uint64 consumed_epoch = 3; uint64 consumed_rows = 4; + uint32 pending_barrier_num = 5; } string request_id = 1; common.Status status = 2; @@ -85,9 +86,15 @@ message StreamingControlStreamResponse { } } +message GetMinUncommittedSstIdRequest {} +message GetMinUncommittedSstIdResponse { + uint64 min_uncommitted_sst_id = 1; +} + service StreamService { rpc WaitEpochCommit(WaitEpochCommitRequest) returns (WaitEpochCommitResponse); rpc StreamingControlStream(stream StreamingControlStreamRequest) returns (stream StreamingControlStreamResponse); + rpc GetMinUncommittedSstId(GetMinUncommittedSstIdRequest) returns (GetMinUncommittedSstIdResponse); } // TODO: Lifecycle management for actors. diff --git a/risedev.yml b/risedev.yml index 4fca6ebd73488..dd3f69a1acf62 100644 --- a/risedev.yml +++ b/risedev.yml @@ -584,6 +584,28 @@ profile: - use: frontend - use: compactor + ci-backfill-3cn-1fe: + config-path: src/config/ci-longer-streaming-upload-timeout.toml + steps: + - use: minio + - use: sqlite + - use: meta-node + meta-backend: sqlite + - use: compute-node + port: 5687 + exporter-port: 1222 + enable-tiered-cache: true + - use: compute-node + port: 5688 + exporter-port: 1223 + enable-tiered-cache: true + - use: compute-node + port: 5689 + exporter-port: 1224 + enable-tiered-cache: true + - use: frontend + - use: compactor + ci-backfill-3cn-1fe-with-monitoring: config-path: src/config/ci-longer-streaming-upload-timeout.toml steps: diff --git a/src/common/src/array/arrow/arrow_iceberg.rs b/src/common/src/array/arrow/arrow_iceberg.rs index 9cb7df6f809a7..ce15c5e3646e6 100644 --- a/src/common/src/array/arrow/arrow_iceberg.rs +++ b/src/common/src/array/arrow/arrow_iceberg.rs @@ -83,8 +83,8 @@ impl IcebergArrowConvert { impl ToArrow for IcebergArrowConvert { #[inline] fn decimal_type_to_arrow(&self, name: &str) -> arrow_schema::Field { - let data_type = - arrow_schema::DataType::Decimal128(arrow_schema::DECIMAL128_MAX_PRECISION, 0); + // Fixed-point decimal; precision P, scale S Scale is fixed, precision must be less than 38. + let data_type = arrow_schema::DataType::Decimal128(28, 10); arrow_schema::Field::new(name, data_type, true) } @@ -178,7 +178,7 @@ impl ToArrow for IcebergCreateTableArrowConvert { // We choose 28 here // The decimal type finally will be converted to an iceberg decimal type. // Iceberg decimal(P,S) - // Fixed-point decimal; precision P, scale S Scale is fixed, precision must be 38 or less. + // Fixed-point decimal; precision P, scale S Scale is fixed, precision must be less than 38. let data_type = arrow_schema::DataType::Decimal128(28, 10); let mut arrow_field = arrow_schema::Field::new(name, data_type, true); diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 5f7bb2bdeff36..541d4896a6def 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -195,6 +195,10 @@ pub struct MetaConfig { #[serde(default = "default::meta::full_gc_object_limit")] pub full_gc_object_limit: u64, + /// Duration in seconds to retain garbage collection history data. + #[serde(default = "default::meta::gc_history_retention_time_sec")] + pub gc_history_retention_time_sec: u64, + /// Max number of inflight time travel query. #[serde(default = "default::meta::max_inflight_time_travel_query")] pub max_inflight_time_travel_query: u64, @@ -1039,8 +1043,7 @@ pub struct StreamingDeveloperConfig { /// Enable arrangement backfill /// If false, the arrangement backfill will be disabled, /// even if session variable set. - /// If true, it will be enabled by default, but session variable - /// can override it. + /// If true, it's decided by session variable `streaming_use_arrangement_backfill` (default true) pub enable_arrangement_backfill: bool, #[serde(default = "default::developer::stream_high_join_amplification_threshold")] @@ -1060,6 +1063,13 @@ pub struct StreamingDeveloperConfig { /// A flag to allow disabling the auto schema change handling #[serde(default = "default::developer::stream_enable_auto_schema_change")] pub enable_auto_schema_change: bool, + + #[serde(default = "default::developer::enable_shared_source")] + /// Enable shared source + /// If false, the shared source will be disabled, + /// even if session variable set. + /// If true, it's decided by session variable `streaming_use_shared_source` (default true) + pub enable_shared_source: bool, } /// The subsections `[batch.developer]`. @@ -1355,6 +1365,10 @@ pub mod default { 3600 * 3 } + pub fn gc_history_retention_time_sec() -> u64 { + 3600 * 6 + } + pub fn full_gc_interval_sec() -> u64 { 600 } @@ -1956,6 +1970,10 @@ pub mod default { true } + pub fn enable_shared_source() -> bool { + true + } + pub fn stream_high_join_amplification_threshold() -> usize { 2048 } diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index 43ac67363dd91..4a16fad1bbe96 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -287,8 +287,8 @@ pub struct SessionConfig { /// /// When enabled, `CREATE SOURCE` will create a source streaming job, and `CREATE MATERIALIZED VIEWS` from the source /// will forward the data from the same source streaming job, and also backfill prior data from the external source. - #[parameter(default = false, alias = "rw_enable_shared_source")] - enable_shared_source: bool, + #[parameter(default = true)] + streaming_use_shared_source: bool, /// Shows the server-side character set encoding. At present, this parameter can be shown but not set, because the encoding is determined at database creation time. #[parameter(default = SERVER_ENCODING)] diff --git a/src/common/src/system_param/mod.rs b/src/common/src/system_param/mod.rs index 2343e8e182dd9..4d3a994631534 100644 --- a/src/common/src/system_param/mod.rs +++ b/src/common/src/system_param/mod.rs @@ -419,6 +419,17 @@ impl ValidateOnSet for OverrideValidateOnSet { } Ok(()) } + + fn time_travel_retention_ms(v: &u64) -> Result<()> { + // This is intended to guarantee that non-time-travel batch query can still function even compute node's recent versions doesn't include the desired version. + let min_retention_ms = 600_000; + if *v < min_retention_ms { + return Err(format!( + "time_travel_retention_ms cannot be less than {min_retention_ms}" + )); + } + Ok(()) + } } for_all_params!(impl_default_from_other_params); diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index e8d403a9693cc..dae132f352c72 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -14,6 +14,7 @@ use await_tree::InstrumentAwait; use futures::{Stream, StreamExt, TryStreamExt}; +use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_pb::stream_service::stream_service_server::StreamService; use risingwave_pb::stream_service::*; use risingwave_storage::dispatch_state_store; @@ -87,4 +88,22 @@ impl StreamService for StreamServiceImpl { self.mgr.handle_new_control_stream(tx, stream, init_request); Ok(Response::new(UnboundedReceiverStream::new(rx))) } + + async fn get_min_uncommitted_sst_id( + &self, + _request: Request, + ) -> Result, Status> { + let min_uncommitted_sst_id = if let Some(hummock) = self.mgr.env.state_store().as_hummock() + { + hummock + .min_uncommitted_sst_id() + .await + .unwrap_or(HummockSstableObjectId::MAX) + } else { + HummockSstableObjectId::MAX + }; + Ok(Response::new(GetMinUncommittedSstIdResponse { + min_uncommitted_sst_id, + })) + } } diff --git a/src/config/ci-time-travel.toml b/src/config/ci-time-travel.toml index d160a8c94c69a..491d2bdecac0a 100644 --- a/src/config/ci-time-travel.toml +++ b/src/config/ci-time-travel.toml @@ -2,4 +2,4 @@ hummock_time_travel_snapshot_interval = 30 [system] -time_travel_retention_ms = 300000 +time_travel_retention_ms = 600000 diff --git a/src/config/docs.md b/src/config/docs.md index 6e5a1359ca671..1c4d6a351ae52 100644 --- a/src/config/docs.md +++ b/src/config/docs.md @@ -39,6 +39,7 @@ This page is automatically generated by `./risedev generate-example-config` | event_log_enabled | | true | | full_gc_interval_sec | Interval of automatic hummock full GC. | 600 | | full_gc_object_limit | Max number of object per full GC job can fetch. | 100000 | +| gc_history_retention_time_sec | Duration in seconds to retain garbage collection history data. | 21600 | | hummock_time_travel_snapshot_interval | The interval at which a Hummock version snapshot is taken for time travel. Larger value indicates less storage overhead but worse query performance. | 100 | | hummock_version_checkpoint_interval_sec | Interval of hummock version checkpoint. | 30 | | hybrid_partition_vnode_count | Count of partitions of tables in default group and materialized view group. The meta node will decide according to some strategy whether to cut the boundaries of the file according to the vnode alignment. Each partition contains aligned data of `vnode_count / hybrid_partition_vnode_count` consecutive virtual-nodes of one state table. Set it zero to disable this feature. | 4 | diff --git a/src/config/example.toml b/src/config/example.toml index 0d01a09d12875..f285720dc6b21 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -17,6 +17,7 @@ dir = "./" min_sst_retention_time_sec = 10800 full_gc_interval_sec = 600 full_gc_object_limit = 100000 +gc_history_retention_time_sec = 21600 max_inflight_time_travel_query = 1000 periodic_compaction_interval_sec = 60 vacuum_interval_sec = 30 @@ -129,6 +130,7 @@ stream_high_join_amplification_threshold = 2048 stream_enable_actor_tokio_metrics = false stream_exchange_connection_pool_size = 1 stream_enable_auto_schema_change = true +stream_enable_shared_source = true [storage] share_buffers_sync_parallelism = 1 diff --git a/src/ctl/src/cmd_impl/hummock/validate_version.rs b/src/ctl/src/cmd_impl/hummock/validate_version.rs index b6ab7f111aaac..62e988f42f1cf 100644 --- a/src/ctl/src/cmd_impl/hummock/validate_version.rs +++ b/src/ctl/src/cmd_impl/hummock/validate_version.rs @@ -206,14 +206,22 @@ pub async fn print_version_delta_in_archive( } fn match_delta(delta: &DeltaType, sst_id: HummockSstableObjectId) -> bool { - let DeltaType::IntraLevel(delta) = delta else { - return false; - }; - delta - .inserted_table_infos - .iter() - .any(|sst| sst.sst_id == sst_id) - || delta.removed_table_ids.iter().any(|sst| *sst == sst_id) + match delta { + DeltaType::GroupConstruct(_) | DeltaType::GroupDestroy(_) | DeltaType::GroupMerge(_) => { + false + } + DeltaType::IntraLevel(delta) => { + delta + .inserted_table_infos + .iter() + .any(|sst| sst.sst_id == sst_id) + || delta.removed_table_ids.iter().any(|sst| *sst == sst_id) + } + DeltaType::NewL0SubLevel(delta) => delta + .inserted_table_infos + .iter() + .any(|sst| sst.sst_id == sst_id), + } } fn print_delta(delta: &DeltaType) { diff --git a/src/frontend/planner_test/tests/testdata/input/shared_source.yml b/src/frontend/planner_test/tests/testdata/input/shared_source.yml index 71c87ab2e3ce1..4d684422d9fd1 100644 --- a/src/frontend/planner_test/tests/testdata/input/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/input/shared_source.yml @@ -9,7 +9,7 @@ ) FORMAT PLAIN ENCODE JSON; expected_outputs: [] - with_config_map: - enable_shared_source: true + streaming_use_shared_source: true sql: | /* The shared source config doesn't affect table with connector. */ EXPLAIN CREATE TABLE s(x int,y int) @@ -25,43 +25,43 @@ # We use with_config_map to control the config when CREATE SOURCE, and use another SET statement to change the config for CREATE MV # # batch: All 4 plans should be the same. -# stream: StreamSourceScan (with backfill) should be used only for the last 2. The first 2 use StreamSource. enable_shared_source changes the behavior of CREATE SOURCE, but not CREATE MATERIALIZED VIEW +# stream: StreamSourceScan (with backfill) should be used only for the last 2. The first 2 use StreamSource. streaming_use_shared_source changes the behavior of CREATE SOURCE, but not CREATE MATERIALIZED VIEW - with_config_map: - enable_shared_source: false + streaming_use_shared_source: false before: - create_source sql: | - SET enable_shared_source = false; + SET streaming_use_shared_source = false; select * from s; expected_outputs: - batch_plan - stream_plan - with_config_map: - enable_shared_source: false + streaming_use_shared_source: false before: - create_source sql: | - SET enable_shared_source = true; + SET streaming_use_shared_source = true; select * from s; expected_outputs: - batch_plan - stream_plan - with_config_map: - enable_shared_source: true + streaming_use_shared_source: true before: - create_source sql: | - SET enable_shared_source = false; + SET streaming_use_shared_source = false; select * from s; expected_outputs: - batch_plan - stream_plan - with_config_map: - enable_shared_source: true + streaming_use_shared_source: true before: - create_source sql: | - SET enable_shared_source = true; + SET streaming_use_shared_source = true; select * from s; expected_outputs: - batch_plan diff --git a/src/frontend/planner_test/tests/testdata/output/shared_source.yml b/src/frontend/planner_test/tests/testdata/output/shared_source.yml index 5083c23952f3b..83fde26bfc7d2 100644 --- a/src/frontend/planner_test/tests/testdata/output/shared_source.yml +++ b/src/frontend/planner_test/tests/testdata/output/shared_source.yml @@ -27,11 +27,11 @@ └─StreamDml { columns: [x, y, _row_id] } └─StreamSource with_config_map: - enable_shared_source: 'true' + streaming_use_shared_source: 'true' - before: - create_source sql: | - SET enable_shared_source = false; + SET streaming_use_shared_source = false; select * from s; batch_plan: |- BatchExchange { order: [], dist: Single } @@ -43,11 +43,11 @@ └─StreamRowIdGen { row_id_index: 3 } └─StreamSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id] } with_config_map: - enable_shared_source: 'false' + streaming_use_shared_source: 'false' - before: - create_source sql: | - SET enable_shared_source = true; + SET streaming_use_shared_source = true; select * from s; batch_plan: |- BatchExchange { order: [], dist: Single } @@ -59,11 +59,11 @@ └─StreamRowIdGen { row_id_index: 3 } └─StreamSource { source: s, columns: [x, y, _rw_kafka_timestamp, _row_id] } with_config_map: - enable_shared_source: 'false' + streaming_use_shared_source: 'false' - before: - create_source sql: | - SET enable_shared_source = false; + SET streaming_use_shared_source = false; select * from s; batch_plan: |- BatchExchange { order: [], dist: Single } @@ -75,11 +75,11 @@ └─StreamRowIdGen { row_id_index: 3 } └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } with_config_map: - enable_shared_source: 'true' + streaming_use_shared_source: 'true' - before: - create_source sql: | - SET enable_shared_source = true; + SET streaming_use_shared_source = true; select * from s; batch_plan: |- BatchExchange { order: [], dist: Single } @@ -91,4 +91,4 @@ └─StreamRowIdGen { row_id_index: 3 } └─StreamSourceScan { columns: [x, y, _rw_kafka_timestamp, _row_id, _rw_kafka_partition, _rw_kafka_offset] } with_config_map: - enable_shared_source: 'true' + streaming_use_shared_source: 'true' diff --git a/src/frontend/planner_test/tests/testdata/output/subquery.yaml b/src/frontend/planner_test/tests/testdata/output/subquery.yaml index 98a0fcbf8fdd3..be23e2a99f11b 100644 --- a/src/frontend/planner_test/tests/testdata/output/subquery.yaml +++ b/src/frontend/planner_test/tests/testdata/output/subquery.yaml @@ -241,7 +241,7 @@ │ │ │ │ │ │ │ └─LogicalProject { exprs: [rw_system_tables.id, rw_system_tables.name, 'system table':Varchar, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } │ │ │ │ │ │ │ └─LogicalSysScan { table: rw_system_tables, columns: [rw_system_tables.id, rw_system_tables.name, rw_system_tables.schema_id, rw_system_tables.owner, rw_system_tables.definition, rw_system_tables.acl] } │ │ │ │ │ │ └─LogicalProject { exprs: [rw_sources.id, rw_sources.name, 'source':Varchar, rw_sources.schema_id, rw_sources.owner, rw_sources.definition, rw_sources.acl] } - │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.associated_table_id, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at, rw_sources.initialized_at_cluster_version, rw_sources.created_at_cluster_version] } + │ │ │ │ │ │ └─LogicalSysScan { table: rw_sources, columns: [rw_sources.id, rw_sources.name, rw_sources.schema_id, rw_sources.owner, rw_sources.connector, rw_sources.columns, rw_sources.format, rw_sources.row_encode, rw_sources.append_only, rw_sources.associated_table_id, rw_sources.connection_id, rw_sources.definition, rw_sources.acl, rw_sources.initialized_at, rw_sources.created_at, rw_sources.initialized_at_cluster_version, rw_sources.created_at_cluster_version, rw_sources.is_shared] } │ │ │ │ │ └─LogicalProject { exprs: [rw_indexes.id, rw_indexes.name, 'index':Varchar, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl] } │ │ │ │ │ └─LogicalSysScan { table: rw_indexes, columns: [rw_indexes.id, rw_indexes.name, rw_indexes.primary_table_id, rw_indexes.key_columns, rw_indexes.include_columns, rw_indexes.schema_id, rw_indexes.owner, rw_indexes.definition, rw_indexes.acl, rw_indexes.initialized_at, rw_indexes.created_at, rw_indexes.initialized_at_cluster_version, rw_indexes.created_at_cluster_version] } │ │ │ │ └─LogicalProject { exprs: [rw_sinks.id, rw_sinks.name, 'sink':Varchar, rw_sinks.schema_id, rw_sinks.owner, rw_sinks.definition, rw_sinks.acl] } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs index 441ac99b48cda..40df3dfc3a849 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_sources.rs @@ -40,6 +40,7 @@ struct RwSource { created_at: Option, initialized_at_cluster_version: Option, created_at_cluster_version: Option, + is_shared: bool, } #[system_catalog(table, "rw_catalog.rw_sources")] @@ -83,6 +84,7 @@ fn read_rw_sources_info(reader: &SysCatalogReaderImpl) -> Result> created_at: source.created_at_epoch.map(|e| e.as_timestamptz()), initialized_at_cluster_version: source.initialized_at_cluster_version.clone(), created_at_cluster_version: source.created_at_cluster_version.clone(), + is_shared: source.info.is_shared(), }) }) .collect()) diff --git a/src/frontend/src/handler/alter_source_column.rs b/src/frontend/src/handler/alter_source_column.rs index 013b27ef8a213..2d2e2c6698282 100644 --- a/src/frontend/src/handler/alter_source_column.rs +++ b/src/frontend/src/handler/alter_source_column.rs @@ -165,13 +165,31 @@ pub mod tests { let session = frontend.session_ref(); let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME); + let sql = r#"create source s_shared (v1 int) with ( + connector = 'kafka', + topic = 'abc', + properties.bootstrap.server = 'localhost:29092', + ) FORMAT PLAIN ENCODE JSON;"#; + + frontend + .run_sql_with_session(session.clone(), sql) + .await + .unwrap(); + + frontend + .run_sql_with_session(session.clone(), "SET streaming_use_shared_source TO false;") + .await + .unwrap(); let sql = r#"create source s (v1 int) with ( connector = 'kafka', topic = 'abc', properties.bootstrap.server = 'localhost:29092', ) FORMAT PLAIN ENCODE JSON;"#; - frontend.run_sql(sql).await.unwrap(); + frontend + .run_sql_with_session(session.clone(), sql) + .await + .unwrap(); let get_source = || { let catalog_reader = session.env().catalog_reader().read_guard(); @@ -189,6 +207,8 @@ pub mod tests { .map(|col| (col.name(), (col.data_type().clone(), col.column_id()))) .collect(); + let sql = "alter source s_shared add column v2 varchar;"; + assert_eq!("Feature is not yet implemented: alter shared source\nTracking issue: https://github.com/risingwavelabs/risingwave/issues/16003", &frontend.run_sql(sql).await.unwrap_err().to_string()); let sql = "alter source s add column v2 varchar;"; frontend.run_sql(sql).await.unwrap(); diff --git a/src/frontend/src/handler/alter_source_with_sr.rs b/src/frontend/src/handler/alter_source_with_sr.rs index 17722625d5347..bf8cf991d1a4f 100644 --- a/src/frontend/src/handler/alter_source_with_sr.rs +++ b/src/frontend/src/handler/alter_source_with_sr.rs @@ -348,7 +348,14 @@ pub mod tests { let session = frontend.session_ref(); let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME); - frontend.run_sql(sql).await.unwrap(); + frontend + .run_sql_with_session(session.clone(), "SET streaming_use_shared_source TO false;") + .await + .unwrap(); + frontend + .run_sql_with_session(session.clone(), sql) + .await + .unwrap(); let get_source = || { let catalog_reader = session.env().catalog_reader().read_guard(); diff --git a/src/frontend/src/handler/create_source.rs b/src/frontend/src/handler/create_source.rs index d6338b89456aa..9fbc557478a8a 100644 --- a/src/frontend/src/handler/create_source.rs +++ b/src/frontend/src/handler/create_source.rs @@ -646,12 +646,6 @@ pub fn handle_addition_columns( )))); } - let latest_col_id: ColumnId = columns - .iter() - .map(|col| col.column_desc.column_id) - .max() - .unwrap(); // there must be at least one column in the column catalog - while let Some(item) = additional_columns.pop() { check_additional_column_compatibility(&item, source_schema)?; @@ -659,7 +653,7 @@ pub fn handle_addition_columns( .header_inner_expect_type .map(|dt| format!("{:?}", dt).to_lowercase()); let col = build_additional_column_desc( - latest_col_id.next(), + ColumnId::placeholder(), connector_name.as_str(), item.column_type.real_value().as_str(), item.column_alias.map(|alias| alias.real_value()), @@ -714,12 +708,6 @@ pub(crate) fn bind_all_columns( "Remove the wildcard or use a source with external schema".to_string(), ))); } - // FIXME(yuhao): cols_from_sql should be None is no `()` is given. - if cols_from_sql.is_empty() { - return Err(RwError::from(ProtocolError( - "Schema definition is required, either from SQL or schema registry.".to_string(), - ))); - } let non_generated_sql_defined_columns = non_generated_sql_columns(col_defs_from_sql); match (&source_schema.format, &source_schema.row_encode) { (Format::DebeziumMongo, Encode::Json) => { @@ -1539,6 +1527,13 @@ pub async fn bind_create_source_or_table_with_connector( &mut columns, false, )?; + + if columns.is_empty() { + return Err(RwError::from(ProtocolError( + "Schema definition is required, either from SQL or schema registry.".to_string(), + ))); + } + // compatible with the behavior that add a hidden column `_rw_kafka_timestamp` to each message from Kafka source if is_create_source { // must behind `handle_addition_columns` @@ -1662,7 +1657,12 @@ pub async fn handle_create_source( let create_cdc_source_job = with_properties.is_shareable_cdc_connector(); let is_shared = create_cdc_source_job || (with_properties.is_shareable_non_cdc_connector() - && session.config().enable_shared_source()); + && session + .env() + .streaming_config() + .developer + .enable_shared_source + && session.config().streaming_use_shared_source()); let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job { bind_columns_from_source_for_cdc(&session, &source_schema)? diff --git a/src/frontend/src/optimizer/plan_node/logical_source.rs b/src/frontend/src/optimizer/plan_node/logical_source.rs index 5bbf1a0216411..81dc1128c2a43 100644 --- a/src/frontend/src/optimizer/plan_node/logical_source.rs +++ b/src/frontend/src/optimizer/plan_node/logical_source.rs @@ -354,7 +354,7 @@ impl ToStream for LogicalSource { } SourceNodeKind::CreateMViewOrBatch => { // Create MV on source. - // We only check enable_shared_source is true when `CREATE SOURCE`. + // We only check streaming_use_shared_source is true when `CREATE SOURCE`. // The value does not affect the behavior of `CREATE MATERIALIZED VIEW` here. let use_shared_source = self.source_catalog().is_some_and(|c| c.info.is_shared()); if use_shared_source { diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index ff1c92227716c..ac8784fde4dd3 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -183,6 +183,7 @@ impl LocalFrontend { res } + /// Creates a new session pub fn session_ref(&self) -> Arc { self.session_user_ref( DEFAULT_DATABASE_NAME.to_string(), diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 4b5caff8a9e6e..4ae39aea29198 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -22,37 +22,10 @@ mod m20240726_063833_auto_schema_change; mod m20240806_143329_add_rate_limit_to_source_catalog; mod m20240820_081248_add_time_travel_per_table_epoch; mod m20240911_083152_variable_vnode_count; +mod m20241016_065621_hummock_gc_history; pub struct Migrator; -#[async_trait::async_trait] -impl MigratorTrait for Migrator { - fn migrations() -> Vec> { - vec![ - Box::new(m20230908_072257_init::Migration), - Box::new(m20231008_020431_hummock::Migration), - Box::new(m20240304_074901_subscription::Migration), - Box::new(m20240410_082733_with_version_column_migration::Migration), - Box::new(m20240410_154406_session_params::Migration), - Box::new(m20240417_062305_subscription_internal_table_name::Migration), - Box::new(m20240418_142249_function_runtime::Migration), - Box::new(m20240506_112555_subscription_partial_ckpt::Migration), - Box::new(m20240525_090457_secret::Migration), - Box::new(m20240617_070131_index_column_properties::Migration), - Box::new(m20240617_071625_sink_into_table_column::Migration), - Box::new(m20240618_072634_function_compressed_binary::Migration), - Box::new(m20240630_131430_remove_parallel_unit::Migration), - Box::new(m20240701_060504_hummock_time_travel::Migration), - Box::new(m20240702_080451_system_param_value::Migration), - Box::new(m20240702_084927_unnecessary_fk::Migration), - Box::new(m20240726_063833_auto_schema_change::Migration), - Box::new(m20240806_143329_add_rate_limit_to_source_catalog::Migration), - Box::new(m20240820_081248_add_time_travel_per_table_epoch::Migration), - Box::new(m20240911_083152_variable_vnode_count::Migration), - ] - } -} - #[macro_export] macro_rules! assert_not_has_tables { ($manager:expr, $( $table:ident ),+) => { @@ -84,3 +57,32 @@ macro_rules! drop_tables { )+ }; } + +#[async_trait::async_trait] +impl MigratorTrait for Migrator { + fn migrations() -> Vec> { + vec![ + Box::new(m20230908_072257_init::Migration), + Box::new(m20231008_020431_hummock::Migration), + Box::new(m20240304_074901_subscription::Migration), + Box::new(m20240410_082733_with_version_column_migration::Migration), + Box::new(m20240410_154406_session_params::Migration), + Box::new(m20240417_062305_subscription_internal_table_name::Migration), + Box::new(m20240418_142249_function_runtime::Migration), + Box::new(m20240506_112555_subscription_partial_ckpt::Migration), + Box::new(m20240525_090457_secret::Migration), + Box::new(m20240617_070131_index_column_properties::Migration), + Box::new(m20240617_071625_sink_into_table_column::Migration), + Box::new(m20240618_072634_function_compressed_binary::Migration), + Box::new(m20240630_131430_remove_parallel_unit::Migration), + Box::new(m20240701_060504_hummock_time_travel::Migration), + Box::new(m20240702_080451_system_param_value::Migration), + Box::new(m20240702_084927_unnecessary_fk::Migration), + Box::new(m20240726_063833_auto_schema_change::Migration), + Box::new(m20240806_143329_add_rate_limit_to_source_catalog::Migration), + Box::new(m20240820_081248_add_time_travel_per_table_epoch::Migration), + Box::new(m20240911_083152_variable_vnode_count::Migration), + Box::new(m20241016_065621_hummock_gc_history::Migration), + ] + } +} diff --git a/src/meta/model_v2/migration/src/m20241016_065621_hummock_gc_history.rs b/src/meta/model_v2/migration/src/m20241016_065621_hummock_gc_history.rs new file mode 100644 index 0000000000000..79254ed8f6661 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20241016_065621_hummock_gc_history.rs @@ -0,0 +1,51 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .create_table( + Table::create() + .table(HummockGcHistory::Table) + .if_not_exists() + .col( + ColumnDef::new(HummockGcHistory::ObjectId) + .big_integer() + .not_null() + .primary_key(), + ) + .col( + ColumnDef::new(HummockGcHistory::MarkDeleteAt) + .date_time() + .not_null(), + ) + .to_owned(), + ) + .await?; + manager + .create_index( + Index::create() + .table(HummockGcHistory::Table) + .name("idx_hummock_gc_history_mark_delete_at") + .col(HummockGcHistory::MarkDeleteAt) + .to_owned(), + ) + .await?; + Ok(()) + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + crate::drop_tables!(manager, HummockGcHistory); + Ok(()) + } +} + +#[derive(DeriveIden)] +enum HummockGcHistory { + Table, + ObjectId, + MarkDeleteAt, +} diff --git a/src/meta/model_v2/src/hummock_gc_history.rs b/src/meta/model_v2/src/hummock_gc_history.rs new file mode 100644 index 0000000000000..5a97d79ae5346 --- /dev/null +++ b/src/meta/model_v2/src/hummock_gc_history.rs @@ -0,0 +1,31 @@ +// 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 sea_orm::entity::prelude::*; +use sea_orm::{DeriveEntityModel, DeriveRelation, EnumIter}; + +use crate::HummockSstableObjectId; + +#[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Default)] +#[sea_orm(table_name = "hummock_gc_history")] +pub struct Model { + #[sea_orm(primary_key, auto_increment = false)] + pub object_id: HummockSstableObjectId, + pub mark_delete_at: DateTime, +} + +impl ActiveModelBehavior for ActiveModel {} + +#[derive(Copy, Clone, Debug, EnumIter, DeriveRelation)] +pub enum Relation {} diff --git a/src/meta/model_v2/src/lib.rs b/src/meta/model_v2/src/lib.rs index e0db07f4709ca..0b99b48914b2a 100644 --- a/src/meta/model_v2/src/lib.rs +++ b/src/meta/model_v2/src/lib.rs @@ -36,6 +36,7 @@ pub mod database; pub mod fragment; pub mod function; pub mod hummock_epoch_to_version; +pub mod hummock_gc_history; pub mod hummock_pinned_snapshot; pub mod hummock_pinned_version; pub mod hummock_sequence; diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 3058a04c15f91..eacd1a65517ee 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -367,6 +367,7 @@ pub fn start( min_sst_retention_time_sec: config.meta.min_sst_retention_time_sec, full_gc_interval_sec: config.meta.full_gc_interval_sec, full_gc_object_limit: config.meta.full_gc_object_limit, + gc_history_retention_time_sec: config.meta.gc_history_retention_time_sec, max_inflight_time_travel_query: config.meta.max_inflight_time_travel_query, enable_committed_sst_sanity_check: config.meta.enable_committed_sst_sanity_check, periodic_compaction_interval_sec: config.meta.periodic_compaction_interval_sec, diff --git a/src/meta/service/src/hummock_service.rs b/src/meta/service/src/hummock_service.rs index c3c3fcd6aa0f0..4ef731a01f03f 100644 --- a/src/meta/service/src/hummock_service.rs +++ b/src/meta/service/src/hummock_service.rs @@ -257,11 +257,16 @@ impl HummockManagerService for HummockServiceImpl { req.total_object_count, req.total_object_size, ); + let pinned_by_metadata_backup = self.vacuum_manager.backup_manager.list_pinned_ssts(); // The following operation takes some time, so we do it in dedicated task and responds the // RPC immediately. tokio::spawn(async move { match hummock_manager - .complete_full_gc(req.object_ids, req.next_start_after) + .complete_full_gc( + req.object_ids, + req.next_start_after, + pinned_by_metadata_backup, + ) .await { Ok(number) => { diff --git a/src/meta/src/barrier/creating_job/barrier_control.rs b/src/meta/src/barrier/creating_job/barrier_control.rs index b0aca04645003..90ac3119f4c37 100644 --- a/src/meta/src/barrier/creating_job/barrier_control.rs +++ b/src/meta/src/barrier/creating_job/barrier_control.rs @@ -12,9 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::Bound::{Excluded, Unbounded}; use std::collections::{BTreeMap, HashSet, VecDeque}; use std::mem::take; +use std::ops::Bound::Unbounded; +use std::ops::{Bound, RangeBounds}; use std::time::Instant; use prometheus::HistogramTimer; @@ -26,22 +27,13 @@ use tracing::debug; use crate::rpc::metrics::MetaMetrics; -#[derive(Debug)] -pub(super) enum CreatingStreamingJobBarrierType { - Snapshot, - LogStore, - Upstream, -} - #[derive(Debug)] struct CreatingStreamingJobEpochState { epoch: u64, node_to_collect: HashSet, resps: Vec, - upstream_epoch_to_notify: Option, is_checkpoint: bool, enqueue_time: Instant, - barrier_type: CreatingStreamingJobBarrierType, } #[derive(Debug)] @@ -49,31 +41,30 @@ pub(super) struct CreatingStreamingJobBarrierControl { table_id: TableId, // key is prev_epoch of barrier inflight_barrier_queue: BTreeMap, + backfill_epoch: u64, initial_epoch: Option, max_collected_epoch: Option, - max_attached_epoch: Option, - // newer epoch at the front. should all be checkpoint barrier + // newer epoch at the front. pending_barriers_to_complete: VecDeque, completing_barrier: Option<(CreatingStreamingJobEpochState, HistogramTimer)>, // metrics consuming_snapshot_barrier_latency: LabelGuardedHistogram<2>, consuming_log_store_barrier_latency: LabelGuardedHistogram<2>, - consuming_upstream_barrier_latency: LabelGuardedHistogram<2>, wait_commit_latency: LabelGuardedHistogram<1>, inflight_barrier_num: LabelGuardedIntGauge<1>, } impl CreatingStreamingJobBarrierControl { - pub(super) fn new(table_id: TableId, metrics: &MetaMetrics) -> Self { + pub(super) fn new(table_id: TableId, backfill_epoch: u64, metrics: &MetaMetrics) -> Self { let table_id_str = format!("{}", table_id.table_id); Self { table_id, inflight_barrier_queue: Default::default(), + backfill_epoch, initial_epoch: None, max_collected_epoch: None, - max_attached_epoch: None, pending_barriers_to_complete: Default::default(), completing_barrier: None, @@ -83,9 +74,6 @@ impl CreatingStreamingJobBarrierControl { consuming_log_store_barrier_latency: metrics .snapshot_backfill_barrier_latency .with_guarded_label_values(&[&table_id_str, "consuming_log_store"]), - consuming_upstream_barrier_latency: metrics - .snapshot_backfill_barrier_latency - .with_guarded_label_values(&[&table_id_str, "consuming_upstream"]), wait_commit_latency: metrics .snapshot_backfill_wait_commit_latency .with_guarded_label_values(&[&table_id_str]), @@ -127,7 +115,6 @@ impl CreatingStreamingJobBarrierControl { epoch: u64, node_to_collect: HashSet, is_checkpoint: bool, - barrier_type: CreatingStreamingJobBarrierType, ) { debug!( epoch, @@ -142,17 +129,12 @@ impl CreatingStreamingJobBarrierControl { if let Some(latest_epoch) = self.latest_epoch() { assert!(epoch > latest_epoch, "{} {}", epoch, latest_epoch); } - if let Some(max_attached_epoch) = self.max_attached_epoch { - assert!(epoch > max_attached_epoch); - } let epoch_state = CreatingStreamingJobEpochState { epoch, node_to_collect, resps: vec![], - upstream_epoch_to_notify: None, is_checkpoint, enqueue_time: Instant::now(), - barrier_type, }; if epoch_state.node_to_collect.is_empty() && self.inflight_barrier_queue.is_empty() { self.add_collected(epoch_state); @@ -163,41 +145,6 @@ impl CreatingStreamingJobBarrierControl { .set(self.inflight_barrier_queue.len() as _); } - pub(super) fn unattached_epochs(&self) -> impl Iterator + '_ { - let range_start = if let Some(max_attached_epoch) = self.max_attached_epoch { - Excluded(max_attached_epoch) - } else { - Unbounded - }; - self.inflight_barrier_queue - .range((range_start, Unbounded)) - .map(|(epoch, state)| (*epoch, state.is_checkpoint)) - } - - /// Attach an `upstream_epoch` to the `epoch` of the creating job. - /// - /// The `upstream_epoch` won't be completed until the `epoch` of the creating job is completed so that - /// the `upstream_epoch` should wait for the progress of creating job, and we can ensure that the downstream - /// creating job can eventually catch up with the upstream. - pub(super) fn attach_upstream_epoch(&mut self, epoch: u64, upstream_epoch: u64) { - debug!( - epoch, - upstream_epoch, - table_id = ?self.table_id.table_id, - "attach epoch" - ); - if let Some(max_attached_epoch) = self.max_attached_epoch { - assert!(epoch > max_attached_epoch); - } - self.max_attached_epoch = Some(epoch); - let epoch_state = self - .inflight_barrier_queue - .get_mut(&epoch) - .expect("should exist"); - assert!(epoch_state.upstream_epoch_to_notify.is_none()); - epoch_state.upstream_epoch_to_notify = Some(upstream_epoch); - } - pub(super) fn collect( &mut self, epoch: u64, @@ -228,46 +175,47 @@ impl CreatingStreamingJobBarrierControl { .set(self.inflight_barrier_queue.len() as _); } - #[expect(clippy::type_complexity)] - /// Return (`upstream_epochs_to_notify`, Some((epoch, resps, `is_first_commit`))) + /// Return Some((epoch, resps, `is_first_commit`)) /// - /// `upstream_epochs_to_notify` is the upstream epochs of non-checkpoint barriers to be notified about barrier completing. - /// These non-checkpoint barriers does not need to call `commit_epoch` and therefore can be completed as long as collected. + /// Only epoch within the `epoch_end_bound` can be started. + /// Usually `epoch_end_bound` is the upstream committed epoch. This is to ensure that + /// the creating job won't have higher committed epoch than the upstream. pub(super) fn start_completing( &mut self, - ) -> (Vec, Option<(u64, Vec, bool)>) { - if self.completing_barrier.is_some() { - return (vec![], None); - } - let mut upstream_epochs_to_notify = vec![]; - while let Some(mut epoch_state) = self.pending_barriers_to_complete.pop_back() { + epoch_end_bound: Bound, + ) -> Option<(u64, Vec, bool)> { + assert!(self.completing_barrier.is_none()); + let epoch_range: (Bound, Bound) = (Unbounded, epoch_end_bound); + while let Some(epoch_state) = self.pending_barriers_to_complete.back() + && epoch_range.contains(&epoch_state.epoch) + { + let mut epoch_state = self + .pending_barriers_to_complete + .pop_back() + .expect("non-empty"); let epoch = epoch_state.epoch; let is_first = self.initial_epoch.expect("should have set") == epoch; if is_first { assert!(epoch_state.is_checkpoint); } else if !epoch_state.is_checkpoint { - if let Some(upstream_epoch) = epoch_state.upstream_epoch_to_notify { - upstream_epochs_to_notify.push(upstream_epoch); - } continue; } let resps = take(&mut epoch_state.resps); self.completing_barrier = Some((epoch_state, self.wait_commit_latency.start_timer())); - return (upstream_epochs_to_notify, Some((epoch, resps, is_first))); + return Some((epoch, resps, is_first)); } - (upstream_epochs_to_notify, None) + None } /// Ack on completing a checkpoint barrier. /// /// Return the upstream epoch to be notified when there is any. - pub(super) fn ack_completed(&mut self, completed_epoch: u64) -> Option { + pub(super) fn ack_completed(&mut self, completed_epoch: u64) { let (epoch_state, wait_commit_timer) = self.completing_barrier.take().expect("should exist"); wait_commit_timer.observe_duration(); assert_eq!(epoch_state.epoch, completed_epoch); - epoch_state.upstream_epoch_to_notify } fn add_collected(&mut self, epoch_state: CreatingStreamingJobEpochState) { @@ -280,10 +228,10 @@ impl CreatingStreamingJobBarrierControl { } self.max_collected_epoch = Some(epoch_state.epoch); let barrier_latency = epoch_state.enqueue_time.elapsed().as_secs_f64(); - let barrier_latency_metrics = match &epoch_state.barrier_type { - CreatingStreamingJobBarrierType::Snapshot => &self.consuming_snapshot_barrier_latency, - CreatingStreamingJobBarrierType::LogStore => &self.consuming_log_store_barrier_latency, - CreatingStreamingJobBarrierType::Upstream => &self.consuming_upstream_barrier_latency, + let barrier_latency_metrics = if epoch_state.epoch < self.backfill_epoch { + &self.consuming_snapshot_barrier_latency + } else { + &self.consuming_log_store_barrier_latency }; barrier_latency_metrics.observe(barrier_latency); self.pending_barriers_to_complete.push_front(epoch_state); diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/creating_job/mod.rs index 08b82148676f8..0598cd319c590 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/creating_job/mod.rs @@ -17,25 +17,21 @@ mod status; use std::cmp::max; use std::collections::HashMap; -use std::mem::take; +use std::ops::Bound::{Excluded, Unbounded}; use std::sync::Arc; -use std::time::Duration; -use prometheus::HistogramTimer; -use risingwave_common::metrics::{LabelGuardedHistogram, LabelGuardedIntGauge}; -use risingwave_common::util::epoch::Epoch; +use risingwave_common::catalog::TableId; +use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_meta_model_v2::WorkerId; use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_service::BarrierCompleteResponse; -use tracing::{debug, info}; +use tracing::info; use crate::barrier::command::CommandContext; -use crate::barrier::creating_job::barrier_control::{ - CreatingStreamingJobBarrierControl, CreatingStreamingJobBarrierType, -}; +use crate::barrier::creating_job::barrier_control::CreatingStreamingJobBarrierControl; use crate::barrier::creating_job::status::{ CreatingJobInjectBarrierInfo, CreatingStreamingJobStatus, }; @@ -52,11 +48,12 @@ pub(super) struct CreatingStreamingJobControl { pub(super) snapshot_backfill_info: SnapshotBackfillInfo, backfill_epoch: u64, + graph_info: InflightGraphInfo, + barrier_control: CreatingStreamingJobBarrierControl, status: CreatingStreamingJobStatus, upstream_lag: LabelGuardedIntGauge<1>, - upstream_wait_progress_latency: LabelGuardedHistogram<1>, } impl CreatingStreamingJobControl { @@ -73,6 +70,7 @@ impl CreatingStreamingJobControl { definition = info.definition, "new creating job" ); + let snapshot_backfill_actors = info.table_fragments.snapshot_backfill_actor_ids(); let mut create_mview_tracker = CreateMviewProgressTracker::default(); create_mview_tracker.update_tracking_jobs(Some((&info, None)), [], version_stat); let fragment_info: HashMap<_, _> = info.new_fragment_info().collect(); @@ -85,14 +83,19 @@ impl CreatingStreamingJobControl { Self { info, snapshot_backfill_info, - barrier_control: CreatingStreamingJobBarrierControl::new(table_id, metrics), + barrier_control: CreatingStreamingJobBarrierControl::new( + table_id, + backfill_epoch, + metrics, + ), backfill_epoch, + graph_info: InflightGraphInfo::new(fragment_info), status: CreatingStreamingJobStatus::ConsumingSnapshot { prev_epoch_fake_physical_time: 0, - pending_commands: vec![], + pending_upstream_barriers: vec![], version_stats: version_stat.clone(), create_mview_tracker, - graph_info: InflightGraphInfo::new(fragment_info), + snapshot_backfill_actors, backfill_epoch, pending_non_checkpoint_barriers: vec![], initial_barrier_info: Some((actors_to_create, initial_mutation)), @@ -100,29 +103,16 @@ impl CreatingStreamingJobControl { upstream_lag: metrics .snapshot_backfill_lag .with_guarded_label_values(&[&table_id_str]), - upstream_wait_progress_latency: metrics - .snapshot_backfill_upstream_wait_progress_latency - .with_guarded_label_values(&[&table_id_str]), } } - pub(super) fn start_wait_progress_timer(&self) -> HistogramTimer { - self.upstream_wait_progress_latency.start_timer() - } - pub(super) fn is_wait_on_worker(&self, worker_id: WorkerId) -> bool { self.barrier_control.is_wait_on_worker(worker_id) - || self - .status - .active_graph_info() - .map(|info| info.contains_worker(worker_id)) - .unwrap_or(false) + || (self.status.is_finishing() && self.graph_info.contains_worker(worker_id)) } pub(super) fn on_new_worker_node_map(&self, node_map: &HashMap) { - if let Some(info) = self.status.active_graph_info() { - info.on_new_worker_node_map(node_map) - } + self.graph_info.on_new_worker_node_map(node_map) } pub(super) fn gen_ddl_progress(&self) -> DdlProgress { @@ -142,32 +132,15 @@ impl CreatingStreamingJobControl { } } CreatingStreamingJobStatus::ConsumingLogStore { - start_consume_log_store_epoch, + log_store_progress_tracker, .. } => { - let max_collected_epoch = max( - self.barrier_control.max_collected_epoch().unwrap_or(0), - self.backfill_epoch, - ); - let lag = Duration::from_millis( - Epoch(*start_consume_log_store_epoch) - .physical_time() - .saturating_sub(Epoch(max_collected_epoch).physical_time()), - ); format!( - "LogStore [remain lag: {:?}, epoch cnt: {}]", - lag, - self.barrier_control.inflight_barrier_count() + "LogStore [{}]", + log_store_progress_tracker.gen_ddl_progress() ) } - CreatingStreamingJobStatus::ConsumingUpstream { .. } => { - format!( - "Upstream [unattached: {}, epoch cnt: {}]", - self.barrier_control.unattached_epochs().count(), - self.barrier_control.inflight_barrier_count(), - ) - } - CreatingStreamingJobStatus::Finishing { .. } => { + CreatingStreamingJobStatus::Finishing(_) => { format!( "Finishing [epoch count: {}]", self.barrier_control.inflight_barrier_count() @@ -182,84 +155,43 @@ impl CreatingStreamingJobControl { } pub(super) fn pinned_upstream_log_epoch(&self) -> Option { - let stop_consume_log_store_epoch = match &self.status { - CreatingStreamingJobStatus::ConsumingSnapshot { .. } - | CreatingStreamingJobStatus::ConsumingLogStore { .. } => None, - CreatingStreamingJobStatus::ConsumingUpstream { - start_consume_upstream_epoch, - .. - } - | CreatingStreamingJobStatus::Finishing { - start_consume_upstream_epoch, - .. - } => Some(*start_consume_upstream_epoch), - }; - if let Some(max_collected_epoch) = self.barrier_control.max_collected_epoch() { - if max_collected_epoch < self.backfill_epoch { - Some(self.backfill_epoch) - } else if let Some(stop_consume_log_store_epoch) = stop_consume_log_store_epoch - && max_collected_epoch >= stop_consume_log_store_epoch - { - None - } else { - Some(max_collected_epoch) - } + if self.status.is_finishing() { + None } else { - Some(self.backfill_epoch) + // TODO: when supporting recoverable snapshot backfill, we should use the max epoch that has committed + Some(max( + self.barrier_control.max_collected_epoch().unwrap_or(0), + self.backfill_epoch, + )) } } - pub(super) fn may_inject_fake_barrier( - &mut self, + fn inject_barrier( + table_id: TableId, control_stream_manager: &mut ControlStreamManager, - upstream_prev_epoch: u64, - is_checkpoint: bool, + barrier_control: &mut CreatingStreamingJobBarrierControl, + pre_applied_graph_info: &InflightGraphInfo, + applied_graph_info: Option<&InflightGraphInfo>, + CreatingJobInjectBarrierInfo { + curr_epoch, + prev_epoch, + kind, + new_actors, + mutation, + }: CreatingJobInjectBarrierInfo, ) -> MetaResult<()> { - if let Some((barriers_to_inject, graph_info)) = - self.status.may_inject_fake_barrier(is_checkpoint) - { - if let Some(graph_info) = graph_info { - info!( - table_id = self.info.table_fragments.table_id().table_id, - upstream_prev_epoch, "start consuming log store" - ); - self.status = CreatingStreamingJobStatus::ConsumingLogStore { - graph_info, - start_consume_log_store_epoch: upstream_prev_epoch, - }; - } - let graph_info = self - .status - .active_graph_info() - .expect("must exist when having barriers to inject"); - let table_id = self.info.table_fragments.table_id(); - for CreatingJobInjectBarrierInfo { - curr_epoch, - prev_epoch, - kind, - new_actors, - mutation, - } in barriers_to_inject - { - let node_to_collect = control_stream_manager.inject_barrier( - Some(table_id), - mutation, - (&curr_epoch, &prev_epoch), - &kind, - graph_info, - Some(graph_info), - new_actors, - vec![], - vec![], - )?; - self.barrier_control.enqueue_epoch( - prev_epoch.value().0, - node_to_collect, - kind.is_checkpoint(), - CreatingStreamingJobBarrierType::Snapshot, - ); - } - } + let node_to_collect = control_stream_manager.inject_barrier( + Some(table_id), + mutation, + (&curr_epoch, &prev_epoch), + &kind, + pre_applied_graph_info, + applied_graph_info, + new_actors, + vec![], + vec![], + )?; + barrier_control.enqueue_epoch(prev_epoch.value().0, node_to_collect, kind.is_checkpoint()); Ok(()) } @@ -267,7 +199,7 @@ impl CreatingStreamingJobControl { &mut self, control_stream_manager: &mut ControlStreamManager, command_ctx: &Arc, - ) -> MetaResult>> { + ) -> MetaResult<()> { let table_id = self.info.table_fragments.table_id(); let start_consume_upstream = if let Command::MergeSnapshotBackfillStreamingJobs( jobs_to_merge, @@ -277,6 +209,13 @@ impl CreatingStreamingJobControl { } else { false }; + if start_consume_upstream { + info!( + table_id = self.info.table_fragments.table_id().table_id, + prev_epoch = command_ctx.prev_epoch.value().0, + "start consuming upstream" + ); + } let progress_epoch = if let Some(max_collected_epoch) = self.barrier_control.max_collected_epoch() { max(max_collected_epoch, self.backfill_epoch) @@ -290,125 +229,24 @@ impl CreatingStreamingJobControl { .0 .saturating_sub(progress_epoch) as _, ); - let graph_to_finish = match &mut self.status { - CreatingStreamingJobStatus::ConsumingSnapshot { - pending_commands, .. - } => { - assert!( - !start_consume_upstream, - "should not start consuming upstream for a job that are consuming snapshot" - ); - pending_commands.push(command_ctx.clone()); - None - } - CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } => { - let node_to_collect = control_stream_manager.inject_barrier( - Some(table_id), - if start_consume_upstream { - // erase the mutation on upstream except the last command - command_ctx.to_mutation() - } else { - None - }, - (&command_ctx.curr_epoch, &command_ctx.prev_epoch), - &command_ctx.kind, - graph_info, - Some(graph_info), - None, - vec![], - vec![], - )?; - self.barrier_control.enqueue_epoch( - command_ctx.prev_epoch.value().0, - node_to_collect, - command_ctx.kind.is_checkpoint(), - CreatingStreamingJobBarrierType::LogStore, - ); - let prev_epoch = command_ctx.prev_epoch.value().0; + if let Some(barrier_to_inject) = self + .status + .on_new_upstream_epoch(command_ctx, start_consume_upstream) + { + Self::inject_barrier( + self.info.table_fragments.table_id(), + control_stream_manager, + &mut self.barrier_control, + &self.graph_info, if start_consume_upstream { - let graph_info = take(graph_info); - info!( - table_id = self.info.table_fragments.table_id().table_id, - prev_epoch, "start consuming upstream" - ); - self.status = CreatingStreamingJobStatus::ConsumingUpstream { - start_consume_upstream_epoch: prev_epoch, - graph_info, - }; - } - None - } - CreatingStreamingJobStatus::ConsumingUpstream { - start_consume_upstream_epoch, - graph_info, - } => { - assert!( - !start_consume_upstream, - "should not start consuming upstream for a job again" - ); - - let should_finish = command_ctx.kind.is_checkpoint() - && self.barrier_control.unattached_epochs().next().is_none(); - let node_to_collect = control_stream_manager.inject_barrier( - Some(table_id), - // do not send the upstream barrier mutation because in `ConsumingUpstream` stage, - // barriers are still injected and collected independently on the creating jobs. - None, - (&command_ctx.curr_epoch, &command_ctx.prev_epoch), - &command_ctx.kind, - graph_info, - if should_finish { - None - } else { - Some(graph_info) - }, - None, - vec![], - vec![], - )?; - let prev_epoch = command_ctx.prev_epoch.value().0; - self.barrier_control.enqueue_epoch( - prev_epoch, - node_to_collect, - command_ctx.kind.is_checkpoint(), - CreatingStreamingJobBarrierType::Upstream, - ); - let graph_info = if should_finish { - info!(prev_epoch, table_id = ?self.info.table_fragments.table_id(), "mark as finishing"); - self.barrier_control - .attach_upstream_epoch(prev_epoch, prev_epoch); - let graph_info = take(graph_info); - self.status = CreatingStreamingJobStatus::Finishing { - start_consume_upstream_epoch: *start_consume_upstream_epoch, - }; - Some(Some(graph_info)) + None } else { - let mut unattached_epochs_iter = self.barrier_control.unattached_epochs(); - let mut epoch_to_attach = unattached_epochs_iter.next().expect("non-empty").0; - let mut remain_count = 5; - while remain_count > 0 - && let Some((epoch, _)) = unattached_epochs_iter.next() - { - remain_count -= 1; - epoch_to_attach = epoch; - } - drop(unattached_epochs_iter); - self.barrier_control - .attach_upstream_epoch(epoch_to_attach, prev_epoch); - Some(None) - }; - - graph_info - } - CreatingStreamingJobStatus::Finishing { .. } => { - assert!( - !start_consume_upstream, - "should not start consuming upstream for a job again" - ); - None - } - }; - Ok(graph_to_finish) + Some(&self.graph_info) + }, + barrier_to_inject, + )?; + } + Ok(()) } pub(super) fn collect( @@ -416,57 +254,99 @@ impl CreatingStreamingJobControl { epoch: u64, worker_id: WorkerId, resp: BarrierCompleteResponse, - ) { - self.status.update_progress(&resp.create_mview_progress); + control_stream_manager: &mut ControlStreamManager, + ) -> MetaResult<()> { + let prev_barriers_to_inject = self.status.update_progress(&resp.create_mview_progress); self.barrier_control.collect(epoch, worker_id, resp); + if let Some(prev_barriers_to_inject) = prev_barriers_to_inject { + let table_id = self.info.table_fragments.table_id(); + for info in prev_barriers_to_inject { + Self::inject_barrier( + table_id, + control_stream_manager, + &mut self.barrier_control, + &self.graph_info, + Some(&self.graph_info), + info, + )?; + } + } + Ok(()) } pub(super) fn should_merge_to_upstream(&self) -> Option { - if let ( - CreatingStreamingJobStatus::ConsumingLogStore { - graph_info, - start_consume_log_store_epoch, - }, - Some(max_collected_epoch), - ) = (&self.status, self.barrier_control.max_collected_epoch()) + if let CreatingStreamingJobStatus::ConsumingLogStore { + ref log_store_progress_tracker, + } = &self.status + && log_store_progress_tracker.is_finished() { - if max_collected_epoch >= *start_consume_log_store_epoch { - Some(graph_info.clone()) - } else { - let lag = Duration::from_millis( - Epoch(*start_consume_log_store_epoch).physical_time() - - Epoch(max_collected_epoch).physical_time(), - ); - debug!( - ?lag, - max_collected_epoch, start_consume_log_store_epoch, "wait consuming log store" - ); - None - } + Some(self.graph_info.clone()) } else { None } } +} - #[expect(clippy::type_complexity)] +pub(super) enum CompleteJobType { + /// The first barrier + First, + Normal, + /// The last barrier to complete + Finished, +} + +impl CreatingStreamingJobControl { pub(super) fn start_completing( &mut self, - ) -> (Vec, Option<(u64, Vec, bool)>) { - self.barrier_control.start_completing() + min_upstream_inflight_epoch: Option, + ) -> Option<(u64, Vec, CompleteJobType)> { + let (finished_at_epoch, epoch_end_bound) = match &self.status { + CreatingStreamingJobStatus::Finishing(finish_at_epoch) => { + let epoch_end_bound = min_upstream_inflight_epoch + .map(|upstream_epoch| { + if upstream_epoch < *finish_at_epoch { + Excluded(upstream_epoch) + } else { + Unbounded + } + }) + .unwrap_or(Unbounded); + (Some(*finish_at_epoch), epoch_end_bound) + } + CreatingStreamingJobStatus::ConsumingSnapshot { .. } + | CreatingStreamingJobStatus::ConsumingLogStore { .. } => ( + None, + min_upstream_inflight_epoch + .map(Excluded) + .unwrap_or(Unbounded), + ), + }; + self.barrier_control.start_completing(epoch_end_bound).map( + |(epoch, resps, is_first_commit)| { + let status = if let Some(finish_at_epoch) = finished_at_epoch { + assert!(!is_first_commit); + if epoch == finish_at_epoch { + self.barrier_control.ack_completed(epoch); + assert!(self.barrier_control.is_empty()); + CompleteJobType::Finished + } else { + CompleteJobType::Normal + } + } else if is_first_commit { + CompleteJobType::First + } else { + CompleteJobType::Normal + }; + (epoch, resps, status) + }, + ) } - pub(super) fn ack_completed(&mut self, completed_epoch: u64) -> Option<(u64, bool)> { - let upstream_epoch_to_notify = self.barrier_control.ack_completed(completed_epoch); - if let Some(upstream_epoch_to_notify) = upstream_epoch_to_notify { - Some((upstream_epoch_to_notify, self.is_finished())) - } else { - assert!(!self.is_finished()); - None - } + pub(super) fn ack_completed(&mut self, completed_epoch: u64) { + self.barrier_control.ack_completed(completed_epoch); } pub(super) fn is_finished(&self) -> bool { - self.barrier_control.is_empty() - && matches!(&self.status, CreatingStreamingJobStatus::Finishing { .. }) + self.barrier_control.is_empty() && self.status.is_finishing() } } diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/creating_job/status.rs index aaae86d0a2144..093747249f1df 100644 --- a/src/meta/src/barrier/creating_job/status.rs +++ b/src/meta/src/barrier/creating_job/status.rs @@ -12,30 +12,101 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::hash_map::Entry; +use std::collections::{HashMap, HashSet}; use std::mem::take; use std::sync::Arc; +use risingwave_common::hash::ActorId; use risingwave_common::util::epoch::Epoch; use risingwave_meta_model_v2::WorkerId; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::StreamActor; -use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; +use risingwave_pb::stream_service::barrier_complete_response::{ + CreateMviewProgress, PbCreateMviewProgress, +}; +use tracing::warn; use crate::barrier::command::CommandContext; -use crate::barrier::info::InflightGraphInfo; use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::{BarrierKind, TracedEpoch}; +#[derive(Debug)] +pub(super) struct CreateMviewLogStoreProgressTracker { + /// `actor_id` -> `pending_barrier_count` + ongoing_actors: HashMap, + finished_actors: HashSet, +} + +impl CreateMviewLogStoreProgressTracker { + fn new(actors: impl Iterator, initial_pending_count: usize) -> Self { + Self { + ongoing_actors: HashMap::from_iter(actors.map(|actor| (actor, initial_pending_count))), + finished_actors: HashSet::new(), + } + } + + pub(super) fn gen_ddl_progress(&self) -> String { + let sum = self.ongoing_actors.values().sum::() as f64; + let count = if self.ongoing_actors.is_empty() { + 1 + } else { + self.ongoing_actors.len() + } as f64; + let avg = sum / count; + format!( + "finished: {}/{}, avg epoch count {}", + self.finished_actors.len(), + self.ongoing_actors.len() + self.finished_actors.len(), + avg + ) + } + + fn update(&mut self, progress: impl IntoIterator) { + for progress in progress { + match self.ongoing_actors.entry(progress.backfill_actor_id) { + Entry::Occupied(mut entry) => { + if progress.done { + entry.remove_entry(); + assert!( + self.finished_actors.insert(progress.backfill_actor_id), + "non-duplicate" + ); + } else { + *entry.get_mut() = progress.pending_barrier_num as _; + } + } + Entry::Vacant(_) => { + if cfg!(debug_assertions) { + panic!( + "reporting progress on non-inflight actor: {:?} {:?}", + progress, self + ); + } else { + warn!(?progress, progress_tracker = ?self, "reporting progress on non-inflight actor"); + } + } + } + } + } + + pub(super) fn is_finished(&self) -> bool { + self.ongoing_actors.is_empty() + } +} + #[derive(Debug)] pub(super) enum CreatingStreamingJobStatus { + /// The creating job is consuming upstream snapshot. + /// Will transit to `ConsumingLogStore` on `update_progress` when + /// the snapshot has been fully consumed after `update_progress`. ConsumingSnapshot { prev_epoch_fake_physical_time: u64, - pending_commands: Vec>, + pending_upstream_barriers: Vec<(TracedEpoch, TracedEpoch, BarrierKind)>, version_stats: HummockVersionStats, create_mview_tracker: CreateMviewProgressTracker, - graph_info: InflightGraphInfo, + snapshot_backfill_actors: HashSet, backfill_epoch: u64, /// The `prev_epoch` of pending non checkpoint barriers pending_non_checkpoint_barriers: Vec, @@ -43,17 +114,16 @@ pub(super) enum CreatingStreamingJobStatus { /// Take the mutation out when injecting the first barrier initial_barrier_info: Option<(HashMap>, Mutation)>, }, + /// The creating job is consuming log store. + /// + /// Will transit to `Finishing` on `on_new_upstream_epoch` when `start_consume_upstream` is `true`. ConsumingLogStore { - graph_info: InflightGraphInfo, - start_consume_log_store_epoch: u64, - }, - ConsumingUpstream { - start_consume_upstream_epoch: u64, - graph_info: InflightGraphInfo, - }, - Finishing { - start_consume_upstream_epoch: u64, + log_store_progress_tracker: CreateMviewLogStoreProgressTracker, }, + /// All backfill actors have started consuming upstream, and the job + /// will be finished when all previously injected barriers have been collected + /// Store the `prev_epoch` that will finish at. + Finishing(u64), } pub(super) struct CreatingJobInjectBarrierInfo { @@ -65,79 +135,138 @@ pub(super) struct CreatingJobInjectBarrierInfo { } impl CreatingStreamingJobStatus { - pub(super) fn active_graph_info(&self) -> Option<&InflightGraphInfo> { - match self { - CreatingStreamingJobStatus::ConsumingSnapshot { graph_info, .. } - | CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } - | CreatingStreamingJobStatus::ConsumingUpstream { graph_info, .. } => Some(graph_info), - CreatingStreamingJobStatus::Finishing { .. } => { - // when entering `Finishing`, the graph will have been added to the upstream graph, - // and therefore the separate graph info is inactive. - None - } - } - } - pub(super) fn update_progress( &mut self, create_mview_progress: impl IntoIterator, - ) { - if let Self::ConsumingSnapshot { - create_mview_tracker, - ref version_stats, - .. - } = self - { - create_mview_tracker.update_tracking_jobs(None, create_mview_progress, version_stats); - } - } - - /// return - /// - Some(vec[(`curr_epoch`, `prev_epoch`, `barrier_kind`)]) of barriers to newly inject - /// - Some(`graph_info`) when the status should transit to `ConsumingLogStore` - pub(super) fn may_inject_fake_barrier( - &mut self, - is_checkpoint: bool, - ) -> Option<(Vec, Option)> { - if let CreatingStreamingJobStatus::ConsumingSnapshot { - prev_epoch_fake_physical_time, - pending_commands, - create_mview_tracker, - graph_info, - pending_non_checkpoint_barriers, - ref backfill_epoch, - initial_barrier_info, - .. - } = self - { - if create_mview_tracker.has_pending_finished_jobs() { - assert!(initial_barrier_info.is_none()); - pending_non_checkpoint_barriers.push(*backfill_epoch); + ) -> Option> { + match self { + Self::ConsumingSnapshot { + create_mview_tracker, + ref version_stats, + prev_epoch_fake_physical_time, + pending_upstream_barriers, + pending_non_checkpoint_barriers, + ref backfill_epoch, + initial_barrier_info, + ref snapshot_backfill_actors, + .. + } => { + create_mview_tracker.update_tracking_jobs( + None, + create_mview_progress, + version_stats, + ); + if create_mview_tracker.has_pending_finished_jobs() { + let (new_actors, mutation) = match initial_barrier_info.take() { + Some((new_actors, mutation)) => (Some(new_actors), Some(mutation)), + None => (None, None), + }; + assert!(initial_barrier_info.is_none()); + pending_non_checkpoint_barriers.push(*backfill_epoch); - let prev_epoch = Epoch::from_physical_time(*prev_epoch_fake_physical_time); - let barriers_to_inject = - [CreatingJobInjectBarrierInfo { + let prev_epoch = Epoch::from_physical_time(*prev_epoch_fake_physical_time); + let barriers_to_inject: Vec<_> = [CreatingJobInjectBarrierInfo { curr_epoch: TracedEpoch::new(Epoch(*backfill_epoch)), prev_epoch: TracedEpoch::new(prev_epoch), kind: BarrierKind::Checkpoint(take(pending_non_checkpoint_barriers)), - new_actors: None, - mutation: None, + new_actors, + mutation, }] .into_iter() - .chain(pending_commands.drain(..).map(|command_ctx| { - CreatingJobInjectBarrierInfo { - curr_epoch: command_ctx.curr_epoch.clone(), - prev_epoch: command_ctx.prev_epoch.clone(), - kind: command_ctx.kind.clone(), + .chain(pending_upstream_barriers.drain(..).map( + |(prev_epoch, curr_epoch, kind)| CreatingJobInjectBarrierInfo { + curr_epoch, + prev_epoch, + kind, new_actors: None, mutation: None, - } - })) + }, + )) .collect(); - let graph_info = take(graph_info); - Some((barriers_to_inject, Some(graph_info))) - } else { + *self = CreatingStreamingJobStatus::ConsumingLogStore { + log_store_progress_tracker: CreateMviewLogStoreProgressTracker::new( + snapshot_backfill_actors.iter().cloned(), + barriers_to_inject.len(), + ), + }; + Some(barriers_to_inject) + } else { + None + } + } + CreatingStreamingJobStatus::ConsumingLogStore { + log_store_progress_tracker, + .. + } => { + log_store_progress_tracker.update(create_mview_progress); + None + } + CreatingStreamingJobStatus::Finishing(_) => None, + } + } + + pub(super) fn on_new_upstream_epoch( + &mut self, + command_ctx: &Arc, + start_consume_upstream: bool, + ) -> Option { + match self { + CreatingStreamingJobStatus::ConsumingSnapshot { + pending_upstream_barriers, + prev_epoch_fake_physical_time, + pending_non_checkpoint_barriers, + initial_barrier_info, + .. + } => { + assert!( + !start_consume_upstream, + "should not start consuming upstream for a job that are consuming snapshot" + ); + pending_upstream_barriers.push(( + command_ctx.prev_epoch.clone(), + command_ctx.curr_epoch.clone(), + command_ctx.kind.clone(), + )); + Some(CreatingStreamingJobStatus::new_fake_barrier( + prev_epoch_fake_physical_time, + pending_non_checkpoint_barriers, + initial_barrier_info, + command_ctx.kind.is_checkpoint(), + )) + } + CreatingStreamingJobStatus::ConsumingLogStore { .. } => { + let prev_epoch = command_ctx.prev_epoch.value().0; + if start_consume_upstream { + assert!(command_ctx.kind.is_checkpoint()); + *self = CreatingStreamingJobStatus::Finishing(prev_epoch); + } + Some(CreatingJobInjectBarrierInfo { + curr_epoch: command_ctx.curr_epoch.clone(), + prev_epoch: command_ctx.prev_epoch.clone(), + kind: command_ctx.kind.clone(), + new_actors: None, + mutation: None, + }) + } + CreatingStreamingJobStatus::Finishing { .. } => { + assert!( + !start_consume_upstream, + "should not start consuming upstream for a job again" + ); + None + } + } + } + + pub(super) fn new_fake_barrier( + prev_epoch_fake_physical_time: &mut u64, + pending_non_checkpoint_barriers: &mut Vec, + initial_barrier_info: &mut Option<(HashMap>, Mutation)>, + is_checkpoint: bool, + ) -> CreatingJobInjectBarrierInfo { + { + { let prev_epoch = TracedEpoch::new(Epoch::from_physical_time(*prev_epoch_fake_physical_time)); *prev_epoch_fake_physical_time += 1; @@ -155,19 +284,18 @@ impl CreatingStreamingJobStatus { } else { Default::default() }; - Some(( - vec![CreatingJobInjectBarrierInfo { - curr_epoch, - prev_epoch, - kind, - new_actors, - mutation, - }], - None, - )) + CreatingJobInjectBarrierInfo { + curr_epoch, + prev_epoch, + kind, + new_actors, + mutation, + } } - } else { - None } } + + pub(super) fn is_finishing(&self) -> bool { + matches!(self, Self::Finishing(_)) + } } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 120c6d8518bbb..d349e7bbfe0a4 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -12,10 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; -use std::future::pending; +use std::future::{pending, Future}; use std::mem::{replace, take}; use std::sync::Arc; use std::time::Duration; @@ -51,10 +50,11 @@ use tracing::{debug, error, info, warn, Instrument}; use self::command::CommandContext; use self::notifier::Notifier; -use crate::barrier::creating_job::CreatingStreamingJobControl; +use crate::barrier::creating_job::{CompleteJobType, CreatingStreamingJobControl}; use crate::barrier::info::InflightGraphInfo; use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob}; use crate::barrier::rpc::{merge_node_rpc_errors, ControlStreamManager}; +use crate::barrier::schedule::ScheduledBarriers; use crate::barrier::state::BarrierManagerState; use crate::error::MetaErrorInner; use crate::hummock::{CommitEpochInfo, HummockManagerRef, NewTableFragmentInfo}; @@ -218,7 +218,7 @@ struct CheckpointControl { /// Command that has been collected but is still completing. /// The join handle of the completing future is stored. - completing_command: CompletingCommand, + completing_task: CompletingTask, hummock_version_stats: HummockVersionStats, @@ -235,7 +235,7 @@ impl CheckpointControl { Self { command_ctx_queue: Default::default(), creating_streaming_job_controls: Default::default(), - completing_command: CompletingCommand::None, + completing_task: CompletingTask::None, hummock_version_stats: context.hummock_manager.get_version_stats().await, create_mview_tracker, context, @@ -244,8 +244,11 @@ impl CheckpointControl { fn total_command_num(&self) -> usize { self.command_ctx_queue.len() - + match &self.completing_command { - CompletingCommand::GlobalStreamingGraph { .. } => 1, + + match &self.completing_task { + CompletingTask::Completing { + command_ctx: Some(_), + .. + } => 1, _ => 0, } } @@ -291,8 +294,7 @@ impl CheckpointControl { command_ctx: Arc, notifiers: Vec, node_to_collect: HashSet, - jobs_to_wait: HashSet, - table_ids_to_commit: HashSet, + creating_jobs_to_wait: HashSet, ) { let timer = self.context.metrics.barrier_latency.start_timer(); @@ -305,27 +307,9 @@ impl CheckpointControl { tracing::trace!( prev_epoch = command_ctx.prev_epoch.value().0, - ?jobs_to_wait, + ?creating_jobs_to_wait, "enqueue command" ); - let creating_jobs_to_wait = jobs_to_wait - .into_iter() - .map(|table_id| { - ( - table_id, - if node_to_collect.is_empty() { - Some( - self.creating_streaming_job_controls - .get(&table_id) - .expect("should exist") - .start_wait_progress_timer(), - ) - } else { - None - }, - ) - }) - .collect(); self.command_ctx_queue.insert( command_ctx.prev_epoch.value().0, EpochNode { @@ -334,8 +318,7 @@ impl CheckpointControl { node_to_collect, resps: vec![], creating_jobs_to_wait, - finished_table_ids: HashMap::new(), - table_ids_to_commit, + finished_jobs: HashMap::new(), }, command_ctx, notifiers, @@ -345,7 +328,11 @@ impl CheckpointControl { /// Change the state of this `prev_epoch` to `Completed`. Return continuous nodes /// with `Completed` starting from first node [`Completed`..`InFlight`) and remove them. - fn barrier_collected(&mut self, resp: BarrierCompleteResponse) { + fn barrier_collected( + &mut self, + resp: BarrierCompleteResponse, + control_stream_manager: &mut ControlStreamManager, + ) -> MetaResult<()> { let worker_id = resp.worker_id; let prev_epoch = resp.epoch; tracing::trace!( @@ -357,19 +344,6 @@ impl CheckpointControl { if resp.partial_graph_id == u32::MAX { if let Some(node) = self.command_ctx_queue.get_mut(&prev_epoch) { assert!(node.state.node_to_collect.remove(&(worker_id as _))); - if node.state.node_to_collect.is_empty() { - node.state - .creating_jobs_to_wait - .iter_mut() - .for_each(|(table_id, timer)| { - *timer = Some( - self.creating_streaming_job_controls - .get(table_id) - .expect("should exist") - .start_wait_progress_timer(), - ); - }); - } node.state.resps.push(resp); } else { panic!( @@ -382,8 +356,9 @@ impl CheckpointControl { self.creating_streaming_job_controls .get_mut(&creating_table_id) .expect("should exist") - .collect(prev_epoch, worker_id as _, resp); + .collect(prev_epoch, worker_id as _, resp, control_stream_manager)?; } + Ok(()) } /// Pause inject barrier until True. @@ -400,11 +375,9 @@ impl CheckpointControl { .command_ctx_queue .last_key_value() .map(|(_, x)| &x.command_ctx) - .or(match &self.completing_command { - CompletingCommand::None - | CompletingCommand::Err(_) - | CompletingCommand::CreatingStreamingJob { .. } => None, - CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => Some(command_ctx), + .or(match &self.completing_task { + CompletingTask::None | CompletingTask::Err(_) => None, + CompletingTask::Completing { command_ctx, .. } => command_ctx.as_ref(), }) .map(|command_ctx| command_ctx.command.should_pause_inject_barrier()) .unwrap_or(false); @@ -413,12 +386,9 @@ impl CheckpointControl { .values() .map(|node| &node.command_ctx) .chain( - match &self.completing_command { - CompletingCommand::None - | CompletingCommand::Err(_) - | CompletingCommand::CreatingStreamingJob { .. } => None, - CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => - Some(command_ctx), + match &self.completing_task { + CompletingTask::None | CompletingTask::Err(_) => None, + CompletingTask::Completing { command_ctx, .. } => command_ctx.as_ref(), } .into_iter() ) @@ -432,32 +402,10 @@ impl CheckpointControl { /// We need to make sure there are no changes when doing recovery pub async fn clear_on_err(&mut self, err: &MetaError) { // join spawned completing command to finish no matter it succeeds or not. - let is_err = match replace(&mut self.completing_command, CompletingCommand::None) { - CompletingCommand::None => false, - CompletingCommand::GlobalStreamingGraph { - command_ctx, - join_handle, - .. - } => { - info!( - prev_epoch = ?command_ctx.prev_epoch, - curr_epoch = ?command_ctx.curr_epoch, - "waiting for completing command to finish in recovery" - ); - match join_handle.await { - Err(e) => { - warn!(err = ?e.as_report(), "failed to join completing task"); - true - } - Ok(Err(e)) => { - warn!(err = ?e.as_report(), "failed to complete barrier during clear"); - true - } - Ok(Ok(_)) => false, - } - } - CompletingCommand::Err(_) => true, - CompletingCommand::CreatingStreamingJob { join_handle, .. } => { + let is_err = match replace(&mut self.completing_task, CompletingTask::None) { + CompletingTask::None => false, + CompletingTask::Completing { join_handle, .. } => { + info!("waiting for completing command to finish in recovery"); match join_handle.await { Err(e) => { warn!(err = ?e.as_report(), "failed to join completing task"); @@ -470,38 +418,19 @@ impl CheckpointControl { Ok(Ok(_)) => false, } } + CompletingTask::Err(_) => true, }; if !is_err { // continue to finish the pending collected barrier. - while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() - { - let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); - let (prev_epoch, curr_epoch) = ( - node.command_ctx.prev_epoch.value().0, - node.command_ctx.curr_epoch.value().0, - ); - let finished_jobs = self - .create_mview_tracker - .apply_collected_command(&node, &self.hummock_version_stats); - if let Err(e) = self - .context - .clone() - .complete_barrier(node, finished_jobs, HashMap::new()) - .await - { + while let Some(task) = self.next_complete_barrier_task(None) { + if let Err(e) = self.context.clone().complete_barrier(task).await { error!( - prev_epoch, - curr_epoch, err = ?e.as_report(), "failed to complete barrier during recovery" ); break; } else { - info!( - prev_epoch, - curr_epoch, "succeed to complete barrier during recovery" - ) + info!("succeed to complete barrier during recovery") } } } @@ -545,11 +474,9 @@ struct BarrierEpochState { resps: Vec, - creating_jobs_to_wait: HashMap>, - - finished_table_ids: HashMap, + creating_jobs_to_wait: HashSet, - table_ids_to_commit: HashSet, + finished_jobs: HashMap)>, } impl BarrierEpochState { @@ -558,22 +485,17 @@ impl BarrierEpochState { } } -enum CompletingCommand { +enum CompletingTask { None, - GlobalStreamingGraph { - command_ctx: Arc, + Completing { + command_ctx: Option>, table_ids_to_finish: HashSet, - require_next_checkpoint: bool, + creating_job_epochs: Vec<(TableId, u64)>, // The join handle of a spawned task that completes the barrier. // The return value indicate whether there is some create streaming job command // that has finished but not checkpointed. If there is any, we will force checkpoint on the next barrier - join_handle: JoinHandle>>, - }, - CreatingStreamingJob { - table_id: TableId, - epoch: u64, - join_handle: JoinHandle>, + join_handle: JoinHandle>, }, #[expect(dead_code)] Err(MetaError), @@ -841,12 +763,8 @@ impl GlobalBarrierManager { } } (worker_id, resp_result) = self.control_stream_manager.next_complete_barrier_response() => { - match resp_result { - Ok(resp) => { - self.checkpoint_control.barrier_collected(resp); - - } - Err(e) => { + if let Err(e) = resp_result.and_then(|resp| self.checkpoint_control.barrier_collected(resp, &mut self.control_stream_manager)) { + { let failed_command = self.checkpoint_control.command_wait_collect_from_worker(worker_id as _); if failed_command.is_some() || self.state.inflight_graph_info.contains_worker(worker_id as _) @@ -863,22 +781,15 @@ impl GlobalBarrierManager { } } } - complete_result = self.checkpoint_control.next_completed_barrier() => { + complete_result = self.checkpoint_control.next_completed_barrier(&mut self.scheduled_barriers) => { match complete_result { - Ok(Some(output)) => { - // If there are remaining commands (that requires checkpoint to finish), we force - // the next barrier to be a checkpoint. - if output.require_next_checkpoint { - assert_matches!(output.command_ctx.kind, BarrierKind::Barrier); - self.scheduled_barriers.force_checkpoint_in_next_barrier(); - } + Ok(output) => { if !output.table_ids_to_finish.is_empty() { self.control_stream_manager.remove_partial_graph( output.table_ids_to_finish.iter().map(|table_id| table_id.table_id).collect() ); } } - Ok(None) => {} Err(e) => { self.failure_recovery(e).await; } @@ -986,19 +897,6 @@ impl GlobalBarrierManager { ); } - // may inject fake barrier - for creating_job in self - .checkpoint_control - .creating_streaming_job_controls - .values_mut() - { - creating_job.may_inject_fake_barrier( - &mut self.control_stream_manager, - prev_epoch.value().0, - checkpoint, - )? - } - self.pending_non_checkpoint_barriers .push(prev_epoch.value().0); let kind = if checkpoint { @@ -1017,8 +915,14 @@ impl GlobalBarrierManager { command = Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge); } - let (pre_applied_graph_info, pre_applied_subscription_info) = - self.state.apply_command(&command); + let command = command; + + let ( + pre_applied_graph_info, + pre_applied_subscription_info, + table_ids_to_commit, + jobs_to_wait, + ) = self.state.apply_command(&command); // Tracing related stuff prev_epoch.span().in_scope(|| { @@ -1026,14 +930,12 @@ impl GlobalBarrierManager { }); span.record("epoch", curr_epoch.value().0); - let table_ids_to_commit: HashSet<_> = pre_applied_graph_info.existing_table_ids().collect(); - let command_ctx = Arc::new(CommandContext::new( self.active_streaming_nodes.current().clone(), pre_applied_subscription_info, prev_epoch.clone(), curr_epoch.clone(), - table_ids_to_commit.clone(), + table_ids_to_commit, self.state.paused_reason(), command, kind, @@ -1043,18 +945,12 @@ impl GlobalBarrierManager { send_latency_timer.observe_duration(); - let mut jobs_to_wait = HashSet::new(); - - for (table_id, creating_job) in &mut self.checkpoint_control.creating_streaming_job_controls + for creating_job in &mut self + .checkpoint_control + .creating_streaming_job_controls + .values_mut() { - if let Some(wait_job) = - creating_job.on_new_command(&mut self.control_stream_manager, &command_ctx)? - { - jobs_to_wait.insert(*table_id); - if let Some(graph_to_finish) = wait_job { - self.state.inflight_graph_info.extend(graph_to_finish); - } - } + creating_job.on_new_command(&mut self.control_stream_manager, &command_ctx)?; } let node_to_collect = match self.control_stream_manager.inject_command_ctx_barrier( @@ -1085,7 +981,6 @@ impl GlobalBarrierManager { notifiers, node_to_collect, jobs_to_wait, - table_ids_to_commit, ); Ok(()) @@ -1150,140 +1045,86 @@ impl GlobalBarrierManager { } } +#[derive(Default)] +struct CompleteBarrierTask { + commit_info: CommitEpochInfo, + finished_jobs: Vec, + notifiers: Vec, + /// Some((`command_ctx`, `enqueue_time`)) + command_context: Option<(Arc, HistogramTimer)>, + table_ids_to_finish: HashSet, + creating_job_epochs: Vec<(TableId, u64)>, +} + impl GlobalBarrierManagerContext { - async fn complete_creating_job_barrier( - self, + fn collect_creating_job_commit_epoch_info( + commit_info: &mut CommitEpochInfo, epoch: u64, resps: Vec, - tables_to_commit: HashSet, + tables_to_commit: impl Iterator, is_first_time: bool, - ) -> MetaResult<()> { + ) { let (sst_to_context, sstables, new_table_watermarks, old_value_sst) = collect_resp_info(resps); assert!(old_value_sst.is_empty()); - let new_table_fragment_info = if is_first_time { - NewTableFragmentInfo::NewCompactionGroup { - table_ids: tables_to_commit.clone(), - } - } else { - NewTableFragmentInfo::None - }; - let info = CommitEpochInfo { - sstables, - new_table_watermarks, - sst_to_context, - new_table_fragment_info, - change_log_delta: Default::default(), - committed_epoch: epoch, - tables_to_commit, + commit_info.sst_to_context.extend(sst_to_context); + commit_info.sstables.extend(sstables); + commit_info + .new_table_watermarks + .extend(new_table_watermarks); + let tables_to_commit: HashSet<_> = tables_to_commit.collect(); + tables_to_commit.iter().for_each(|table_id| { + commit_info + .tables_to_commit + .try_insert(*table_id, epoch) + .expect("non duplicate"); + }); + if is_first_time { + commit_info + .new_table_fragment_infos + .push(NewTableFragmentInfo::NewCompactionGroup { + table_ids: tables_to_commit, + }); }; - self.hummock_manager.commit_epoch(info).await?; - Ok(()) } - async fn complete_barrier( - self, - node: EpochNode, - mut finished_jobs: Vec, - backfill_pinned_log_epoch: HashMap)>, - ) -> MetaResult> { - tracing::trace!( - prev_epoch = node.command_ctx.prev_epoch.value().0, - kind = ?node.command_ctx.kind, - "complete barrier" - ); - let EpochNode { - command_ctx, - notifiers, - enqueue_time, - state, - .. - } = node; - assert!(state.node_to_collect.is_empty()); - assert!(state.creating_jobs_to_wait.is_empty()); - let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); - if !state.finished_table_ids.is_empty() { - assert!(command_ctx.kind.is_checkpoint()); - } - finished_jobs.extend(state.finished_table_ids.into_values().map(|info| { - TrackingJob::New(TrackingCommand { - info, - replace_table_info: None, - }) - })); - - let result = self - .update_snapshot( - &command_ctx, - state.table_ids_to_commit, - state.resps, - backfill_pinned_log_epoch, - ) - .await; - - let version_stats = match result { - Ok(version_stats) => version_stats, - Err(e) => { - for notifier in notifiers { - notifier.notify_collection_failed(e.clone()); - } - return Err(e); + async fn complete_barrier(self, task: CompleteBarrierTask) -> MetaResult { + let result: MetaResult<()> = try { + let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); + self.hummock_manager.commit_epoch(task.commit_info).await?; + if let Some((command_ctx, _)) = &task.command_context { + command_ctx.post_collect().await?; } + + wait_commit_timer.observe_duration(); }; - notifiers.into_iter().for_each(|notifier| { - notifier.notify_collected(); - }); - try_join_all(finished_jobs.into_iter().map(|finished_job| { - let metadata_manager = &self.metadata_manager; - async move { finished_job.finish(metadata_manager).await } - })) - .await?; - let duration_sec = enqueue_time.stop_and_record(); - self.report_complete_event(duration_sec, &command_ctx); - wait_commit_timer.observe_duration(); - self.metrics - .last_committed_barrier_time - .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); - Ok(version_stats) - } - async fn update_snapshot( - &self, - command_ctx: &CommandContext, - tables_to_commit: HashSet, - resps: Vec, - backfill_pinned_log_epoch: HashMap)>, - ) -> MetaResult> { { - { - match &command_ctx.kind { - BarrierKind::Initial => {} - BarrierKind::Checkpoint(epochs) => { - let commit_info = collect_commit_epoch_info( - resps, - command_ctx, - epochs, - backfill_pinned_log_epoch, - tables_to_commit, - ); - self.hummock_manager.commit_epoch(commit_info).await?; - } - BarrierKind::Barrier => { - // if we collect a barrier(checkpoint = false), - // we need to ensure that command is Plain and the notifier's checkpoint is - // false - assert!(!command_ctx.command.need_checkpoint()); - } + if let Err(e) = result { + for notifier in task.notifiers { + notifier.notify_collection_failed(e.clone()); } - - command_ctx.post_collect().await?; - Ok(if command_ctx.kind.is_checkpoint() { - Some(self.hummock_manager.get_version_stats().await) - } else { - None - }) + return Err(e); + } + task.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + try_join_all( + task.finished_jobs + .into_iter() + .map(|finished_job| finished_job.finish(&self.metadata_manager)), + ) + .await?; + if let Some((command_ctx, enqueue_time)) = task.command_context { + let duration_sec = enqueue_time.stop_and_record(); + self.report_complete_event(duration_sec, &command_ctx); + self.metrics + .last_committed_barrier_time + .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); } } + + Ok(self.hummock_manager.get_version_stats().await) } pub fn hummock_manager(&self) -> &HummockManagerRef { @@ -1350,8 +1191,6 @@ impl GlobalBarrierManagerContext { } struct BarrierCompleteOutput { - command_ctx: Arc, - require_next_checkpoint: bool, table_ids_to_finish: HashSet, } @@ -1381,84 +1220,167 @@ impl CheckpointControl { .collect() } - pub(super) async fn next_completed_barrier( + fn next_complete_barrier_task( &mut self, - ) -> MetaResult> { - if matches!(&self.completing_command, CompletingCommand::None) { - // If there is no completing barrier, try to start completing the earliest barrier if - // it has been collected. - if let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() + mut scheduled_barriers: Option<&mut ScheduledBarriers>, + ) -> Option { + // `Vec::new` is a const fn, and do not have memory allocation, and therefore is lightweight enough + let mut creating_jobs_task = vec![]; + { + // `Vec::new` is a const fn, and do not have memory allocation, and therefore is lightweight enough + let mut finished_jobs = Vec::new(); + let min_upstream_inflight_barrier = self + .command_ctx_queue + .first_key_value() + .map(|(epoch, _)| *epoch); + for (table_id, job) in &mut self.creating_streaming_job_controls { + if let Some((epoch, resps, status)) = + job.start_completing(min_upstream_inflight_barrier) + { + let is_first_time = match status { + CompleteJobType::First => true, + CompleteJobType::Normal => false, + CompleteJobType::Finished => { + finished_jobs.push((*table_id, epoch, resps)); + continue; + } + }; + creating_jobs_task.push((*table_id, epoch, resps, is_first_time)); + } + } + for (table_id, epoch, resps) in finished_jobs { + let epoch_state = &mut self + .command_ctx_queue + .get_mut(&epoch) + .expect("should exist") + .state; + assert!(epoch_state.creating_jobs_to_wait.remove(&table_id)); + debug!(epoch, ?table_id, "finish creating job"); + // It's safe to remove the creating job, because on CompleteJobType::Finished, + // all previous barriers have been collected and completed. + let creating_streaming_job = self + .creating_streaming_job_controls + .remove(&table_id) + .expect("should exist"); + assert!(creating_streaming_job.is_finished()); + assert!(epoch_state + .finished_jobs + .insert(table_id, (creating_streaming_job.info, resps)) + .is_none()); + } + } + let mut task = None; + while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() + && !state.is_inflight() + { { - let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); + let (_, mut node) = self.command_ctx_queue.pop_first().expect("non-empty"); assert!(node.state.creating_jobs_to_wait.is_empty()); - let table_ids_to_finish = node.state.finished_table_ids.keys().cloned().collect(); - let finished_jobs = self + assert!(node.state.node_to_collect.is_empty()); + let mut finished_jobs = self .create_mview_tracker .apply_collected_command(&node, &self.hummock_version_stats); - let command_ctx = node.command_ctx.clone(); - let join_handle = tokio::spawn(self.context.clone().complete_barrier( - node, - finished_jobs, - self.collect_backfill_pinned_upstream_log_epoch(), - )); - let require_next_checkpoint = - if self.create_mview_tracker.has_pending_finished_jobs() { - self.command_ctx_queue + if !node.command_ctx.kind.is_checkpoint() { + assert!(finished_jobs.is_empty()); + node.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + if let Some(scheduled_barriers) = &mut scheduled_barriers + && self.create_mview_tracker.has_pending_finished_jobs() + && self + .command_ctx_queue .values() .all(|node| !node.command_ctx.kind.is_checkpoint()) - } else { - false - }; - self.completing_command = CompletingCommand::GlobalStreamingGraph { - command_ctx, - require_next_checkpoint, - join_handle, - table_ids_to_finish, - }; - } else { - for (table_id, job) in &mut self.creating_streaming_job_controls { - let (upstream_epochs_to_notify, commit_info) = job.start_completing(); - for upstream_epoch in upstream_epochs_to_notify { - let wait_progress_timer = self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .creating_jobs_to_wait - .remove(table_id) - .expect("should exist"); - if let Some(timer) = wait_progress_timer { - timer.observe_duration(); - } - } - if let Some((epoch, resps, is_first_time)) = commit_info { - let tables_to_commit = job - .info - .table_fragments - .all_table_ids() - .map(TableId::new) - .collect(); - let join_handle = - tokio::spawn(self.context.clone().complete_creating_job_barrier( - epoch, - resps, - tables_to_commit, - is_first_time, - )); - self.completing_command = CompletingCommand::CreatingStreamingJob { - table_id: *table_id, - epoch, - join_handle, - }; - break; + { + scheduled_barriers.force_checkpoint_in_next_barrier(); } + continue; } + let table_ids_to_finish = node + .state + .finished_jobs + .drain() + .map(|(table_id, (info, resps))| { + node.state.resps.extend(resps); + finished_jobs.push(TrackingJob::New(TrackingCommand { + info, + replace_table_info: None, + })); + table_id + }) + .collect(); + let commit_info = collect_commit_epoch_info( + take(&mut node.state.resps), + &node.command_ctx, + self.collect_backfill_pinned_upstream_log_epoch(), + ); + task = Some(CompleteBarrierTask { + commit_info, + finished_jobs, + notifiers: node.notifiers, + command_context: Some((node.command_ctx, node.enqueue_time)), + table_ids_to_finish, + creating_job_epochs: vec![], + }); + break; } } + if !creating_jobs_task.is_empty() { + let task = task.get_or_insert_default(); + for (table_id, epoch, resps, is_first_time) in creating_jobs_task { + GlobalBarrierManagerContext::collect_creating_job_commit_epoch_info( + &mut task.commit_info, + epoch, + resps, + self.creating_streaming_job_controls[&table_id] + .info + .table_fragments + .all_table_ids() + .map(TableId::new), + is_first_time, + ); + task.creating_job_epochs.push((table_id, epoch)); + } + } + task + } + + pub(super) fn next_completed_barrier<'a>( + &'a mut self, + scheduled_barriers: &mut ScheduledBarriers, + ) -> impl Future> + 'a { + // If there is no completing barrier, try to start completing the earliest barrier if + // it has been collected. + if let CompletingTask::None = &self.completing_task { + if let Some(task) = self.next_complete_barrier_task(Some(scheduled_barriers)) { + { + let command_ctx = task + .command_context + .as_ref() + .map(|(command_ctx, _)| command_ctx.clone()); + let table_ids_to_finish = task.table_ids_to_finish.clone(); + let creating_job_epochs = task.creating_job_epochs.clone(); + let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); + self.completing_task = CompletingTask::Completing { + command_ctx, + join_handle, + table_ids_to_finish, + creating_job_epochs, + }; + } + } + } + + self.next_completed_barrier_inner() + } + + async fn next_completed_barrier_inner(&mut self) -> MetaResult { + let CompletingTask::Completing { join_handle, .. } = &mut self.completing_task else { + return pending().await; + }; - match &mut self.completing_command { - CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { + let (table_ids_to_finish, creating_job_epochs) = { + { let join_result: MetaResult<_> = try { join_handle .await @@ -1467,85 +1389,33 @@ impl CheckpointControl { // It's important to reset the completing_command after await no matter the result is err // or not, and otherwise the join handle will be polled again after ready. let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) + CompletingTask::Err(e.clone()) } else { - CompletingCommand::None + CompletingTask::None }; let completed_command = - replace(&mut self.completing_command, next_completing_command_status); - join_result.map(move | version_stats| { - if let Some(new_version_stats) = version_stats { - self.hummock_version_stats = new_version_stats; - } - must_match!( - completed_command, - CompletingCommand::GlobalStreamingGraph { command_ctx, table_ids_to_finish, require_next_checkpoint, .. } => { - Some(BarrierCompleteOutput { - command_ctx, - require_next_checkpoint, - table_ids_to_finish, - }) - } - ) - }) + replace(&mut self.completing_task, next_completing_command_status); + self.hummock_version_stats = join_result?; + + must_match!(completed_command, CompletingTask::Completing { + table_ids_to_finish, + creating_job_epochs, + .. + } => (table_ids_to_finish, creating_job_epochs)) } - CompletingCommand::CreatingStreamingJob { - table_id, - epoch, - join_handle, - } => { - let table_id = *table_id; - let epoch = *epoch; - let join_result: MetaResult<_> = try { - join_handle - .await - .context("failed to join completing command")?? - }; - // It's important to reset the completing_command after await no matter the result is err - // or not, and otherwise the join handle will be polled again after ready. - let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) - } else { - CompletingCommand::None - }; - self.completing_command = next_completing_command_status; - if let Some((upstream_epoch, is_finished)) = self - .creating_streaming_job_controls + }; + + { + for (table_id, epoch) in creating_job_epochs { + self.creating_streaming_job_controls .get_mut(&table_id) .expect("should exist") .ack_completed(epoch) - { - let wait_progress_timer = self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .creating_jobs_to_wait - .remove(&table_id) - .expect("should exist"); - if let Some(timer) = wait_progress_timer { - timer.observe_duration(); - } - if is_finished { - debug!(epoch, ?table_id, "finish creating job"); - let creating_streaming_job = self - .creating_streaming_job_controls - .remove(&table_id) - .expect("should exist"); - assert!(creating_streaming_job.is_finished()); - assert!(self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .finished_table_ids - .insert(table_id, creating_streaming_job.info) - .is_none()); - } - } - join_result.map(|_| None) } - CompletingCommand::None | CompletingCommand::Err(_) => pending().await, + + Ok(BarrierCompleteOutput { + table_ids_to_finish, + }) } } } @@ -1682,28 +1552,26 @@ fn collect_resp_info( fn collect_commit_epoch_info( resps: Vec, command_ctx: &CommandContext, - epochs: &Vec, backfill_pinned_log_epoch: HashMap)>, - tables_to_commit: HashSet, ) -> CommitEpochInfo { let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) = collect_resp_info(resps); - let new_table_fragment_info = if let Command::CreateStreamingJob { info, job_type } = + let new_table_fragment_infos = if let Command::CreateStreamingJob { info, job_type } = &command_ctx.command && !matches!(job_type, CreateStreamingJobType::SnapshotBackfill(_)) { let table_fragments = &info.table_fragments; - NewTableFragmentInfo::Normal { + vec![NewTableFragmentInfo::Normal { mv_table_id: table_fragments.mv_table_id().map(TableId::new), internal_table_ids: table_fragments .internal_table_ids() .into_iter() .map(TableId::new) .collect(), - } + }] } else { - NewTableFragmentInfo::None + vec![] }; let mut mv_log_store_truncate_epoch = HashMap::new(); @@ -1739,19 +1607,23 @@ fn collect_commit_epoch_info( let table_new_change_log = build_table_change_log_delta( old_value_ssts.into_iter(), synced_ssts.iter().map(|sst| &sst.sst_info), - epochs, + must_match!(&command_ctx.kind, BarrierKind::Checkpoint(epochs) => epochs), mv_log_store_truncate_epoch.into_iter(), ); let epoch = command_ctx.prev_epoch.value().0; + let tables_to_commit = command_ctx + .table_ids_to_commit + .iter() + .map(|table_id| (*table_id, epoch)) + .collect(); CommitEpochInfo { sstables: synced_ssts, new_table_watermarks, sst_to_context, - new_table_fragment_info, + new_table_fragment_infos, change_log_delta: table_new_change_log, - committed_epoch: epoch, tables_to_commit, } } diff --git a/src/meta/src/barrier/state.rs b/src/meta/src/barrier/state.rs index db2ded5629d7a..d9fe6f13c963c 100644 --- a/src/meta/src/barrier/state.rs +++ b/src/meta/src/barrier/state.rs @@ -12,6 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; + +use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::Epoch; use risingwave_pb::meta::PausedReason; @@ -83,10 +86,17 @@ impl BarrierManagerState { /// Returns the inflight actor infos that have included the newly added actors in the given command. The dropped actors /// will be removed from the state after the info get resolved. + /// + /// Return (`graph_info`, `subscription_info`, `table_ids_to_commit`, `jobs_to_wait`) pub fn apply_command( &mut self, command: &Command, - ) -> (InflightGraphInfo, InflightSubscriptionInfo) { + ) -> ( + InflightGraphInfo, + InflightSubscriptionInfo, + HashSet, + HashSet, + ) { // update the fragment_infos outside pre_apply let fragment_changes = if let Command::CreateStreamingJob { job_type: CreateStreamingJobType::SnapshotBackfill(_), @@ -108,8 +118,19 @@ impl BarrierManagerState { if let Some(fragment_changes) = fragment_changes { self.inflight_graph_info.post_apply(&fragment_changes); } + + let mut table_ids_to_commit: HashSet<_> = info.existing_table_ids().collect(); + let mut jobs_to_wait = HashSet::new(); + if let Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge) = command { + for (table_id, (_, graph_info)) in jobs_to_merge { + jobs_to_wait.insert(*table_id); + table_ids_to_commit.extend(graph_info.existing_table_ids()); + self.inflight_graph_info.extend(graph_info.clone()); + } + } + self.inflight_subscription_info.post_apply(command); - (info, subscription_info) + (info, subscription_info, table_ids_to_commit, jobs_to_wait) } } diff --git a/src/meta/src/hummock/manager/checkpoint.rs b/src/meta/src/hummock/manager/checkpoint.rs index f678014d440c8..3949331098441 100644 --- a/src/meta/src/hummock/manager/checkpoint.rs +++ b/src/meta/src/hummock/manager/checkpoint.rs @@ -17,10 +17,8 @@ use std::ops::Bound::{Excluded, Included}; use std::ops::{Deref, DerefMut}; use std::sync::atomic::Ordering; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::{ - object_size_map, summarize_group_deltas, -}; -use risingwave_hummock_sdk::version::HummockVersion; +use risingwave_hummock_sdk::compaction_group::hummock_version_ext::object_size_map; +use risingwave_hummock_sdk::version::{GroupDeltaCommon, HummockVersion}; use risingwave_hummock_sdk::HummockVersionId; use risingwave_pb::hummock::hummock_version_checkpoint::{PbStaleObjects, StaleObjects}; use risingwave_pb::hummock::{ @@ -140,7 +138,6 @@ impl HummockManager { drop(versioning_guard); let versioning = self.versioning.read().await; let context_info = self.context_info.read().await; - versioning.mark_objects_for_deletion(&context_info, &self.delete_object_tracker); let min_pinned_version_id = context_info.min_pinned_version_id(); trigger_gc_stat(&self.metrics, &versioning.checkpoint, min_pinned_version_id); return Ok(0); @@ -156,13 +153,27 @@ impl HummockManager { .hummock_version_deltas .range((Excluded(old_checkpoint_id), Included(new_checkpoint_id))) { - for (group_id, group_deltas) in &version_delta.group_deltas { - let summary = summarize_group_deltas(group_deltas, *group_id); + for group_deltas in version_delta.group_deltas.values() { object_sizes.extend( - summary - .insert_table_infos + group_deltas + .group_deltas .iter() - .map(|t| (t.object_id, t.file_size)) + .flat_map(|delta| { + match delta { + GroupDeltaCommon::IntraLevel(level_delta) => { + Some(level_delta.inserted_table_infos.iter()) + } + GroupDeltaCommon::NewL0SubLevel(inserted_table_infos) => { + Some(inserted_table_infos.iter()) + } + GroupDeltaCommon::GroupConstruct(_) + | GroupDeltaCommon::GroupDestroy(_) + | GroupDeltaCommon::GroupMerge(_) => None, + } + .into_iter() + .flatten() + .map(|t| (t.object_id, t.file_size)) + }) .chain( version_delta .change_log_delta @@ -209,12 +220,9 @@ impl HummockManager { .collect(), }); } - // Whenever data archive or time travel is enabled, we can directly discard reference to stale objects that will no longer be used. - if self.env.opts.enable_hummock_data_archive || self.time_travel_enabled().await { - let context_info = self.context_info.read().await; - let min_pinned_version_id = context_info.min_pinned_version_id(); - stale_objects.retain(|version_id, _| *version_id >= min_pinned_version_id); - } + // We can directly discard reference to stale objects that will no longer be used. + let min_pinned_version_id = self.context_info.read().await.min_pinned_version_id(); + stale_objects.retain(|version_id, _| *version_id >= min_pinned_version_id); let new_checkpoint = HummockVersionCheckpoint { version: current_version.clone(), stale_objects, @@ -234,15 +242,9 @@ impl HummockManager { // 3. hold write lock and update in memory state let mut versioning_guard = self.versioning.write().await; let versioning = versioning_guard.deref_mut(); - let context_info = self.context_info.read().await; assert!(new_checkpoint.version.id > versioning.checkpoint.version.id); versioning.checkpoint = new_checkpoint; - // Not delete stale objects when archive or time travel is enabled - if !self.env.opts.enable_hummock_data_archive && !self.time_travel_enabled().await { - versioning.mark_objects_for_deletion(&context_info, &self.delete_object_tracker); - } - - let min_pinned_version_id = context_info.min_pinned_version_id(); + let min_pinned_version_id = self.context_info.read().await.min_pinned_version_id(); trigger_gc_stat(&self.metrics, &versioning.checkpoint, min_pinned_version_id); trigger_split_stat(&self.metrics, &versioning.current_version); drop(versioning_guard); diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index cb353d32e0890..da97a29c06f8c 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{BTreeMap, HashMap, HashSet}; +use std::sync::Arc; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::change_log::ChangeLogDelta; @@ -28,6 +29,7 @@ use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockSstableObjectId, LocalSstableInfo, }; use risingwave_pb::hummock::compact_task::{self}; +use risingwave_pb::hummock::CompactionConfig; use sea_orm::TransactionTrait; use crate::hummock::error::{Error, Result}; @@ -47,7 +49,6 @@ use crate::hummock::{ }; pub enum NewTableFragmentInfo { - None, Normal { mv_table_id: Option, internal_table_ids: Vec, @@ -57,14 +58,15 @@ pub enum NewTableFragmentInfo { }, } +#[derive(Default)] pub struct CommitEpochInfo { pub sstables: Vec, pub new_table_watermarks: HashMap, pub sst_to_context: HashMap, - pub new_table_fragment_info: NewTableFragmentInfo, + pub new_table_fragment_infos: Vec, pub change_log_delta: HashMap, - pub committed_epoch: u64, - pub tables_to_commit: HashSet, + /// `table_id` -> `committed_epoch` + pub tables_to_commit: HashMap, } impl HummockManager { @@ -75,9 +77,8 @@ impl HummockManager { mut sstables, new_table_watermarks, sst_to_context, - new_table_fragment_info, + new_table_fragment_infos, change_log_delta, - committed_epoch, tables_to_commit, } = commit_info; let mut versioning_guard = self.versioning.write().await; @@ -91,7 +92,6 @@ impl HummockManager { let versioning: &mut Versioning = &mut versioning_guard; self.commit_epoch_sanity_check( - committed_epoch, &tables_to_commit, &sstables, &sst_to_context, @@ -124,15 +124,18 @@ impl HummockManager { let state_table_info = &version.latest_version().state_table_info; let mut table_compaction_group_mapping = state_table_info.build_table_compaction_group_id(); + let mut new_table_ids = HashMap::new(); + let mut new_compaction_groups = HashMap::new(); + let mut compaction_group_manager_txn = None; + let mut compaction_group_config: Option> = None; // Add new table - let (new_table_ids, new_compaction_group, compaction_group_manager_txn) = + for new_table_fragment_info in new_table_fragment_infos { match new_table_fragment_info { NewTableFragmentInfo::Normal { mv_table_id, internal_table_ids, } => { - let mut new_table_ids = HashMap::new(); on_handle_add_new_table( state_table_info, &internal_table_ids, @@ -148,24 +151,40 @@ impl HummockManager { &mut table_compaction_group_mapping, &mut new_table_ids, )?; - (new_table_ids, None, None) } NewTableFragmentInfo::NewCompactionGroup { table_ids } => { - let compaction_group_manager_guard = - self.compaction_group_manager.write().await; - let compaction_group_config = - compaction_group_manager_guard.default_compaction_config(); - let mut compaction_group_manager = - CompactionGroupManager::start_owned_compaction_groups_txn( - compaction_group_manager_guard, - ); - let mut new_table_ids = HashMap::new(); + let (compaction_group_manager, compaction_group_config) = + if let Some(compaction_group_manager) = &mut compaction_group_manager_txn { + ( + compaction_group_manager, + (*compaction_group_config + .as_ref() + .expect("must be set with compaction_group_manager_txn")) + .clone(), + ) + } else { + let compaction_group_manager_guard = + self.compaction_group_manager.write().await; + let new_compaction_group_config = + compaction_group_manager_guard.default_compaction_config(); + compaction_group_config = Some(new_compaction_group_config.clone()); + ( + compaction_group_manager_txn.insert( + CompactionGroupManager::start_owned_compaction_groups_txn( + compaction_group_manager_guard, + ), + ), + new_compaction_group_config, + ) + }; let new_compaction_group_id = next_compaction_group_id(&self.env).await?; + new_compaction_groups + .insert(new_compaction_group_id, compaction_group_config.clone()); compaction_group_manager.insert( new_compaction_group_id, CompactionGroup { group_id: new_compaction_group_id, - compaction_config: compaction_group_config.clone(), + compaction_config: compaction_group_config, }, ); @@ -176,14 +195,9 @@ impl HummockManager { &mut table_compaction_group_mapping, &mut new_table_ids, )?; - ( - new_table_ids, - Some((new_compaction_group_id, (*compaction_group_config).clone())), - Some(compaction_group_manager), - ) } - NewTableFragmentInfo::None => (HashMap::new(), None, None), - }; + } + } let commit_sstables = self .correct_commit_ssts(sstables, &table_compaction_group_mapping) @@ -192,9 +206,8 @@ impl HummockManager { let modified_compaction_groups: Vec<_> = commit_sstables.keys().cloned().collect(); let time_travel_delta = version.pre_commit_epoch( - committed_epoch, &tables_to_commit, - new_compaction_group, + new_compaction_groups, commit_sstables, &new_table_ids, new_table_watermarks, @@ -251,9 +264,14 @@ impl HummockManager { .values() .map(|g| (g.group_id, g.parent_group_id)) .collect(); - let time_travel_tables_to_commit = table_compaction_group_mapping - .iter() - .filter(|(table_id, _)| tables_to_commit.contains(table_id)); + let time_travel_tables_to_commit = + table_compaction_group_mapping + .iter() + .filter_map(|(table_id, cg_id)| { + tables_to_commit + .get(table_id) + .map(|committed_epoch| (table_id, cg_id, *committed_epoch)) + }); let mut txn = self.env.meta_store_ref().conn.begin().await?; let version_snapshot_sst_ids = self .write_time_travel_metadata( @@ -263,7 +281,6 @@ impl HummockManager { &group_parents, &versioning.last_time_travel_snapshot_sst_ids, time_travel_tables_to_commit, - committed_epoch, ) .await?; commit_multi_var_with_provided_txn!( diff --git a/src/meta/src/hummock/manager/compaction/mod.rs b/src/meta/src/hummock/manager/compaction/mod.rs index 09d6e6badea52..faabbb80427b3 100644 --- a/src/meta/src/hummock/manager/compaction/mod.rs +++ b/src/meta/src/hummock/manager/compaction/mod.rs @@ -41,7 +41,6 @@ use rand::seq::SliceRandom; use rand::thread_rng; use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::compact_task::{CompactTask, ReportTask}; -use risingwave_hummock_sdk::compaction_group::hummock_version_ext::HummockLevelsExt; use risingwave_hummock_sdk::key_range::KeyRange; use risingwave_hummock_sdk::level::{InputLevel, Level, Levels}; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -153,17 +152,15 @@ impl<'a> HummockVersionTransaction<'a> { .entry(compact_task.compaction_group_id) .or_default() .group_deltas; - let mut removed_table_ids_map: BTreeMap> = BTreeMap::default(); + let mut removed_table_ids_map: BTreeMap> = BTreeMap::default(); for level in &compact_task.input_ssts { let level_idx = level.level_idx; - let mut removed_table_ids = - level.table_infos.iter().map(|sst| sst.sst_id).collect_vec(); removed_table_ids_map .entry(level_idx) .or_default() - .append(&mut removed_table_ids); + .extend(level.table_infos.iter().map(|sst| sst.sst_id)); } for (level_idx, removed_table_ids) in removed_table_ids_map { @@ -181,7 +178,7 @@ impl<'a> HummockVersionTransaction<'a> { let group_delta = GroupDelta::IntraLevel(IntraLevelDelta::new( compact_task.target_level, compact_task.target_sub_level_id, - vec![], // default + HashSet::new(), // default compact_task.sorted_output_ssts.clone(), compact_task.split_weight_by_vnode, )); diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index b76bd47c49b74..0a12f57a17d8b 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -15,14 +15,17 @@ use std::collections::{BTreeMap, HashMap, HashSet}; use fail::fail_point; +use futures::{stream, StreamExt}; use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ - HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, + HummockContextId, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, INVALID_VERSION_ID, }; +use risingwave_meta_model_v2::hummock_gc_history; use risingwave_pb::hummock::{HummockPinnedVersion, ValidationTask}; +use sea_orm::{DatabaseConnection, EntityTrait}; use crate::controller::SqlMetaStore; use crate::hummock::error::{Error, Result}; @@ -189,8 +192,7 @@ impl HummockManager { pub async fn commit_epoch_sanity_check( &self, - committed_epoch: HummockEpoch, - tables_to_commit: &HashSet, + tables_to_commit: &HashMap, sstables: &[LocalSstableInfo], sst_to_context: &HashMap, current_version: &HummockVersion, @@ -216,9 +218,9 @@ impl HummockManager { } // sanity check on monotonically increasing table committed epoch - for table_id in tables_to_commit { + for (table_id, committed_epoch) in tables_to_commit { if let Some(info) = current_version.state_table_info.info().get(table_id) { - if committed_epoch <= info.committed_epoch { + if *committed_epoch <= info.committed_epoch { return Err(anyhow::anyhow!( "table {} Epoch {} <= committed_epoch {}", table_id, @@ -232,7 +234,8 @@ impl HummockManager { // HummockManager::now requires a write to the meta store. Thus, it should be avoided whenever feasible. if !sstables.is_empty() { - // sanity check to ensure SSTs to commit have not been full GCed yet. + // Sanity check to ensure SSTs to commit have not been full GCed yet. + // TODO: since HummockManager::complete_full_gc have already filtered out SSTs by min uncommitted SST id, this sanity check can be removed. let now = self.now().await?; check_sst_retention( now, @@ -241,6 +244,10 @@ impl HummockManager { .iter() .map(|s| (s.sst_info.object_id, s.created_at)), )?; + if self.env.opts.gc_history_retention_time_sec != 0 { + let ids = sstables.iter().map(|s| s.sst_info.object_id).collect_vec(); + check_gc_history(&self.meta_store_ref().conn, ids).await?; + } } async { @@ -265,7 +272,6 @@ impl HummockManager { .send_event(ResponseEvent::ValidationTask(ValidationTask { sst_infos: sst_infos.into_iter().map(|sst| sst.into()).collect_vec(), sst_id_to_worker_id: sst_to_context.clone(), - epoch: committed_epoch, })) .is_err() { @@ -293,7 +299,12 @@ impl HummockManager { now, self.env.opts.min_sst_retention_time_sec, object_timestamps.iter().map(|(k, v)| (*k, *v)), - ) + )?; + if self.env.opts.gc_history_retention_time_sec != 0 { + let ids = object_timestamps.iter().map(|(id, _)| *id).collect_vec(); + check_gc_history(&self.meta_store_ref().conn, ids).await?; + } + Ok(()) } } @@ -311,6 +322,34 @@ fn check_sst_retention( Ok(()) } +async fn check_gc_history( + db: &DatabaseConnection, + // need IntoIterator to work around stream's "implementation of `std::iter::Iterator` is not general enough" error. + object_ids: impl IntoIterator, +) -> Result<()> { + let futures = object_ids.into_iter().map(|id| async move { + let id: risingwave_meta_model_v2::HummockSstableObjectId = id.try_into().unwrap(); + hummock_gc_history::Entity::find_by_id(id) + .one(db) + .await + .map_err(Error::from) + }); + let res: Vec<_> = stream::iter(futures).buffer_unordered(10).collect().await; + let res: Result> = res.into_iter().collect(); + let mut expired_object_ids = res?.into_iter().flatten().peekable(); + if expired_object_ids.peek().is_none() { + return Ok(()); + } + let expired_object_ids: Vec<_> = expired_object_ids.collect(); + tracing::error!( + ?expired_object_ids, + "new SSTs are rejected because they have already been GCed" + ); + Err(Error::InvalidSst( + expired_object_ids[0].object_id as HummockSstableObjectId, + )) +} + // pin and unpin method impl HummockManager { /// Pin the current greatest hummock version. The pin belongs to `context_id` diff --git a/src/meta/src/hummock/manager/gc.rs b/src/meta/src/hummock/manager/gc.rs index 85d7d8660dbf5..e57e9479c9487 100644 --- a/src/meta/src/hummock/manager/gc.rs +++ b/src/meta/src/hummock/manager/gc.rs @@ -18,19 +18,24 @@ use std::ops::Bound::{Excluded, Included}; use std::ops::DerefMut; use std::time::{Duration, SystemTime}; +use chrono::DateTime; +use futures::future::try_join_all; use itertools::Itertools; use parking_lot::Mutex; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_meta_model_migration::OnConflict; -use risingwave_meta_model_v2::hummock_sequence; use risingwave_meta_model_v2::hummock_sequence::HUMMOCK_NOW; +use risingwave_meta_model_v2::{hummock_gc_history, hummock_sequence}; use risingwave_pb::hummock::subscribe_compaction_event_response::Event as ResponseEvent; use risingwave_pb::hummock::FullScanTask; -use sea_orm::{ActiveValue, EntityTrait}; +use risingwave_pb::stream_service::GetMinUncommittedSstIdRequest; +use risingwave_rpc_client::StreamClientPool; +use sea_orm::{ActiveValue, ColumnTrait, EntityTrait, QueryFilter, Set}; use crate::hummock::error::{Error, Result}; use crate::hummock::manager::commit_multi_var; use crate::hummock::HummockManager; +use crate::manager::MetadataManager; use crate::model::BTreeMapTransaction; #[derive(Default)] @@ -133,7 +138,7 @@ impl HummockManager { pub async fn extend_objects_to_delete_from_scan( &self, object_ids: &[HummockSstableObjectId], - ) -> usize { + ) -> Result { let tracked_object_ids: HashSet = { let versioning = self.versioning.read().await; let context_info = self.context_info.read().await; @@ -163,13 +168,15 @@ impl HummockManager { let to_delete = object_ids .iter() .filter(|object_id| !tracked_object_ids.contains(object_id)) + .copied() .collect_vec(); + let to_delete_num = to_delete.len(); + self.write_gc_history(to_delete.iter().cloned()).await?; // This lock ensures that during commit_epoch or report_compact_tasks, where versioning lock is held, // no new objects will be marked for deletion here. let _versioning = self.versioning.read().await; - self.delete_object_tracker - .add(to_delete.iter().map(|id| **id)); - to_delete.len() + self.delete_object_tracker.add(to_delete.into_iter()); + Ok(to_delete_num) } /// Starts a full GC. @@ -228,7 +235,14 @@ impl HummockManager { &self, object_ids: Vec, next_start_after: Option, + pinned_by_metadata_backup: HashSet, ) -> Result { + // It's crucial to collect_min_uncommitted_sst_id (i.e. `min_sst_id`) only after LIST object store (i.e. `object_ids`). + // Because after getting `min_sst_id`, new compute nodes may join and generate new uncommitted SSTs that are not covered by `min_sst_id`. + // By getting `min_sst_id` after `object_ids`, it's ensured `object_ids` won't include any SSTs from those new compute nodes. + let min_sst_id = + collect_min_uncommitted_sst_id(&self.metadata_manager, self.env.stream_client_pool()) + .await?; self.full_gc_state.set_next_start_after(next_start_after); if object_ids.is_empty() { tracing::info!("SST full scan returns no SSTs."); @@ -246,19 +260,38 @@ impl HummockManager { self.metrics .time_travel_object_count .set(pinned_object_ids.len() as _); + // filter by SST id watermark, i.e. minimum id of uncommitted SSTs reported by compute nodes. + let object_ids = object_ids + .into_iter() + .filter(|id| *id < min_sst_id) + .collect_vec(); + let after_min_sst_id = object_ids.len(); // filter by time travel archive let object_ids = object_ids .into_iter() .filter(|s| !pinned_object_ids.contains(s)) .collect_vec(); let after_time_travel = object_ids.len(); + // filter by metadata backup + let object_ids = object_ids + .into_iter() + .filter(|s| !pinned_by_metadata_backup.contains(s)) + .collect_vec(); + let after_metadata_backup = object_ids.len(); // filter by version - let selected_object_number = self.extend_objects_to_delete_from_scan(&object_ids).await; + let after_version = self.extend_objects_to_delete_from_scan(&object_ids).await?; metrics .full_gc_selected_object_count - .observe(selected_object_number as _); - tracing::info!("Object full scan returns {candidate_object_number} objects. {after_time_travel} remains after filtered by time travel archives. {selected_object_number} remains after filtered by hummock version."); - Ok(selected_object_number) + .observe(after_version as _); + tracing::info!( + candidate_object_number, + after_min_sst_id, + after_time_travel, + after_metadata_backup, + after_version, + "complete full gc" + ); + Ok(after_version) } pub async fn now(&self) -> Result { @@ -325,6 +358,67 @@ impl HummockManager { self.metrics.total_object_count.set(total_object_count as _); } } + + async fn write_gc_history( + &self, + object_ids: impl Iterator, + ) -> Result<()> { + if self.env.opts.gc_history_retention_time_sec == 0 { + return Ok(()); + } + let now = self.now().await?; + let dt = DateTime::from_timestamp(now.try_into().unwrap(), 0).unwrap(); + let models = object_ids.map(|o| hummock_gc_history::ActiveModel { + object_id: Set(o.try_into().unwrap()), + mark_delete_at: Set(dt.naive_utc()), + }); + let db = &self.meta_store_ref().conn; + let gc_history_low_watermark = DateTime::from_timestamp( + now.saturating_sub(self.env.opts.gc_history_retention_time_sec) + .try_into() + .unwrap(), + 0, + ) + .unwrap(); + hummock_gc_history::Entity::delete_many() + .filter(hummock_gc_history::Column::MarkDeleteAt.lt(gc_history_low_watermark)) + .exec(db) + .await?; + hummock_gc_history::Entity::insert_many(models) + .on_conflict( + OnConflict::column(hummock_gc_history::Column::ObjectId) + .do_nothing() + .to_owned(), + ) + .do_nothing() + .exec(db) + .await?; + Ok(()) + } +} + +async fn collect_min_uncommitted_sst_id( + metadata_manager: &MetadataManager, + client_pool: &StreamClientPool, +) -> Result { + let futures = metadata_manager + .list_active_streaming_compute_nodes() + .await + .map_err(|err| Error::MetaStore(err.into()))? + .into_iter() + .map(|worker_node| async move { + let client = client_pool.get(&worker_node).await?; + let request = GetMinUncommittedSstIdRequest {}; + client.get_min_uncommitted_sst_id(request).await + }); + let min_watermark = try_join_all(futures) + .await + .map_err(|err| Error::Internal(err.into()))? + .into_iter() + .map(|resp| resp.min_uncommitted_sst_id) + .min() + .unwrap_or(HummockSstableObjectId::MAX); + Ok(min_watermark) } pub struct FullGcState { @@ -429,6 +523,7 @@ impl PagedMetric { #[cfg(test)] mod tests { + use std::collections::HashSet; use std::sync::Arc; use std::time::Duration; @@ -490,15 +585,20 @@ mod tests { // Empty input results immediate return, without waiting heartbeat. hummock_manager - .complete_full_gc(vec![], None) + .complete_full_gc(vec![], None, HashSet::default()) .await .unwrap(); // LSMtree is empty. All input SST ids should be treated as garbage. + // Use fake object ids, because they'll be written to GC history and they shouldn't affect later commit. assert_eq!( 3, hummock_manager - .complete_full_gc(vec![1, 2, 3], None) + .complete_full_gc( + vec![i64::MAX as u64 - 2, i64::MAX as u64 - 1, i64::MAX as u64], + None, + HashSet::new(), + ) .await .unwrap() ); @@ -523,7 +623,8 @@ mod tests { hummock_manager .complete_full_gc( [committed_object_ids, vec![max_committed_object_id + 1]].concat(), - None + None, + HashSet::default(), ) .await .unwrap() diff --git a/src/meta/src/hummock/manager/mod.rs b/src/meta/src/hummock/manager/mod.rs index e1f23ba540158..9db4c20383fb7 100644 --- a/src/meta/src/hummock/manager/mod.rs +++ b/src/meta/src/hummock/manager/mod.rs @@ -410,11 +410,6 @@ impl HummockManager { .collect(); self.delete_object_tracker.clear(); - // Not delete stale objects when archive or time travel is enabled - if !self.env.opts.enable_hummock_data_archive && !self.time_travel_enabled().await { - versioning_guard.mark_objects_for_deletion(context_info, &self.delete_object_tracker); - } - self.initial_compaction_group_config_after_load( versioning_guard, self.compaction_group_manager.write().await.deref_mut(), diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 5abf162ea7d16..2743824df98bc 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -1192,7 +1192,8 @@ async fn test_extend_objects_to_delete() { assert_eq!( hummock_manager .extend_objects_to_delete_from_scan(&all_object_ids) - .await, + .await + .unwrap(), orphan_sst_num as usize ); assert_eq!( @@ -1213,7 +1214,8 @@ async fn test_extend_objects_to_delete() { assert_eq!( hummock_manager .extend_objects_to_delete_from_scan(&all_object_ids) - .await, + .await + .unwrap(), orphan_sst_num as usize ); let objects_to_delete = hummock_manager.get_objects_to_delete(); @@ -1242,7 +1244,8 @@ async fn test_extend_objects_to_delete() { assert_eq!( hummock_manager .extend_objects_to_delete_from_scan(&all_object_ids) - .await, + .await + .unwrap(), orphan_sst_num as usize ); let objects_to_delete = hummock_manager.get_objects_to_delete(); @@ -1270,7 +1273,8 @@ async fn test_extend_objects_to_delete() { assert_eq!( hummock_manager .extend_objects_to_delete_from_scan(&all_object_ids) - .await, + .await + .unwrap(), orphan_sst_num as usize + 3 ); let objects_to_delete = hummock_manager.get_objects_to_delete(); diff --git a/src/meta/src/hummock/manager/time_travel.rs b/src/meta/src/hummock/manager/time_travel.rs index c8df0c93b85ec..ed9a2676e445f 100644 --- a/src/meta/src/hummock/manager/time_travel.rs +++ b/src/meta/src/hummock/manager/time_travel.rs @@ -359,8 +359,7 @@ impl HummockManager { delta: HummockVersionDelta, group_parents: &HashMap, skip_sst_ids: &HashSet, - tables_to_commit: impl Iterator, - committed_epoch: u64, + tables_to_commit: impl Iterator, ) -> Result>> { let select_groups = group_parents .iter() @@ -397,7 +396,7 @@ impl HummockManager { Ok(count) } - for (table_id, cg_id) in tables_to_commit { + for (table_id, cg_id, committed_epoch) in tables_to_commit { if !select_groups.contains(cg_id) { continue; } @@ -449,7 +448,7 @@ impl HummockManager { } let written = write_sstable_infos( delta - .newly_added_sst_infos(&select_groups) + .newly_added_sst_infos(Some(&select_groups)) .filter(|s| !skip_sst_ids.contains(&s.sst_id)), txn, ) diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 04d5e237a11df..57a228f35805f 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -12,20 +12,17 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap}; use std::ops::{Deref, DerefMut}; +use std::sync::Arc; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::change_log::ChangeLogDelta; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::TableWatermarks; -use risingwave_hummock_sdk::version::{ - GroupDelta, HummockVersion, HummockVersionDelta, IntraLevelDelta, -}; -use risingwave_hummock_sdk::{ - CompactionGroupId, FrontendHummockVersionDelta, HummockEpoch, HummockVersionId, -}; +use risingwave_hummock_sdk::version::{GroupDelta, HummockVersion, HummockVersionDelta}; +use risingwave_hummock_sdk::{CompactionGroupId, FrontendHummockVersionDelta, HummockVersionId}; use risingwave_pb::hummock::{ CompactionConfig, CompatibilityVersion, GroupConstruct, HummockVersionDeltas, HummockVersionStats, StateTableInfoDelta, @@ -113,9 +110,8 @@ impl<'a> HummockVersionTransaction<'a> { /// Returns a duplicate delta, used by time travel. pub(super) fn pre_commit_epoch( &mut self, - committed_epoch: HummockEpoch, - tables_to_commit: &HashSet, - new_compaction_group: Option<(CompactionGroupId, CompactionConfig)>, + tables_to_commit: &HashMap, + new_compaction_groups: HashMap>, commit_sstables: BTreeMap>, new_table_ids: &HashMap, new_table_watermarks: HashMap, @@ -125,7 +121,7 @@ impl<'a> HummockVersionTransaction<'a> { new_version_delta.new_table_watermarks = new_table_watermarks; new_version_delta.change_log_delta = change_log_delta; - if let Some((compaction_group_id, compaction_group_config)) = new_compaction_group { + for (compaction_group_id, compaction_group_config) in new_compaction_groups { { let group_deltas = &mut new_version_delta .group_deltas @@ -135,7 +131,7 @@ impl<'a> HummockVersionTransaction<'a> { #[expect(deprecated)] group_deltas.push(GroupDelta::GroupConstruct(GroupConstruct { - group_config: Some(compaction_group_config.clone()), + group_config: Some((*compaction_group_config).clone()), group_id: compaction_group_id, parent_group_id: StaticCompactionGroupId::NewCompactionGroup as CompactionGroupId, @@ -154,13 +150,7 @@ impl<'a> HummockVersionTransaction<'a> { .entry(compaction_group_id) .or_default() .group_deltas; - let group_delta = GroupDelta::IntraLevel(IntraLevelDelta::new( - 0, - 0, // l0_sub_level_id will be generated during apply_version_delta - vec![], // default - inserted_table_infos, - 0, // default - )); + let group_delta = GroupDelta::NewL0SubLevel(inserted_table_infos); group_deltas.push(group_delta); } @@ -173,6 +163,7 @@ impl<'a> HummockVersionTransaction<'a> { "newly added table exists previously: {:?}", table_id ); + let committed_epoch = *tables_to_commit.get(table_id).expect("newly added table must exist in tables_to_commit"); delta.state_table_info_delta.insert( *table_id, StateTableInfoDelta { @@ -182,7 +173,7 @@ impl<'a> HummockVersionTransaction<'a> { ); } - for table_id in tables_to_commit { + for (table_id, committed_epoch) in tables_to_commit { if new_table_ids.contains_key(table_id) { continue; } @@ -194,7 +185,7 @@ impl<'a> HummockVersionTransaction<'a> { .insert( *table_id, StateTableInfoDelta { - committed_epoch, + committed_epoch: *committed_epoch, compaction_group_id: info.compaction_group_id, } ) diff --git a/src/meta/src/hummock/manager/versioning.rs b/src/meta/src/hummock/manager/versioning.rs index 4d81689b9f705..7fa93f72d02b8 100644 --- a/src/meta/src/hummock/manager/versioning.rs +++ b/src/meta/src/hummock/manager/versioning.rs @@ -36,7 +36,6 @@ use crate::hummock::error::Result; use crate::hummock::manager::checkpoint::HummockVersionCheckpoint; use crate::hummock::manager::commit_multi_var; use crate::hummock::manager::context::ContextInfo; -use crate::hummock::manager::gc::DeleteObjectTracker; use crate::hummock::manager::transaction::HummockVersionTransaction; use crate::hummock::metrics_utils::{trigger_write_stop_stats, LocalTableMetrics}; use crate::hummock::model::CompactionGroup; @@ -82,22 +81,6 @@ impl ContextInfo { } impl Versioning { - /// Marks all objects <= `min_pinned_version_id` for deletion. - pub(super) fn mark_objects_for_deletion( - &self, - context_info: &ContextInfo, - delete_object_tracker: &DeleteObjectTracker, - ) { - let min_pinned_version_id = context_info.min_pinned_version_id(); - delete_object_tracker.add( - self.checkpoint - .stale_objects - .iter() - .filter(|(version_id, _)| **version_id <= min_pinned_version_id) - .flat_map(|(_, stale_objects)| stale_objects.id.iter().cloned()), - ); - } - pub(super) fn mark_next_time_travel_version_snapshot(&mut self) { self.time_travel_snapshot_interval_counter = u64::MAX; } diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index 54b26fa20a665..805db163587a0 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -171,20 +171,20 @@ impl HummockMetaClient for MockHummockMetaClient { .chain(table_ids.iter().cloned()) .collect::>(); - let new_table_fragment_info = if commit_table_ids + let new_table_fragment_infos = if commit_table_ids .iter() .all(|table_id| table_ids.contains(table_id)) { - NewTableFragmentInfo::None + vec![] } else { - NewTableFragmentInfo::Normal { + vec![NewTableFragmentInfo::Normal { mv_table_id: None, internal_table_ids: commit_table_ids .iter() .cloned() .map(TableId::from) .collect_vec(), - } + }] }; let sst_to_context = sync_result @@ -215,13 +215,12 @@ impl HummockMetaClient for MockHummockMetaClient { sstables: sync_result.uncommitted_ssts, new_table_watermarks: new_table_watermark, sst_to_context, - new_table_fragment_info, + new_table_fragment_infos, change_log_delta: table_change_log, - committed_epoch: epoch, tables_to_commit: commit_table_ids .iter() .cloned() - .map(TableId::from) + .map(|table_id| (TableId::new(table_id), epoch)) .collect(), }) .await diff --git a/src/meta/src/hummock/vacuum.rs b/src/meta/src/hummock/vacuum.rs index dcb2d95d7ca1d..bc04aa53b6a87 100644 --- a/src/meta/src/hummock/vacuum.rs +++ b/src/meta/src/hummock/vacuum.rs @@ -35,7 +35,7 @@ pub type VacuumManagerRef = Arc; pub struct VacuumManager { env: MetaSrvEnv, hummock_manager: HummockManagerRef, - backup_manager: BackupManagerRef, + pub backup_manager: BackupManagerRef, /// Use the `CompactorManager` to dispatch `VacuumTask`. compactor_manager: CompactorManagerRef, /// SST object ids which have been dispatched to vacuum nodes but are not replied yet. @@ -112,8 +112,7 @@ impl VacuumManager { pending_object_ids } else { // 2. If no pending SST objects, then fetch new ones. - let mut objects_to_delete = self.hummock_manager.get_objects_to_delete(); - self.filter_out_pinned_ssts(&mut objects_to_delete).await?; + let objects_to_delete = self.hummock_manager.get_objects_to_delete(); if objects_to_delete.is_empty() { return Ok(vec![]); } @@ -178,29 +177,6 @@ impl VacuumManager { Ok(sent_batch) } - async fn filter_out_pinned_ssts( - &self, - objects_to_delete: &mut Vec, - ) -> MetaResult<()> { - if objects_to_delete.is_empty() { - return Ok(()); - } - let reject = self.backup_manager.list_pinned_ssts(); - // Ack these SSTs immediately, because they tend to be pinned for long time. - // They will be GCed during full GC when they are no longer pinned. - let to_ack = objects_to_delete - .iter() - .filter(|s| reject.contains(s)) - .cloned() - .collect_vec(); - if to_ack.is_empty() { - return Ok(()); - } - self.hummock_manager.ack_deleted_objects(&to_ack).await?; - objects_to_delete.retain(|s| !reject.contains(s)); - Ok(()) - } - /// Acknowledges deletion of SSTs and deletes corresponding metadata. pub async fn report_vacuum_task(&self, vacuum_task: VacuumTask) -> MetaResult<()> { let deleted_object_ids = self @@ -225,6 +201,7 @@ impl VacuumManager { #[cfg(test)] mod tests { + use std::collections::HashSet; use std::sync::Arc; use itertools::Itertools; @@ -275,7 +252,31 @@ mod tests { .await .unwrap(); hummock_manager.create_version_checkpoint(0).await.unwrap(); - assert!(!hummock_manager.get_objects_to_delete().is_empty()); + assert!(hummock_manager.get_objects_to_delete().is_empty()); + hummock_manager + .complete_full_gc( + sst_infos + .iter() + .flat_map(|ssts| ssts.iter().map(|s| s.object_id)) + .collect(), + None, + HashSet::default(), + ) + .await + .unwrap(); + assert_eq!(hummock_manager.get_objects_to_delete().len(), 3); + assert_eq!( + hummock_manager + .get_objects_to_delete() + .into_iter() + .sorted() + .collect::>(), + sst_infos[0] + .iter() + .map(|s| s.object_id) + .sorted() + .collect::>() + ); // No SST deletion is scheduled because no available worker. assert_eq!(vacuum.vacuum_object().await.unwrap().len(), 0); let _receiver = compactor_manager.add_compactor(context_id); diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index 88dc47d1f30e8..69d9dc21a075a 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -31,6 +31,7 @@ #![feature(const_option)] #![feature(anonymous_lifetime_in_impl_trait)] #![feature(duration_millis_float)] +#![feature(option_get_or_insert_default)] pub mod backup_restore; pub mod barrier; diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 380ab94bd9210..9e9bbcf41da4d 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -126,6 +126,8 @@ pub struct MetaOpts { pub full_gc_interval_sec: u64, /// Max number of object per full GC job can fetch. pub full_gc_object_limit: u64, + /// Duration in seconds to retain garbage collection history data. + pub gc_history_retention_time_sec: u64, /// Max number of inflight time travel query. pub max_inflight_time_travel_query: u64, /// Enable sanity check when SSTs are committed @@ -262,6 +264,7 @@ impl MetaOpts { min_sst_retention_time_sec: 3600 * 24 * 7, full_gc_interval_sec: 3600 * 24 * 7, full_gc_object_limit: 100_000, + gc_history_retention_time_sec: 3600 * 24 * 7, max_inflight_time_travel_query: 1000, enable_committed_sst_sanity_check: false, periodic_compaction_interval_sec: 60, diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index ccceddfb0799c..e9fa97b1f87a0 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -83,9 +83,6 @@ pub struct MetaMetrics { pub snapshot_backfill_barrier_latency: LabelGuardedHistogramVec<2>, // (table_id, barrier_type) /// The latency of commit epoch of `table_id` pub snapshot_backfill_wait_commit_latency: LabelGuardedHistogramVec<1>, // (table_id, ) - /// The latency that the upstream waits on the snapshot backfill progress after the upstream - /// has collected the barrier. - pub snapshot_backfill_upstream_wait_progress_latency: LabelGuardedHistogramVec<1>, /* (table_id, ) */ /// The lags between the upstream epoch and the downstream epoch. pub snapshot_backfill_lag: LabelGuardedIntGaugeVec<1>, // (table_id, ) /// The number of inflight barriers of `table_id` @@ -282,13 +279,7 @@ impl MetaMetrics { ); let snapshot_backfill_wait_commit_latency = register_guarded_histogram_vec_with_registry!(opts, &["table_id"], registry).unwrap(); - let opts = histogram_opts!( - "meta_snapshot_backfill_upstream_wait_progress_latency", - "snapshot backfill upstream_wait_progress_latency", - exponential_buckets(0.1, 1.5, 20).unwrap() // max 221s - ); - let snapshot_backfill_upstream_wait_progress_latency = - register_guarded_histogram_vec_with_registry!(opts, &["table_id"], registry).unwrap(); + let snapshot_backfill_lag = register_guarded_int_gauge_vec_with_registry!( "meta_snapshot_backfill_upstream_lag", "snapshot backfill upstream_lag", @@ -759,7 +750,6 @@ impl MetaMetrics { last_committed_barrier_time, snapshot_backfill_barrier_latency, snapshot_backfill_wait_commit_latency, - snapshot_backfill_upstream_wait_progress_latency, snapshot_backfill_lag, snapshot_backfill_inflight_barrier_num, recovery_failure_cnt, diff --git a/src/prost/build.rs b/src/prost/build.rs index ee04705ef19e5..0e1b2ea5c1db6 100644 --- a/src/prost/build.rs +++ b/src/prost/build.rs @@ -183,6 +183,7 @@ fn main() -> Result<(), Box> { .type_attribute("hummock.GroupTableChange", "#[derive(Eq)]") .type_attribute("hummock.GroupMerge", "#[derive(Eq)]") .type_attribute("hummock.GroupDelta", "#[derive(Eq)]") + .type_attribute("hummock.NewL0SubLevel", "#[derive(Eq)]") .type_attribute("hummock.LevelHandler.RunningCompactTask", "#[derive(Eq)]") .type_attribute("hummock.LevelHandler", "#[derive(Eq)]") .type_attribute("hummock.TableOption", "#[derive(Eq)]") diff --git a/src/rpc_client/src/stream_client.rs b/src/rpc_client/src/stream_client.rs index dcd4a8edbf729..42eeaa780d099 100644 --- a/src/rpc_client/src/stream_client.rs +++ b/src/rpc_client/src/stream_client.rs @@ -73,7 +73,8 @@ pub type StreamClientPoolRef = Arc; macro_rules! for_all_stream_rpc { ($macro:ident) => { $macro! { - { 0, wait_epoch_commit, WaitEpochCommitRequest, WaitEpochCommitResponse } + { 0, wait_epoch_commit, WaitEpochCommitRequest, WaitEpochCommitResponse }, + { 0, get_min_uncommitted_sst_id, GetMinUncommittedSstIdRequest, GetMinUncommittedSstIdResponse } } }; } diff --git a/src/storage/hummock_sdk/src/compact_task.rs b/src/storage/hummock_sdk/src/compact_task.rs index 162895a38ac91..82a9b1904d5af 100644 --- a/src/storage/hummock_sdk/src/compact_task.rs +++ b/src/storage/hummock_sdk/src/compact_task.rs @@ -326,7 +326,6 @@ impl From<&CompactTask> for PbCompactTask { pub struct ValidationTask { pub sst_infos: Vec, pub sst_id_to_worker_id: HashMap, - pub epoch: u64, } impl From for ValidationTask { @@ -338,7 +337,6 @@ impl From for ValidationTask { .map(SstableInfo::from) .collect_vec(), sst_id_to_worker_id: pb_validation_task.sst_id_to_worker_id.clone(), - epoch: pb_validation_task.epoch, } } } @@ -352,7 +350,6 @@ impl From for PbValidationTask { .map(|sst| sst.into()) .collect_vec(), sst_id_to_worker_id: validation_task.sst_id_to_worker_id.clone(), - epoch: validation_task.epoch, } } } diff --git a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs index 0ffdd15eca498..1af5c42ac7d4a 100644 --- a/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs +++ b/src/storage/hummock_sdk/src/compaction_group/hummock_version_ext.rs @@ -22,8 +22,7 @@ use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::hash::VnodeBitmapExt; use risingwave_pb::hummock::{ - CompactionConfig, CompatibilityVersion, GroupConstruct, GroupMerge, PbLevelType, - StateTableInfo, StateTableInfoDelta, + CompactionConfig, CompatibilityVersion, PbLevelType, StateTableInfo, StateTableInfoDelta, }; use tracing::warn; @@ -36,83 +35,11 @@ use crate::level::{Level, Levels, OverlappingLevel}; use crate::sstable_info::SstableInfo; use crate::table_watermark::{ReadTableWatermark, TableWatermarks}; use crate::version::{ - GroupDelta, GroupDeltas, HummockVersion, HummockVersionCommon, HummockVersionDelta, - HummockVersionStateTableInfo, IntraLevelDelta, + GroupDelta, GroupDeltaCommon, HummockVersion, HummockVersionCommon, HummockVersionDelta, + HummockVersionStateTableInfo, IntraLevelDelta, IntraLevelDeltaCommon, }; use crate::{can_concat, CompactionGroupId, HummockSstableId, HummockSstableObjectId}; -pub struct GroupDeltasSummary { - pub delete_sst_levels: Vec, - pub delete_sst_ids_set: HashSet, - pub insert_sst_level_id: u32, - pub insert_sub_level_id: u64, - pub insert_table_infos: Vec, - pub group_construct: Option, - pub group_destroy: Option, - pub new_vnode_partition_count: u32, - pub group_merge: Option, -} - -pub fn summarize_group_deltas( - group_deltas: &GroupDeltas, - compaction_group_id: CompactionGroupId, -) -> GroupDeltasSummary { - let mut delete_sst_levels = Vec::with_capacity(group_deltas.group_deltas.len()); - let mut delete_sst_ids_set = HashSet::new(); - let mut insert_sst_level_id = u32::MAX; - let mut insert_sub_level_id = u64::MAX; - let mut insert_table_infos = vec![]; - let mut group_construct = None; - let mut group_destroy = None; - let mut new_vnode_partition_count = 0; - let mut group_merge = None; - - for group_delta in &group_deltas.group_deltas { - match group_delta { - GroupDelta::IntraLevel(intra_level) => { - if !intra_level.removed_table_ids.is_empty() { - delete_sst_levels.push(intra_level.level_idx); - delete_sst_ids_set.extend(intra_level.removed_table_ids.iter().clone()); - } - if !intra_level.inserted_table_infos.is_empty() { - insert_sst_level_id = intra_level.level_idx; - insert_sub_level_id = intra_level.l0_sub_level_id; - insert_table_infos.extend(intra_level.inserted_table_infos.iter().cloned()); - } - new_vnode_partition_count = intra_level.vnode_partition_count; - } - GroupDelta::GroupConstruct(construct_delta) => { - assert!(group_construct.is_none()); - group_construct = Some(construct_delta.clone()); - } - GroupDelta::GroupDestroy(_) => { - assert!(group_destroy.is_none()); - group_destroy = Some(compaction_group_id); - } - GroupDelta::GroupMerge(merge_delta) => { - assert!(group_merge.is_none()); - group_merge = Some(*merge_delta); - group_destroy = Some(merge_delta.right_group_id); - } - } - } - - delete_sst_levels.sort(); - delete_sst_levels.dedup(); - - GroupDeltasSummary { - delete_sst_levels, - delete_sst_ids_set, - insert_sst_level_id, - insert_sub_level_id, - insert_table_infos, - group_construct, - group_destroy, - new_vnode_partition_count, - group_merge, - } -} - #[derive(Clone, Default)] pub struct TableGroupInfo { pub group_id: CompactionGroupId, @@ -493,11 +420,12 @@ impl HummockVersion { let mut removed_ssts: BTreeMap> = BTreeMap::new(); // Build only if all deltas are intra level deltas. - if !group_deltas - .group_deltas - .iter() - .all(|delta| matches!(delta, GroupDelta::IntraLevel(_))) - { + if !group_deltas.group_deltas.iter().all(|delta| { + matches!( + delta, + GroupDelta::IntraLevel(_) | GroupDelta::NewL0SubLevel(_) + ) + }) { continue; } @@ -505,24 +433,36 @@ impl HummockVersion { // current `hummock::manager::gen_version_delta` implementation. Better refactor the // struct to reduce conventions. for group_delta in &group_deltas.group_deltas { - if let GroupDelta::IntraLevel(intra_level) = group_delta { - if !intra_level.inserted_table_infos.is_empty() { - info.insert_sst_level = intra_level.level_idx; - info.insert_sst_infos - .extend(intra_level.inserted_table_infos.iter().cloned()); + match group_delta { + GroupDeltaCommon::NewL0SubLevel(inserted_table_infos) => { + if !inserted_table_infos.is_empty() { + info.insert_sst_level = 0; + info.insert_sst_infos + .extend(inserted_table_infos.iter().cloned()); + } } - if !intra_level.removed_table_ids.is_empty() { - for id in &intra_level.removed_table_ids { - if intra_level.level_idx == 0 { - removed_l0_ssts.insert(*id); - } else { - removed_ssts - .entry(intra_level.level_idx) - .or_default() - .insert(*id); + GroupDeltaCommon::IntraLevel(intra_level) => { + if !intra_level.inserted_table_infos.is_empty() { + info.insert_sst_level = intra_level.level_idx; + info.insert_sst_infos + .extend(intra_level.inserted_table_infos.iter().cloned()); + } + if !intra_level.removed_table_ids.is_empty() { + for id in &intra_level.removed_table_ids { + if intra_level.level_idx == 0 { + removed_l0_ssts.insert(*id); + } else { + removed_ssts + .entry(intra_level.level_idx) + .or_default() + .insert(*id); + } } } } + GroupDeltaCommon::GroupConstruct(_) + | GroupDeltaCommon::GroupDestroy(_) + | GroupDeltaCommon::GroupMerge(_) => {} } } @@ -587,97 +527,129 @@ impl HummockVersion { // apply to `levels`, which is different compaction groups for (compaction_group_id, group_deltas) in &version_delta.group_deltas { - let summary = summarize_group_deltas(group_deltas, *compaction_group_id); - if let Some(group_construct) = &summary.group_construct { - let mut new_levels = build_initial_compaction_group_levels( - *compaction_group_id, - group_construct.get_group_config().unwrap(), - ); - let parent_group_id = group_construct.parent_group_id; - new_levels.parent_group_id = parent_group_id; - #[expect(deprecated)] - // for backward-compatibility of previous hummock version delta - new_levels - .member_table_ids - .clone_from(&group_construct.table_ids); - self.levels.insert(*compaction_group_id, new_levels); - let member_table_ids = - if group_construct.version >= CompatibilityVersion::NoMemberTableIds as _ { - self.state_table_info - .compaction_group_member_table_ids(*compaction_group_id) - .iter() - .map(|table_id| table_id.table_id) - .collect() - } else { + let mut is_applied_l0_compact = false; + for group_delta in &group_deltas.group_deltas { + match group_delta { + GroupDeltaCommon::GroupConstruct(group_construct) => { + let mut new_levels = build_initial_compaction_group_levels( + *compaction_group_id, + group_construct.get_group_config().unwrap(), + ); + let parent_group_id = group_construct.parent_group_id; + new_levels.parent_group_id = parent_group_id; #[expect(deprecated)] // for backward-compatibility of previous hummock version delta - BTreeSet::from_iter(group_construct.table_ids.clone()) - }; - - if group_construct.version >= CompatibilityVersion::SplitGroupByTableId as _ { - let split_key = if group_construct.split_key.is_some() { - Some(Bytes::from(group_construct.split_key.clone().unwrap())) - } else { - None - }; - self.init_with_parent_group_v2( - parent_group_id, - *compaction_group_id, - group_construct.get_new_sst_start_id(), - split_key.clone(), - ); - } else { - // for backward-compatibility of previous hummock version delta - self.init_with_parent_group( - parent_group_id, - *compaction_group_id, - member_table_ids, - group_construct.get_new_sst_start_id(), - ); - } - } else if let Some(group_merge) = &summary.group_merge { - tracing::info!( - "group_merge left {:?} right {:?}", - group_merge.left_group_id, - group_merge.right_group_id - ); - self.merge_compaction_group(group_merge.left_group_id, group_merge.right_group_id) - } - let group_destroy = summary.group_destroy; - let levels = self.levels.get_mut(compaction_group_id).unwrap_or_else(|| { - panic!("compaction group {} does not exist", compaction_group_id) - }); + new_levels + .member_table_ids + .clone_from(&group_construct.table_ids); + self.levels.insert(*compaction_group_id, new_levels); + let member_table_ids = if group_construct.version + >= CompatibilityVersion::NoMemberTableIds as _ + { + self.state_table_info + .compaction_group_member_table_ids(*compaction_group_id) + .iter() + .map(|table_id| table_id.table_id) + .collect() + } else { + #[expect(deprecated)] + // for backward-compatibility of previous hummock version delta + BTreeSet::from_iter(group_construct.table_ids.clone()) + }; + + if group_construct.version >= CompatibilityVersion::SplitGroupByTableId as _ + { + let split_key = if group_construct.split_key.is_some() { + Some(Bytes::from(group_construct.split_key.clone().unwrap())) + } else { + None + }; + self.init_with_parent_group_v2( + parent_group_id, + *compaction_group_id, + group_construct.get_new_sst_start_id(), + split_key.clone(), + ); + } else { + // for backward-compatibility of previous hummock version delta + self.init_with_parent_group( + parent_group_id, + *compaction_group_id, + member_table_ids, + group_construct.get_new_sst_start_id(), + ); + } + } + GroupDeltaCommon::GroupMerge(group_merge) => { + tracing::info!( + "group_merge left {:?} right {:?}", + group_merge.left_group_id, + group_merge.right_group_id + ); + self.merge_compaction_group( + group_merge.left_group_id, + group_merge.right_group_id, + ) + } + GroupDeltaCommon::IntraLevel(level_delta) => { + let levels = + self.levels.get_mut(compaction_group_id).unwrap_or_else(|| { + panic!("compaction group {} does not exist", compaction_group_id) + }); + if is_commit_epoch { + assert!( + level_delta.removed_table_ids.is_empty(), + "no sst should be deleted when committing an epoch" + ); - if is_commit_epoch { - let GroupDeltasSummary { - delete_sst_levels, - delete_sst_ids_set, - .. - } = summary; + let IntraLevelDelta { + level_idx, + l0_sub_level_id, + inserted_table_infos, + .. + } = level_delta; + { + assert_eq!( + *level_idx, 0, + "we should only add to L0 when we commit an epoch." + ); + if !inserted_table_infos.is_empty() { + insert_new_sub_level( + &mut levels.l0, + *l0_sub_level_id, + PbLevelType::Overlapping, + inserted_table_infos.clone(), + None, + ); + } + } + } else { + // The delta is caused by compaction. + levels.apply_compact_ssts( + level_delta, + self.state_table_info + .compaction_group_member_table_ids(*compaction_group_id), + ); + if level_delta.level_idx == 0 { + is_applied_l0_compact = true; + } + } + } + GroupDeltaCommon::NewL0SubLevel(inserted_table_infos) => { + let levels = + self.levels.get_mut(compaction_group_id).unwrap_or_else(|| { + panic!("compaction group {} does not exist", compaction_group_id) + }); + assert!(is_commit_epoch); - assert!( - delete_sst_levels.is_empty() && delete_sst_ids_set.is_empty() - || group_destroy.is_some(), - "no sst should be deleted when committing an epoch" - ); - let mut next_l0_sub_level_id = levels - .l0 - .sub_levels - .last() - .map(|level| level.sub_level_id + 1) - .unwrap_or(1); - for group_delta in &group_deltas.group_deltas { - if let GroupDelta::IntraLevel(IntraLevelDelta { - level_idx, - inserted_table_infos, - .. - }) = group_delta - { - assert_eq!( - *level_idx, 0, - "we should only add to L0 when we commit an epoch." - ); if !inserted_table_infos.is_empty() { + let next_l0_sub_level_id = levels + .l0 + .sub_levels + .last() + .map(|level| level.sub_level_id + 1) + .unwrap_or(1); + insert_new_sub_level( &mut levels.l0, next_l0_sub_level_id, @@ -685,20 +657,16 @@ impl HummockVersion { inserted_table_infos.clone(), None, ); - next_l0_sub_level_id += 1; } } + GroupDeltaCommon::GroupDestroy(_) => { + self.levels.remove(compaction_group_id); + } } - } else { - // The delta is caused by compaction. - levels.apply_compact_ssts( - summary, - self.state_table_info - .compaction_group_member_table_ids(*compaction_group_id), - ); } - if let Some(destroy_group_id) = &group_destroy { - self.levels.remove(destroy_group_id); + if is_applied_l0_compact && let Some(levels) = self.levels.get_mut(compaction_group_id) + { + levels.post_apply_l0_compact(); } } self.id = version_delta.id; @@ -1005,54 +973,53 @@ impl HummockVersionCommon { } } -#[easy_ext::ext(HummockLevelsExt)] impl Levels { - pub fn apply_compact_ssts( + pub(crate) fn apply_compact_ssts( &mut self, - summary: GroupDeltasSummary, + level_delta: &IntraLevelDeltaCommon, member_table_ids: &BTreeSet, ) { - let GroupDeltasSummary { - delete_sst_levels, - delete_sst_ids_set, - insert_sst_level_id, - insert_sub_level_id, - insert_table_infos, - new_vnode_partition_count, + let IntraLevelDeltaCommon { + level_idx, + l0_sub_level_id, + inserted_table_infos: insert_table_infos, + vnode_partition_count, + removed_table_ids: delete_sst_ids_set, .. - } = summary; + } = level_delta; + let new_vnode_partition_count = *vnode_partition_count; - if !self.check_deleted_sst_exist(&delete_sst_levels, delete_sst_ids_set.clone()) { + if !self.check_deleted_sst_exist(&[*level_idx], delete_sst_ids_set.clone()) { warn!( "This VersionDelta may be committed by an expired compact task. Please check it. \n - delete_sst_levels: {:?}\n, - delete_sst_ids_set: {:?}\n, insert_sst_level_id: {}\n, insert_sub_level_id: {}\n, - insert_table_infos: {:?}\n", - delete_sst_levels, - delete_sst_ids_set, - insert_sst_level_id, - insert_sub_level_id, + insert_table_infos: {:?}\n, + delete_sst_ids_set: {:?}\n", + level_idx, + l0_sub_level_id, insert_table_infos .iter() .map(|sst| (sst.sst_id, sst.object_id)) - .collect_vec() + .collect_vec(), + delete_sst_ids_set, ); return; } - for level_idx in &delete_sst_levels { + if !delete_sst_ids_set.is_empty() { if *level_idx == 0 { for level in &mut self.l0.sub_levels { - level_delete_ssts(level, &delete_sst_ids_set); + level_delete_ssts(level, delete_sst_ids_set); } } else { let idx = *level_idx as usize - 1; - level_delete_ssts(&mut self.levels[idx], &delete_sst_ids_set); + level_delete_ssts(&mut self.levels[idx], delete_sst_ids_set); } } if !insert_table_infos.is_empty() { + let insert_sst_level_id = *level_idx; + let insert_sub_level_id = *l0_sub_level_id; if insert_sst_level_id == 0 { let l0 = &mut self.l0; let index = l0 @@ -1093,7 +1060,10 @@ impl Levels { level_insert_ssts(&mut self.levels[idx], insert_table_infos); } } - if delete_sst_levels.iter().any(|level_id| *level_id == 0) { + } + + pub(crate) fn post_apply_l0_compact(&mut self) { + { self.l0 .sub_levels .retain(|level| !level.table_infos.is_empty()); @@ -1358,7 +1328,7 @@ fn level_delete_ssts( original_len != operand.table_infos.len() } -fn level_insert_ssts(operand: &mut Level, insert_table_infos: Vec) { +fn level_insert_ssts(operand: &mut Level, insert_table_infos: &Vec) { operand.total_file_size += insert_table_infos .iter() .map(|sst| sst.sst_size) @@ -1367,7 +1337,9 @@ fn level_insert_ssts(operand: &mut Level, insert_table_infos: Vec) .iter() .map(|sst| sst.uncompressed_file_size) .sum::(); - operand.table_infos.extend(insert_table_infos); + operand + .table_infos + .extend(insert_table_infos.iter().cloned()); operand .table_infos .sort_by(|sst1, sst2| sst1.key_range.cmp(&sst2.key_range)); @@ -1501,7 +1473,7 @@ pub fn validate_version(version: &HummockVersion) -> Vec { #[cfg(test)] mod tests { - use std::collections::HashMap; + use std::collections::{HashMap, HashSet}; use bytes::Bytes; use risingwave_common::catalog::TableId; @@ -1655,7 +1627,7 @@ mod tests { group_deltas: vec![GroupDelta::IntraLevel(IntraLevelDelta::new( 1, 0, - vec![], + HashSet::new(), vec![SstableInfo { object_id: 1, sst_id: 1, diff --git a/src/storage/hummock_sdk/src/table_watermark.rs b/src/storage/hummock_sdk/src/table_watermark.rs index 324e8a91cf4a3..bbc0ae22148c0 100644 --- a/src/storage/hummock_sdk/src/table_watermark.rs +++ b/src/storage/hummock_sdk/src/table_watermark.rs @@ -259,7 +259,7 @@ impl TableWatermarksIndex { } } if self.latest_epoch < committed_epoch { - warn!( + debug!( latest_epoch = self.latest_epoch, committed_epoch, "committed_epoch exceed table watermark latest_epoch" ); diff --git a/src/storage/hummock_sdk/src/version.rs b/src/storage/hummock_sdk/src/version.rs index 64206d9b45b55..b106563cdc7ac 100644 --- a/src/storage/hummock_sdk/src/version.rs +++ b/src/storage/hummock_sdk/src/version.rs @@ -21,12 +21,12 @@ use std::sync::{Arc, LazyLock}; use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::INVALID_EPOCH; -use risingwave_pb::hummock::group_delta::PbDeltaType; +use risingwave_pb::hummock::group_delta::{DeltaType, PbDeltaType}; use risingwave_pb::hummock::hummock_version_delta::PbGroupDeltas; use risingwave_pb::hummock::{ CompactionConfig, PbGroupConstruct, PbGroupDelta, PbGroupDestroy, PbGroupMerge, - PbHummockVersion, PbHummockVersionDelta, PbIntraLevelDelta, PbSstableInfo, PbStateTableInfo, - StateTableInfo, StateTableInfoDelta, + PbHummockVersion, PbHummockVersionDelta, PbIntraLevelDelta, PbNewL0SubLevel, PbSstableInfo, + PbStateTableInfo, StateTableInfo, StateTableInfoDelta, }; use tracing::warn; @@ -512,76 +512,45 @@ impl HummockVersionDelta { /// Note: the result can be false positive because we only collect the set of sst object ids in the `inserted_table_infos`, /// but it is possible that the object is moved or split from other compaction groups or levels. pub fn newly_added_object_ids(&self) -> HashSet { - self.group_deltas - .values() - .flat_map(|group_deltas| { - group_deltas.group_deltas.iter().flat_map(|group_delta| { - static EMPTY_VEC: Vec = Vec::new(); - let sst_slice = if let GroupDelta::IntraLevel(level_delta) = &group_delta { - &level_delta.inserted_table_infos - } else { - &EMPTY_VEC - }; - sst_slice.iter().map(|sst| sst.object_id) - }) - }) - .chain(self.change_log_delta.values().flat_map(|delta| { - let new_log = delta.new_log.as_ref().unwrap(); - new_log - .new_value - .iter() - .map(|sst| sst.object_id) - .chain(new_log.old_value.iter().map(|sst| sst.object_id)) - })) + self.newly_added_sst_infos(None) + .map(|sst| sst.object_id) .collect() } pub fn newly_added_sst_ids(&self) -> HashSet { - let ssts_from_group_deltas = self.group_deltas.values().flat_map(|group_deltas| { - group_deltas.group_deltas.iter().flat_map(|group_delta| { - static EMPTY_VEC: Vec = Vec::new(); - let sst_slice = if let GroupDelta::IntraLevel(level_delta) = &group_delta { - &level_delta.inserted_table_infos - } else { - &EMPTY_VEC - }; - sst_slice.iter() - }) - }); - - let ssts_from_change_log = self.change_log_delta.values().flat_map(|delta| { - let new_log = delta.new_log.as_ref().unwrap(); - new_log.new_value.iter().chain(new_log.old_value.iter()) - }); - - ssts_from_group_deltas - .chain(ssts_from_change_log) + self.newly_added_sst_infos(None) .map(|sst| sst.sst_id) .collect() } pub fn newly_added_sst_infos<'a>( &'a self, - select_group: &'a HashSet, + select_group: Option<&'a HashSet>, ) -> impl Iterator + 'a { self.group_deltas .iter() - .filter_map(|(cg_id, group_deltas)| { - if select_group.contains(cg_id) { - Some(group_deltas) - } else { + .filter_map(move |(cg_id, group_deltas)| { + if let Some(select_group) = select_group + && !select_group.contains(cg_id) + { None + } else { + Some(group_deltas) } }) .flat_map(|group_deltas| { group_deltas.group_deltas.iter().flat_map(|group_delta| { - static EMPTY_VEC: Vec = Vec::new(); - let sst_slice = if let GroupDelta::IntraLevel(level_delta) = &group_delta { - &level_delta.inserted_table_infos - } else { - &EMPTY_VEC + let sst_slice = match &group_delta { + GroupDeltaCommon::NewL0SubLevel(inserted_table_infos) + | GroupDeltaCommon::IntraLevel(IntraLevelDeltaCommon { + inserted_table_infos, + .. + }) => Some(inserted_table_infos.iter()), + GroupDeltaCommon::GroupConstruct(_) + | GroupDeltaCommon::GroupDestroy(_) + | GroupDeltaCommon::GroupMerge(_) => None, }; - sst_slice.iter() + sst_slice.into_iter().flatten() }) }) .chain(self.change_log_delta.values().flat_map(|delta| { @@ -785,7 +754,7 @@ where pub struct IntraLevelDeltaCommon { pub level_idx: u32, pub l0_sub_level_id: u64, - pub removed_table_ids: Vec, + pub removed_table_ids: HashSet, pub inserted_table_infos: Vec, pub vnode_partition_count: u32, } @@ -814,7 +783,7 @@ where Self { level_idx: pb_intra_level_delta.level_idx, l0_sub_level_id: pb_intra_level_delta.l0_sub_level_id, - removed_table_ids: pb_intra_level_delta.removed_table_ids, + removed_table_ids: HashSet::from_iter(pb_intra_level_delta.removed_table_ids), inserted_table_infos: pb_intra_level_delta .inserted_table_infos .into_iter() @@ -833,7 +802,7 @@ where Self { level_idx: intra_level_delta.level_idx, l0_sub_level_id: intra_level_delta.l0_sub_level_id, - removed_table_ids: intra_level_delta.removed_table_ids, + removed_table_ids: intra_level_delta.removed_table_ids.into_iter().collect(), inserted_table_infos: intra_level_delta .inserted_table_infos .into_iter() @@ -852,7 +821,11 @@ where Self { level_idx: intra_level_delta.level_idx, l0_sub_level_id: intra_level_delta.l0_sub_level_id, - removed_table_ids: intra_level_delta.removed_table_ids.clone(), + removed_table_ids: intra_level_delta + .removed_table_ids + .iter() + .cloned() + .collect(), inserted_table_infos: intra_level_delta .inserted_table_infos .iter() @@ -871,7 +844,9 @@ where Self { level_idx: pb_intra_level_delta.level_idx, l0_sub_level_id: pb_intra_level_delta.l0_sub_level_id, - removed_table_ids: pb_intra_level_delta.removed_table_ids.clone(), + removed_table_ids: HashSet::from_iter( + pb_intra_level_delta.removed_table_ids.iter().cloned(), + ), inserted_table_infos: pb_intra_level_delta .inserted_table_infos .iter() @@ -886,7 +861,7 @@ impl IntraLevelDelta { pub fn new( level_idx: u32, l0_sub_level_id: u64, - removed_table_ids: Vec, + removed_table_ids: HashSet, inserted_table_infos: Vec, vnode_partition_count: u32, ) -> Self { @@ -902,6 +877,7 @@ impl IntraLevelDelta { #[derive(Debug, PartialEq, Clone)] pub enum GroupDeltaCommon { + NewL0SubLevel(Vec), IntraLevel(IntraLevelDeltaCommon), GroupConstruct(PbGroupConstruct), GroupDestroy(PbGroupDestroy), @@ -928,6 +904,13 @@ where Some(PbDeltaType::GroupMerge(pb_group_merge)) => { GroupDeltaCommon::GroupMerge(pb_group_merge) } + Some(DeltaType::NewL0SubLevel(pb_new_sub_level)) => GroupDeltaCommon::NewL0SubLevel( + pb_new_sub_level + .inserted_table_infos + .into_iter() + .map(T::from) + .collect(), + ), None => panic!("delta_type is not set"), } } @@ -951,6 +934,14 @@ where GroupDeltaCommon::GroupMerge(pb_group_merge) => PbGroupDelta { delta_type: Some(PbDeltaType::GroupMerge(pb_group_merge)), }, + GroupDeltaCommon::NewL0SubLevel(new_sub_level) => PbGroupDelta { + delta_type: Some(PbDeltaType::NewL0SubLevel(PbNewL0SubLevel { + inserted_table_infos: new_sub_level + .into_iter() + .map(PbSstableInfo::from) + .collect(), + })), + }, } } } @@ -973,6 +964,11 @@ where GroupDeltaCommon::GroupMerge(pb_group_merge) => PbGroupDelta { delta_type: Some(PbDeltaType::GroupMerge(*pb_group_merge)), }, + GroupDeltaCommon::NewL0SubLevel(new_sub_level) => PbGroupDelta { + delta_type: Some(PbDeltaType::NewL0SubLevel(PbNewL0SubLevel { + inserted_table_infos: new_sub_level.iter().map(PbSstableInfo::from).collect(), + })), + }, } } } @@ -995,6 +991,13 @@ where Some(PbDeltaType::GroupMerge(pb_group_merge)) => { GroupDeltaCommon::GroupMerge(*pb_group_merge) } + Some(DeltaType::NewL0SubLevel(pb_new_sub_level)) => GroupDeltaCommon::NewL0SubLevel( + pb_new_sub_level + .inserted_table_infos + .iter() + .map(T::from) + .collect(), + ), None => panic!("delta_type is not set"), } } diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 7c70721f04d82..4e6ab26a539c6 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -2585,9 +2585,12 @@ async fn test_commit_multi_epoch() { let initial_epoch = INVALID_EPOCH; let commit_epoch = - |epoch, sst: SstableInfo, new_table_fragment_info, tables_to_commit: &[TableId]| { + |epoch, sst: SstableInfo, new_table_fragment_infos, tables_to_commit: &[TableId]| { let manager = &test_env.manager; - let tables_to_commit = tables_to_commit.iter().cloned().collect(); + let tables_to_commit = tables_to_commit + .iter() + .map(|table_id| (*table_id, epoch)) + .collect(); async move { manager .commit_epoch(CommitEpochInfo { @@ -2610,9 +2613,8 @@ async fn test_commit_multi_epoch() { sst_info: sst, created_at: u64::MAX, }], - new_table_fragment_info, + new_table_fragment_infos, change_log_delta: Default::default(), - committed_epoch: epoch, tables_to_commit, }) .await @@ -2633,10 +2635,10 @@ async fn test_commit_multi_epoch() { commit_epoch( epoch1, sst1_epoch1.clone(), - NewTableFragmentInfo::Normal { + vec![NewTableFragmentInfo::Normal { mv_table_id: None, internal_table_ids: vec![existing_table_id], - }, + }], &[existing_table_id], ) .await; @@ -2678,13 +2680,7 @@ async fn test_commit_multi_epoch() { let epoch2 = epoch1.next_epoch(); - commit_epoch( - epoch2, - sst1_epoch2.clone(), - NewTableFragmentInfo::None, - &[existing_table_id], - ) - .await; + commit_epoch(epoch2, sst1_epoch2.clone(), vec![], &[existing_table_id]).await; { let version = test_env.manager.get_current_version().await; @@ -2727,9 +2723,9 @@ async fn test_commit_multi_epoch() { commit_epoch( epoch1, sst2_epoch1.clone(), - NewTableFragmentInfo::NewCompactionGroup { + vec![NewTableFragmentInfo::NewCompactionGroup { table_ids: HashSet::from_iter([new_table_id]), - }, + }], &[new_table_id], ) .await; @@ -2764,13 +2760,7 @@ async fn test_commit_multi_epoch() { ..Default::default() }; - commit_epoch( - epoch2, - sst2_epoch2.clone(), - NewTableFragmentInfo::None, - &[new_table_id], - ) - .await; + commit_epoch(epoch2, sst2_epoch2.clone(), vec![], &[new_table_id]).await; { let version = test_env.manager.get_current_version().await; @@ -2804,7 +2794,7 @@ async fn test_commit_multi_epoch() { commit_epoch( epoch3, sst_epoch3.clone(), - NewTableFragmentInfo::None, + vec![], &[existing_table_id, new_table_id], ) .await; diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index d3e552a76213f..6734235225654 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::ops::Bound; use std::ops::Bound::{Excluded, Included, Unbounded}; use std::sync::Arc; @@ -29,7 +29,7 @@ use risingwave_common::util::epoch::{test_epoch, EpochExt, INVALID_EPOCH, MAX_EP use risingwave_hummock_sdk::key::{prefixed_range_with_vnode, TableKeyRange}; use risingwave_hummock_sdk::{HummockReadEpoch, LocalSstableInfo, SyncResult}; use risingwave_meta::hummock::test_utils::setup_compute_env; -use risingwave_meta::hummock::{CommitEpochInfo, NewTableFragmentInfo}; +use risingwave_meta::hummock::CommitEpochInfo; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::iterator::change_log::test_utils::{ apply_test_log_data, gen_test_data, @@ -1384,10 +1384,9 @@ async fn test_replicated_local_hummock_storage() { sstables: vec![], new_table_watermarks: Default::default(), sst_to_context: Default::default(), - new_table_fragment_info: NewTableFragmentInfo::None, + new_table_fragment_infos: vec![], change_log_delta: Default::default(), - committed_epoch: epoch0, - tables_to_commit: HashSet::from_iter([TEST_TABLE_ID]), + tables_to_commit: HashMap::from_iter([(TEST_TABLE_ID, epoch0)]), }) .await .unwrap(); diff --git a/src/storage/src/hummock/event_handler/hummock_event_handler.rs b/src/storage/src/hummock/event_handler/hummock_event_handler.rs index 7744b102761de..7a33ed81b4373 100644 --- a/src/storage/src/hummock/event_handler/hummock_event_handler.rs +++ b/src/storage/src/hummock/event_handler/hummock_event_handler.rs @@ -796,6 +796,13 @@ impl HummockEventHandler { self.uploader.may_destroy_instance(instance_id); self.destroy_read_version(instance_id); } + HummockEvent::GetMinUncommittedSstId { result_tx } => { + let _ = result_tx + .send(self.uploader.min_uncommitted_sst_id()) + .inspect_err(|e| { + error!("unable to send get_min_uncommitted_sst_id result: {:?}", e); + }); + } } } diff --git a/src/storage/src/hummock/event_handler/mod.rs b/src/storage/src/hummock/event_handler/mod.rs index 60f2e0c02d07e..f0a4b2a899874 100644 --- a/src/storage/src/hummock/event_handler/mod.rs +++ b/src/storage/src/hummock/event_handler/mod.rs @@ -18,7 +18,7 @@ use std::sync::Arc; use parking_lot::{RwLock, RwLockReadGuard}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::TableId; -use risingwave_hummock_sdk::{HummockEpoch, HummockVersionId}; +use risingwave_hummock_sdk::{HummockEpoch, HummockSstableObjectId, HummockVersionId}; use thiserror_ext::AsReport; use tokio::sync::oneshot; @@ -105,6 +105,10 @@ pub enum HummockEvent { DestroyReadVersion { instance_id: LocalInstanceId, }, + + GetMinUncommittedSstId { + result_tx: oneshot::Sender>, + }, } impl HummockEvent { @@ -164,6 +168,7 @@ impl HummockEvent { #[cfg(any(test, feature = "test"))] HummockEvent::FlushEvent(_) => "FlushEvent".to_string(), + HummockEvent::GetMinUncommittedSstId { .. } => "GetMinSpilledSstId".to_string(), } } } diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index cad5708aa831e..6a20f96008db6 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -36,7 +36,7 @@ use risingwave_common::must_match; use risingwave_hummock_sdk::table_watermark::{ TableWatermarks, VnodeWatermark, WatermarkDirection, }; -use risingwave_hummock_sdk::{HummockEpoch, LocalSstableInfo}; +use risingwave_hummock_sdk::{HummockEpoch, HummockSstableObjectId, LocalSstableInfo}; use task_manager::{TaskManager, UploadingTaskStatus}; use thiserror_ext::AsReport; use tokio::sync::oneshot; @@ -1089,6 +1089,21 @@ impl UploaderData { send_sync_result(syncing_data.sync_result_sender, Err(err())); } } + + fn min_uncommitted_sst_id(&self) -> Option { + self.spilled_data + .values() + .map(|(s, _)| s) + .chain(self.syncing_data.values().flat_map(|s| s.uploaded.iter())) + .filter_map(|s| { + s.sstable_infos() + .iter() + .chain(s.old_value_sstable_infos()) + .map(|s| s.sst_info.sst_id) + .min() + }) + .min() + } } struct ErrState { @@ -1329,6 +1344,14 @@ impl HummockUploader { } data.check_upload_task_consistency(); } + + pub(crate) fn min_uncommitted_sst_id(&self) -> Option { + if let UploaderState::Working(ref u) = self.state { + u.min_uncommitted_sst_id() + } else { + None + } + } } impl UploaderData { diff --git a/src/storage/src/hummock/store/hummock_storage.rs b/src/storage/src/hummock/store/hummock_storage.rs index ade7df01a6496..665b064181687 100644 --- a/src/storage/src/hummock/store/hummock_storage.rs +++ b/src/storage/src/hummock/store/hummock_storage.rs @@ -30,7 +30,7 @@ use risingwave_hummock_sdk::key::{ use risingwave_hummock_sdk::sstable_info::SstableInfo; use risingwave_hummock_sdk::table_watermark::TableWatermarksIndex; use risingwave_hummock_sdk::version::HummockVersion; -use risingwave_hummock_sdk::{HummockReadEpoch, HummockVersionId}; +use risingwave_hummock_sdk::{HummockReadEpoch, HummockSstableObjectId, HummockVersionId}; use risingwave_rpc_client::HummockMetaClient; use thiserror_ext::AsReport; use tokio::sync::mpsc::{unbounded_channel, UnboundedSender}; @@ -555,6 +555,14 @@ impl HummockStorage { pub fn compaction_await_tree_reg(&self) -> Option<&await_tree::Registry> { self.compact_await_tree_reg.as_ref() } + + pub async fn min_uncommitted_sst_id(&self) -> Option { + let (tx, rx) = oneshot::channel(); + self.hummock_event_sender + .send(HummockEvent::GetMinUncommittedSstId { result_tx: tx }) + .expect("should send success"); + rx.await.expect("should await success") + } } impl StateStoreRead for HummockStorage { diff --git a/src/storage/src/hummock/validator.rs b/src/storage/src/hummock/validator.rs index cc95b7089b664..2c0efbb3ca934 100644 --- a/src/storage/src/hummock/validator.rs +++ b/src/storage/src/hummock/validator.rs @@ -38,12 +38,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) .sst_id_to_worker_id .get(&sst.object_id) .expect("valid worker_id"); - tracing::debug!( - "Validating SST {} from worker {}, epoch {}", - sst.object_id, - worker_id, - task.epoch - ); + tracing::debug!("Validating SST {} from worker {}", sst.object_id, worker_id,); let holder = match sstable_store.sstable(&sst, unused.borrow_mut()).await { Ok(holder) => holder, Err(_err) => { @@ -100,12 +95,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) break; } } - tracing::debug!( - "Validated {} keys for SST {}, epoch {}", - key_counts, - sst.object_id, - task.epoch - ); + tracing::debug!("Validated {} keys for SST {}", key_counts, sst.object_id,); iter.collect_local_statistic(&mut unused); unused.ignore(); } diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index 89801a3cf4133..eb1325141fdfd 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -12,17 +12,16 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::min; use std::collections::VecDeque; use std::future::{pending, Future}; -use std::mem::replace; +use std::mem::{replace, take}; use std::sync::Arc; use anyhow::anyhow; use futures::future::Either; use futures::{pin_mut, Stream, TryStreamExt}; -use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::TableId; use risingwave_common::metrics::LabelGuardedIntCounter; use risingwave_common::row::OwnedRow; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -32,7 +31,6 @@ use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::ChangeLogRow; use risingwave_storage::StateStore; use tokio::select; -use tokio::sync::mpsc; use tokio::sync::mpsc::UnboundedReceiver; use crate::executor::backfill::utils::{create_builder, mapping_chunk}; @@ -40,7 +38,7 @@ use crate::executor::monitor::StreamingMetrics; use crate::executor::prelude::{try_stream, StreamExt}; use crate::executor::{ expect_first_barrier, ActorContextRef, BackfillExecutor, Barrier, BoxedMessageStream, - DispatcherBarrier, DispatcherMessage, Execute, MergeExecutorInput, Message, Mutation, + DispatcherBarrier, DispatcherMessage, Execute, MergeExecutorInput, Message, StreamExecutorError, StreamExecutorResult, }; use crate::task::CreateMviewProgressReporter; @@ -101,35 +99,14 @@ impl SnapshotBackfillExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { debug!("snapshot backfill executor start"); - let upstream_table_id = self.upstream_table.table_id(); let first_barrier = expect_first_barrier(&mut self.upstream).await?; debug!(epoch = ?first_barrier.epoch, "get first upstream barrier"); let first_recv_barrier = receive_next_barrier(&mut self.barrier_rx).await?; debug!(epoch = ?first_recv_barrier.epoch, "get first inject barrier"); let should_backfill = first_barrier.epoch != first_recv_barrier.epoch; - let mut barrier_epoch = { + let (mut barrier_epoch, mut need_report_finish) = { if should_backfill { - let subscriber_ids = first_recv_barrier - .added_subscriber_on_mv_table(upstream_table_id) - .collect_vec(); - let snapshot_backfill_table_fragment_id = match subscriber_ids.as_slice() { - [] => { - return Err(anyhow!( - "first recv barrier on backfill should add subscriber on upstream" - ) - .into()); - } - [snapshot_backfill_table_fragment_id] => *snapshot_backfill_table_fragment_id, - multiple => { - return Err(anyhow!( - "first recv barrier on backfill have multiple subscribers {:?} on upstream table {}", - multiple, upstream_table_id.table_id - ) - .into()); - } - }; - let table_id_str = format!("{}", self.upstream_table.table_id().table_id); let actor_id_str = format!("{}", self.actor_ctx.id); @@ -138,12 +115,8 @@ impl SnapshotBackfillExecutor { .snapshot_backfill_consume_row_count .with_guarded_label_values(&[&table_id_str, &actor_id_str, "consume_upstream"]); - let mut upstream_buffer = UpstreamBuffer::new( - &mut self.upstream, - upstream_table_id, - snapshot_backfill_table_fragment_id, - consume_upstream_row_count, - ); + let mut upstream_buffer = + UpstreamBuffer::new(&mut self.upstream, consume_upstream_row_count); let first_barrier_epoch = first_barrier.epoch; @@ -165,7 +138,7 @@ impl SnapshotBackfillExecutor { self.rate_limit, &mut self.barrier_rx, &self.output_indices, - self.progress, + &mut self.progress, first_recv_barrier, ); @@ -187,12 +160,11 @@ impl SnapshotBackfillExecutor { yield Message::Barrier(recv_barrier); } - let mut upstream_buffer = - upstream_buffer.start_consuming_log_store(&mut self.barrier_rx); + let mut upstream_buffer = upstream_buffer.start_consuming_log_store(); let mut barrier_epoch = first_barrier_epoch; - let initial_pending_barrier = upstream_buffer.state.barrier_count(); + let initial_pending_barrier = upstream_buffer.barrier_count(); info!( ?barrier_epoch, table_id = self.upstream_table.table_id().table_id, @@ -210,37 +182,50 @@ impl SnapshotBackfillExecutor { ]); // Phase 2: consume upstream log store - while let Some(barrier) = upstream_buffer.take_buffered_barrier().await? { - assert_eq!(barrier_epoch.curr, barrier.epoch.prev); - barrier_epoch = barrier.epoch; - - debug!(?barrier_epoch, kind = ?barrier.kind, "before consume change log"); - // use `upstream_buffer.run_future` to poll upstream concurrently so that we won't have back-pressure - // on the upstream. Otherwise, in `batch_iter_log_with_pk_bounds`, we may wait upstream epoch to be committed, - // and the back-pressure may cause the upstream unable to consume the barrier and then cause deadlock. - let stream = upstream_buffer - .run_future(self.upstream_table.batch_iter_log_with_pk_bounds( - barrier_epoch.prev, - HummockReadEpoch::Committed(barrier_epoch.prev), - )) - .await?; - let data_types = self.upstream_table.schema().data_types(); - let builder = create_builder(None, self.chunk_size, data_types); - let stream = read_change_log(stream, builder); - pin_mut!(stream); - while let Some(chunk) = upstream_buffer.run_future(stream.try_next()).await? { - debug!( - ?barrier_epoch, - size = chunk.cardinality(), - "consume change log yield chunk", - ); - consuming_log_store_row_count.inc_by(chunk.cardinality() as _); - yield Message::Chunk(chunk); - } + while let Some(upstream_barriers) = + upstream_buffer.next_checkpoint_barrier().await? + { + for upstream_barrier in upstream_barriers { + let barrier = receive_next_barrier(&mut self.barrier_rx).await?; + assert_eq!(upstream_barrier.epoch, barrier.epoch); + assert_eq!(barrier_epoch.curr, barrier.epoch.prev); + barrier_epoch = barrier.epoch; + + debug!(?barrier_epoch, kind = ?barrier.kind, "before consume change log"); + // use `upstream_buffer.run_future` to poll upstream concurrently so that we won't have back-pressure + // on the upstream. Otherwise, in `batch_iter_log_with_pk_bounds`, we may wait upstream epoch to be committed, + // and the back-pressure may cause the upstream unable to consume the barrier and then cause deadlock. + let stream = upstream_buffer + .run_future(self.upstream_table.batch_iter_log_with_pk_bounds( + barrier_epoch.prev, + HummockReadEpoch::Committed(barrier_epoch.prev), + )) + .await?; + let data_types = self.upstream_table.schema().data_types(); + let builder = create_builder(None, self.chunk_size, data_types); + let stream = read_change_log(stream, builder); + pin_mut!(stream); + while let Some(chunk) = + upstream_buffer.run_future(stream.try_next()).await? + { + debug!( + ?barrier_epoch, + size = chunk.cardinality(), + "consume change log yield chunk", + ); + consuming_log_store_row_count.inc_by(chunk.cardinality() as _); + yield Message::Chunk(chunk); + } + + debug!(?barrier_epoch, "after consume change log"); - debug!(?barrier_epoch, "after consume change log"); + self.progress.update_create_mview_log_store_progress( + barrier.epoch, + upstream_buffer.barrier_count(), + ); - yield Message::Barrier(barrier); + yield Message::Barrier(barrier); + } } info!( @@ -248,7 +233,7 @@ impl SnapshotBackfillExecutor { table_id = self.upstream_table.table_id().table_id, "finish consuming log store" ); - barrier_epoch + (barrier_epoch, true) } else { info!( table_id = self.upstream_table.table_id().table_id, @@ -257,7 +242,7 @@ impl SnapshotBackfillExecutor { let first_recv_barrier = receive_next_barrier(&mut self.barrier_rx).await?; assert_eq!(first_barrier.epoch, first_recv_barrier.epoch); yield Message::Barrier(first_recv_barrier); - first_barrier.epoch + (first_barrier.epoch, false) } }; let mut upstream = self.upstream.into_executor(self.barrier_rx).execute(); @@ -266,6 +251,10 @@ impl SnapshotBackfillExecutor { if let Message::Barrier(barrier) = &msg { assert_eq!(barrier.epoch.prev, barrier_epoch.curr); barrier_epoch = barrier.epoch; + if need_report_finish { + need_report_finish = false; + self.progress.finish_consuming_log_store(barrier_epoch); + } } yield msg; } @@ -328,146 +317,84 @@ async fn read_change_log( } } -trait UpstreamBufferState { - // The future must be cancellation-safe - async fn is_finished(&mut self) -> StreamExecutorResult; - fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier); -} - -struct StateOfConsumingSnapshot { - pending_barriers: Vec, -} - -impl UpstreamBufferState for StateOfConsumingSnapshot { - async fn is_finished(&mut self) -> StreamExecutorResult { - // never finish when consuming snapshot - Ok(false) - } - - fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier) { - self.pending_barriers.push(upstream_barrier) - } -} - -struct StateOfConsumingLogStore<'a> { - barrier_rx: &'a mut mpsc::UnboundedReceiver, - /// Barriers received from upstream but not yet received the barrier from local barrier worker - /// newer barrier at the front - upstream_pending_barriers: VecDeque, - /// Barriers received from both upstream and local barrier worker - /// newer barrier at the front - barriers: VecDeque, - is_finished: bool, - current_subscriber_id: u32, - upstream_table_id: TableId, -} - -impl<'a> StateOfConsumingLogStore<'a> { - fn barrier_count(&self) -> usize { - self.upstream_pending_barriers.len() + self.barriers.len() - } - - async fn handle_one_pending_barrier(&mut self) -> StreamExecutorResult { - assert!(!self.is_finished); - let barrier = receive_next_barrier(self.barrier_rx).await?; - assert_eq!( - self.upstream_pending_barriers - .pop_back() - .expect("non-empty") - .epoch, - barrier.epoch - ); - if is_finish_barrier(&barrier, self.current_subscriber_id, self.upstream_table_id) { - self.is_finished = true; - } - Ok(barrier) - } -} - -impl<'a> UpstreamBufferState for StateOfConsumingLogStore<'a> { - async fn is_finished(&mut self) -> StreamExecutorResult { - while !self.upstream_pending_barriers.is_empty() { - let barrier = self.handle_one_pending_barrier().await?; - self.barriers.push_front(barrier); - } - if self.is_finished { - assert!(self.upstream_pending_barriers.is_empty()); - } - Ok(self.is_finished) - } - - fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier) { - self.upstream_pending_barriers.push_front(upstream_barrier); - } -} +struct ConsumingSnapshot; +struct ConsumingLogStore; struct UpstreamBuffer<'a, S> { upstream: &'a mut MergeExecutorInput, - state: S, + max_pending_checkpoint_barrier_num: usize, + pending_non_checkpoint_barriers: Vec, + /// Barriers received from upstream but not yet received the barrier from local barrier worker. + /// + /// In the outer `VecDeque`, newer barriers at the front. + /// In the inner `Vec`, newer barrier at the back, with the last barrier as checkpoint barrier, + /// and others as non-checkpoint barrier + upstream_pending_barriers: VecDeque>, + /// Whether we have started polling any upstream data before the next barrier. + /// When `true`, we should continue polling until the next barrier, because + /// some data in this epoch have been discarded and data in this epoch + /// must be read from log store + is_polling_epoch_data: bool, consume_upstream_row_count: LabelGuardedIntCounter<3>, - upstream_table_id: TableId, - current_subscriber_id: u32, + _phase: S, } -impl<'a> UpstreamBuffer<'a, StateOfConsumingSnapshot> { +impl<'a> UpstreamBuffer<'a, ConsumingSnapshot> { fn new( upstream: &'a mut MergeExecutorInput, - upstream_table_id: TableId, - current_subscriber_id: u32, consume_upstream_row_count: LabelGuardedIntCounter<3>, ) -> Self { Self { upstream, - state: StateOfConsumingSnapshot { - pending_barriers: vec![], - }, + is_polling_epoch_data: false, consume_upstream_row_count, - upstream_table_id, - current_subscriber_id, + pending_non_checkpoint_barriers: vec![], + upstream_pending_barriers: Default::default(), + // no limit on the number of pending barrier in the beginning + max_pending_checkpoint_barrier_num: usize::MAX, + _phase: ConsumingSnapshot {}, } } - fn start_consuming_log_store<'s>( - self, - barrier_rx: &'s mut UnboundedReceiver, - ) -> UpstreamBuffer<'a, StateOfConsumingLogStore<'s>> { - let StateOfConsumingSnapshot { pending_barriers } = self.state; - let mut upstream_pending_barriers = VecDeque::with_capacity(pending_barriers.len()); - for pending_barrier in pending_barriers { - upstream_pending_barriers.push_front(pending_barrier); - } + fn start_consuming_log_store(self) -> UpstreamBuffer<'a, ConsumingLogStore> { + let max_pending_barrier_num = self.barrier_count(); UpstreamBuffer { upstream: self.upstream, - state: StateOfConsumingLogStore { - barrier_rx, - upstream_pending_barriers, - barriers: Default::default(), - is_finished: false, - current_subscriber_id: self.current_subscriber_id, - upstream_table_id: self.upstream_table_id, - }, + pending_non_checkpoint_barriers: self.pending_non_checkpoint_barriers, + upstream_pending_barriers: self.upstream_pending_barriers, + max_pending_checkpoint_barrier_num: max_pending_barrier_num, + is_polling_epoch_data: self.is_polling_epoch_data, consume_upstream_row_count: self.consume_upstream_row_count, - upstream_table_id: self.upstream_table_id, - current_subscriber_id: self.current_subscriber_id, + _phase: ConsumingLogStore {}, } } } -impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { +impl<'a, S> UpstreamBuffer<'a, S> { async fn concurrently_consume_upstream(&mut self) -> StreamExecutorError { - if let Err(e) = try { - while !self.state.is_finished().await? { - self.consume_until_next_barrier().await?; + { + loop { + if let Err(e) = try { + if self.upstream_pending_barriers.len() + >= self.max_pending_checkpoint_barrier_num + { + // pause the future to block consuming upstream + return pending().await; + } + let barrier = self.consume_until_next_checkpoint_barrier().await?; + self.upstream_pending_barriers.push_front(barrier); + } { + break e; + } } - } { - return e; } - pending().await } /// Consume the upstream until seeing the next barrier. /// `pending_barriers` must be non-empty after this method returns. - async fn consume_until_next_barrier(&mut self) -> StreamExecutorResult<()> { + async fn consume_until_next_checkpoint_barrier( + &mut self, + ) -> StreamExecutorResult> { loop { let msg: DispatcherMessage = self .upstream @@ -476,63 +403,54 @@ impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { .ok_or_else(|| anyhow!("end of upstream"))?; match msg { DispatcherMessage::Chunk(chunk) => { + self.is_polling_epoch_data = true; self.consume_upstream_row_count .inc_by(chunk.cardinality() as _); } DispatcherMessage::Barrier(barrier) => { - self.state.on_upstream_barrier(barrier); - break Ok(()); + let is_checkpoint = barrier.kind.is_checkpoint(); + self.pending_non_checkpoint_barriers.push(barrier); + if is_checkpoint { + self.is_polling_epoch_data = false; + break Ok(take(&mut self.pending_non_checkpoint_barriers)); + } else { + self.is_polling_epoch_data = true; + } + } + DispatcherMessage::Watermark(_) => { + self.is_polling_epoch_data = true; } - DispatcherMessage::Watermark(_) => {} } } } } -impl<'a, 's> UpstreamBuffer<'a, StateOfConsumingLogStore<'s>> { - async fn take_buffered_barrier(&mut self) -> StreamExecutorResult> { - Ok(if let Some(barrier) = self.state.barriers.pop_back() { - Some(barrier) - } else if !self.state.upstream_pending_barriers.is_empty() { - let barrier = self.state.handle_one_pending_barrier().await?; - Some(barrier) - } else if self.state.is_finished { - None - } else { - self.consume_until_next_barrier().await?; - let barrier = self.state.handle_one_pending_barrier().await?; - Some(barrier) - }) - } -} - -fn is_finish_barrier( - barrier: &Barrier, - current_subscriber_id: u32, - upstream_table_id: TableId, -) -> bool { - if let Some(Mutation::DropSubscriptions { - subscriptions_to_drop, - }) = barrier.mutation.as_deref() - { - let is_finished = subscriptions_to_drop - .iter() - .any(|(subscriber_id, _)| *subscriber_id == current_subscriber_id); - if is_finished { - assert!(subscriptions_to_drop.iter().any( - |(subscriber_id, subscribed_upstream_table_id)| { - *subscriber_id == current_subscriber_id - && upstream_table_id == *subscribed_upstream_table_id +impl<'a> UpstreamBuffer<'a, ConsumingLogStore> { + async fn next_checkpoint_barrier( + &mut self, + ) -> StreamExecutorResult>> { + Ok( + if let Some(barriers) = self.upstream_pending_barriers.pop_back() { + // sub(1) to ensure that the lag is monotonically decreasing. + self.max_pending_checkpoint_barrier_num = min( + self.upstream_pending_barriers.len(), + self.max_pending_checkpoint_barrier_num.saturating_sub(1), + ); + Some(barriers) + } else { + self.max_pending_checkpoint_barrier_num = 0; + if self.is_polling_epoch_data { + let barriers = self.consume_until_next_checkpoint_barrier().await?; + Some(barriers) + } else { + None } - )) - } - is_finished - } else { - false + }, + ) } } -impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { +impl<'a, S> UpstreamBuffer<'a, S> { /// Run a future while concurrently polling the upstream so that the upstream /// won't be back-pressured. async fn run_future>( @@ -550,6 +468,10 @@ impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { } } } + + fn barrier_count(&self) -> usize { + self.upstream_pending_barriers.len() + } } async fn receive_next_barrier( @@ -589,7 +511,7 @@ async fn make_consume_snapshot_stream<'a, S: StateStore>( rate_limit: Option, barrier_rx: &'a mut UnboundedReceiver, output_indices: &'a [usize], - mut progress: CreateMviewProgressReporter, + progress: &'a mut CreateMviewProgressReporter, first_recv_barrier: Barrier, ) { let mut barrier_epoch = first_recv_barrier.epoch; diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index bd166f8082c4c..987e0827da965 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -54,9 +54,21 @@ pub enum BackfillState { SourceCachingUp(String), Finished, } -pub type BackfillStates = HashMap; +pub type BackfillStates = HashMap; -impl BackfillState { +/// Only `state` field is the real state for fail-over. +/// Other fields are for observability (but we still need to persist them). +#[derive(Clone, Debug, Deserialize, Serialize, PartialEq)] +pub struct BackfillStateWithProgress { + pub state: BackfillState, + pub num_consumed_rows: u64, + /// The latest offset from upstream (inclusive). After we reach this offset, we can stop backfilling. + /// This is initialized with the latest available offset in the connector (if the connector provides the ability to fetch it) + /// so that we can finish backfilling even when upstream doesn't emit any data. + pub target_offset: Option, +} + +impl BackfillStateWithProgress { pub fn encode_to_json(self) -> JsonbVal { serde_json::to_value(self).unwrap().into() } @@ -108,13 +120,13 @@ struct BackfillStage { /// /// Note: the offsets are not updated. Should use `state`'s offset to update before using it (`get_latest_unfinished_splits`). splits: Vec, - /// The latest offset from upstream (inclusive). After we reach this offset, we can stop backfilling. - /// This is initialized with the latest available offset in the connector (if the connector provides the ability to fetch it) - /// so that we can finish backfilling even when upstream doesn't emit any data. - target_offsets: HashMap>, } impl BackfillStage { + fn total_backfilled_rows(&self) -> u64 { + self.states.values().map(|s| s.num_consumed_rows).sum() + } + fn debug_assert_consistent(&self) { if cfg!(debug_assertions) { let all_splits: HashSet<_> = @@ -123,10 +135,6 @@ impl BackfillStage { self.states.keys().cloned().collect::>(), all_splits ); - assert_eq!( - self.target_offsets.keys().cloned().collect::>(), - all_splits - ); } } @@ -134,7 +142,7 @@ impl BackfillStage { fn get_latest_unfinished_splits(&self) -> StreamExecutorResult> { let mut unfinished_splits = Vec::new(); for split in &self.splits { - let state = self.states.get(split.id().as_ref()).unwrap(); + let state = &self.states.get(split.id().as_ref()).unwrap().state; match state { BackfillState::Backfilling(Some(offset)) => { let mut updated_split = split.clone(); @@ -152,7 +160,8 @@ impl BackfillStage { fn handle_upstream_row(&mut self, split_id: &str, offset: &str) -> bool { let mut vis = false; let state = self.states.get_mut(split_id).unwrap(); - match state { + let state_inner = &mut state.state; + match state_inner { BackfillState::Backfilling(None) => { // backfilling for this split is not started yet. Ignore this row } @@ -163,12 +172,12 @@ impl BackfillStage { } Ordering::Equal => { // backfilling for this split is finished just right. - *state = BackfillState::Finished; + *state_inner = BackfillState::Finished; } Ordering::Greater => { // backfilling for this split produced more data than current source's progress. // We should stop backfilling, and filter out rows from upstream with offset <= backfill_offset. - *state = BackfillState::SourceCachingUp(backfill_offset.clone()); + *state_inner = BackfillState::SourceCachingUp(backfill_offset.clone()); } } } @@ -178,11 +187,11 @@ impl BackfillStage { // Source caught up, but doesn't contain the last backfilled row. // This may happen e.g., if Kafka performed compaction. vis = true; - *state = BackfillState::Finished; + *state_inner = BackfillState::Finished; } Ordering::Equal => { // Source just caught up with backfilling. - *state = BackfillState::Finished; + *state_inner = BackfillState::Finished; } Ordering::Greater => { // Source is still behind backfilling. @@ -194,11 +203,11 @@ impl BackfillStage { // This split's backfilling is finished, we are waiting for other splits } } - if matches!(state, BackfillState::Backfilling(_)) { - *self.target_offsets.get_mut(split_id).unwrap() = Some(offset.to_string()); + if matches!(state_inner, BackfillState::Backfilling(_)) { + state.target_offset = Some(offset.to_string()); } if vis { - debug_assert_eq!(*state, BackfillState::Finished); + debug_assert_eq!(*state_inner, BackfillState::Finished); } vis } @@ -206,10 +215,11 @@ impl BackfillStage { /// Updates backfill states and returns whether the row backfilled from external system is visible. fn handle_backfill_row(&mut self, split_id: &str, offset: &str) -> bool { let state = self.states.get_mut(split_id).unwrap(); - match state { + state.num_consumed_rows += 1; + let state_inner = &mut state.state; + match state_inner { BackfillState::Backfilling(_old_offset) => { - let target_offset = self.target_offsets.get(split_id).unwrap(); - if let Some(target_offset) = target_offset + if let Some(target_offset) = &state.target_offset && compare_kafka_offset(offset, target_offset).is_ge() { // Note1: If target_offset = offset, it seems we can mark the state as Finished without waiting for upstream to catch up @@ -221,9 +231,9 @@ impl BackfillStage { // // Note3: if target_offset is None (e.g., when upstream doesn't emit messages at all), we will // keep backfilling. - *state = BackfillState::SourceCachingUp(offset.to_string()); + *state_inner = BackfillState::SourceCachingUp(offset.to_string()); } else { - *state = BackfillState::Backfilling(Some(offset.to_string())); + *state_inner = BackfillState::Backfilling(Some(offset.to_string())); } true } @@ -336,22 +346,20 @@ impl SourceBackfillExecutorInner { self.backfill_state_store.init_epoch(barrier.epoch); let mut backfill_states: BackfillStates = HashMap::new(); - for split in &owned_splits { let split_id = split.id(); let backfill_state = self .backfill_state_store .try_recover_from_state_store(&split_id) .await? - .unwrap_or(BackfillState::Backfilling(None)); + .unwrap_or(BackfillStateWithProgress { + state: BackfillState::Backfilling(None), + num_consumed_rows: 0, + target_offset: None, // init with None + }); backfill_states.insert(split_id, backfill_state); } let mut backfill_stage = BackfillStage { - // init with None - target_offsets: backfill_states - .keys() - .map(|split_id| (split_id.clone(), None)) - .collect(), states: backfill_states, splits: owned_splits, }; @@ -368,14 +376,14 @@ impl SourceBackfillExecutorInner { .instrument_await("source_build_reader") .await?; for (split_id, info) in &backfill_info { + let state = backfill_stage.states.get_mut(split_id).unwrap(); match info { BackfillInfo::NoDataToBackfill => { - *backfill_stage.states.get_mut(split_id).unwrap() = BackfillState::Finished; + state.state = BackfillState::Finished; } BackfillInfo::HasDataToBackfill { latest_offset } => { // Note: later we will override it with the offset from the source message, and it's possible to become smaller than this value. - *backfill_stage.target_offsets.get_mut(split_id).unwrap() = - Some(latest_offset.clone()); + state.target_offset = Some(latest_offset.clone()); } } } @@ -586,8 +594,10 @@ impl SourceBackfillExecutorInner { // progress based on the number of consumed rows and an estimated total number of rows from hummock. // For now, we just rely on the same code path, and for source backfill, the progress will always be 99.99%. tracing::debug!("progress finish"); - let epoch = barrier.epoch; - self.progress.finish(epoch, 114514); + self.progress.finish( + barrier.epoch, + backfill_stage.total_backfilled_rows(), + ); // yield barrier after reporting progress yield Message::Barrier(barrier); @@ -671,16 +681,9 @@ impl SourceBackfillExecutorInner { } } - let mut splits: HashSet = backfill_stage.states.keys().cloned().collect(); + let mut states = backfill_stage.states; // Make sure `Finished` state is persisted. - self.backfill_state_store - .set_states( - splits - .iter() - .map(|s| (s.clone(), BackfillState::Finished)) - .collect(), - ) - .await?; + self.backfill_state_store.set_states(states.clone()).await?; // All splits finished backfilling. Now we only forward the source data. #[for_await] @@ -700,7 +703,7 @@ impl SourceBackfillExecutorInner { ); self.apply_split_change_forward_stage( actor_splits, - &mut splits, + &mut states, true, ) .await?; @@ -708,7 +711,7 @@ impl SourceBackfillExecutorInner { Mutation::Update(UpdateMutation { actor_splits, .. }) => { self.apply_split_change_forward_stage( actor_splits, - &mut splits, + &mut states, false, ) .await?; @@ -743,7 +746,7 @@ impl SourceBackfillExecutorInner { fn should_report_finished(&self, states: &BackfillStates) -> bool { states.values().all(|state| { matches!( - state, + state.state, BackfillState::Finished | BackfillState::SourceCachingUp(_) ) }) @@ -763,13 +766,13 @@ impl SourceBackfillExecutorInner { async fn backfill_finished(&self, states: &BackfillStates) -> StreamExecutorResult { Ok(states .values() - .all(|state| matches!(state, BackfillState::Finished)) + .all(|state| matches!(state.state, BackfillState::Finished)) && self .backfill_state_store .scan() .await? .into_iter() - .all(|state| matches!(state, BackfillState::Finished))) + .all(|state| matches!(state.state, BackfillState::Finished))) } /// For newly added splits, we do not need to backfill and can directly forward from upstream. @@ -823,8 +826,15 @@ impl SourceBackfillExecutorInner { match backfill_state { None => { // Newly added split. We don't need to backfill. - // Note that this branch is different from the initial barrier (BackfillState::Backfilling(None) there). - target_state.insert(split_id, BackfillState::Finished); + // Note that this branch is different from the initial barrier (BackfillStateInner::Backfilling(None) there). + target_state.insert( + split_id, + BackfillStateWithProgress { + state: BackfillState::Finished, + num_consumed_rows: 0, + target_offset: None, + }, + ); } Some(backfill_state) => { // Migrated split. Backfill if unfinished. @@ -858,17 +868,6 @@ impl SourceBackfillExecutorInner { } stage.states = target_state; stage.splits = target_splits; - let old_target_offsets = std::mem::take(&mut stage.target_offsets); - stage.target_offsets = stage - .states - .keys() - .map(|split_id| { - ( - split_id.clone(), - old_target_offsets.get(split_id).cloned().flatten(), - ) - }) - .collect(); stage.debug_assert_consistent(); Ok(split_changed) } @@ -878,12 +877,12 @@ impl SourceBackfillExecutorInner { async fn apply_split_change_forward_stage( &mut self, split_assignment: &HashMap>, - splits: &mut HashSet, + states: &mut BackfillStates, should_trim_state: bool, ) -> StreamExecutorResult<()> { self.source_split_change_count.inc(); if let Some(target_splits) = split_assignment.get(&self.actor_ctx.id).cloned() { - self.update_state_if_changed_forward_stage(target_splits, splits, should_trim_state) + self.update_state_if_changed_forward_stage(target_splits, states, should_trim_state) .await?; } @@ -893,7 +892,7 @@ impl SourceBackfillExecutorInner { async fn update_state_if_changed_forward_stage( &mut self, target_splits: Vec, - current_splits: &mut HashSet, + states: &mut BackfillStates, should_trim_state: bool, ) -> StreamExecutorResult<()> { let target_splits: HashSet = target_splits @@ -902,23 +901,25 @@ impl SourceBackfillExecutorInner { .collect(); let mut split_changed = false; + let mut newly_added_splits = vec![]; // Checks added splits for split_id in &target_splits { - if !current_splits.contains(split_id) { + if !states.contains_key(split_id) { split_changed = true; let backfill_state = self .backfill_state_store .try_recover_from_state_store(split_id) .await?; - match backfill_state { + match &backfill_state { None => { // Newly added split. We don't need to backfill! + newly_added_splits.push(split_id.clone()); } Some(backfill_state) => { // Migrated split. It should also be finished since we are in forwarding stage. - match backfill_state { + match backfill_state.state { BackfillState::Finished => {} _ => { return Err(anyhow::anyhow!( @@ -930,11 +931,19 @@ impl SourceBackfillExecutorInner { } } } + states.insert( + split_id.clone(), + backfill_state.unwrap_or(BackfillStateWithProgress { + state: BackfillState::Finished, + num_consumed_rows: 0, + target_offset: None, + }), + ); } } // Checks dropped splits - for existing_split_id in current_splits.iter() { + for existing_split_id in states.keys() { if !target_splits.contains(existing_split_id) { tracing::info!("split dropping detected: {}", existing_split_id); split_changed = true; @@ -947,19 +956,31 @@ impl SourceBackfillExecutorInner { "apply split change" ); - let dropped_splits = - current_splits.extract_if(|split_id| !target_splits.contains(split_id)); + let dropped_splits = states.extract_if(|split_id, _| !target_splits.contains(split_id)); if should_trim_state { // trim dropped splits' state - self.backfill_state_store.trim_state(dropped_splits).await?; + self.backfill_state_store + .trim_state(dropped_splits.map(|(k, _v)| k)) + .await?; } + // For migrated splits, and existing splits, we do not need to update + // state store, but only for newly added splits. self.backfill_state_store .set_states( - target_splits + newly_added_splits .into_iter() - .map(|split_id| (split_id, BackfillState::Finished)) + .map(|split_id| { + ( + split_id, + BackfillStateWithProgress { + state: BackfillState::Finished, + num_consumed_rows: 0, + target_offset: None, + }, + ) + }) .collect(), ) .await?; diff --git a/src/stream/src/executor/source/source_backfill_state_table.rs b/src/stream/src/executor/source/source_backfill_state_table.rs index 3579aff2ec4fb..b0ca8d363b9d7 100644 --- a/src/stream/src/executor/source/source_backfill_state_table.rs +++ b/src/stream/src/executor/source/source_backfill_state_table.rs @@ -23,7 +23,7 @@ use risingwave_connector::source::SplitId; use risingwave_pb::catalog::PbTable; use risingwave_storage::StateStore; -use super::source_backfill_executor::{BackfillState, BackfillStates}; +use super::source_backfill_executor::{BackfillStateWithProgress, BackfillStates}; use crate::common::table::state_table::StateTable; use crate::executor::error::StreamExecutorError; use crate::executor::StreamExecutorResult; @@ -56,7 +56,7 @@ impl BackfillStateTableHandler { } /// XXX: we might get stale data for other actors' writes, but it's fine? - pub async fn scan(&self) -> StreamExecutorResult> { + pub async fn scan(&self) -> StreamExecutorResult> { let sub_range: &(Bound, Bound) = &(Bound::Unbounded, Bound::Unbounded); let state_table_iter = self @@ -70,7 +70,7 @@ impl BackfillStateTableHandler { let row = item?.into_owned_row(); let state = match row.datum_at(1) { Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { - BackfillState::restore_from_json(jsonb_ref.to_owned_scalar())? + BackfillStateWithProgress::restore_from_json(jsonb_ref.to_owned_scalar())? } _ => unreachable!(), }; @@ -80,7 +80,11 @@ impl BackfillStateTableHandler { Ok(ret) } - async fn set(&mut self, key: SplitId, state: BackfillState) -> StreamExecutorResult<()> { + async fn set( + &mut self, + key: SplitId, + state: BackfillStateWithProgress, + ) -> StreamExecutorResult<()> { let row = [ Some(Self::string_to_scalar(key.as_ref())), Some(ScalarImpl::Jsonb(state.encode_to_json())), @@ -126,13 +130,13 @@ impl BackfillStateTableHandler { pub async fn try_recover_from_state_store( &mut self, split_id: &SplitId, - ) -> StreamExecutorResult> { + ) -> StreamExecutorResult> { Ok(self .get(split_id) .await? .map(|row| match row.datum_at(1) { Some(ScalarRefImpl::Jsonb(jsonb_ref)) => { - BackfillState::restore_from_json(jsonb_ref.to_owned_scalar()) + BackfillStateWithProgress::restore_from_json(jsonb_ref.to_owned_scalar()) } _ => unreachable!(), }) diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 9b2820bb3bfed..a91f9b8476111 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::assert_matches::assert_matches; use std::fmt::{Display, Formatter}; use risingwave_common::util::epoch::EpochPair; @@ -27,23 +28,30 @@ type ConsumedRows = u64; #[derive(Debug, Clone, Copy)] pub(crate) enum BackfillState { - ConsumingUpstream(ConsumedEpoch, ConsumedRows), - Done(ConsumedRows), + ConsumingUpstreamTable(ConsumedEpoch, ConsumedRows), + DoneConsumingUpstreamTable(ConsumedRows), + ConsumingLogStore { pending_barrier_num: usize }, + DoneConsumingLogStore, } impl BackfillState { pub fn to_pb(self, actor_id: ActorId) -> PbCreateMviewProgress { + let (done, consumed_epoch, consumed_rows, pending_barrier_num) = match self { + BackfillState::ConsumingUpstreamTable(consumed_epoch, consumed_rows) => { + (false, consumed_epoch, consumed_rows, 0) + } + BackfillState::DoneConsumingUpstreamTable(consumed_rows) => (true, 0, consumed_rows, 0), /* unused field for done */ + BackfillState::ConsumingLogStore { + pending_barrier_num, + } => (false, 0, 0, pending_barrier_num as _), + BackfillState::DoneConsumingLogStore => (true, 0, 0, 0), + }; PbCreateMviewProgress { backfill_actor_id: actor_id, - done: matches!(self, BackfillState::Done(_)), - consumed_epoch: match self { - BackfillState::ConsumingUpstream(consumed_epoch, _) => consumed_epoch, - BackfillState::Done(_) => 0, // unused field for done - }, - consumed_rows: match self { - BackfillState::ConsumingUpstream(_, consumed_rows) => consumed_rows, - BackfillState::Done(consumed_rows) => consumed_rows, - }, + done, + consumed_epoch, + consumed_rows, + pending_barrier_num, } } } @@ -51,10 +59,27 @@ impl BackfillState { impl Display for BackfillState { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - BackfillState::ConsumingUpstream(epoch, rows) => { - write!(f, "ConsumingUpstream(epoch: {}, rows: {})", epoch, rows) + BackfillState::ConsumingUpstreamTable(epoch, rows) => { + write!( + f, + "ConsumingUpstreamTable(epoch: {}, rows: {})", + epoch, rows + ) + } + BackfillState::DoneConsumingUpstreamTable(rows) => { + write!(f, "DoneConsumingUpstreamTable(rows: {})", rows) + } + BackfillState::ConsumingLogStore { + pending_barrier_num, + } => { + write!( + f, + "ConsumingLogStore(pending_barrier_num: {pending_barrier_num})" + ) + } + BackfillState::DoneConsumingLogStore => { + write!(f, "DoneConsumingLogStore") } - BackfillState::Done(rows) => write!(f, "Done(rows: {})", rows), } } } @@ -165,7 +190,7 @@ impl CreateMviewProgressReporter { current_consumed_rows: ConsumedRows, ) { match self.state { - Some(BackfillState::ConsumingUpstream(last, last_consumed_rows)) => { + Some(BackfillState::ConsumingUpstreamTable(last, last_consumed_rows)) => { assert!( last < consumed_epoch, "last_epoch: {:#?} must be greater than consumed epoch: {:#?}", @@ -174,22 +199,61 @@ impl CreateMviewProgressReporter { ); assert!(last_consumed_rows <= current_consumed_rows); } - Some(BackfillState::Done(_)) => unreachable!(), + Some(state) => { + panic!( + "should not update consuming progress at invalid state: {:?}", + state + ) + } None => {} }; self.update_inner( epoch, - BackfillState::ConsumingUpstream(consumed_epoch, current_consumed_rows), + BackfillState::ConsumingUpstreamTable(consumed_epoch, current_consumed_rows), ); } /// Finish the progress. If the progress is already finished, then perform no-op. /// `current_epoch` should be provided to locate the barrier under concurrent checkpoint. pub fn finish(&mut self, epoch: EpochPair, current_consumed_rows: ConsumedRows) { - if let Some(BackfillState::Done(_)) = self.state { + if let Some(BackfillState::DoneConsumingUpstreamTable(_)) = self.state { return; } - self.update_inner(epoch, BackfillState::Done(current_consumed_rows)); + self.update_inner( + epoch, + BackfillState::DoneConsumingUpstreamTable(current_consumed_rows), + ); + } + + pub(crate) fn update_create_mview_log_store_progress( + &mut self, + epoch: EpochPair, + pending_barrier_num: usize, + ) { + assert_matches!( + self.state, + Some(BackfillState::DoneConsumingUpstreamTable(_)) + | Some(BackfillState::ConsumingLogStore { .. }), + "cannot update log store progress at state {:?}", + self.state + ); + self.update_inner( + epoch, + BackfillState::ConsumingLogStore { + pending_barrier_num, + }, + ); + } + + pub(crate) fn finish_consuming_log_store(&mut self, epoch: EpochPair) { + assert_matches!( + self.state, + Some(BackfillState::DoneConsumingUpstreamTable(_)) + | Some(BackfillState::ConsumingLogStore { .. }), + "cannot finish log store progress at state {:?}", + self.state + ); + self.update_inner(epoch, BackfillState::DoneConsumingLogStore); } } @@ -201,7 +265,7 @@ impl LocalBarrierManager { /// /// When all backfill executors of the creating mview finish, the creation progress will be done at /// frontend and the mview will be exposed to the user. - pub fn register_create_mview_progress( + pub(crate) fn register_create_mview_progress( &self, backfill_actor_id: ActorId, ) -> CreateMviewProgressReporter { diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index 3b41c82afab06..2a6118970ccdd 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -158,7 +158,7 @@ impl Configuration { pub fn for_scale_shared_source() -> Self { let mut conf = Self::for_scale(); - conf.per_session_queries = vec!["SET ENABLE_SHARED_SOURCE = true;".into()].into(); + conf.per_session_queries = vec!["SET STREAMING_USE_SHARED_SOURCE = true;".into()].into(); conf }