diff --git a/.typos.toml b/.typos.toml index 4d4bbfca1c082..498d954a55d88 100644 --- a/.typos.toml +++ b/.typos.toml @@ -36,4 +36,5 @@ extend-exclude = [ # We don't want to fix "fals" here, but may want in other places. # Ideally, we should just ignore that line: https://github.com/crate-ci/typos/issues/316 "src/common/src/cast/mod.rs", + "src/tests/simulation/tests/integration_tests/scale/shared_source.rs", ] diff --git a/Cargo.lock b/Cargo.lock index c1b1ec57fdece..32c5fe29fc5aa 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -9216,9 +9216,12 @@ version = "0.14.0" source = "registry+https://github.com/rust-lang/crates.io-index" checksum = "55a6a9143ae25c25fa7b6a48d6cc08b10785372060009c25140a4e7c340e95af" dependencies = [ + "base64 0.22.0", "once_cell", "prost 0.13.1", "prost-types 0.13.1", + "serde", + "serde-value", ] [[package]] @@ -11210,6 +11213,7 @@ dependencies = [ "comfy-table", "crepe", "easy-ext", + "educe", "either", "enum-as-inner 0.6.0", "expect-test", @@ -11557,6 +11561,7 @@ dependencies = [ "madsim-etcd-client", "madsim-rdkafka", "madsim-tokio", + "maplit", "paste", "pin-project", "pretty_assertions", diff --git a/ci/scripts/e2e-source-test.sh b/ci/scripts/e2e-source-test.sh index 56a06ac756931..29f2a0ac7b5ce 100755 --- a/ci/scripts/e2e-source-test.sh +++ b/ci/scripts/e2e-source-test.sh @@ -130,7 +130,7 @@ echo "> inserted new rows into postgres" # start cluster w/o clean-data unset RISINGWAVE_CI -export RUST_LOG="risingwave_stream=debug,risingwave_batch=info,risingwave_storage=info" \ +export RUST_LOG="risingwave_stream=debug,risingwave_batch=info,risingwave_storage=info" risedev dev ci-1cn-1fe-with-recovery echo "> wait for cluster recovery finish" diff --git a/e2e_test/backup_restore/tpch_snapshot_create.slt b/e2e_test/backup_restore/tpch_snapshot_create.slt index c1fad2a2e0759..bb14dd369b837 100644 --- a/e2e_test/backup_restore/tpch_snapshot_create.slt +++ b/e2e_test/backup_restore/tpch_snapshot_create.slt @@ -1,5 +1,8 @@ include ../tpch/create_tables.slt.part +statement ok +CREATE SECRET secret1 WITH (backend = 'meta') AS 'demo-secret' + # First, insert the data into the tables include ../tpch/insert_customer.slt.part include ../tpch/insert_lineitem.slt.part diff --git a/e2e_test/backup_restore/tpch_snapshot_drop.slt b/e2e_test/backup_restore/tpch_snapshot_drop.slt index 0e593371347b7..27d271c35c617 100644 --- a/e2e_test/backup_restore/tpch_snapshot_drop.slt +++ b/e2e_test/backup_restore/tpch_snapshot_drop.slt @@ -1,3 +1,6 @@ +statement ok +DROP SECRET secret1; + statement ok drop materialized view tpch_q7; diff --git a/e2e_test/batch/catalog/pg_settings.slt.part b/e2e_test/batch/catalog/pg_settings.slt.part index 3482ce4850246..e05d466c3a4d6 100644 --- a/e2e_test/batch/catalog/pg_settings.slt.part +++ b/e2e_test/batch/catalog/pg_settings.slt.part @@ -22,6 +22,7 @@ user backfill_rate_limit user background_ddl user batch_enable_distributed_dml user batch_parallelism +user bypass_cluster_limits user bytea_output user cdc_source_wait_streaming_start_timeout user client_encoding diff --git a/e2e_test/commands/risectl b/e2e_test/commands/risectl new file mode 100755 index 0000000000000..2bb462d83fbab --- /dev/null +++ b/e2e_test/commands/risectl @@ -0,0 +1,3 @@ +#!/usr/bin/env bash + +RUST_LOG="error" .risingwave/bin/risingwave/risectl "$@" diff --git a/e2e_test/source_inline/kafka/shared_source.slt b/e2e_test/source_inline/kafka/shared_source.slt index 51a9f1e5ee1b3..5d1072df2cfaa 100644 --- a/e2e_test/source_inline/kafka/shared_source.slt +++ b/e2e_test/source_inline/kafka/shared_source.slt @@ -230,6 +230,87 @@ internal_table.mjs --name s0 --type source 3,"{""split_info"": {""partition"": 3, ""start_offset"": 11, ""stop_offset"": null, ""topic"": ""shared_source""}, ""split_type"": ""kafka""}" +# # Note: the parallelism depends on the risedev profile. +# # So scale tests below are commented out. + +# query ??? +# select name, flags, parallelism from rw_fragments JOIN rw_relations ON rw_fragments.table_id = rw_relations.id order by name; +# ---- +# mv_1 {MVIEW,SOURCE_SCAN} 5 +# mv_2 {MVIEW,SOURCE_SCAN} 5 +# s0 {SOURCE} 5 + + +# system ok +# risectl meta source-split-info --ignore-id +# ---- +# Table +# Fragment (Source) +# Actor (1 splits): [0] +# Actor (1 splits): [2] +# Actor (1 splits): [3] +# Actor (1 splits): [1] +# Actor (0 splits): [] +# Table +# Fragment (SourceScan) +# Actor (1 splits): [0] <- Upstream Actor #1055: [0] +# Actor (1 splits): [2] <- Upstream Actor #1056: [2] +# Actor (1 splits): [3] <- Upstream Actor #1057: [3] +# Actor (1 splits): [1] <- Upstream Actor #1058: [1] +# Actor (0 splits): [] <- Upstream Actor #1059: [] +# Table +# Fragment (SourceScan) +# Actor (1 splits): [0] <- Upstream Actor #1055: [0] +# Actor (1 splits): [2] <- Upstream Actor #1056: [2] +# Actor (1 splits): [3] <- Upstream Actor #1057: [3] +# Actor (1 splits): [1] <- Upstream Actor #1058: [1] +# Actor (0 splits): [] <- Upstream Actor #1059: [] + + +# # scale down +# statement ok +# ALTER MATERIALIZED VIEW mv_1 SET PARALLELISM TO 2; + +# # should have no effect, because of NoShuffle +# # TODO: support ALTER SOURCE SET PARALLELISM, then we can +# query ??? +# select name, flags, parallelism from rw_fragments JOIN rw_relations ON rw_fragments.table_id = rw_relations.id order by name; +# ---- +# mv_1 {MVIEW,SOURCE_SCAN} 5 +# mv_2 {MVIEW,SOURCE_SCAN} 5 +# s0 {SOURCE} 5 + +# system ok +# risectl meta source-split-info --ignore-id +# ---- +# Table +# Fragment (Source) +# Actor (1 splits): [0] +# Actor (1 splits): [2] +# Actor (1 splits): [3] +# Actor (1 splits): [1] +# Actor (0 splits): [] +# Table +# Fragment (SourceScan) +# Actor (1 splits): [0] <- Upstream Actor #1055: [0] +# Actor (1 splits): [2] <- Upstream Actor #1056: [2] +# Actor (1 splits): [3] <- Upstream Actor #1057: [3] +# Actor (1 splits): [1] <- Upstream Actor #1058: [1] +# Actor (0 splits): [] <- Upstream Actor #1059: [] +# Table +# Fragment (SourceScan) +# Actor (1 splits): [0] <- Upstream Actor #1055: [0] +# Actor (1 splits): [2] <- Upstream Actor #1056: [2] +# Actor (1 splits): [3] <- Upstream Actor #1057: [3] +# Actor (1 splits): [1] <- Upstream Actor #1058: [1] +# Actor (0 splits): [] <- Upstream Actor #1059: [] + + +# # Manual test: change the parallelism of the compute node, kill and restart, and check +# # risedev ctl meta source-split-info --ignore-id +# # risedev psql -c "select name, flags, parallelism from rw_fragments JOIN rw_relations ON rw_fragments.table_id = rw_relations.id order by name;" + + statement ok drop source s0 cascade; diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java index 10aa371c50aec..02297a4ea57dd 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSink.java @@ -71,12 +71,13 @@ public JDBCSink(JDBCSinkConfig config, TableSchema tableSchema) { .collect(Collectors.toList()); LOG.info( - "schema = {}, table = {}, tableSchema = {}, columnSqlTypes = {}, pkIndices = {}", + "schema = {}, table = {}, tableSchema = {}, columnSqlTypes = {}, pkIndices = {}, queryTimeout = {}", config.getSchemaName(), config.getTableName(), tableSchema, columnSqlTypes, - pkIndices); + pkIndices, + config.getQueryTimeout()); if (factory.isPresent()) { this.jdbcDialect = factory.get().create(columnSqlTypes, pkIndices); @@ -92,7 +93,7 @@ public JDBCSink(JDBCSinkConfig config, TableSchema tableSchema) { // Commit the `getTransactionIsolation` conn.commit(); - jdbcStatements = new JdbcStatements(conn); + jdbcStatements = new JdbcStatements(conn, config.getQueryTimeout()); } catch (SQLException e) { throw Status.INTERNAL .withDescription( @@ -173,7 +174,7 @@ public boolean write(Iterable rows) { conn = JdbcUtils.getConnection(config.getJdbcUrl()); // reset the flag since we will retry to prepare the batch again updateFlag = false; - jdbcStatements = new JdbcStatements(conn); + jdbcStatements = new JdbcStatements(conn, config.getQueryTimeout()); } else { throw io.grpc.Status.INTERNAL .withDescription( @@ -206,13 +207,15 @@ public boolean write(Iterable rows) { * across multiple batches if only the JDBC connection is valid. */ class JdbcStatements implements AutoCloseable { + private final int queryTimeoutSecs; private PreparedStatement deleteStatement; private PreparedStatement upsertStatement; private PreparedStatement insertStatement; private final Connection conn; - public JdbcStatements(Connection conn) throws SQLException { + public JdbcStatements(Connection conn, int queryTimeoutSecs) throws SQLException { + this.queryTimeoutSecs = queryTimeoutSecs; this.conn = conn; var schemaTableName = jdbcDialect.createSchemaTableName( @@ -339,6 +342,9 @@ private void executeStatement(PreparedStatement stmt) throws SQLException { if (stmt == null) { return; } + // if timeout occurs, a SQLTimeoutException will be thrown + // and we will retry to write the stream chunk in `JDBCSink.write` + stmt.setQueryTimeout(queryTimeoutSecs); LOG.debug("Executing statement: {}", stmt); stmt.executeBatch(); stmt.clearParameters(); diff --git a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkConfig.java b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkConfig.java index ca74ac6a8eb74..94eb5cdc7e0ff 100644 --- a/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkConfig.java +++ b/java/connector-node/risingwave-sink-jdbc/src/main/java/com/risingwave/connector/JDBCSinkConfig.java @@ -32,6 +32,9 @@ public class JDBCSinkConfig extends CommonSinkConfig { @JsonProperty(value = "schema.name") private String schemaName; + @JsonProperty(value = "jdbc.query.timeout") + private int queryTimeoutSeconds = 600; + @JsonCreator public JDBCSinkConfig( @JsonProperty(value = "jdbc.url") String jdbcUrl, @@ -62,4 +65,8 @@ public String getSinkType() { public boolean isUpsertSink() { return this.isUpsertSink; } + + public int getQueryTimeout() { + return queryTimeoutSeconds; + } } diff --git a/proto/meta.proto b/proto/meta.proto index 8932dcbc9e033..98a7f267c0124 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -791,3 +791,30 @@ message RelationIdInfos { // relation_id -> FragmentIdToActorIdMap map map = 1; } + +message ActorCountPerParallelism { + message WorkerActorCount { + uint64 actor_count = 1; + uint64 parallelism = 2; + } + map worker_id_to_actor_count = 1; + uint64 hard_limit = 2; + uint64 soft_limit = 3; +} + +message ClusterLimit { + oneof limit { + ActorCountPerParallelism actor_count = 1; + // TODO: limit DDL using compaction pending bytes + } +} + +message GetClusterLimitsRequest {} + +message GetClusterLimitsResponse { + repeated ClusterLimit active_limits = 1; +} + +service ClusterLimitService { + rpc GetClusterLimits(GetClusterLimitsRequest) returns (GetClusterLimitsResponse); +} diff --git a/proto/stream_service.proto b/proto/stream_service.proto index a9351a08bbae5..ad52b0148856a 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -17,16 +17,6 @@ message BuildActorInfo { map related_subscriptions = 2; } -message DropActorsRequest { - string request_id = 1; - repeated uint32 actor_ids = 2; -} - -message DropActorsResponse { - string request_id = 1; - common.Status status = 2; -} - message InjectBarrierRequest { string request_id = 1; stream_plan.Barrier barrier = 2; @@ -99,7 +89,6 @@ message StreamingControlStreamResponse { } service StreamService { - rpc DropActors(DropActorsRequest) returns (DropActorsResponse); rpc WaitEpochCommit(WaitEpochCommitRequest) returns (WaitEpochCommitResponse); rpc StreamingControlStream(stream StreamingControlStreamRequest) returns (stream StreamingControlStreamResponse); } diff --git a/src/batch/src/executor/join/distributed_lookup_join.rs b/src/batch/src/executor/join/distributed_lookup_join.rs index 1068ffd7f3349..74d7843013e4d 100644 --- a/src/batch/src/executor/join/distributed_lookup_join.rs +++ b/src/batch/src/executor/join/distributed_lookup_join.rs @@ -17,8 +17,9 @@ use std::mem::swap; use futures::pin_mut; use itertools::Itertools; +use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, Schema}; -use risingwave_common::hash::{HashKey, HashKeyDispatcher}; +use risingwave_common::hash::{HashKey, HashKeyDispatcher, VirtualNode}; use risingwave_common::memory::MemoryContext; use risingwave_common::row::OwnedRow; use risingwave_common::types::{DataType, Datum}; @@ -30,7 +31,7 @@ use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::common::BatchQueryEpoch; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::{TableDistribution, TableIter}; +use risingwave_storage::table::TableIter; use risingwave_storage::{dispatch_state_store, StateStore}; use crate::error::Result; @@ -194,7 +195,8 @@ impl BoxedExecutorBuilder for DistributedLookupJoinExecutorBuilder { .collect(); // Lookup Join always contains distribution key, so we don't need vnode bitmap - let vnodes = Some(TableDistribution::all_vnodes()); + // TODO(var-vnode): use vnode count from table desc + let vnodes = Some(Bitmap::ones(VirtualNode::COUNT).into()); dispatch_state_store!(source.context().state_store(), state_store, { let table = StorageTable::new_partial(state_store, column_ids, vnodes, table_desc); let inner_side_builder = InnerSideExecutorBuilder::new( diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index a3be00fc39a22..7c7a08af5d873 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -17,7 +17,7 @@ use std::marker::PhantomData; use anyhow::Context; use itertools::Itertools; -use risingwave_common::bitmap::BitmapBuilder; +use risingwave_common::bitmap::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::{ColumnDesc, Field, Schema}; use risingwave_common::hash::table_distribution::TableDistribution; use risingwave_common::hash::{ @@ -408,12 +408,11 @@ impl BoxedExecutorBuilder for LocalLookupJoinExecutorBuilder { }) .collect(); + // TODO(var-vnode): use vnode count from table desc + let vnodes = Some(Bitmap::ones(VirtualNode::COUNT).into()); let inner_side_builder = InnerSideExecutorBuilder { table_desc: table_desc.clone(), - table_distribution: TableDistribution::new_from_storage_table_desc( - Some(TableDistribution::all_vnodes()), - table_desc, - ), + table_distribution: TableDistribution::new_from_storage_table_desc(vnodes, table_desc), vnode_mapping, outer_side_key_types, inner_side_schema, diff --git a/src/batch/src/executor/log_row_seq_scan.rs b/src/batch/src/executor/log_row_seq_scan.rs index 7106eaec1b760..be2a11b756946 100644 --- a/src/batch/src/executor/log_row_seq_scan.rs +++ b/src/batch/src/executor/log_row_seq_scan.rs @@ -22,13 +22,14 @@ use prometheus::Histogram; use risingwave_common::array::{DataChunk, Op}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::{ColumnId, Field, Schema}; +use risingwave_common::hash::VirtualNode; use risingwave_common::row::{Row, RowExt}; use risingwave_common::types::ScalarImpl; use risingwave_pb::batch_plan::plan_node::NodeBody; use risingwave_pb::common::{batch_query_epoch, BatchQueryEpoch}; use risingwave_pb::plan_common::StorageTableDesc; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::{collect_data_chunk, TableDistribution}; +use risingwave_storage::table::collect_data_chunk; use risingwave_storage::{dispatch_state_store, StateStore}; use super::{BoxedDataChunkStream, BoxedExecutor, BoxedExecutorBuilder, Executor, ExecutorBuilder}; @@ -106,7 +107,8 @@ impl BoxedExecutorBuilder for LogStoreRowSeqScanExecutorBuilder { Some(vnodes) => Some(Bitmap::from(vnodes).into()), // This is possible for dml. vnode_bitmap is not filled by scheduler. // Or it's single distribution, e.g., distinct agg. We scan in a single executor. - None => Some(TableDistribution::all_vnodes()), + // TODO(var-vnode): use vnode count from table desc + None => Some(Bitmap::ones(VirtualNode::COUNT).into()), }; let chunk_size = source.context.get_config().developer.chunk_size as u32; diff --git a/src/batch/src/executor/row_seq_scan.rs b/src/batch/src/executor/row_seq_scan.rs index b897dbd813787..7c7244d954764 100644 --- a/src/batch/src/executor/row_seq_scan.rs +++ b/src/batch/src/executor/row_seq_scan.rs @@ -21,6 +21,7 @@ use prometheus::Histogram; use risingwave_common::array::DataChunk; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::{ColumnId, Schema}; +use risingwave_common::hash::VirtualNode; use risingwave_common::row::{OwnedRow, Row}; use risingwave_common::types::{DataType, Datum}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -32,7 +33,6 @@ use risingwave_pb::plan_common::as_of::AsOfType; use risingwave_pb::plan_common::{as_of, PbAsOf, StorageTableDesc}; use risingwave_storage::store::PrefetchOptions; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::TableDistribution; use risingwave_storage::{dispatch_state_store, StateStore}; use crate::error::{BatchError, Result}; @@ -210,7 +210,8 @@ impl BoxedExecutorBuilder for RowSeqScanExecutorBuilder { Some(vnodes) => Some(Bitmap::from(vnodes).into()), // This is possible for dml. vnode_bitmap is not filled by scheduler. // Or it's single distribution, e.g., distinct agg. We scan in a single executor. - None => Some(TableDistribution::all_vnodes()), + // TODO(var-vnode): use vnode count from table desc + None => Some(Bitmap::ones(VirtualNode::COUNT).into()), }; let scan_ranges = { diff --git a/src/batch/src/task/consistent_hash_shuffle_channel.rs b/src/batch/src/task/consistent_hash_shuffle_channel.rs index ad0fdbaa8b70a..32d91a7acc09b 100644 --- a/src/batch/src/task/consistent_hash_shuffle_channel.rs +++ b/src/batch/src/task/consistent_hash_shuffle_channel.rs @@ -59,6 +59,7 @@ fn generate_hash_values( .iter() .map(|idx| *idx as usize) .collect::>(), + consistent_hash_info.vmap.len(), ); let hash_values = vnodes diff --git a/src/common/src/config.rs b/src/common/src/config.rs index 88ea110869b79..ed7ac8619252c 100644 --- a/src/common/src/config.rs +++ b/src/common/src/config.rs @@ -466,6 +466,16 @@ pub struct MetaDeveloperConfig { #[serde(default = "default::developer::max_get_task_probe_times")] pub max_get_task_probe_times: usize, + + /// Max number of actor allowed per parallelism (default = 100). + /// CREATE MV/Table will be noticed when the number of actors exceeds this limit. + #[serde(default = "default::developer::actor_cnt_per_worker_parallelism_soft_limit")] + pub actor_cnt_per_worker_parallelism_soft_limit: usize, + + /// Max number of actor allowed per parallelism (default = 400). + /// CREATE MV/Table will be rejected when the number of actors exceeds this limit. + #[serde(default = "default::developer::actor_cnt_per_worker_parallelism_hard_limit")] + pub actor_cnt_per_worker_parallelism_hard_limit: usize, } /// The section `[server]` in `risingwave.toml`. @@ -1859,6 +1869,14 @@ pub mod default { 5 } + pub fn actor_cnt_per_worker_parallelism_soft_limit() -> usize { + 100 + } + + pub fn actor_cnt_per_worker_parallelism_hard_limit() -> usize { + 400 + } + pub fn memory_controller_threshold_aggressive() -> f64 { 0.9 } diff --git a/src/common/src/hash/consistent_hash/bitmap.rs b/src/common/src/hash/consistent_hash/bitmap.rs index 773231ba36a89..eee6a64a2b42c 100644 --- a/src/common/src/hash/consistent_hash/bitmap.rs +++ b/src/common/src/hash/consistent_hash/bitmap.rs @@ -15,6 +15,7 @@ use std::ops::RangeInclusive; use crate::bitmap::Bitmap; +use crate::hash::table_distribution::SINGLETON_VNODE; use crate::hash::VirtualNode; /// An extension trait for `Bitmap` to support virtual node operations. @@ -36,4 +37,17 @@ impl Bitmap { self.high_ranges() .map(|r| (VirtualNode::from_index(*r.start())..=VirtualNode::from_index(*r.end()))) } + + /// Returns whether only the [`SINGLETON_VNODE`] is set in the bitmap. + /// + /// Note that this method returning `true` does not imply that the bitmap was created by + /// [`VnodeBitmapExt::singleton`], or that the bitmap has length 1. + pub fn is_singleton(&self) -> bool { + self.count_ones() == 1 && self.iter_vnodes().next().unwrap() == SINGLETON_VNODE + } + + /// Creates a bitmap with length 1 and the single bit set. + pub fn singleton() -> Self { + Self::ones(1) + } } diff --git a/src/common/src/hash/consistent_hash/mapping.rs b/src/common/src/hash/consistent_hash/mapping.rs index a462acb291853..0ab8f9e18fd2e 100644 --- a/src/common/src/hash/consistent_hash/mapping.rs +++ b/src/common/src/hash/consistent_hash/mapping.rs @@ -105,26 +105,26 @@ impl VnodeMapping { /// /// For example, if `items` is `[0, 1, 2]`, and the total vnode count is 10, we'll generate /// mapping like `[0, 0, 0, 0, 1, 1, 1, 2, 2, 2]`. - pub fn new_uniform(items: impl ExactSizeIterator) -> Self { + pub fn new_uniform(items: impl ExactSizeIterator, vnode_count: usize) -> Self { // If the number of items is greater than the total vnode count, no vnode will be mapped to // some items and the mapping will be invalid. - assert!(items.len() <= VirtualNode::COUNT); + assert!(items.len() <= vnode_count); let mut original_indices = Vec::with_capacity(items.len()); let mut data = Vec::with_capacity(items.len()); - let hash_shard_size = VirtualNode::COUNT / items.len(); - let mut one_more_count = VirtualNode::COUNT % items.len(); + let hash_shard_size = vnode_count / items.len(); + let mut one_more_count = vnode_count % items.len(); let mut init_bound = 0; for item in items { - let vnode_count = if one_more_count > 0 { + let count = if one_more_count > 0 { one_more_count -= 1; hash_shard_size + 1 } else { hash_shard_size }; - init_bound += vnode_count; + init_bound += count; original_indices.push(init_bound as u32 - 1); data.push(item); @@ -141,10 +141,11 @@ impl VnodeMapping { /// Create a vnode mapping where all vnodes are mapped to the same single item. pub fn new_single(item: T::Item) -> Self { - Self::new_uniform(std::iter::once(item)) + // TODO(var-vnode): always 1 correct? + Self::new_uniform(std::iter::once(item), 1) } - /// The length of the vnode in this mapping, typically [`VirtualNode::COUNT`]. + /// The length (or count) of the vnode in this mapping. pub fn len(&self) -> usize { self.original_indices .last() @@ -204,12 +205,13 @@ impl VnodeMapping { /// Convert this vnode mapping to a mapping from items to bitmaps, where each bitmap represents /// the vnodes mapped to the item. pub fn to_bitmaps(&self) -> HashMap { + let vnode_count = self.len(); let mut vnode_bitmaps = HashMap::new(); for (vnode, item) in self.iter_with_vnode() { vnode_bitmaps .entry(item) - .or_insert_with(|| BitmapBuilder::zeroed(VirtualNode::COUNT)) + .or_insert_with(|| BitmapBuilder::zeroed(vnode_count)) .set(vnode.to_index(), true); } @@ -222,10 +224,11 @@ impl VnodeMapping { /// Create a vnode mapping from the given mapping from items to bitmaps, where each bitmap /// represents the vnodes mapped to the item. pub fn from_bitmaps(bitmaps: &HashMap) -> Self { - let mut items = vec![None; VirtualNode::COUNT]; + let vnode_count = bitmaps.values().next().expect("empty bitmaps").len(); + let mut items = vec![None; vnode_count]; for (&item, bitmap) in bitmaps { - assert_eq!(bitmap.len(), VirtualNode::COUNT); + assert_eq!(bitmap.len(), vnode_count); for idx in bitmap.iter_ones() { if let Some(prev) = items[idx].replace(item) { panic!("mapping at index `{idx}` is set to both `{prev:?}` and `{item:?}`"); @@ -241,9 +244,8 @@ impl VnodeMapping { Self::from_expanded(&items) } - /// Create a vnode mapping from the expanded slice of items with length [`VirtualNode::COUNT`]. + /// Create a vnode mapping from the expanded slice of items. pub fn from_expanded(items: &[T::Item]) -> Self { - assert_eq!(items.len(), VirtualNode::COUNT); let (original_indices, data) = compress_data(items); Self { original_indices, @@ -251,7 +253,7 @@ impl VnodeMapping { } } - /// Convert this vnode mapping to a expanded vector of items with length [`VirtualNode::COUNT`]. + /// Convert this vnode mapping to a expanded vector of items. pub fn to_expanded(&self) -> ExpandedMapping { self.iter().collect() } @@ -353,8 +355,8 @@ impl ActorMapping { impl WorkerSlotMapping { /// Create a uniform worker mapping from the given worker ids - pub fn build_from_ids(worker_slot_ids: &[WorkerSlotId]) -> Self { - Self::new_uniform(worker_slot_ids.iter().cloned()) + pub fn build_from_ids(worker_slot_ids: &[WorkerSlotId], vnode_count: usize) -> Self { + Self::new_uniform(worker_slot_ids.iter().cloned(), vnode_count) } /// Create a worker mapping from the protobuf representation. @@ -403,18 +405,18 @@ mod tests { type TestMapping = VnodeMapping; type Test2Mapping = VnodeMapping; - const COUNTS: &[usize] = &[1, 3, 12, 42, VirtualNode::COUNT]; + const COUNTS: &[usize] = &[1, 3, 12, 42, VirtualNode::COUNT_FOR_TEST]; fn uniforms() -> impl Iterator { COUNTS .iter() - .map(|&count| TestMapping::new_uniform(0..count as u32)) + .map(|&count| TestMapping::new_uniform(0..count as u32, VirtualNode::COUNT_FOR_TEST)) } fn randoms() -> impl Iterator { COUNTS.iter().map(|&count| { let raw = repeat_with(|| rand::thread_rng().gen_range(0..count as u32)) - .take(VirtualNode::COUNT) + .take(VirtualNode::COUNT_FOR_TEST) .collect_vec(); TestMapping::from_expanded(&raw) }) @@ -427,7 +429,7 @@ mod tests { #[test] fn test_uniform() { for vnode_mapping in uniforms() { - assert_eq!(vnode_mapping.len(), VirtualNode::COUNT); + assert_eq!(vnode_mapping.len(), VirtualNode::COUNT_FOR_TEST); let item_count = vnode_mapping.iter_unique().count(); let mut check: HashMap> = HashMap::new(); diff --git a/src/common/src/hash/consistent_hash/vnode.rs b/src/common/src/hash/consistent_hash/vnode.rs index f528544689f31..685f99d6cf4f4 100644 --- a/src/common/src/hash/consistent_hash/vnode.rs +++ b/src/common/src/hash/consistent_hash/vnode.rs @@ -30,26 +30,45 @@ use crate::util::row_id::extract_vnode_id_from_row_id; pub struct VirtualNode(VirtualNodeInner); /// The internal representation of a virtual node id. +/// +/// Note: not all bits of the inner representation might be used. type VirtualNodeInner = u16; -static_assertions::const_assert!(VirtualNodeInner::BITS >= VirtualNode::BITS as u32); -impl From for VirtualNode { - fn from(hash_code: Crc32HashCode) -> Self { +/// `vnode_count` must be provided to convert a hash code to a virtual node. +/// +/// Use [`Crc32HashCodeToVnodeExt::to_vnode`] instead. +impl !From for VirtualNode {} + +#[easy_ext::ext(Crc32HashCodeToVnodeExt)] +impl Crc32HashCode { + /// Converts the hash code to a virtual node, based on the given total count of vnodes. + fn to_vnode(self, vnode_count: usize) -> VirtualNode { // Take the least significant bits of the hash code. // TODO: should we use the most significant bits? - let inner = (hash_code.value() % Self::COUNT as u64) as VirtualNodeInner; + let inner = (self.value() % vnode_count as u64) as VirtualNodeInner; VirtualNode(inner) } } impl VirtualNode { - /// The number of bits used to represent a virtual node. - /// - /// Note: Not all bits of the inner representation are used. One should rely on this constant - /// to determine the count of virtual nodes. - pub const BITS: usize = 8; /// The total count of virtual nodes. - pub const COUNT: usize = 1 << Self::BITS; + // TODO(var-vnode): remove this and only keep `COUNT_FOR_TEST` + pub const COUNT: usize = 1 << 8; + /// The maximum value of the virtual node. + // TODO(var-vnode): remove this and only keep `MAX_FOR_TEST` + pub const MAX: VirtualNode = VirtualNode::from_index(Self::COUNT - 1); +} + +impl VirtualNode { + /// The total count of virtual nodes, for testing purposes. + pub const COUNT_FOR_TEST: usize = Self::COUNT; + /// The maximum value of the virtual node, for testing purposes. + pub const MAX_FOR_TEST: VirtualNode = Self::MAX; +} + +impl VirtualNode { + /// The maximum count of virtual nodes that fits in [`VirtualNodeInner`]. + pub const MAX_COUNT: usize = 1 << VirtualNodeInner::BITS; /// The size of a virtual node in bytes, in memory or serialized representation. pub const SIZE: usize = std::mem::size_of::(); } @@ -58,8 +77,6 @@ impl VirtualNode { pub type AllVirtualNodeIter = std::iter::Map, fn(usize) -> VirtualNode>; impl VirtualNode { - /// The maximum value of the virtual node. - pub const MAX: VirtualNode = VirtualNode::from_index(Self::COUNT - 1); /// We may use `VirtualNode` as a datum in a stream, or store it as a column. /// Hence this reifies it as a RW datatype. pub const RW_TYPE: DataType = DataType::Int16; @@ -68,7 +85,7 @@ impl VirtualNode { /// Creates a virtual node from the `usize` index. pub const fn from_index(index: usize) -> Self { - debug_assert!(index < Self::COUNT); + debug_assert!(index < Self::MAX_COUNT); Self(index as _) } @@ -79,7 +96,6 @@ impl VirtualNode { /// Creates a virtual node from the given scalar representation. Used by `VNODE` expression. pub const fn from_scalar(scalar: i16) -> Self { - debug_assert!((scalar as usize) < Self::COUNT); Self(scalar as _) } @@ -99,7 +115,6 @@ impl VirtualNode { /// Creates a virtual node from the given big-endian bytes representation. pub const fn from_be_bytes(bytes: [u8; Self::SIZE]) -> Self { let inner = VirtualNodeInner::from_be_bytes(bytes); - debug_assert!((inner as usize) < Self::COUNT); Self(inner) } @@ -109,8 +124,8 @@ impl VirtualNode { } /// Iterates over all virtual nodes. - pub fn all() -> AllVirtualNodeIter { - (0..Self::COUNT).map(Self::from_index) + pub fn all(vnode_count: usize) -> AllVirtualNodeIter { + (0..vnode_count).map(Self::from_index) } } @@ -119,7 +134,11 @@ impl VirtualNode { // chunk. When only one column is provided and its type is `Serial`, we consider the column to // be the one that contains RowId, and use a special method to skip the calculation of Hash // and directly extract the `VirtualNode` from `RowId`. - pub fn compute_chunk(data_chunk: &DataChunk, keys: &[usize]) -> Vec { + pub fn compute_chunk( + data_chunk: &DataChunk, + keys: &[usize], + vnode_count: usize, + ) -> Vec { if let Ok(idx) = keys.iter().exactly_one() && let ArrayImpl::Serial(serial_array) = &**data_chunk.column_at(*idx) { @@ -135,7 +154,7 @@ impl VirtualNode { // This process doesn’t guarantee the order of rows, producing indeterminate results in some cases, // such as when `distinct on` is used without an `order by`. let (row, _) = data_chunk.row_at(idx); - row.hash(Crc32FastBuilder).into() + row.hash(Crc32FastBuilder).to_vnode(vnode_count) } }) .collect(); @@ -144,19 +163,29 @@ impl VirtualNode { data_chunk .get_hash_values(keys, Crc32FastBuilder) .into_iter() - .map(|hash| hash.into()) + .map(|hash| hash.to_vnode(vnode_count)) .collect() } + /// Equivalent to [`Self::compute_chunk`] with [`VirtualNode::COUNT_FOR_TEST`] as the vnode count. + pub fn compute_chunk_for_test(data_chunk: &DataChunk, keys: &[usize]) -> Vec { + Self::compute_chunk(data_chunk, keys, Self::COUNT_FOR_TEST) + } + // `compute_row` is used to calculate the `VirtualNode` for the corresponding columns in a // `Row`. Similar to `compute_chunk`, it also contains special handling for serial columns. - pub fn compute_row(row: impl Row, indices: &[usize]) -> VirtualNode { + pub fn compute_row(row: impl Row, indices: &[usize], vnode_count: usize) -> VirtualNode { let project = row.project(indices); if let Ok(Some(ScalarRefImpl::Serial(s))) = project.iter().exactly_one().as_ref() { return extract_vnode_id_from_row_id(s.as_row_id()); } - project.hash(Crc32FastBuilder).into() + project.hash(Crc32FastBuilder).to_vnode(vnode_count) + } + + /// Equivalent to [`Self::compute_row`] with [`VirtualNode::COUNT_FOR_TEST`] as the vnode count. + pub fn compute_row_for_test(row: impl Row, indices: &[usize]) -> VirtualNode { + Self::compute_row(row, indices, Self::COUNT_FOR_TEST) } } @@ -179,7 +208,7 @@ mod tests { ); let chunk = DataChunk::from_pretty(chunk.as_str()); - let vnodes = VirtualNode::compute_chunk(&chunk, &[0]); + let vnodes = VirtualNode::compute_chunk_for_test(&chunk, &[0]); assert_eq!( vnodes.as_slice(), @@ -195,7 +224,7 @@ mod tests { Some(ScalarImpl::Int64(12345)), ]); - let vnode = VirtualNode::compute_row(&row, &[0]); + let vnode = VirtualNode::compute_row_for_test(&row, &[0]); assert_eq!(vnode, VirtualNode::from_index(100)); } @@ -216,7 +245,7 @@ mod tests { ); let chunk = DataChunk::from_pretty(chunk.as_str()); - let vnodes = VirtualNode::compute_chunk(&chunk, &[0]); + let vnodes = VirtualNode::compute_chunk_for_test(&chunk, &[0]); assert_eq!( vnodes.as_slice(), diff --git a/src/common/src/hash/table_distribution.rs b/src/common/src/hash/table_distribution.rs index 9be9cd2abafb2..5275aca04adb3 100644 --- a/src/common/src/hash/table_distribution.rs +++ b/src/common/src/hash/table_distribution.rs @@ -13,30 +13,34 @@ // limitations under the License. use std::mem::replace; -use std::ops::Deref; use std::sync::{Arc, LazyLock}; use itertools::Itertools; use risingwave_pb::plan_common::StorageTableDesc; -use tracing::warn; use crate::array::{Array, DataChunk, PrimitiveArray}; -use crate::bitmap::{Bitmap, BitmapBuilder}; +use crate::bitmap::Bitmap; use crate::hash::VirtualNode; use crate::row::Row; use crate::util::iter_util::ZipEqFast; -/// For tables without distribution (singleton), the `DEFAULT_VNODE` is encoded. -pub const DEFAULT_VNODE: VirtualNode = VirtualNode::ZERO; +/// For tables without distribution (singleton), the `SINGLETON_VNODE` is encoded. +pub const SINGLETON_VNODE: VirtualNode = VirtualNode::ZERO; + +use super::VnodeBitmapExt; #[derive(Debug, Clone)] enum ComputeVnode { Singleton, DistKeyIndices { + /// Virtual nodes that the table is partitioned into. + vnodes: Arc, /// Indices of distribution key for computing vnode, based on the pk columns of the table. dist_key_in_pk_indices: Vec, }, VnodeColumnIndex { + /// Virtual nodes that the table is partitioned into. + vnodes: Arc, /// Index of vnode column. vnode_col_idx_in_pk: usize, }, @@ -47,13 +51,8 @@ enum ComputeVnode { pub struct TableDistribution { /// The way to compute vnode provided primary key compute_vnode: ComputeVnode, - - /// Virtual nodes that the table is partitioned into. - vnodes: Arc, } -pub const SINGLETON_VNODE: VirtualNode = DEFAULT_VNODE; - impl TableDistribution { pub fn new_from_storage_table_desc( vnodes: Option>, @@ -75,69 +74,32 @@ impl TableDistribution { ) -> Self { let compute_vnode = if let Some(vnode_col_idx_in_pk) = vnode_col_idx_in_pk { ComputeVnode::VnodeColumnIndex { + vnodes: vnodes.unwrap_or_else(|| Bitmap::singleton().into()), vnode_col_idx_in_pk, } } else if !dist_key_in_pk_indices.is_empty() { ComputeVnode::DistKeyIndices { + vnodes: vnodes.expect("vnodes must be `Some` as dist key indices are set"), dist_key_in_pk_indices, } } else { ComputeVnode::Singleton }; - let vnodes = vnodes.unwrap_or_else(Self::singleton_vnode_bitmap); - if let ComputeVnode::Singleton = &compute_vnode { - if &vnodes != Self::singleton_vnode_bitmap_ref() && &vnodes != Self::all_vnodes_ref() { - warn!( - ?vnodes, - "singleton distribution get non-singleton vnode bitmap" - ); - } - } - - Self { - compute_vnode, - vnodes, - } + Self { compute_vnode } } pub fn is_singleton(&self) -> bool { matches!(&self.compute_vnode, ComputeVnode::Singleton) } - pub fn singleton_vnode_bitmap_ref() -> &'static Arc { - /// A bitmap that only the default vnode is set. - static SINGLETON_VNODES: LazyLock> = LazyLock::new(|| { - let mut vnodes = BitmapBuilder::zeroed(VirtualNode::COUNT); - vnodes.set(SINGLETON_VNODE.to_index(), true); - vnodes.finish().into() - }); - - SINGLETON_VNODES.deref() - } - - pub fn singleton_vnode_bitmap() -> Arc { - Self::singleton_vnode_bitmap_ref().clone() - } - - pub fn all_vnodes_ref() -> &'static Arc { - /// A bitmap that all vnodes are set. - static ALL_VNODES: LazyLock> = - LazyLock::new(|| Bitmap::ones(VirtualNode::COUNT).into()); - &ALL_VNODES - } - - pub fn all_vnodes() -> Arc { - Self::all_vnodes_ref().clone() - } - /// Distribution that accesses all vnodes, mainly used for tests. - pub fn all(dist_key_in_pk_indices: Vec) -> Self { + pub fn all(dist_key_in_pk_indices: Vec, vnode_count: usize) -> Self { Self { compute_vnode: ComputeVnode::DistKeyIndices { + vnodes: Bitmap::ones(vnode_count).into(), dist_key_in_pk_indices, }, - vnodes: Self::all_vnodes(), } } @@ -145,20 +107,39 @@ impl TableDistribution { pub fn singleton() -> Self { Self { compute_vnode: ComputeVnode::Singleton, - vnodes: Self::singleton_vnode_bitmap(), } } pub fn update_vnode_bitmap(&mut self, new_vnodes: Arc) -> Arc { - if self.is_singleton() && &new_vnodes != Self::singleton_vnode_bitmap_ref() { - warn!(?new_vnodes, "update vnode on singleton distribution"); + match &mut self.compute_vnode { + ComputeVnode::Singleton => { + if !new_vnodes.is_singleton() { + panic!( + "update vnode bitmap on singleton distribution to non-singleton: {:?}", + new_vnodes + ); + } + self.vnodes().clone() // not updated + } + + ComputeVnode::DistKeyIndices { vnodes, .. } + | ComputeVnode::VnodeColumnIndex { vnodes, .. } => { + assert_eq!(vnodes.len(), new_vnodes.len()); + replace(vnodes, new_vnodes) + } } - assert_eq!(self.vnodes.len(), new_vnodes.len()); - replace(&mut self.vnodes, new_vnodes) } + /// Get vnode bitmap if distributed, or a dummy [`Bitmap::singleton()`] if singleton. pub fn vnodes(&self) -> &Arc { - &self.vnodes + static SINGLETON_VNODES: LazyLock> = + LazyLock::new(|| Bitmap::singleton().into()); + + match &self.compute_vnode { + ComputeVnode::DistKeyIndices { vnodes, .. } => vnodes, + ComputeVnode::VnodeColumnIndex { vnodes, .. } => vnodes, + ComputeVnode::Singleton => &SINGLETON_VNODES, + } } /// Get vnode value with given primary key. @@ -166,11 +147,13 @@ impl TableDistribution { match &self.compute_vnode { ComputeVnode::Singleton => SINGLETON_VNODE, ComputeVnode::DistKeyIndices { + vnodes, dist_key_in_pk_indices, - } => compute_vnode(pk, dist_key_in_pk_indices, &self.vnodes), + } => compute_vnode(pk, dist_key_in_pk_indices, vnodes), ComputeVnode::VnodeColumnIndex { + vnodes, vnode_col_idx_in_pk, - } => get_vnode_from_row(pk, *vnode_col_idx_in_pk, &self.vnodes), + } => get_vnode_from_row(pk, *vnode_col_idx_in_pk, vnodes), } } @@ -178,22 +161,20 @@ impl TableDistribution { match &self.compute_vnode { ComputeVnode::Singleton => Some(SINGLETON_VNODE), ComputeVnode::DistKeyIndices { + vnodes, dist_key_in_pk_indices, } => dist_key_in_pk_indices .iter() .all(|&d| d < pk_prefix.len()) - .then(|| compute_vnode(pk_prefix, dist_key_in_pk_indices, &self.vnodes)), + .then(|| compute_vnode(pk_prefix, dist_key_in_pk_indices, vnodes)), ComputeVnode::VnodeColumnIndex { + vnodes, vnode_col_idx_in_pk, } => { if *vnode_col_idx_in_pk >= pk_prefix.len() { None } else { - Some(get_vnode_from_row( - pk_prefix, - *vnode_col_idx_in_pk, - &self.vnodes, - )) + Some(get_vnode_from_row(pk_prefix, *vnode_col_idx_in_pk, vnodes)) } } } @@ -203,7 +184,7 @@ impl TableDistribution { /// Get vnode value with `indices` on the given `row`. pub fn compute_vnode(row: impl Row, indices: &[usize], vnodes: &Bitmap) -> VirtualNode { assert!(!indices.is_empty()); - let vnode = VirtualNode::compute_row(&row, indices); + let vnode = VirtualNode::compute_row(&row, indices, vnodes.len()); check_vnode_is_set(vnode, vnodes); tracing::debug!(target: "events::storage::storage_table", "compute vnode: {:?} key {:?} => {}", row, indices, vnode); @@ -230,6 +211,7 @@ impl TableDistribution { vec![SINGLETON_VNODE; chunk.capacity()] } ComputeVnode::DistKeyIndices { + vnodes, dist_key_in_pk_indices, } => { let dist_key_indices = dist_key_in_pk_indices @@ -237,19 +219,20 @@ impl TableDistribution { .map(|idx| pk_indices[*idx]) .collect_vec(); - VirtualNode::compute_chunk(chunk, &dist_key_indices) + VirtualNode::compute_chunk(chunk, &dist_key_indices, vnodes.len()) .into_iter() .zip_eq_fast(chunk.visibility().iter()) .map(|(vnode, vis)| { // Ignore the invisible rows. if vis { - check_vnode_is_set(vnode, &self.vnodes); + check_vnode_is_set(vnode, vnodes); } vnode }) .collect() } ComputeVnode::VnodeColumnIndex { + vnodes, vnode_col_idx_in_pk, } => { let array: &PrimitiveArray = @@ -262,7 +245,7 @@ impl TableDistribution { let vnode = VirtualNode::from_scalar(vnode); if vis { assert!(exist); - check_vnode_is_set(vnode, &self.vnodes); + check_vnode_is_set(vnode, vnodes); } vnode }) diff --git a/src/common/src/session_config/mod.rs b/src/common/src/session_config/mod.rs index ffdbe6753acb5..163aa18799390 100644 --- a/src/common/src/session_config/mod.rs +++ b/src/common/src/session_config/mod.rs @@ -292,6 +292,12 @@ pub struct SessionConfig { #[parameter(default = "hex", check_hook = check_bytea_output)] bytea_output: String, + + /// Bypass checks on cluster limits + /// + /// When enabled, `CREATE MATERIALIZED VIEW` will not fail if the cluster limit is hit. + #[parameter(default = false)] + bypass_cluster_limits: bool, } fn check_timezone(val: &str) -> Result<(), String> { diff --git a/src/common/src/util/cluster_limit.rs b/src/common/src/util/cluster_limit.rs new file mode 100644 index 0000000000000..048ea4fdab305 --- /dev/null +++ b/src/common/src/util/cluster_limit.rs @@ -0,0 +1,134 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::fmt::{self, Display, Formatter}; + +use risingwave_pb::meta::actor_count_per_parallelism::PbWorkerActorCount; +use risingwave_pb::meta::cluster_limit::PbLimit; +use risingwave_pb::meta::{PbActorCountPerParallelism, PbClusterLimit}; +pub enum ClusterLimit { + ActorCount(ActorCountPerParallelism), +} + +impl From for PbClusterLimit { + fn from(limit: ClusterLimit) -> Self { + match limit { + ClusterLimit::ActorCount(actor_count_per_parallelism) => PbClusterLimit { + limit: Some(PbLimit::ActorCount(actor_count_per_parallelism.into())), + }, + } + } +} + +impl From for ClusterLimit { + fn from(pb_limit: PbClusterLimit) -> Self { + match pb_limit.limit.unwrap() { + PbLimit::ActorCount(actor_count_per_parallelism) => { + ClusterLimit::ActorCount(actor_count_per_parallelism.into()) + } + } + } +} + +#[derive(Debug)] +pub struct WorkerActorCount { + pub actor_count: usize, + pub parallelism: usize, +} + +impl From for PbWorkerActorCount { + fn from(worker_actor_count: WorkerActorCount) -> Self { + PbWorkerActorCount { + actor_count: worker_actor_count.actor_count as u64, + parallelism: worker_actor_count.parallelism as u64, + } + } +} + +impl From for WorkerActorCount { + fn from(pb_worker_actor_count: PbWorkerActorCount) -> Self { + WorkerActorCount { + actor_count: pb_worker_actor_count.actor_count as usize, + parallelism: pb_worker_actor_count.parallelism as usize, + } + } +} + +pub struct ActorCountPerParallelism { + pub worker_id_to_actor_count: HashMap, + pub hard_limit: usize, + pub soft_limit: usize, +} + +impl From for PbActorCountPerParallelism { + fn from(actor_count_per_parallelism: ActorCountPerParallelism) -> Self { + PbActorCountPerParallelism { + worker_id_to_actor_count: actor_count_per_parallelism + .worker_id_to_actor_count + .into_iter() + .map(|(k, v)| (k, v.into())) + .collect(), + hard_limit: actor_count_per_parallelism.hard_limit as u64, + soft_limit: actor_count_per_parallelism.soft_limit as u64, + } + } +} + +impl From for ActorCountPerParallelism { + fn from(pb_actor_count_per_parallelism: PbActorCountPerParallelism) -> Self { + ActorCountPerParallelism { + worker_id_to_actor_count: pb_actor_count_per_parallelism + .worker_id_to_actor_count + .into_iter() + .map(|(k, v)| (k, v.into())) + .collect(), + hard_limit: pb_actor_count_per_parallelism.hard_limit as usize, + soft_limit: pb_actor_count_per_parallelism.soft_limit as usize, + } + } +} + +impl ActorCountPerParallelism { + pub fn exceed_hard_limit(&self) -> bool { + self.worker_id_to_actor_count + .values() + .any(|v| v.actor_count > self.hard_limit.saturating_mul(v.parallelism)) + } + + pub fn exceed_soft_limit(&self) -> bool { + self.worker_id_to_actor_count + .values() + .any(|v| v.actor_count > self.soft_limit.saturating_mul(v.parallelism)) + } + + pub fn exceed_limit(&self) -> bool { + self.exceed_soft_limit() || self.exceed_hard_limit() + } +} + +impl Display for ActorCountPerParallelism { + fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result { + let worker_id_to_actor_count_str: Vec<_> = self + .worker_id_to_actor_count + .iter() + .map(|(k, v)| format!("{} -> {:?}", k, v)) + .collect(); + write!( + f, + "ActorCountPerParallelism {{ critical limit: {:?}, recommended limit: {:?}. worker_id_to_actor_count: {:?} }}", + self.hard_limit, self.soft_limit, worker_id_to_actor_count_str + ) + } +} diff --git a/src/common/src/util/mod.rs b/src/common/src/util/mod.rs index 20dac5906c91d..bfa15c8327037 100644 --- a/src/common/src/util/mod.rs +++ b/src/common/src/util/mod.rs @@ -42,3 +42,4 @@ pub mod tracing; pub mod value_encoding; pub mod worker_util; pub use tokio_util; +pub mod cluster_limit; diff --git a/src/common/src/util/row_id.rs b/src/common/src/util/row_id.rs index 508f418903413..7f22c17e925e4 100644 --- a/src/common/src/util/row_id.rs +++ b/src/common/src/util/row_id.rs @@ -52,6 +52,7 @@ pub struct RowIdGenerator { pub type RowId = i64; +// TODO(var-vnode): how should we handle this for different virtual node counts? #[inline] pub fn extract_vnode_id_from_row_id(id: RowId) -> VirtualNode { let vnode_id = ((id >> VNODE_ID_SHIFT_BITS) & (VNODE_ID_UPPER_BOUND as i64 - 1)) as u32; diff --git a/src/common/src/util/scan_range.rs b/src/common/src/util/scan_range.rs index fd056f1790444..cfe209cf2c22a 100644 --- a/src/common/src/util/scan_range.rs +++ b/src/common/src/util/scan_range.rs @@ -159,7 +159,7 @@ mod tests { let pk = vec![1, 3, 2]; let dist_key_idx_in_pk = crate::catalog::get_dist_key_in_pk_indices(&dist_key, &pk).unwrap(); - let dist = TableDistribution::all(dist_key_idx_in_pk); + let dist = TableDistribution::all(dist_key_idx_in_pk, VirtualNode::COUNT_FOR_TEST); let mut scan_range = ScanRange::full_table_scan(); assert!(scan_range.try_compute_vnode(&dist).is_none()); @@ -173,7 +173,7 @@ mod tests { Some(ScalarImpl::from(514)), ]); - let vnode = VirtualNode::compute_row(&row, &[0, 1]); + let vnode = VirtualNode::compute_row_for_test(&row, &[0, 1]); assert_eq!(scan_range.try_compute_vnode(&dist), Some(vnode)); } @@ -185,7 +185,7 @@ mod tests { let pk = vec![1, 3, 2]; let dist_key_idx_in_pk = crate::catalog::get_dist_key_in_pk_indices(&dist_key, &pk).unwrap(); - let dist = TableDistribution::all(dist_key_idx_in_pk); + let dist = TableDistribution::all(dist_key_idx_in_pk, VirtualNode::COUNT_FOR_TEST); let mut scan_range = ScanRange::full_table_scan(); assert!(scan_range.try_compute_vnode(&dist).is_none()); @@ -203,7 +203,7 @@ mod tests { Some(ScalarImpl::from(114514)), ]); - let vnode = VirtualNode::compute_row(&row, &[2, 1]); + let vnode = VirtualNode::compute_row_for_test(&row, &[2, 1]); assert_eq!(scan_range.try_compute_vnode(&dist), Some(vnode)); } diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 5619ffc6e0f96..ccaf67db6a3c8 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -123,7 +123,7 @@ pub fn place_vnode( } None => { // No hint is provided, assign all vnodes to `temp_pu`. - for vnode in VirtualNode::all() { + for vnode in VirtualNode::all(VirtualNode::COUNT) { temp_slot.balance += 1; temp_slot.builder.set(vnode.to_index(), true); } diff --git a/src/compute/src/rpc/service/stream_service.rs b/src/compute/src/rpc/service/stream_service.rs index eb055a174b3ea..6253cfe74c730 100644 --- a/src/compute/src/rpc/service/stream_service.rs +++ b/src/compute/src/rpc/service/stream_service.rs @@ -40,20 +40,6 @@ impl StreamService for StreamServiceImpl { type StreamingControlStreamStream = impl Stream>; - #[cfg_attr(coverage, coverage(off))] - async fn drop_actors( - &self, - request: Request, - ) -> std::result::Result, Status> { - let req = request.into_inner(); - let actors = req.actor_ids; - self.mgr.drop_actors(actors).await?; - Ok(Response::new(DropActorsResponse { - request_id: req.request_id, - status: None, - })) - } - #[cfg_attr(coverage, coverage(off))] async fn wait_epoch_commit( &self, diff --git a/src/config/example.toml b/src/config/example.toml index c81b35163eafa..f3c127cdc7825 100644 --- a/src/config/example.toml +++ b/src/config/example.toml @@ -81,6 +81,8 @@ meta_enable_trivial_move = true meta_enable_check_task_level_overlap = false meta_max_trivial_move_task_count_per_loop = 256 meta_max_get_task_probe_times = 5 +meta_actor_cnt_per_worker_parallelism_soft_limit = 100 +meta_actor_cnt_per_worker_parallelism_hard_limit = 400 [batch] enable_barrier_read = false diff --git a/src/connector/Cargo.toml b/src/connector/Cargo.toml index d87e89c1cf65d..a77e9cb929d17 100644 --- a/src/connector/Cargo.toml +++ b/src/connector/Cargo.toml @@ -103,7 +103,7 @@ pg_bigdecimal = { git = "https://github.com/risingwavelabs/rust-pg_bigdecimal", postgres-openssl = "0.5.0" prometheus = { version = "0.13", features = ["process"] } prost = { workspace = true, features = ["no-recursion-limit"] } -prost-reflect = "0.14" +prost-reflect = { version = "0.14", features = ["serde"] } prost-types = "0.13" protobuf-native = "0.2.2" pulsar = { version = "6.3", default-features = false, features = [ diff --git a/src/connector/codec/src/decoder/mod.rs b/src/connector/codec/src/decoder/mod.rs index 814e06a166c6c..bbfdbf0a90d79 100644 --- a/src/connector/codec/src/decoder/mod.rs +++ b/src/connector/codec/src/decoder/mod.rs @@ -38,6 +38,9 @@ pub enum AccessError { #[error("Unsupported additional column `{name}`")] UnsupportedAdditionalColumn { name: String }, + #[error("Fail to convert protobuf Any into jsonb: {0}")] + ProtobufAnyToJson(#[source] serde_json::Error), + /// Errors that are not categorized into variants above. #[error("{message}")] Uncategorized { message: String }, diff --git a/src/connector/src/connector_common/common.rs b/src/connector/src/connector_common/common.rs index b522ae2eda560..9f4211aedd4d9 100644 --- a/src/connector/src/connector_common/common.rs +++ b/src/connector/src/connector_common/common.rs @@ -192,14 +192,26 @@ pub struct KafkaCommon { #[serde(rename = "properties.ssl.ca.location")] ssl_ca_location: Option, + /// CA certificate string (PEM format) for verifying the broker's key. + #[serde(rename = "properties.ssl.ca.pem")] + ssl_ca_pem: Option, + /// Path to client's certificate file (PEM). #[serde(rename = "properties.ssl.certificate.location")] ssl_certificate_location: Option, + /// Client's public key string (PEM format) used for authentication. + #[serde(rename = "properties.ssl.certificate.pem")] + ssl_certificate_pem: Option, + /// Path to client's private key file (PEM). #[serde(rename = "properties.ssl.key.location")] ssl_key_location: Option, + /// Client's private key string (PEM format) used for authentication. + #[serde(rename = "properties.ssl.key.pem")] + ssl_key_pem: Option, + /// Passphrase of client's private key. #[serde(rename = "properties.ssl.key.password")] ssl_key_password: Option, @@ -325,12 +337,21 @@ impl KafkaCommon { if let Some(ssl_ca_location) = self.ssl_ca_location.as_ref() { config.set("ssl.ca.location", ssl_ca_location); } + if let Some(ssl_ca_pem) = self.ssl_ca_pem.as_ref() { + config.set("ssl.ca.pem", ssl_ca_pem); + } if let Some(ssl_certificate_location) = self.ssl_certificate_location.as_ref() { config.set("ssl.certificate.location", ssl_certificate_location); } + if let Some(ssl_certificate_pem) = self.ssl_certificate_pem.as_ref() { + config.set("ssl.certificate.pem", ssl_certificate_pem); + } if let Some(ssl_key_location) = self.ssl_key_location.as_ref() { config.set("ssl.key.location", ssl_key_location); } + if let Some(ssl_key_pem) = self.ssl_key_pem.as_ref() { + config.set("ssl.key.pem", ssl_key_pem); + } if let Some(ssl_key_password) = self.ssl_key_password.as_ref() { config.set("ssl.key.password", ssl_key_password); } diff --git a/src/connector/src/parser/protobuf/parser.rs b/src/connector/src/parser/protobuf/parser.rs index 8be25074f6295..ec8c747cafd5a 100644 --- a/src/connector/src/parser/protobuf/parser.rs +++ b/src/connector/src/parser/protobuf/parser.rs @@ -12,8 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::sync::Arc; - use anyhow::Context; use itertools::Itertools; use prost_reflect::{ @@ -22,8 +20,7 @@ use prost_reflect::{ }; use risingwave_common::array::{ListValue, StructValue}; use risingwave_common::types::{ - DataType, Datum, DatumCow, Decimal, JsonbRef, JsonbVal, ScalarImpl, ScalarRefImpl, ToDatumRef, - ToOwnedDatum, F32, F64, + DataType, DatumCow, Decimal, JsonbVal, ScalarImpl, ToOwnedDatum, F32, F64, }; use risingwave_common::{bail, try_match_expand}; use risingwave_pb::plan_common::{AdditionalColumn, ColumnDesc, ColumnDescVersion}; @@ -32,9 +29,7 @@ use thiserror_ext::{AsReport, Macro}; use crate::error::ConnectorResult; use crate::parser::unified::protobuf::ProtobufAccess; -use crate::parser::unified::{ - bail_uncategorized, uncategorized, AccessError, AccessImpl, AccessResult, -}; +use crate::parser::unified::{uncategorized, AccessError, AccessImpl, AccessResult}; use crate::parser::util::bytes_from_url; use crate::parser::{AccessBuilder, EncodingProperties}; use crate::schema::schema_registry::{extract_schema_id, handle_sr_list, Client, WireFormatError}; @@ -44,7 +39,6 @@ use crate::schema::SchemaLoader; pub struct ProtobufAccessBuilder { confluent_wire_type: bool, message_descriptor: MessageDescriptor, - descriptor_pool: Arc, } impl AccessBuilder for ProtobufAccessBuilder { @@ -59,10 +53,7 @@ impl AccessBuilder for ProtobufAccessBuilder { let message = DynamicMessage::decode(self.message_descriptor.clone(), payload) .context("failed to parse message")?; - Ok(AccessImpl::Protobuf(ProtobufAccess::new( - message, - Arc::clone(&self.descriptor_pool), - ))) + Ok(AccessImpl::Protobuf(ProtobufAccess::new(message))) } } @@ -71,13 +62,11 @@ impl ProtobufAccessBuilder { let ProtobufParserConfig { confluent_wire_type, message_descriptor, - descriptor_pool, } = config; Ok(Self { confluent_wire_type, message_descriptor, - descriptor_pool, }) } } @@ -86,8 +75,6 @@ impl ProtobufAccessBuilder { pub struct ProtobufParserConfig { confluent_wire_type: bool, pub(crate) message_descriptor: MessageDescriptor, - /// Note that the pub(crate) here is merely for testing - pub(crate) descriptor_pool: Arc, } impl ProtobufParserConfig { @@ -132,7 +119,6 @@ impl ProtobufParserConfig { Ok(Self { message_descriptor, confluent_wire_type: protobuf_config.use_schema_registry, - descriptor_pool: Arc::new(pool), }) } @@ -216,141 +202,9 @@ fn detect_loop_and_push( Ok(()) } -fn extract_any_info(dyn_msg: &DynamicMessage) -> (String, Value) { - debug_assert!( - dyn_msg.fields().count() == 2, - "Expected only two fields for Any Type MessageDescriptor" - ); - - let type_url = dyn_msg - .get_field_by_name("type_url") - .expect("Expect type_url in dyn_msg") - .to_string() - .split('/') - .nth(1) - .map(|part| part[..part.len() - 1].to_string()) - .unwrap_or_default(); - - let payload = dyn_msg - .get_field_by_name("value") - .expect("Expect value (payload) in dyn_msg") - .as_ref() - .clone(); - - (type_url, payload) -} - -/// TODO: Resolve the potential naming conflict in the map -/// i.e., If the two anonymous type shares the same key (e.g., "Int32"), -/// the latter will overwrite the former one in `serde_json::Map`. -/// Possible solution, maintaining a global id map, for the same types -/// In the same level of fields, add the unique id at the tail of the name. -/// e.g., "Int32.1" & "Int32.2" in the above example -fn recursive_parse_json( - fields: &[Datum], - full_name_vec: Option>, - full_name: Option, -) -> serde_json::Value { - // Note that the key is of no order - let mut ret: serde_json::Map = serde_json::Map::new(); - - // The hidden type hint for user's convenience - // i.e., `"_type": message.full_name()` - if let Some(full_name) = full_name { - ret.insert("_type".to_string(), serde_json::Value::String(full_name)); - } - - for (idx, field) in fields.iter().enumerate() { - let mut key; - if let Some(k) = full_name_vec.as_ref() { - key = k[idx].to_string(); - } else { - key = "".to_string(); - } - - match field.clone() { - Some(ScalarImpl::Int16(v)) => { - if key.is_empty() { - key = "Int16".to_string(); - } - ret.insert(key, serde_json::Value::Number(serde_json::Number::from(v))); - } - Some(ScalarImpl::Int32(v)) => { - if key.is_empty() { - key = "Int32".to_string(); - } - ret.insert(key, serde_json::Value::Number(serde_json::Number::from(v))); - } - Some(ScalarImpl::Int64(v)) => { - if key.is_empty() { - key = "Int64".to_string(); - } - ret.insert(key, serde_json::Value::Number(serde_json::Number::from(v))); - } - Some(ScalarImpl::Bool(v)) => { - if key.is_empty() { - key = "Bool".to_string(); - } - ret.insert(key, serde_json::Value::Bool(v)); - } - Some(ScalarImpl::Bytea(v)) => { - if key.is_empty() { - key = "Bytea".to_string(); - } - let s = String::from_utf8(v.to_vec()).unwrap(); - ret.insert(key, serde_json::Value::String(s)); - } - Some(ScalarImpl::Float32(v)) => { - if key.is_empty() { - key = "Int16".to_string(); - } - ret.insert( - key, - serde_json::Value::Number( - serde_json::Number::from_f64(v.into_inner() as f64).unwrap(), - ), - ); - } - Some(ScalarImpl::Float64(v)) => { - if key.is_empty() { - key = "Float64".to_string(); - } - ret.insert( - key, - serde_json::Value::Number( - serde_json::Number::from_f64(v.into_inner()).unwrap(), - ), - ); - } - Some(ScalarImpl::Utf8(v)) => { - if key.is_empty() { - key = "Utf8".to_string(); - } - ret.insert(key, serde_json::Value::String(v.to_string())); - } - Some(ScalarImpl::Struct(v)) => { - if key.is_empty() { - key = "Struct".to_string(); - } - ret.insert(key, recursive_parse_json(v.fields(), None, None)); - } - Some(ScalarImpl::Jsonb(v)) => { - if key.is_empty() { - key = "Jsonb".to_string(); - } - ret.insert(key, v.take()); - } - r#type => panic!("Not yet support ScalarImpl type: {:?}", r#type), - } - } - - serde_json::Value::Object(ret) -} - pub fn from_protobuf_value<'a>( field_desc: &FieldDescriptor, value: &'a Value, - descriptor_pool: &Arc, ) -> AccessResult> { let kind = field_desc.kind(); @@ -382,62 +236,9 @@ pub fn from_protobuf_value<'a>( } Value::Message(dyn_msg) => { if dyn_msg.descriptor().full_name() == "google.protobuf.Any" { - // If the fields are not presented, default value is an empty string - if !dyn_msg.has_field_by_name("type_url") || !dyn_msg.has_field_by_name("value") { - borrowed!(JsonbRef::empty_string()); - } - - // Sanity check - debug_assert!( - dyn_msg.has_field_by_name("type_url") && dyn_msg.has_field_by_name("value"), - "`type_url` & `value` must exist in fields of `dyn_msg`" - ); - - // The message is of type `Any` - let (type_url, payload) = extract_any_info(dyn_msg); - - let payload_field_desc = dyn_msg.descriptor().get_field_by_name("value").unwrap(); - - let payload = from_protobuf_value(&payload_field_desc, &payload, descriptor_pool)?; - let Some(ScalarRefImpl::Bytea(payload)) = payload.to_datum_ref() else { - bail_uncategorized!("expected bytes for dynamic message payload"); - }; - - // Get the corresponding schema from the descriptor pool - let msg_desc = descriptor_pool - .get_message_by_name(&type_url) - .ok_or_else(|| { - uncategorized!("message `{type_url}` not found in descriptor pool") - })?; - - let f = msg_desc - .clone() - .fields() - .map(|f| f.name().to_string()) - .collect::>(); - - let full_name = msg_desc.clone().full_name().to_string(); - - // Decode the payload based on the `msg_desc` - let decoded_value = DynamicMessage::decode(msg_desc, payload).unwrap(); - let decoded_value = from_protobuf_value( - field_desc, - &Value::Message(decoded_value), - descriptor_pool, - )? - .to_owned_datum() - .unwrap(); - - // Extract the struct value - let ScalarImpl::Struct(v) = decoded_value else { - panic!("Expect ScalarImpl::Struct"); - }; - - ScalarImpl::Jsonb(JsonbVal::from(serde_json::json!(recursive_parse_json( - v.fields(), - Some(f), - Some(full_name), - )))) + ScalarImpl::Jsonb(JsonbVal::from( + serde_json::to_value(dyn_msg).map_err(AccessError::ProtobufAnyToJson)?, + )) } else { let mut rw_values = Vec::with_capacity(dyn_msg.descriptor().fields().len()); // fields is a btree map in descriptor @@ -454,9 +255,7 @@ pub fn from_protobuf_value<'a>( } // use default value if dyn_msg doesn't has this field let value = dyn_msg.get_field(&field_desc); - rw_values.push( - from_protobuf_value(&field_desc, &value, descriptor_pool)?.to_owned_datum(), - ); + rw_values.push(from_protobuf_value(&field_desc, &value)?.to_owned_datum()); } ScalarImpl::Struct(StructValue::new(rw_values)) } @@ -466,7 +265,7 @@ pub fn from_protobuf_value<'a>( .map_err(|e| uncategorized!("{}", e.to_report_string()))?; let mut builder = data_type.as_list().create_array_builder(values.len()); for value in values { - builder.append(from_protobuf_value(field_desc, value, descriptor_pool)?); + builder.append(from_protobuf_value(field_desc, value)?); } ScalarImpl::List(ListValue::new(builder.finish())) } @@ -498,25 +297,18 @@ fn protobuf_type_mapping( } Kind::Uint64 | Kind::Fixed64 => DataType::Decimal, Kind::String => DataType::Varchar, - Kind::Message(m) => { - let fields = m - .fields() - .map(|f| protobuf_type_mapping(&f, parse_trace)) - .try_collect()?; - let field_names = m.fields().map(|f| f.name().to_string()).collect_vec(); - - // Note that this part is useful for actual parsing - // Since RisingWave will parse message to `ScalarImpl::Jsonb` - // Please do NOT modify it - if field_names.len() == 2 - && field_names.contains(&"value".to_string()) - && field_names.contains(&"type_url".to_string()) - { - DataType::Jsonb - } else { + Kind::Message(m) => match m.full_name() { + // Well-Known Types are identified by their full name + "google.protobuf.Any" => DataType::Jsonb, + _ => { + let fields = m + .fields() + .map(|f| protobuf_type_mapping(&f, parse_trace)) + .try_collect()?; + let field_names = m.fields().map(|f| f.name().to_string()).collect_vec(); DataType::new_struct(fields, field_names) } - } + }, Kind::Enum(_) => DataType::Varchar, Kind::Bytes => DataType::Bytea, }; @@ -973,10 +765,9 @@ mod test { // This is of no use let field = value.fields().next().unwrap().0; - if let Some(ret) = - from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool) - .unwrap() - .to_owned_datum() + if let Some(ret) = from_protobuf_value(&field, &Value::Message(value)) + .unwrap() + .to_owned_datum() { println!("Decoded Value for ANY_GEN_PROTO_DATA: {:#?}", ret); println!("---------------------------"); @@ -1000,7 +791,7 @@ mod test { assert_eq!( jv, JsonbVal::from(json!({ - "_type": "test.StringValue", + "@type": "type.googleapis.com/test.StringValue", "value": "John Doe" })) ); @@ -1036,10 +827,9 @@ mod test { // This is of no use let field = value.fields().next().unwrap().0; - if let Some(ret) = - from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool) - .unwrap() - .to_owned_datum() + if let Some(ret) = from_protobuf_value(&field, &Value::Message(value)) + .unwrap() + .to_owned_datum() { println!("Decoded Value for ANY_GEN_PROTO_DATA: {:#?}", ret); println!("---------------------------"); @@ -1063,7 +853,7 @@ mod test { assert_eq!( jv, JsonbVal::from(json!({ - "_type": "test.Int32Value", + "@type": "type.googleapis.com/test.Int32Value", "value": 114514 })) ); @@ -1110,10 +900,9 @@ mod test { // This is of no use let field = value.fields().next().unwrap().0; - if let Some(ret) = - from_protobuf_value(&field, &Value::Message(value), &conf.descriptor_pool) - .unwrap() - .to_owned_datum() + if let Some(ret) = from_protobuf_value(&field, &Value::Message(value)) + .unwrap() + .to_owned_datum() { println!("Decoded Value for ANY_RECURSIVE_GEN_PROTO_DATA: {:#?}", ret); println!("---------------------------"); @@ -1137,13 +926,13 @@ mod test { assert_eq!( jv, JsonbVal::from(json!({ - "_type": "test.AnyValue", - "any_value_1": { - "_type": "test.StringValue", + "@type": "type.googleapis.com/test.AnyValue", + "anyValue1": { + "@type": "type.googleapis.com/test.StringValue", "value": "114514", }, - "any_value_2": { - "_type": "test.Int32Value", + "anyValue2": { + "@type": "type.googleapis.com/test.Int32Value", "value": 114514, } })) @@ -1156,6 +945,37 @@ mod test { Ok(()) } + // id: 12345 + // any_value: { + // type_url: "type.googleapis.com/test.StringXalue" + // value: "\n\010John Doe" + // } + static ANY_GEN_PROTO_DATA_INVALID: &[u8] = b"\x08\xb9\x60\x12\x32\x0a\x24\x74\x79\x70\x65\x2e\x67\x6f\x6f\x67\x6c\x65\x61\x70\x69\x73\x2e\x63\x6f\x6d\x2f\x74\x65\x73\x74\x2e\x53\x74\x72\x69\x6e\x67\x58\x61\x6c\x75\x65\x12\x0a\x0a\x08\x4a\x6f\x68\x6e\x20\x44\x6f\x65"; + + #[tokio::test] + async fn test_any_invalid() -> crate::error::ConnectorResult<()> { + let conf = create_recursive_pb_parser_config("/any-schema.pb", "test.TestAny").await; + + let value = + DynamicMessage::decode(conf.message_descriptor.clone(), ANY_GEN_PROTO_DATA_INVALID) + .unwrap(); + + // The top-level `Value` is not a proto field, but we need a dummy one. + let field = value.fields().next().unwrap().0; + + let err = from_protobuf_value(&field, &Value::Message(value)).unwrap_err(); + + let expected = expect_test::expect![[r#" + Fail to convert protobuf Any into jsonb + + Caused by: + message 'test.StringXalue' not found + "#]]; + expected.assert_eq(err.to_report_string_pretty().as_str()); + + Ok(()) + } + #[test] fn test_decode_varint_zigzag() { // 1. Positive number diff --git a/src/connector/src/parser/unified/mod.rs b/src/connector/src/parser/unified/mod.rs index 8045ce0132401..fdfe3aae6aaee 100644 --- a/src/connector/src/parser/unified/mod.rs +++ b/src/connector/src/parser/unified/mod.rs @@ -17,9 +17,7 @@ use auto_impl::auto_impl; use risingwave_common::types::{DataType, DatumCow}; use risingwave_connector_codec::decoder::avro::AvroAccess; -pub use risingwave_connector_codec::decoder::{ - bail_uncategorized, uncategorized, Access, AccessError, AccessResult, -}; +pub use risingwave_connector_codec::decoder::{uncategorized, Access, AccessError, AccessResult}; use self::bytes::BytesAccess; use self::json::JsonAccess; diff --git a/src/connector/src/parser/unified/protobuf.rs b/src/connector/src/parser/unified/protobuf.rs index 02febc22db247..b1d34746b5029 100644 --- a/src/connector/src/parser/unified/protobuf.rs +++ b/src/connector/src/parser/unified/protobuf.rs @@ -13,9 +13,9 @@ // limitations under the License. use std::borrow::Cow; -use std::sync::{Arc, LazyLock}; +use std::sync::LazyLock; -use prost_reflect::{DescriptorPool, DynamicMessage, ReflectMessage}; +use prost_reflect::{DynamicMessage, ReflectMessage}; use risingwave_common::log::LogSuppresser; use risingwave_common::types::{DataType, DatumCow, ToOwnedDatum}; use thiserror_ext::AsReport; @@ -26,15 +26,11 @@ use crate::parser::unified::uncategorized; pub struct ProtobufAccess { message: DynamicMessage, - descriptor_pool: Arc, } impl ProtobufAccess { - pub fn new(message: DynamicMessage, descriptor_pool: Arc) -> Self { - Self { - message, - descriptor_pool, - } + pub fn new(message: DynamicMessage) -> Self { + Self { message } } } @@ -59,10 +55,10 @@ impl Access for ProtobufAccess { })?; match self.message.get_field(&field_desc) { - Cow::Borrowed(value) => from_protobuf_value(&field_desc, value, &self.descriptor_pool), + Cow::Borrowed(value) => from_protobuf_value(&field_desc, value), // `Owned` variant occurs only if there's no such field and the default value is returned. - Cow::Owned(value) => from_protobuf_value(&field_desc, &value, &self.descriptor_pool) + Cow::Owned(value) => from_protobuf_value(&field_desc, &value) // enforce `Owned` variant to avoid returning a reference to a temporary value .map(|d| d.to_owned_datum().into()), } diff --git a/src/connector/with_options_sink.yaml b/src/connector/with_options_sink.yaml index cc92f9a0a664a..e8a8efff68801 100644 --- a/src/connector/with_options_sink.yaml +++ b/src/connector/with_options_sink.yaml @@ -373,14 +373,26 @@ KafkaConfig: field_type: String comments: Path to CA certificate file for verifying the broker's key. required: false + - name: properties.ssl.ca.pem + field_type: String + comments: CA certificate string (PEM format) for verifying the broker's key. + required: false - name: properties.ssl.certificate.location field_type: String comments: Path to client's certificate file (PEM). required: false + - name: properties.ssl.certificate.pem + field_type: String + comments: Client's public key string (PEM format) used for authentication. + required: false - name: properties.ssl.key.location field_type: String comments: Path to client's private key file (PEM). required: false + - name: properties.ssl.key.pem + field_type: String + comments: Client's private key string (PEM format) used for authentication. + required: false - name: properties.ssl.key.password field_type: String comments: Passphrase of client's private key. diff --git a/src/connector/with_options_source.yaml b/src/connector/with_options_source.yaml index 4eaf1e0d3db4b..a6a19e80c89a3 100644 --- a/src/connector/with_options_source.yaml +++ b/src/connector/with_options_source.yaml @@ -199,14 +199,26 @@ KafkaProperties: field_type: String comments: Path to CA certificate file for verifying the broker's key. required: false + - name: properties.ssl.ca.pem + field_type: String + comments: CA certificate string (PEM format) for verifying the broker's key. + required: false - name: properties.ssl.certificate.location field_type: String comments: Path to client's certificate file (PEM). required: false + - name: properties.ssl.certificate.pem + field_type: String + comments: Client's public key string (PEM format) used for authentication. + required: false - name: properties.ssl.key.location field_type: String comments: Path to client's private key file (PEM). required: false + - name: properties.ssl.key.pem + field_type: String + comments: Client's private key string (PEM format) used for authentication. + required: false - name: properties.ssl.key.password field_type: String comments: Passphrase of client's private key. diff --git a/src/ctl/src/cmd_impl/meta/cluster_info.rs b/src/ctl/src/cmd_impl/meta/cluster_info.rs index cbc21ca6ec610..76b91d37fbd3c 100644 --- a/src/ctl/src/cmd_impl/meta/cluster_info.rs +++ b/src/ctl/src/cmd_impl/meta/cluster_info.rs @@ -31,7 +31,7 @@ pub async fn get_cluster_info(context: &CtlContext) -> anyhow::Result anyhow::Result<()> { +pub async fn source_split_info(context: &CtlContext, ignore_id: bool) -> anyhow::Result<()> { let GetClusterInfoResponse { worker_nodes: _, source_infos: _, @@ -40,37 +40,113 @@ pub async fn source_split_info(context: &CtlContext) -> anyhow::Result<()> { revision: _, } = get_cluster_info(context).await?; + let mut actor_splits_map: BTreeMap = BTreeMap::new(); + + // build actor_splits_map for table_fragment in &table_fragments { if table_fragment.actor_splits.is_empty() { continue; } - println!("Table #{}", table_fragment.table_id); - for fragment in table_fragment.fragments.values() { let fragment_type_mask = fragment.fragment_type_mask; if fragment_type_mask & FragmentTypeFlag::Source as u32 == 0 - || fragment_type_mask & FragmentTypeFlag::Dml as u32 != 0 + && fragment_type_mask & FragmentTypeFlag::SourceScan as u32 == 0 { + // no source or source backfill + continue; + } + if fragment_type_mask & FragmentTypeFlag::Dml as u32 != 0 { // skip dummy source for dml fragment continue; } - println!("\tFragment #{}", fragment.fragment_id); for actor in &fragment.actors { if let Some(ConnectorSplits { splits }) = actor_splits.remove(&actor.actor_id) { let splits = splits .iter() .map(|split| SplitImpl::try_from(split).unwrap()) .map(|split| split.id()) - .collect_vec(); + .collect_vec() + .join(","); + actor_splits_map.insert(actor.actor_id, (splits.len(), splits)); + } + } + } + } + // print in the second iteration. Otherwise we don't have upstream splits info + for table_fragment in &table_fragments { + if table_fragment.actor_splits.is_empty() { + continue; + } + if ignore_id { + println!("Table"); + } else { + println!("Table #{}", table_fragment.table_id); + } + for fragment in table_fragment.fragments.values() { + let fragment_type_mask = fragment.fragment_type_mask; + if fragment_type_mask & FragmentTypeFlag::Source as u32 == 0 + && fragment_type_mask & FragmentTypeFlag::SourceScan as u32 == 0 + { + // no source or source backfill + continue; + } + if fragment_type_mask & FragmentTypeFlag::Dml as u32 != 0 { + // skip dummy source for dml fragment + continue; + } + + println!( + "\tFragment{} ({})", + if ignore_id { + "".to_string() + } else { + format!(" #{}", fragment.fragment_id) + }, + if fragment_type_mask == FragmentTypeFlag::Source as u32 { + "Source" + } else { + "SourceScan" + } + ); + for actor in &fragment.actors { + if let Some((split_count, splits)) = actor_splits_map.get(&actor.actor_id) { println!( - "\t\tActor #{:<3} ({}): [{}]", - actor.actor_id, - splits.len(), - splits.join(",") + "\t\tActor{} ({} splits): [{}]{}", + if ignore_id { + "".to_string() + } else { + format!(" #{:<3}", actor.actor_id,) + }, + split_count, + splits, + if !actor.upstream_actor_id.is_empty() { + assert!( + actor.upstream_actor_id.len() == 1, + "should have only one upstream actor, got {actor:?}" + ); + let upstream_splits = + actor_splits_map.get(&actor.upstream_actor_id[0]).unwrap(); + format!( + " <- Upstream Actor{}: [{}]", + if ignore_id { + "".to_string() + } else { + format!(" #{}", actor.upstream_actor_id[0]) + }, + upstream_splits.1 + ) + } else { + "".to_string() + } ); + } else { + println!( + "\t\tError: Actor #{:<3} (not found in actor_splits)", + actor.actor_id, + ) } } } diff --git a/src/ctl/src/cmd_impl/table/scan.rs b/src/ctl/src/cmd_impl/table/scan.rs index e5bba170bf97a..f5cee710a40fc 100644 --- a/src/ctl/src/cmd_impl/table/scan.rs +++ b/src/ctl/src/cmd_impl/table/scan.rs @@ -14,6 +14,8 @@ use anyhow::{anyhow, Result}; use futures::{pin_mut, StreamExt}; +use risingwave_common::bitmap::Bitmap; +use risingwave_common::hash::VirtualNode; use risingwave_frontend::TableCatalog; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_rpc_client::MetaClient; @@ -63,7 +65,8 @@ pub async fn make_state_table(hummock: S, table: &TableCatalog) - .collect(), table.pk().iter().map(|x| x.order_type).collect(), table.pk().iter().map(|x| x.column_index).collect(), - TableDistribution::all(table.distribution_key().to_vec()), // scan all vnodes + // TODO(var-vnode): use vnode count from table desc + TableDistribution::all(table.distribution_key().to_vec(), VirtualNode::COUNT), // scan all vnodes Some(table.value_indices.clone()), ) .await @@ -81,7 +84,8 @@ pub fn make_storage_table( Ok(StorageTable::new_partial( hummock, output_columns_ids, - Some(TableDistribution::all_vnodes()), + // TODO(var-vnode): use vnode count from table desc + Some(Bitmap::ones(VirtualNode::COUNT).into()), &table.table_desc().try_to_protobuf()?, )) } diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index d1deba4f99140..34c5be6ace21b 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -404,7 +404,10 @@ enum MetaCommands { /// get cluster info ClusterInfo, /// get source split info - SourceSplitInfo, + SourceSplitInfo { + #[clap(long)] + ignore_id: bool, + }, /// Reschedule the actors in the stream graph /// /// The format is `fragment_id-[worker_id:count]+[worker_id:count]` @@ -808,8 +811,8 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { Commands::Meta(MetaCommands::Pause) => cmd_impl::meta::pause(context).await?, Commands::Meta(MetaCommands::Resume) => cmd_impl::meta::resume(context).await?, Commands::Meta(MetaCommands::ClusterInfo) => cmd_impl::meta::cluster_info(context).await?, - Commands::Meta(MetaCommands::SourceSplitInfo) => { - cmd_impl::meta::source_split_info(context).await? + Commands::Meta(MetaCommands::SourceSplitInfo { ignore_id }) => { + cmd_impl::meta::source_split_info(context, ignore_id).await? } Commands::Meta(MetaCommands::Reschedule { from, diff --git a/src/expr/impl/src/scalar/vnode.rs b/src/expr/impl/src/scalar/vnode.rs index e544c39f62499..edd4caa39970e 100644 --- a/src/expr/impl/src/scalar/vnode.rs +++ b/src/expr/impl/src/scalar/vnode.rs @@ -43,7 +43,8 @@ impl Expression for VnodeExpression { } async fn eval(&self, input: &DataChunk) -> Result { - let vnodes = VirtualNode::compute_chunk(input, &self.dist_key_indices); + // TODO(var-vnode): get vnode count from context + let vnodes = VirtualNode::compute_chunk(input, &self.dist_key_indices, VirtualNode::COUNT); let mut builder = I16ArrayBuilder::new(input.capacity()); vnodes .into_iter() @@ -52,8 +53,9 @@ impl Expression for VnodeExpression { } async fn eval_row(&self, input: &OwnedRow) -> Result { + // TODO(var-vnode): get vnode count from context Ok(Some( - VirtualNode::compute_row(input, &self.dist_key_indices) + VirtualNode::compute_row(input, &self.dist_key_indices, VirtualNode::COUNT) .to_scalar() .into(), )) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs index 879e375e2b762..5e3261c06d186 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/mod.rs @@ -59,3 +59,4 @@ mod rw_worker_nodes; mod rw_actor_id_to_ddl; mod rw_fragment_id_to_ddl; +mod rw_worker_actor_count; diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_actor_count.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_actor_count.rs new file mode 100644 index 0000000000000..a336f69b2029f --- /dev/null +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_actor_count.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 risingwave_common::types::Fields; +use risingwave_frontend_macro::system_catalog; + +#[system_catalog( + view, + "rw_catalog.rw_worker_actor_count", + "SELECT t2.id as worker_id, parallelism, count(*) as actor_count + FROM rw_actors t1, rw_worker_nodes t2 + where t1.worker_id = t2.id + GROUP BY t2.id, t2.parallelism;" +)] +#[derive(Fields)] +struct RwWorkerActorCount { + worker_id: i32, + parallelism: i32, + actor_count: i64, +} diff --git a/src/frontend/src/handler/create_mv.rs b/src/frontend/src/handler/create_mv.rs index 4399d80811c19..9d48f2a429cca 100644 --- a/src/frontend/src/handler/create_mv.rs +++ b/src/frontend/src/handler/create_mv.rs @@ -205,6 +205,9 @@ pub async fn handle_create_mv_bound( ) -> Result { let session = handler_args.session.clone(); + // Check cluster limits + session.check_cluster_limits().await?; + if let Either::Right(resp) = session.check_relation_name_duplicated( name.clone(), StatementType::CREATE_MATERIALIZED_VIEW, diff --git a/src/frontend/src/handler/create_sink.rs b/src/frontend/src/handler/create_sink.rs index d0bd1d0cc8f2f..bb8d528ab1205 100644 --- a/src/frontend/src/handler/create_sink.rs +++ b/src/frontend/src/handler/create_sink.rs @@ -419,6 +419,8 @@ pub async fn handle_create_sink( ) -> Result { let session = handle_args.session.clone(); + session.check_cluster_limits().await?; + if let Either::Right(resp) = session.check_relation_name_duplicated( stmt.sink_name.clone(), StatementType::CREATE_SINK, diff --git a/src/frontend/src/handler/create_table.rs b/src/frontend/src/handler/create_table.rs index a10453a43ea4e..386d50e791886 100644 --- a/src/frontend/src/handler/create_table.rs +++ b/src/frontend/src/handler/create_table.rs @@ -1235,6 +1235,8 @@ pub async fn handle_create_table( session.notice_to_user("APPEND ONLY TABLE is currently an experimental feature."); } + session.check_cluster_limits().await?; + if let Either::Right(resp) = session.check_relation_name_duplicated( table_name.clone(), StatementType::CREATE_TABLE, diff --git a/src/frontend/src/meta_client.rs b/src/frontend/src/meta_client.rs index 60fa992bdbe2d..020e3380b29b7 100644 --- a/src/frontend/src/meta_client.rs +++ b/src/frontend/src/meta_client.rs @@ -17,6 +17,7 @@ use std::collections::HashMap; use anyhow::Context; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; +use risingwave_common::util::cluster_limit::ClusterLimit; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_hummock_sdk::HummockVersionId; use risingwave_pb::backup_service::MetaSnapshotMetadata; @@ -136,6 +137,8 @@ pub trait FrontendMetaClient: Send + Sync { ) -> Result>; async fn get_cluster_recovery_status(&self) -> Result; + + async fn get_cluster_limits(&self) -> Result>; } pub struct FrontendMetaClientImpl(pub MetaClient); @@ -345,4 +348,8 @@ impl FrontendMetaClient for FrontendMetaClientImpl { async fn get_cluster_recovery_status(&self) -> Result { self.0.get_cluster_recovery_status().await } + + async fn get_cluster_limits(&self) -> Result> { + self.0.get_cluster_limits().await + } } diff --git a/src/frontend/src/scheduler/plan_fragmenter.rs b/src/frontend/src/scheduler/plan_fragmenter.rs index 09e4cbc0bfa03..2ecae1d7f7642 100644 --- a/src/frontend/src/scheduler/plan_fragmenter.rs +++ b/src/frontend/src/scheduler/plan_fragmenter.rs @@ -1250,7 +1250,8 @@ fn derive_partitions( } let table_distribution = TableDistribution::new_from_storage_table_desc( - Some(TableDistribution::all_vnodes()), + // TODO(var-vnode): use vnode count from table desc + Some(Bitmap::ones(VirtualNode::COUNT).into()), &table_desc.try_to_protobuf()?, ); diff --git a/src/frontend/src/session.rs b/src/frontend/src/session.rs index 16f0c7226be21..a1150798951cb 100644 --- a/src/frontend/src/session.rs +++ b/src/frontend/src/session.rs @@ -59,9 +59,10 @@ use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::telemetry_env_enabled; use risingwave_common::types::DataType; use risingwave_common::util::addr::HostAddr; +use risingwave_common::util::cluster_limit::ActorCountPerParallelism; use risingwave_common::util::iter_util::ZipEqFast; -use risingwave_common::util::resource_util; use risingwave_common::util::runtime::BackgroundShutdownRuntime; +use risingwave_common::util::{cluster_limit, resource_util}; use risingwave_common::{GIT_SHA, RW_VERSION}; use risingwave_common_heap_profiling::HeapProfiler; use risingwave_common_service::{MetricsManager, ObserverManager}; @@ -1194,6 +1195,47 @@ impl SessionImpl { pub fn temporary_source_manager(&self) -> TemporarySourceManager { self.temporary_source_manager.lock().clone() } + + pub async fn check_cluster_limits(&self) -> Result<()> { + if self.config().bypass_cluster_limits() { + return Ok(()); + } + + let gen_message = |violated_limit: &ActorCountPerParallelism, + exceed_hard_limit: bool| + -> String { + let (limit_type, action) = if exceed_hard_limit { + ("critical", "Please scale the cluster before proceeding!") + } else { + ("recommended", "Scaling the cluster is recommended.") + }; + format!( + "\n- {}\n- {}\n- {}\n- {}\n- {}\n{}", + format_args!("Actor count per parallelism exceeds the {} limit.", limit_type), + format_args!("Depending on your workload, this may overload the cluster and cause performance/stability issues. {}", action), + "Contact us via slack or https://risingwave.com/contact-us/ for further enquiry.", + "You can bypass this check via SQL `SET bypass_cluster_limits TO true`.", + "You can check actor count distribution via SQL `SELECT * FROM rw_worker_actor_count`.", + violated_limit, + ) + }; + + let limits = self.env().meta_client().get_cluster_limits().await?; + for limit in limits { + match limit { + cluster_limit::ClusterLimit::ActorCount(l) => { + if l.exceed_hard_limit() { + return Err(RwError::from(ErrorCode::ProtocolError(gen_message( + &l, true, + )))); + } else if l.exceed_soft_limit() { + self.notice_to_user(gen_message(&l, false)); + } + } + } + } + Ok(()) + } } pub static SESSION_MANAGER: std::sync::OnceLock> = diff --git a/src/frontend/src/test_utils.rs b/src/frontend/src/test_utils.rs index ee6ff589e0cdb..10dad2105ed94 100644 --- a/src/frontend/src/test_utils.rs +++ b/src/frontend/src/test_utils.rs @@ -30,6 +30,7 @@ use risingwave_common::catalog::{ }; use risingwave_common::session_config::SessionConfig; use risingwave_common::system_param::reader::SystemParamsReader; +use risingwave_common::util::cluster_limit::ClusterLimit; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_hummock_sdk::version::{HummockVersion, HummockVersionDelta}; use risingwave_pb::backup_service::MetaSnapshotMetadata; @@ -1065,7 +1066,7 @@ impl FrontendMetaClient for MockFrontendMetaClient { } async fn list_all_nodes(&self) -> RpcResult> { - unimplemented!() + Ok(vec![]) } async fn list_compact_task_progress(&self) -> RpcResult> { @@ -1097,6 +1098,10 @@ impl FrontendMetaClient for MockFrontendMetaClient { ) -> RpcResult> { unimplemented!() } + + async fn get_cluster_limits(&self) -> RpcResult> { + Ok(vec![]) + } } #[cfg(test)] diff --git a/src/meta/Cargo.toml b/src/meta/Cargo.toml index 4511e9f61d894..a7f37bf505910 100644 --- a/src/meta/Cargo.toml +++ b/src/meta/Cargo.toml @@ -28,6 +28,7 @@ clap = { workspace = true } comfy-table = "7" crepe = "0.1" easy-ext = "1" +educe = "0.6" either = "1" enum-as-inner = "0.6" etcd-client = { workspace = true } diff --git a/src/meta/node/src/lib.rs b/src/meta/node/src/lib.rs index 049519372c81e..88a76d1a1c706 100644 --- a/src/meta/node/src/lib.rs +++ b/src/meta/node/src/lib.rs @@ -457,6 +457,14 @@ pub fn start( table_info_statistic_history_times: config .storage .table_info_statistic_history_times, + actor_cnt_per_worker_parallelism_hard_limit: config + .meta + .developer + .actor_cnt_per_worker_parallelism_hard_limit, + actor_cnt_per_worker_parallelism_soft_limit: config + .meta + .developer + .actor_cnt_per_worker_parallelism_soft_limit, }, config.system.into_init_system_params(), Default::default(), diff --git a/src/meta/node/src/server.rs b/src/meta/node/src/server.rs index 1f0f7f6a3fe8e..11b22014f9f98 100644 --- a/src/meta/node/src/server.rs +++ b/src/meta/node/src/server.rs @@ -27,7 +27,6 @@ use risingwave_common::telemetry::manager::TelemetryManager; use risingwave_common::telemetry::{report_scarf_enabled, report_to_scarf, telemetry_env_enabled}; use risingwave_common::util::tokio_util::sync::CancellationToken; use risingwave_common_service::{MetricsManager, TracingExtractLayer}; -use risingwave_meta::barrier::StreamRpcManager; use risingwave_meta::controller::catalog::CatalogController; use risingwave_meta::controller::cluster::ClusterController; use risingwave_meta::manager::{ @@ -40,6 +39,7 @@ use risingwave_meta::stream::ScaleController; use risingwave_meta::MetaStoreBackend; use risingwave_meta_service::backup_service::BackupServiceImpl; use risingwave_meta_service::cloud_service::CloudServiceImpl; +use risingwave_meta_service::cluster_limit_service::ClusterLimitServiceImpl; use risingwave_meta_service::cluster_service::ClusterServiceImpl; use risingwave_meta_service::ddl_service::DdlServiceImpl; use risingwave_meta_service::event_log_service::EventLogServiceImpl; @@ -63,6 +63,7 @@ use risingwave_pb::connector_service::sink_coordination_service_server::SinkCoor use risingwave_pb::ddl_service::ddl_service_server::DdlServiceServer; use risingwave_pb::health::health_server::HealthServer; use risingwave_pb::hummock::hummock_manager_service_server::HummockManagerServiceServer; +use risingwave_pb::meta::cluster_limit_service_server::ClusterLimitServiceServer; use risingwave_pb::meta::cluster_service_server::ClusterServiceServer; use risingwave_pb::meta::event_log_service_server::EventLogServiceServer; use risingwave_pb::meta::heartbeat_service_server::HeartbeatServiceServer; @@ -550,12 +551,9 @@ pub async fn start_service_as_election_leader( // TODO(shutdown): remove this as there's no need to gracefully shutdown some of these sub-tasks. let mut sub_tasks = vec![shutdown_handle]; - let stream_rpc_manager = StreamRpcManager::new(env.clone()); - let scale_controller = Arc::new(ScaleController::new( &metadata_manager, source_manager.clone(), - stream_rpc_manager.clone(), env.clone(), )); @@ -567,7 +565,6 @@ pub async fn start_service_as_election_leader( source_manager.clone(), sink_manager.clone(), meta_metrics.clone(), - stream_rpc_manager.clone(), scale_controller.clone(), ) .await; @@ -585,7 +582,6 @@ pub async fn start_service_as_election_leader( metadata_manager.clone(), barrier_scheduler.clone(), source_manager.clone(), - stream_rpc_manager, scale_controller.clone(), ) .unwrap(), @@ -657,6 +653,7 @@ pub async fn start_service_as_election_leader( ServingServiceImpl::new(serving_vnode_mapping.clone(), metadata_manager.clone()); let cloud_srv = CloudServiceImpl::new(metadata_manager.clone(), aws_cli); let event_log_srv = EventLogServiceImpl::new(env.event_log_manager_ref()); + let cluster_limit_srv = ClusterLimitServiceImpl::new(env.clone(), metadata_manager.clone()); if let Some(prometheus_addr) = address_info.prometheus_addr { MetricsManager::boot_metrics_service(prometheus_addr.to_string()) @@ -795,7 +792,8 @@ pub async fn start_service_as_election_leader( .add_service(ServingServiceServer::new(serving_srv)) .add_service(CloudServiceServer::new(cloud_srv)) .add_service(SinkCoordinationServiceServer::new(sink_coordination_srv)) - .add_service(EventLogServiceServer::new(event_log_srv)); + .add_service(EventLogServiceServer::new(event_log_srv)) + .add_service(ClusterLimitServiceServer::new(cluster_limit_srv)); #[cfg(not(madsim))] // `otlp-embedded` does not use madsim-patched tonic let server_builder = server_builder.add_service(TraceServiceServer::new(trace_srv)); diff --git a/src/meta/service/src/cluster_limit_service.rs b/src/meta/service/src/cluster_limit_service.rs new file mode 100644 index 0000000000000..df19b24b234e6 --- /dev/null +++ b/src/meta/service/src/cluster_limit_service.rs @@ -0,0 +1,107 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; + +use risingwave_common::util::cluster_limit::{ + ActorCountPerParallelism, ClusterLimit, WorkerActorCount, +}; +use risingwave_meta::manager::{MetaSrvEnv, MetadataManager, WorkerId}; +use risingwave_meta::MetaResult; +use risingwave_pb::common::worker_node::State; +use risingwave_pb::common::WorkerType; +use risingwave_pb::meta::cluster_limit_service_server::ClusterLimitService; +use risingwave_pb::meta::{GetClusterLimitsRequest, GetClusterLimitsResponse}; +use tonic::{Request, Response, Status}; + +#[derive(Clone)] +pub struct ClusterLimitServiceImpl { + env: MetaSrvEnv, + metadata_manager: MetadataManager, +} + +impl ClusterLimitServiceImpl { + pub fn new(env: MetaSrvEnv, metadata_manager: MetadataManager) -> Self { + ClusterLimitServiceImpl { + env, + metadata_manager, + } + } + + async fn get_active_actor_limit(&self) -> MetaResult> { + let (soft_limit, hard_limit) = ( + self.env.opts.actor_cnt_per_worker_parallelism_soft_limit, + self.env.opts.actor_cnt_per_worker_parallelism_hard_limit, + ); + + let running_worker_parallelism: HashMap = self + .metadata_manager + .list_worker_node(Some(WorkerType::ComputeNode), Some(State::Running)) + .await? + .into_iter() + .map(|e| (e.id, e.parallelism())) + .collect(); + let worker_actor_count: HashMap = self + .metadata_manager + .worker_actor_count() + .await? + .into_iter() + .filter_map(|(worker_id, actor_count)| { + running_worker_parallelism + .get(&worker_id) + .map(|parallelism| { + ( + worker_id, + WorkerActorCount { + actor_count, + parallelism: *parallelism, + }, + ) + }) + }) + .collect(); + + let limit = ActorCountPerParallelism { + worker_id_to_actor_count: worker_actor_count, + hard_limit, + soft_limit, + }; + + if limit.exceed_limit() { + Ok(Some(ClusterLimit::ActorCount(limit))) + } else { + Ok(None) + } + } +} + +#[async_trait::async_trait] +impl ClusterLimitService for ClusterLimitServiceImpl { + #[cfg_attr(coverage, coverage(off))] + async fn get_cluster_limits( + &self, + _request: Request, + ) -> Result, Status> { + // TODO: support more limits + match self.get_active_actor_limit().await { + Ok(Some(limit)) => Ok(Response::new(GetClusterLimitsResponse { + active_limits: vec![limit.into()], + })), + Ok(None) => Ok(Response::new(GetClusterLimitsResponse { + active_limits: vec![], + })), + Err(e) => Err(e.into()), + } + } +} diff --git a/src/meta/service/src/lib.rs b/src/meta/service/src/lib.rs index 9ab248802772e..e2f57d4a26bbb 100644 --- a/src/meta/service/src/lib.rs +++ b/src/meta/service/src/lib.rs @@ -21,6 +21,7 @@ use risingwave_meta::*; pub mod backup_service; pub mod cloud_service; +pub mod cluster_limit_service; pub mod cluster_service; pub mod ddl_service; pub mod event_log_service; diff --git a/src/meta/src/backup_restore/restore_impl/v2.rs b/src/meta/src/backup_restore/restore_impl/v2.rs index a887293e0c8ef..938050ce4d300 100644 --- a/src/meta/src/backup_restore/restore_impl/v2.rs +++ b/src/meta/src/backup_restore/restore_impl/v2.rs @@ -106,8 +106,8 @@ impl Writer for WriterModelV2ToMetaStoreV2 { insert_models(metadata.workers.clone(), db).await?; insert_models(metadata.worker_properties.clone(), db).await?; insert_models(metadata.users.clone(), db).await?; - insert_models(metadata.user_privileges.clone(), db).await?; insert_models(metadata.objects.clone(), db).await?; + insert_models(metadata.user_privileges.clone(), db).await?; insert_models(metadata.object_dependencies.clone(), db).await?; insert_models(metadata.databases.clone(), db).await?; insert_models(metadata.schemas.clone(), db).await?; diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 6e4ebe40b93b0..577a0bef25360 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -16,7 +16,6 @@ use std::collections::{HashMap, HashSet}; use std::fmt::Formatter; use futures::future::try_join_all; -use itertools::Itertools; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::TableId; use risingwave_common::hash::ActorMapping; @@ -147,8 +146,10 @@ impl ReplaceTablePlan { } } -#[derive(Debug, Clone)] +#[derive(educe::Educe, Clone)] +#[educe(Debug)] pub struct CreateStreamingJobCommandInfo { + #[educe(Debug(ignore))] pub table_fragments: TableFragments, /// Refer to the doc on [`MetadataManager::get_upstream_root_fragments`] for the meaning of "root". pub upstream_root_actors: HashMap>, @@ -959,19 +960,6 @@ impl Command { } impl CommandContext { - /// Clean up actors in CNs if needed, used by drop, cancel and reschedule commands. - async fn clean_up(&self, actors: Vec) -> MetaResult<()> { - self.barrier_manager_context - .stream_rpc_manager - .drop_actors( - &self.node_map, - self.node_map - .keys() - .map(|worker_id| (*worker_id, actors.clone())), - ) - .await - } - pub async fn wait_epoch_commit(&self, epoch: HummockEpoch) -> MetaResult<()> { let futures = self.node_map.values().map(|worker_node| async { let client = self @@ -1021,13 +1009,9 @@ impl CommandContext { } Command::DropStreamingJobs { - actors, unregistered_state_table_ids, .. } => { - // Tell compute nodes to drop actors. - self.clean_up(actors.clone()).await?; - self.barrier_manager_context .hummock_manager .unregister_table_ids(unregistered_state_table_ids.iter().cloned()) @@ -1036,7 +1020,6 @@ impl CommandContext { Command::CancelStreamingJob(table_fragments) => { tracing::debug!(id = ?table_fragments.table_id(), "cancelling stream job"); - self.clean_up(table_fragments.actor_ids()).await?; // NOTE(kwannoel): At this point, meta has already registered the table ids. // We should unregister them. @@ -1136,8 +1119,6 @@ impl CommandContext { .. }) = job_type { - self.clean_up(old_table_fragments.actor_ids()).await?; - // Drop fragment info in meta store. mgr.fragment_manager .post_replace_table( @@ -1164,13 +1145,9 @@ impl CommandContext { new_table_fragments, dispatchers, init_split_assignment, - old_table_fragments, .. }) = job_type { - // Tell compute nodes to drop actors. - self.clean_up(old_table_fragments.actor_ids()).await?; - mgr.catalog_controller .post_collect_table_fragments( new_table_fragments.table_id().table_id as _, @@ -1201,11 +1178,6 @@ impl CommandContext { table_parallelism, .. } => { - let removed_actors = reschedules - .values() - .flat_map(|reschedule| reschedule.removed_actors.clone().into_iter()) - .collect_vec(); - self.clean_up(removed_actors).await?; self.barrier_manager_context .scale_controller .post_apply_reschedule(reschedules, table_parallelism) @@ -1220,8 +1192,6 @@ impl CommandContext { init_split_assignment, .. }) => { - self.clean_up(old_table_fragments.actor_ids()).await?; - match &self.barrier_manager_context.metadata_manager { MetadataManager::V1(mgr) => { // Drop fragment info in meta store. diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 32ce0776eff17..6f97a5fb627f6 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -86,7 +86,6 @@ pub use self::command::{ Reschedule, SnapshotBackfillInfo, }; pub use self::info::InflightSubscriptionInfo; -pub use self::rpc::StreamRpcManager; pub use self::schedule::BarrierScheduler; pub use self::trace::TracedEpoch; @@ -172,8 +171,6 @@ pub struct GlobalBarrierManagerContext { pub(super) metrics: Arc, - stream_rpc_manager: StreamRpcManager, - env: MetaSrvEnv, } @@ -596,7 +593,6 @@ impl GlobalBarrierManager { source_manager: SourceManagerRef, sink_manager: SinkCoordinatorManager, metrics: Arc, - stream_rpc_manager: StreamRpcManager, scale_controller: ScaleControllerRef, ) -> Self { let enable_recovery = env.opts.enable_recovery; @@ -624,7 +620,6 @@ impl GlobalBarrierManager { scale_controller, sink_manager, metrics, - stream_rpc_manager, env: env.clone(), }; diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 5754e4c60e364..2e1b6f9dc397a 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -55,6 +55,7 @@ pub(super) struct Progress { upstream_mv_count: HashMap, /// Total key count in the upstream materialized view + /// TODO: implement this for source backfill upstream_total_key_count: u64, /// Consumed rows @@ -122,6 +123,12 @@ impl Progress { /// Returns whether all backfill executors are done. fn is_done(&self) -> bool { + tracing::trace!( + "Progress::is_done? {}, {}, {:?}", + self.done_count, + self.states.len(), + self.states + ); self.done_count == self.states.len() } @@ -274,6 +281,7 @@ pub(super) struct TrackingCommand { /// 4. With `actor_map` we can use an actor's `ActorId` to find the ID of the `StreamJob`. #[derive(Default, Debug)] pub(super) struct CreateMviewProgressTracker { + // TODO: add a specialized progress for source /// Progress of the create-mview DDL indicated by the `TableId`. progress_map: HashMap, @@ -494,6 +502,7 @@ impl CreateMviewProgressTracker { replace_table: Option<&ReplaceTablePlan>, version_stats: &HummockVersionStats, ) -> Option { + tracing::trace!(?info, "add job to track"); let (info, actors, replace_table_info) = { let CreateStreamingJobCommandInfo { table_fragments, .. @@ -596,6 +605,7 @@ impl CreateMviewProgressTracker { progress: &CreateMviewProgress, version_stats: &HummockVersionStats, ) -> Option { + tracing::trace!(?progress, "update progress"); let actor = progress.backfill_actor_id; let Some(table_id) = self.actor_map.get(&actor).copied() else { // On restart, backfill will ALWAYS notify CreateMviewProgressTracker, diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 140f92cdb6296..1863e86319c82 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -1120,6 +1120,14 @@ impl GlobalBarrierManagerContext { return Err(anyhow!("actors dropped during update").into()); } + { + for (node_id, actors) in &info.actor_map { + if !actors.is_empty() && !all_node_actors.contains_key(node_id) { + return Err(anyhow!("streaming job dropped during update").into()); + } + } + } + Ok(all_node_actors) } } diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 91237890c62d8..cb1b703fb6ec5 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -21,10 +21,9 @@ use anyhow::anyhow; use fail::fail_point; use futures::future::try_join_all; use futures::stream::{BoxStream, FuturesUnordered}; -use futures::{pin_mut, FutureExt, StreamExt}; +use futures::{FutureExt, StreamExt}; use itertools::Itertools; use risingwave_common::catalog::TableId; -use risingwave_common::hash::ActorId; use risingwave_common::util::tracing::TracingContext; use risingwave_hummock_sdk::HummockVersionId; use risingwave_pb::common::{ActorInfo, WorkerNode}; @@ -34,11 +33,9 @@ use risingwave_pb::stream_service::build_actor_info::SubscriptionIds; use risingwave_pb::stream_service::streaming_control_stream_request::RemovePartialGraphRequest; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, BarrierCompleteResponse, - BuildActorInfo, DropActorsRequest, InjectBarrierRequest, StreamingControlStreamRequest, + BuildActorInfo, InjectBarrierRequest, StreamingControlStreamRequest, StreamingControlStreamResponse, }; -use risingwave_rpc_client::error::RpcError; -use risingwave_rpc_client::StreamClient; use rw_futures_util::pending_on_none; use thiserror_ext::AsReport; use tokio::sync::mpsc::UnboundedSender; @@ -50,7 +47,7 @@ use uuid::Uuid; use super::command::CommandContext; use super::{BarrierKind, GlobalBarrierManagerContext, TracedEpoch}; use crate::barrier::info::InflightGraphInfo; -use crate::manager::{MetaSrvEnv, WorkerId}; +use crate::manager::WorkerId; use crate::{MetaError, MetaResult}; const COLLECT_ERROR_TIMEOUT: Duration = Duration::from_secs(3); @@ -387,7 +384,7 @@ impl ControlStreamManager { request: Some( streaming_control_stream_request::Request::InjectBarrier( InjectBarrierRequest { - request_id: StreamRpcManager::new_request_id(), + request_id: Uuid::new_v4().to_string(), barrier: Some(barrier), actor_ids_to_collect, table_ids_to_sync, @@ -499,95 +496,6 @@ impl GlobalBarrierManagerContext { } } -#[derive(Clone)] -pub struct StreamRpcManager { - env: MetaSrvEnv, -} - -impl StreamRpcManager { - pub fn new(env: MetaSrvEnv) -> Self { - Self { env } - } - - async fn make_request> + 'static>( - &self, - request: impl Iterator, - f: impl Fn(StreamClient, REQ) -> Fut, - ) -> MetaResult> { - let pool = self.env.stream_client_pool(); - let f = &f; - let iters = request.map(|(node, input)| async move { - let client = pool.get(node).await.map_err(|e| (node.id, e))?; - f(client, input).await.map_err(|e| (node.id, e)) - }); - let result = try_join_all_with_error_timeout(iters, COLLECT_ERROR_TIMEOUT).await; - result.map_err(|results_err| merge_node_rpc_errors("merged RPC Error", results_err)) - } - - fn new_request_id() -> String { - Uuid::new_v4().to_string() - } - - pub async fn drop_actors( - &self, - node_map: &HashMap, - node_actors: impl Iterator)>, - ) -> MetaResult<()> { - self.make_request( - node_actors - .map(|(worker_id, actor_ids)| (node_map.get(&worker_id).unwrap(), actor_ids)), - |client, actor_ids| async move { - client - .drop_actors(DropActorsRequest { - request_id: Self::new_request_id(), - actor_ids, - }) - .await - }, - ) - .await?; - Ok(()) - } -} - -/// This function is similar to `try_join_all`, but it attempts to collect as many error as possible within `error_timeout`. -async fn try_join_all_with_error_timeout( - iters: I, - error_timeout: Duration, -) -> Result, Vec> -where - I: IntoIterator, - F: Future>, -{ - let stream = FuturesUnordered::from_iter(iters); - pin_mut!(stream); - let mut results_ok = vec![]; - let mut results_err = vec![]; - while let Some(result) = stream.next().await { - match result { - Ok(rsp) => { - results_ok.push(rsp); - } - Err(err) => { - results_err.push(err); - break; - } - } - } - if results_err.is_empty() { - return Ok(results_ok); - } - let _ = timeout(error_timeout, async { - while let Some(result) = stream.next().await { - if let Err(err) = result { - results_err.push(err); - } - } - }) - .await; - Err(results_err) -} - pub(super) fn merge_node_rpc_errors( message: &str, errors: impl IntoIterator, diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index 16228a06d0a9a..31575e72804f9 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -1411,7 +1411,7 @@ mod tests { use std::collections::{BTreeMap, HashMap}; use itertools::Itertools; - use risingwave_common::hash::ActorMapping; + use risingwave_common::hash::{ActorMapping, VirtualNode}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; @@ -1497,8 +1497,11 @@ mod tests { }) .collect(); - let actor_bitmaps = - ActorMapping::new_uniform((0..actor_count).map(|i| i as _)).to_bitmaps(); + let actor_bitmaps = ActorMapping::new_uniform( + (0..actor_count).map(|i| i as _), + VirtualNode::COUNT_FOR_TEST, + ) + .to_bitmaps(); let pb_actors = (0..actor_count) .map(|actor_id| { @@ -1610,8 +1613,11 @@ mod tests { }) .collect(); - let mut actor_bitmaps = - ActorMapping::new_uniform((0..actor_count).map(|i| i as _)).to_bitmaps(); + let mut actor_bitmaps = ActorMapping::new_uniform( + (0..actor_count).map(|i| i as _), + VirtualNode::COUNT_FOR_TEST, + ) + .to_bitmaps(); let actors = (0..actor_count) .map(|actor_id| { diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 08428e5472e23..e7bca768437f2 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::change_log::ChangeLogDelta; @@ -220,23 +220,8 @@ impl HummockManager { NewTableFragmentInfo::None => (HashMap::new(), None, None), }; - let mut group_members_table_ids: HashMap> = HashMap::new(); - { - // expand group_members_table_ids - for (table_id, group_id) in &table_compaction_group_mapping { - group_members_table_ids - .entry(*group_id) - .or_default() - .insert(*table_id); - } - } - let commit_sstables = self - .correct_commit_ssts( - sstables, - &table_compaction_group_mapping, - &group_members_table_ids, - ) + .correct_commit_ssts(sstables, &table_compaction_group_mapping) .await?; let modified_compaction_groups: Vec<_> = commit_sstables.keys().cloned().collect(); @@ -389,7 +374,6 @@ impl HummockManager { &self, sstables: Vec, table_compaction_group_mapping: &HashMap, - group_members_table_ids: &HashMap>, ) -> Result>> { let mut new_sst_id_number = 0; let mut sst_to_cg_vec = Vec::with_capacity(sstables.len()); @@ -413,7 +397,7 @@ impl HummockManager { } } - new_sst_id_number += group_table_ids.len(); + new_sst_id_number += group_table_ids.len() * 2; // `split_sst` will split the SST into two parts and consumer 2 SST IDs sst_to_cg_vec.push((commit_sst, group_table_ids)); } @@ -424,17 +408,16 @@ impl HummockManager { let mut commit_sstables: BTreeMap> = BTreeMap::new(); for (mut sst, group_table_ids) in sst_to_cg_vec { - for (group_id, match_ids) in group_table_ids { - let group_members_table_ids = group_members_table_ids.get(&group_id).unwrap(); - if match_ids - .iter() - .all(|id| group_members_table_ids.contains(&TableId::new(*id))) - { + let len = group_table_ids.len(); + for (index, (group_id, match_ids)) in group_table_ids.into_iter().enumerate() { + if sst.sst_info.table_ids == match_ids { + // The SST contains all the tables in the group should be last key + assert!(index == len - 1); commit_sstables .entry(group_id) .or_default() - .push(sst.sst_info.clone()); - continue; + .push(sst.sst_info); + break; } let origin_sst_size = sst.sst_info.sst_size; diff --git a/src/meta/src/hummock/manager/tests.rs b/src/meta/src/hummock/manager/tests.rs index 56b4836f585a1..dca7311f4778f 100644 --- a/src/meta/src/hummock/manager/tests.rs +++ b/src/meta/src/hummock/manager/tests.rs @@ -1327,7 +1327,22 @@ async fn test_split_compaction_group_on_commit() { sst_size: 100, ..Default::default() }, - table_stats: Default::default(), + table_stats: HashMap::from([ + ( + 100, + TableStats { + total_compressed_size: 50, + ..Default::default() + }, + ), + ( + 101, + TableStats { + total_compressed_size: 50, + ..Default::default() + }, + ), + ]), }; hummock_manager .commit_epoch_for_test(30, vec![sst_1], HashMap::from([(10, context_id)])) diff --git a/src/meta/src/manager/catalog/user.rs b/src/meta/src/manager/catalog/user.rs index 81181b0fc1e17..68e5e31395c0d 100644 --- a/src/meta/src/manager/catalog/user.rs +++ b/src/meta/src/manager/catalog/user.rs @@ -74,6 +74,7 @@ impl UserManager { .values() .map(|connection| connection.owner), ) + .chain(database.secrets.values().map(|secret| secret.owner)) .for_each(|owner_id| user_manager.increase_ref(owner_id)); Ok(user_manager) diff --git a/src/meta/src/manager/env.rs b/src/meta/src/manager/env.rs index 22f88bd9c0a75..ed18be6b0f483 100644 --- a/src/meta/src/manager/env.rs +++ b/src/meta/src/manager/env.rs @@ -294,6 +294,10 @@ pub struct MetaOpts { pub temp_secret_file_dir: String, pub table_info_statistic_history_times: usize, + + // Cluster limits + pub actor_cnt_per_worker_parallelism_hard_limit: usize, + pub actor_cnt_per_worker_parallelism_soft_limit: usize, } impl MetaOpts { @@ -358,6 +362,8 @@ impl MetaOpts { secret_store_private_key: Some("0123456789abcdef".as_bytes().to_vec()), temp_secret_file_dir: "./secrets".to_string(), table_info_statistic_history_times: 240, + actor_cnt_per_worker_parallelism_hard_limit: usize::MAX, + actor_cnt_per_worker_parallelism_soft_limit: usize::MAX, } } } @@ -408,9 +414,11 @@ impl MetaSrvEnv { (ClusterId::new(), true) }; - // For new clusters, the name of the object store needs to be prefixed according to the object id. - // For old clusters, the prefix is ​​not divided for the sake of compatibility. - + // For new clusters: + // - the name of the object store needs to be prefixed according to the object id. + // + // For old clusters + // - the prefix is ​​not divided for the sake of compatibility. init_system_params.use_new_object_prefix_strategy = Some(cluster_first_launch); let system_params_manager = Arc::new( SystemParamsManager::new( @@ -455,7 +463,7 @@ impl MetaSrvEnv { } } MetaStoreImpl::Sql(sql_meta_store) => { - let is_sql_backend_cluster_first_launch = + let cluster_first_launch = is_first_launch_for_sql_backend_cluster(sql_meta_store).await?; // Try to upgrade if any new model changes are added. Migrator::up(&sql_meta_store.conn, None) @@ -469,10 +477,14 @@ impl MetaSrvEnv { .await? .map(|c| c.cluster_id.to_string().into()) .unwrap(); - init_system_params.use_new_object_prefix_strategy = - Some(is_sql_backend_cluster_first_launch); - // For new clusters, the name of the object store needs to be prefixed according to the object id. - // For old clusters, the prefix is ​​not divided for the sake of compatibility. + + // For new clusters: + // - the name of the object store needs to be prefixed according to the object id. + // + // For old clusters + // - the prefix is ​​not divided for the sake of compatibility. + init_system_params.use_new_object_prefix_strategy = Some(cluster_first_launch); + let system_param_controller = Arc::new( SystemParamsController::new( sql_meta_store.clone(), diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index 52fc811787d30..935d4773865ed 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -917,6 +917,7 @@ impl MetadataManager { &self, job: &StreamingJob, ) -> MetaResult { + tracing::debug!("wait_streaming_job_finished: {job:?}"); match self { MetadataManager::V1(mgr) => mgr.wait_streaming_job_finished(job).await, MetadataManager::V2(mgr) => mgr.wait_streaming_job_finished(job.id() as _).await, diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 447cf5cf85645..aaff076688785 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -363,7 +363,9 @@ impl TableFragments { return vec![]; } if (fragment.fragment_type_mask - & (FragmentTypeFlag::Values as u32 | FragmentTypeFlag::StreamScan as u32)) + & (FragmentTypeFlag::Values as u32 + | FragmentTypeFlag::StreamScan as u32 + | FragmentTypeFlag::SourceScan as u32)) != 0 { actor_ids.extend(fragment.actors.iter().map(|actor| actor.actor_id)); diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index feb7a959083bb..e1605b0aa61dc 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -368,12 +368,14 @@ impl DdlController { } } + #[tracing::instrument(skip(self), level = "debug")] pub async fn alter_parallelism( &self, table_id: u32, parallelism: PbTableParallelism, mut deferred: bool, ) -> MetaResult<()> { + tracing::info!("alter parallelism"); if self.barrier_manager.check_status_running().is_err() { tracing::info!( "alter parallelism is set to deferred mode because the system is in recovery state" diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index d10fa83710d85..4c296bd772467 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -49,7 +49,7 @@ use tokio::sync::{oneshot, RwLock, RwLockReadGuard, RwLockWriteGuard}; use tokio::task::JoinHandle; use tokio::time::{Instant, MissedTickBehavior}; -use crate::barrier::{Command, Reschedule, StreamRpcManager}; +use crate::barrier::{Command, Reschedule}; use crate::controller::scale::RescheduleWorkingSet; use crate::manager::{ IdCategory, IdGenManagerImpl, LocalNotification, MetaSrvEnv, MetadataManager, @@ -183,17 +183,26 @@ impl CustomFragmentInfo { } } +use educe::Educe; + +// The debug implementation is arbitrary. Just used in debug logs. +#[derive(Educe)] +#[educe(Debug)] pub struct RescheduleContext { /// Meta information for all Actors + #[educe(Debug(ignore))] actor_map: HashMap, /// Status of all Actors, used to find the location of the `Actor` actor_status: BTreeMap, /// Meta information of all `Fragment`, used to find the `Fragment`'s `Actor` + #[educe(Debug(ignore))] fragment_map: HashMap, /// Index of all `Actor` upstreams, specific to `Dispatcher` upstream_dispatchers: HashMap>, - /// Fragments with stream source + /// Fragments with `StreamSource` stream_source_fragment_ids: HashSet, + /// Fragments with `StreamSourceBackfill` + stream_source_backfill_fragment_ids: HashSet, /// Target fragments in `NoShuffle` relation no_shuffle_target_fragment_ids: HashSet, /// Source fragments in `NoShuffle` relation @@ -437,10 +446,10 @@ pub struct ScaleController { pub source_manager: SourceManagerRef, - pub stream_rpc_manager: StreamRpcManager, - pub env: MetaSrvEnv, + /// We will acquire lock during DDL to prevent scaling operations on jobs that are in the creating state. + /// e.g., a MV cannot be rescheduled during foreground backfill. pub reschedule_lock: RwLock<()>, } @@ -448,11 +457,9 @@ impl ScaleController { pub fn new( metadata_manager: &MetadataManager, source_manager: SourceManagerRef, - stream_rpc_manager: StreamRpcManager, env: MetaSrvEnv, ) -> Self { Self { - stream_rpc_manager, metadata_manager: metadata_manager.clone(), source_manager, env, @@ -770,6 +777,7 @@ impl ScaleController { } let mut stream_source_fragment_ids = HashSet::new(); + let mut stream_source_backfill_fragment_ids = HashSet::new(); let mut no_shuffle_reschedule = HashMap::new(); for (fragment_id, WorkerReschedule { worker_actor_diff }) in &*reschedule { let fragment = fragment_map @@ -798,6 +806,7 @@ impl ScaleController { // correspondence, so we need to clone the reschedule plan to the downstream of all // cascading relations. if no_shuffle_source_fragment_ids.contains(fragment_id) { + // This fragment is a NoShuffle's upstream. let mut queue: VecDeque<_> = fragment_dispatcher_map .get(fragment_id) .unwrap() @@ -887,6 +896,17 @@ impl ScaleController { "reschedule plan rewritten with NoShuffle reschedule {:?}", no_shuffle_reschedule ); + + for noshuffle_downstream in no_shuffle_reschedule.keys() { + let fragment = fragment_map.get(noshuffle_downstream).unwrap(); + // SourceScan is always a NoShuffle downstream, rescheduled together with the upstream Source. + if (fragment.get_fragment_type_mask() & FragmentTypeFlag::SourceScan as u32) != 0 { + let stream_node = fragment.actor_template.nodes.as_ref().unwrap(); + if stream_node.find_source_backfill().is_some() { + stream_source_backfill_fragment_ids.insert(fragment.fragment_id); + } + } + } } // Modifications for NoShuffle downstream. @@ -898,6 +918,7 @@ impl ScaleController { fragment_map, upstream_dispatchers, stream_source_fragment_ids, + stream_source_backfill_fragment_ids, no_shuffle_target_fragment_ids, no_shuffle_source_fragment_ids, fragment_dispatcher_map, @@ -924,9 +945,11 @@ impl ScaleController { HashMap, HashMap>, )> { + tracing::debug!("build_reschedule_context, reschedules: {:#?}", reschedules); let ctx = self .build_reschedule_context(&mut reschedules, options, table_parallelisms) .await?; + tracing::debug!("reschedule context: {:#?}", ctx); let reschedules = reschedules; // Here, the plan for both upstream and downstream of the NO_SHUFFLE Fragment should already have been populated. @@ -1264,9 +1287,9 @@ impl ScaleController { } } - // For stream source fragments, we need to reallocate the splits. + // For stream source & source backfill fragments, we need to reallocate the splits. // Because we are in the Pause state, so it's no problem to reallocate - let mut fragment_stream_source_actor_splits = HashMap::new(); + let mut fragment_actor_splits = HashMap::new(); for fragment_id in reschedules.keys() { let actors_after_reschedule = fragment_actors_after_reschedule.get(fragment_id).unwrap(); @@ -1284,13 +1307,51 @@ impl ScaleController { let actor_splits = self .source_manager - .migrate_splits(*fragment_id, &prev_actor_ids, &curr_actor_ids) + .migrate_splits_for_source_actors( + *fragment_id, + &prev_actor_ids, + &curr_actor_ids, + ) .await?; - fragment_stream_source_actor_splits.insert(*fragment_id, actor_splits); + tracing::debug!( + "source actor splits: {:?}, fragment_id: {}", + actor_splits, + fragment_id + ); + fragment_actor_splits.insert(*fragment_id, actor_splits); + } + } + // We use 2 iterations to make sure source actors are migrated first, and then align backfill actors + if !ctx.stream_source_backfill_fragment_ids.is_empty() { + for fragment_id in reschedules.keys() { + let actors_after_reschedule = + fragment_actors_after_reschedule.get(fragment_id).unwrap(); + + if ctx + .stream_source_backfill_fragment_ids + .contains(fragment_id) + { + let fragment = ctx.fragment_map.get(fragment_id).unwrap(); + + let curr_actor_ids = actors_after_reschedule.keys().cloned().collect_vec(); + + let actor_splits = self.source_manager.migrate_splits_for_backfill_actors( + *fragment_id, + &fragment.upstream_fragment_ids, + &curr_actor_ids, + &fragment_actor_splits, + &no_shuffle_upstream_actor_map, + )?; + tracing::debug!( + "source backfill actor splits: {:?}, fragment_id: {}", + actor_splits, + fragment_id + ); + fragment_actor_splits.insert(*fragment_id, actor_splits); + } } } - // TODO: support migrate splits for SourceBackfill // Generate fragment reschedule plan let mut reschedule_fragment: HashMap = @@ -1428,7 +1489,7 @@ impl ScaleController { let upstream_fragment_dispatcher_ids = upstream_fragment_dispatcher_set.into_iter().collect_vec(); - let actor_splits = fragment_stream_source_actor_splits + let actor_splits = fragment_actor_splits .get(&fragment_id) .cloned() .unwrap_or_default(); @@ -1479,6 +1540,8 @@ impl ScaleController { .pre_apply_reschedules(fragment_created_actors) .await; + tracing::debug!("analyze_reschedule_plan result: {:#?}", reschedule_fragment); + Ok((reschedule_fragment, applied_reschedules)) } @@ -1867,12 +1930,12 @@ impl ScaleController { actor_location: &mut HashMap, table_fragment_id_map: &mut HashMap>, fragment_actor_id_map: &mut HashMap>, - table_fragments: &BTreeMap, + all_table_fragments: &BTreeMap, ) -> MetaResult<()> { // This is only for assertion purposes and will be removed once the dispatcher_id is guaranteed to always correspond to the downstream fragment_id, // such as through the foreign key constraints in the SQL backend. let mut actor_fragment_id_map_for_check = HashMap::new(); - for table_fragments in table_fragments.values() { + for table_fragments in all_table_fragments.values() { for (fragment_id, fragment) in &table_fragments.fragments { for actor in &fragment.actors { let prev = @@ -1883,7 +1946,7 @@ impl ScaleController { } } - for (table_id, table_fragments) in table_fragments { + for (table_id, table_fragments) in all_table_fragments { for (fragment_id, fragment) in &table_fragments.fragments { for actor in &fragment.actors { fragment_actor_id_map @@ -1911,8 +1974,15 @@ impl ScaleController { dispatcher.dispatcher_id as FragmentId ); } else { + tracing::error!( + "downstream actor id {} from actor {} (fragment {}) not found in actor_fragment_id_map_for_check: {actor_fragment_id_map_for_check:?}\n\ndispatchers: {:#?}", + downstream_actor_id, + actor.actor_id, + actor.fragment_id, + actor.dispatcher + ); bail!( - "downstream actor id {} from actor {} not found in fragment_actor_id_map", + "downstream actor id {} from actor {} not found", downstream_actor_id, actor.actor_id, ); @@ -2029,6 +2099,17 @@ impl ScaleController { .await?; } } + tracing::debug!( + ?worker_ids, + ?table_parallelisms, + ?no_shuffle_source_fragment_ids, + ?no_shuffle_target_fragment_ids, + ?fragment_distribution_map, + ?actor_location, + ?table_fragment_id_map, + ?fragment_actor_id_map, + "generate_table_resize_plan, after build_index" + ); let mut target_plan = HashMap::new(); @@ -2149,7 +2230,10 @@ impl ScaleController { } target_plan.retain(|_, plan| !plan.worker_actor_diff.is_empty()); - + tracing::debug!( + ?target_plan, + "generate_table_resize_plan finished target_plan" + ); Ok(target_plan) } @@ -2380,6 +2464,7 @@ impl ScaleController { /// At present, for table level scaling, we use the strategy `TableResizePolicy`. /// Currently, this is used as an internal interface, so it won’t be included in Protobuf. +#[derive(Debug)] pub struct TableResizePolicy { pub(crate) worker_ids: BTreeSet, pub(crate) table_parallelisms: HashMap, diff --git a/src/meta/src/stream/source_manager.rs b/src/meta/src/stream/source_manager.rs index a383bfee8e46a..751ee92beebc1 100644 --- a/src/meta/src/stream/source_manager.rs +++ b/src/meta/src/stream/source_manager.rs @@ -231,7 +231,8 @@ pub struct SourceManagerCore { /// `source_id` -> `(fragment_id, upstream_fragment_id)` backfill_fragments: HashMap>, - /// Splits assigned per actor + /// Splits assigned per actor, + /// incl. both `Source` and `SourceBackfill`. actor_splits: HashMap>, } @@ -468,13 +469,13 @@ impl Default for SplitDiffOptions { } /// Reassigns splits if there are new splits or dropped splits, -/// i.e., `actor_splits` and `discovered_splits` differ. +/// i.e., `actor_splits` and `discovered_splits` differ, or actors are rescheduled. /// /// The existing splits will remain unmoved in their currently assigned actor. /// /// If an actor has an upstream actor, it should be a backfill executor, -/// and its splits should be aligned with the upstream actor. `reassign_splits` should not be used in this case. -/// Use `align_backfill_splits` instead. +/// and its splits should be aligned with the upstream actor. **`reassign_splits` should not be used in this case. +/// Use `align_backfill_splits` instead.** /// /// - `fragment_id`: just for logging /// @@ -790,11 +791,10 @@ impl SourceManager { /// Migrates splits from previous actors to the new actors for a rescheduled fragment. /// - /// Very occasionally split removal may happen - /// during scaling, in which case we need to use the old splits for reallocation instead of the - /// latest splits (which may be missing), so that we can resolve the split removal in the next - /// command. - pub async fn migrate_splits( + /// Very occasionally split removal may happen during scaling, in which case we need to + /// use the old splits for reallocation instead of the latest splits (which may be missing), + /// so that we can resolve the split removal in the next command. + pub async fn migrate_splits_for_source_actors( &self, fragment_id: FragmentId, prev_actor_ids: &[ActorId], @@ -817,7 +817,7 @@ impl SourceManager { fragment_id, empty_actor_splits, &prev_splits, - // pre-allocate splits is the first time getting splits and it does not have scale in scene + // pre-allocate splits is the first time getting splits and it does not have scale-in scene SplitDiffOptions::default(), ) .unwrap_or_default(); @@ -825,6 +825,43 @@ impl SourceManager { Ok(diff) } + /// Migrates splits from previous actors to the new actors for a rescheduled fragment. + pub fn migrate_splits_for_backfill_actors( + &self, + fragment_id: FragmentId, + upstream_fragment_ids: &Vec, + curr_actor_ids: &[ActorId], + fragment_actor_splits: &HashMap>>, + no_shuffle_upstream_actor_map: &HashMap>, + ) -> MetaResult>> { + // align splits for backfill fragments with its upstream source fragment + debug_assert!(upstream_fragment_ids.len() == 1); + let upstream_fragment_id = upstream_fragment_ids[0]; + let actors = no_shuffle_upstream_actor_map + .iter() + .filter(|(id, _)| curr_actor_ids.contains(id)) + .map(|(id, upstream_fragment_actors)| { + debug_assert!(upstream_fragment_actors.len() == 1); + ( + *id, + vec![*upstream_fragment_actors.get(&upstream_fragment_id).unwrap()], + ) + }); + let upstream_assignment = fragment_actor_splits.get(&upstream_fragment_id).unwrap(); + tracing::info!( + fragment_id, + upstream_fragment_id, + ?upstream_assignment, + "migrate_splits_for_backfill_actors" + ); + Ok(align_backfill_splits( + actors, + upstream_assignment, + fragment_id, + upstream_fragment_id, + )?) + } + /// Allocates splits to actors for a newly created source executor. pub async fn allocate_splits(&self, table_id: &TableId) -> MetaResult { let core = self.core.lock().await; diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index 0f9e473c26486..f338dd27725ca 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -25,7 +25,7 @@ use either::Either; use enum_as_inner::EnumAsInner; use itertools::Itertools; use risingwave_common::bitmap::Bitmap; -use risingwave_common::hash::{ActorMapping, WorkerSlotId, WorkerSlotMapping}; +use risingwave_common::hash::{ActorMapping, VirtualNode, WorkerSlotId, WorkerSlotMapping}; use risingwave_common::{bail, hash}; use risingwave_pb::common::{ActorInfo, WorkerNode}; use risingwave_pb::meta::table_fragments::fragment::{ @@ -235,7 +235,8 @@ impl Scheduler { assert_eq!(scheduled_worker_slots.len(), parallelism); // Build the default hash mapping uniformly. - let default_hash_mapping = WorkerSlotMapping::build_from_ids(&scheduled_worker_slots); + let default_hash_mapping = + WorkerSlotMapping::build_from_ids(&scheduled_worker_slots, VirtualNode::COUNT); let single_scheduled = schedule_units_for_slots(&slots, 1, streaming_job_id)?; let default_single_worker_id = single_scheduled.keys().exactly_one().cloned().unwrap(); diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index a8e8bc47752a5..5dc174106197c 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -31,7 +31,7 @@ use tracing::Instrument; use super::{Locations, RescheduleOptions, ScaleControllerRef, TableResizePolicy}; use crate::barrier::{ BarrierScheduler, Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, - ReplaceTablePlan, SnapshotBackfillInfo, StreamRpcManager, + ReplaceTablePlan, SnapshotBackfillInfo, }; use crate::manager::{DdlType, MetaSrvEnv, MetadataManager, NotificationVersion, StreamingJob}; use crate::model::{ActorId, FragmentId, MetadataModel, TableFragments, TableParallelism}; @@ -203,8 +203,6 @@ pub struct GlobalStreamManager { creating_job_info: CreatingStreamingJobInfoRef, pub scale_controller: ScaleControllerRef, - - pub stream_rpc_manager: StreamRpcManager, } impl GlobalStreamManager { @@ -213,7 +211,6 @@ impl GlobalStreamManager { metadata_manager: MetadataManager, barrier_scheduler: BarrierScheduler, source_manager: SourceManagerRef, - stream_rpc_manager: StreamRpcManager, scale_controller: ScaleControllerRef, ) -> MetaResult { Ok(Self { @@ -223,7 +220,6 @@ impl GlobalStreamManager { source_manager, creating_job_info: Arc::new(CreatingStreamingJobInfo::default()), scale_controller, - stream_rpc_manager, }) } @@ -764,8 +760,7 @@ mod tests { use std::time::Duration; use futures::{Stream, TryStreamExt}; - use risingwave_common::hash; - use risingwave_common::hash::{ActorMapping, WorkerSlotId}; + use risingwave_common::hash::{self, ActorMapping, VirtualNode, WorkerSlotId}; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::meta::add_worker_node_request::Property; @@ -816,13 +811,6 @@ mod tests { type StreamingControlStreamStream = impl Stream>; - async fn drop_actors( - &self, - _request: Request, - ) -> std::result::Result, Status> { - Ok(Response::new(DropActorsResponse::default())) - } - async fn streaming_control_stream( &self, request: Request>, @@ -989,11 +977,9 @@ mod tests { let (sink_manager, _) = SinkCoordinatorManager::start_worker(); - let stream_rpc_manager = StreamRpcManager::new(env.clone()); let scale_controller = Arc::new(ScaleController::new( &metadata_manager, source_manager.clone(), - stream_rpc_manager.clone(), env.clone(), )); @@ -1005,7 +991,6 @@ mod tests { source_manager.clone(), sink_manager, meta_metrics.clone(), - stream_rpc_manager.clone(), scale_controller.clone(), ) .await; @@ -1015,7 +1000,6 @@ mod tests { metadata_manager, barrier_scheduler.clone(), source_manager.clone(), - stream_rpc_manager, scale_controller.clone(), )?; @@ -1137,12 +1121,14 @@ mod tests { } fn make_mview_stream_actors(table_id: &TableId, count: usize) -> Vec { - let mut actor_bitmaps: HashMap<_, _> = - ActorMapping::new_uniform((0..count).map(|i| i as hash::ActorId)) - .to_bitmaps() - .into_iter() - .map(|(actor_id, bitmap)| (actor_id, bitmap.to_protobuf())) - .collect(); + let mut actor_bitmaps: HashMap<_, _> = ActorMapping::new_uniform( + (0..count).map(|i| i as hash::ActorId), + VirtualNode::COUNT_FOR_TEST, + ) + .to_bitmaps() + .into_iter() + .map(|(actor_id, bitmap)| (actor_id, bitmap.to_protobuf())) + .collect(); (0..count) .map(|i| StreamActor { diff --git a/src/meta/src/stream/test_scale.rs b/src/meta/src/stream/test_scale.rs index 0dc0bced84005..54e619e473cd0 100644 --- a/src/meta/src/stream/test_scale.rs +++ b/src/meta/src/stream/test_scale.rs @@ -26,7 +26,7 @@ mod tests { use crate::stream::CustomActorInfo; fn simulated_parallelism(min: Option, max: Option) -> Vec { - let mut raw = vec![1, 3, 12, 42, VirtualNode::COUNT]; + let mut raw = vec![1, 3, 12, 42, VirtualNode::COUNT_FOR_TEST]; if let Some(min) = min { raw.retain(|n| *n > min); raw.push(min); @@ -39,7 +39,9 @@ mod tests { } fn build_fake_actors(actor_ids: Vec) -> Vec { - let actor_bitmaps = ActorMapping::new_uniform(actor_ids.clone().into_iter()).to_bitmaps(); + let actor_bitmaps = + ActorMapping::new_uniform(actor_ids.clone().into_iter(), VirtualNode::COUNT_FOR_TEST) + .to_bitmaps(); actor_ids .iter() .map(|actor_id| CustomActorInfo { @@ -55,7 +57,7 @@ mod tests { fn check_affinity_for_scale_in(bitmap: &Bitmap, actor: &CustomActorInfo) { let prev_bitmap = Bitmap::from(actor.vnode_bitmap.as_ref().unwrap()); - for idx in 0..VirtualNode::COUNT { + for idx in 0..VirtualNode::COUNT_FOR_TEST { if prev_bitmap.is_set(idx) { assert!(bitmap.is_set(idx)); } @@ -63,7 +65,9 @@ mod tests { } fn check_bitmaps(bitmaps: &HashMap) { - let mut target = (0..VirtualNode::COUNT).map(|_| false).collect_vec(); + let mut target = (0..VirtualNode::COUNT_FOR_TEST) + .map(|_| false) + .collect_vec(); for bitmap in bitmaps.values() { for (idx, pos) in target.iter_mut().enumerate() { @@ -89,9 +93,10 @@ mod tests { fn test_build_actor_mapping() { for parallelism in simulated_parallelism(None, None) { let actor_ids = (0..parallelism as ActorId).collect_vec(); - let actor_mapping = ActorMapping::new_uniform(actor_ids.into_iter()); + let actor_mapping = + ActorMapping::new_uniform(actor_ids.into_iter(), VirtualNode::COUNT_FOR_TEST); - assert_eq!(actor_mapping.len(), VirtualNode::COUNT); + assert_eq!(actor_mapping.len(), VirtualNode::COUNT_FOR_TEST); let mut check: HashMap> = HashMap::new(); for (vnode, actor_id) in actor_mapping.iter_with_vnode() { @@ -178,7 +183,7 @@ mod tests { #[test] fn test_rebalance_scale_out() { - for parallelism in simulated_parallelism(Some(3), Some(VirtualNode::COUNT - 1)) { + for parallelism in simulated_parallelism(Some(3), Some(VirtualNode::COUNT_FOR_TEST - 1)) { let actors = build_fake_actors((0..parallelism as ActorId).collect_vec()); // add 1 @@ -189,8 +194,9 @@ mod tests { let actors = build_fake_actors((0..parallelism as ActorId).collect_vec()); - // add to VirtualNode::COUNT - let actors_to_add = (parallelism as ActorId..VirtualNode::COUNT as ActorId).collect(); + // add to VirtualNode::COUNT_FOR_TEST + let actors_to_add = + (parallelism as ActorId..VirtualNode::COUNT_FOR_TEST as ActorId).collect(); let result = rebalance_actor_vnode(&actors, &BTreeSet::new(), &actors_to_add); assert_eq!(result.len(), actors.len() + actors_to_add.len()); check_bitmaps(&result); @@ -275,7 +281,7 @@ mod tests { #[test] fn test_rebalance_scale_real() { - let actor_ids = (0..(VirtualNode::COUNT - 1) as ActorId).collect_vec(); + let actor_ids = (0..(VirtualNode::COUNT_FOR_TEST - 1) as ActorId).collect_vec(); let actors = build_fake_actors(actor_ids); let actors_to_remove = btreeset! {0, 1}; let actors_to_add = btreeset! {255}; diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index b4e06d8690b72..db66e60c91eeb 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -22,6 +22,7 @@ use std::time::{Duration, SystemTime}; use anyhow::{anyhow, Context}; use async_trait::async_trait; +use cluster_limit_service_client::ClusterLimitServiceClient; use either::Either; use futures::stream::BoxStream; use lru::LruCache; @@ -1436,6 +1437,14 @@ impl MetaClient { let resp = self.inner.get_version_by_epoch(req).await?; Ok(resp.version.unwrap()) } + + pub async fn get_cluster_limits( + &self, + ) -> Result> { + let req = GetClusterLimitsRequest {}; + let resp = self.inner.get_cluster_limits(req).await?; + Ok(resp.active_limits.into_iter().map(|l| l.into()).collect()) + } } #[async_trait] @@ -1636,6 +1645,7 @@ struct GrpcMetaClientCore { cloud_client: CloudServiceClient, sink_coordinate_client: SinkCoordinationRpcClient, event_log_client: EventLogServiceClient, + cluster_limit_client: ClusterLimitServiceClient, } impl GrpcMetaClientCore { @@ -1662,7 +1672,8 @@ impl GrpcMetaClientCore { let serving_client = ServingServiceClient::new(channel.clone()); let cloud_client = CloudServiceClient::new(channel.clone()); let sink_coordinate_client = SinkCoordinationServiceClient::new(channel.clone()); - let event_log_client = EventLogServiceClient::new(channel); + let event_log_client = EventLogServiceClient::new(channel.clone()); + let cluster_limit_client = ClusterLimitServiceClient::new(channel); GrpcMetaClientCore { cluster_client, @@ -1682,6 +1693,7 @@ impl GrpcMetaClientCore { cloud_client, sink_coordinate_client, event_log_client, + cluster_limit_client, } } } @@ -2126,6 +2138,7 @@ macro_rules! for_all_meta_rpc { ,{ cloud_client, rw_cloud_validate_source, RwCloudValidateSourceRequest, RwCloudValidateSourceResponse } ,{ event_log_client, list_event_log, ListEventLogRequest, ListEventLogResponse } ,{ event_log_client, add_event_log, AddEventLogRequest, AddEventLogResponse } + ,{ cluster_limit_client, get_cluster_limits, GetClusterLimitsRequest, GetClusterLimitsResponse } } }; } diff --git a/src/rpc_client/src/stream_client.rs b/src/rpc_client/src/stream_client.rs index 920b6f0777f37..40a6d48dacb37 100644 --- a/src/rpc_client/src/stream_client.rs +++ b/src/rpc_client/src/stream_client.rs @@ -70,8 +70,7 @@ pub type StreamClientPoolRef = Arc; macro_rules! for_all_stream_rpc { ($macro:ident) => { $macro! { - { 0, drop_actors, DropActorsRequest, DropActorsResponse } - ,{ 0, wait_epoch_commit, WaitEpochCommitRequest, WaitEpochCommitResponse } + { 0, wait_epoch_commit, WaitEpochCommitRequest, WaitEpochCommitResponse } } }; } diff --git a/src/storage/backup/integration_tests/test_basic.sh b/src/storage/backup/integration_tests/test_basic.sh index afaee3ac6c507..9674807e62c6e 100644 --- a/src/storage/backup/integration_tests/test_basic.sh +++ b/src/storage/backup/integration_tests/test_basic.sh @@ -34,12 +34,20 @@ if ! psql -h localhost -p 4566 -d dev -U root -c "show materialized views;" | gr echo "expect 0 MV" exit 1 fi +if ! psql -h localhost -p 4566 -d dev -U root -c "show secrets;" | grep -q "0 row"; then + echo "expect 0 SECRET" + exit 1 +fi echo "restore snapshot ${job_id_1} succeeded" restore "${job_id_2}" start_cluster if ! psql -h localhost -p 4566 -d dev -U root -c "show materialized views;" | grep -q "1 row"; then - echo "expect 1 MVs" + echo "expect 1 MV" + exit 1 +fi +if ! psql -h localhost -p 4566 -d dev -U root -c "show secrets;" | grep -q "1 row"; then + echo "expect 1 SECRET" exit 1 fi echo "restore snapshot ${job_id_2} succeeded" @@ -55,6 +63,10 @@ if ! psql -h localhost -p 4566 -d dev -U root -c "show materialized views;" | gr echo "expect 0 MV" exit 1 fi +if ! psql -h localhost -p 4566 -d dev -U root -c "show secrets;" | grep -q "0 row"; then + echo "expect 0 SECRET" + exit 1 +fi echo "restore snapshot ${job_id_3} succeeded" echo "test succeeded" 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 ca6585f46fd51..c54dd05b25d28 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 @@ -354,7 +354,7 @@ impl HummockVersion { &mut self, parent_group_id: CompactionGroupId, group_id: CompactionGroupId, - member_table_ids: HashSet, + member_table_ids: BTreeSet, new_sst_start_id: u64, ) { let mut new_sst_id = new_sst_start_id; @@ -594,7 +594,7 @@ impl HummockVersion { } else { #[expect(deprecated)] // for backward-compatibility of previous hummock version delta - HashSet::from_iter(group_construct.table_ids.clone()) + BTreeSet::from_iter(group_construct.table_ids.clone()) }; self.init_with_parent_group( @@ -614,7 +614,7 @@ impl HummockVersion { self.init_with_parent_group( group_change.origin_group_id, group_change.target_group_id, - HashSet::from_iter(group_change.table_ids.clone()), + BTreeSet::from_iter(group_change.table_ids.clone()), group_change.new_sst_start_id, ); @@ -998,7 +998,7 @@ pub fn build_initial_compaction_group_levels( } fn split_sst_info_for_level( - member_table_ids: &HashSet, + member_table_ids: &BTreeSet, level: &mut Level, new_sst_id: &mut u64, ) -> Vec { @@ -1338,21 +1338,25 @@ pub fn split_sst( new_sst_id: &mut u64, old_sst_size: u64, new_sst_size: u64, - new_sst_table_ids: Vec, + new_table_ids: Vec, ) -> SstableInfo { let mut branch_table_info = sst_info.clone(); branch_table_info.sst_id = *new_sst_id; branch_table_info.sst_size = new_sst_size; + *new_sst_id += 1; - sst_info.sst_id = *new_sst_id + 1; + sst_info.sst_id = *new_sst_id; sst_info.sst_size = old_sst_size; + *new_sst_id += 1; { // related github.com/risingwavelabs/risingwave/pull/17898/ // This is a temporary implementation that will update `table_ids`` based on the new split rule after PR 17898 - - let set1: HashSet<_> = sst_info.table_ids.iter().cloned().collect(); - let set2: HashSet<_> = new_sst_table_ids.iter().cloned().collect(); + // sst_info.table_ids = vec[1, 2, 3]; + // new_table_ids = vec[2, 3, 4]; + // branch_table_info.table_ids = vec[1, 2, 3] ∩ vec[2, 3, 4] = vec[2, 3] + let set1: BTreeSet<_> = sst_info.table_ids.iter().cloned().collect(); + let set2: BTreeSet<_> = new_table_ids.into_iter().collect(); let intersection: Vec<_> = set1.intersection(&set2).cloned().collect(); // Update table_ids @@ -1362,8 +1366,6 @@ pub fn split_sst( .retain(|table_id| !branch_table_info.table_ids.contains(table_id)); } - *new_sst_id += 1; - branch_table_info } diff --git a/src/storage/hummock_sdk/src/sstable_info.rs b/src/storage/hummock_sdk/src/sstable_info.rs index 2f64508e57314..20943e4dd101a 100644 --- a/src/storage/hummock_sdk/src/sstable_info.rs +++ b/src/storage/hummock_sdk/src/sstable_info.rs @@ -63,6 +63,7 @@ impl SstableInfo { impl From for SstableInfo { fn from(pb_sstable_info: PbSstableInfo) -> Self { + assert!(pb_sstable_info.table_ids.is_sorted()); Self { object_id: pb_sstable_info.object_id, sst_id: pb_sstable_info.sst_id, @@ -100,6 +101,7 @@ impl From for SstableInfo { impl From<&PbSstableInfo> for SstableInfo { fn from(pb_sstable_info: &PbSstableInfo) -> Self { + assert!(pb_sstable_info.table_ids.is_sorted()); Self { object_id: pb_sstable_info.object_id, sst_id: pb_sstable_info.sst_id, @@ -136,7 +138,8 @@ impl From<&PbSstableInfo> for SstableInfo { impl From for PbSstableInfo { fn from(sstable_info: SstableInfo) -> Self { - assert_ne!(0, sstable_info.sst_size); + assert!(sstable_info.sst_size > 0 || sstable_info.is_stripped()); + assert!(sstable_info.table_ids.is_sorted()); PbSstableInfo { object_id: sstable_info.object_id, sst_id: sstable_info.sst_id, @@ -174,7 +177,8 @@ impl From for PbSstableInfo { impl From<&SstableInfo> for PbSstableInfo { fn from(sstable_info: &SstableInfo) -> Self { - assert_ne!(0, sstable_info.sst_size); + assert!(sstable_info.sst_size > 0 || sstable_info.is_stripped()); + assert!(sstable_info.table_ids.is_sorted()); PbSstableInfo { object_id: sstable_info.object_id, sst_id: sstable_info.sst_id, @@ -212,3 +216,10 @@ impl SstableInfo { self.key_range = KeyRange::default(); } } + +// Time travel +impl SstableInfo { + pub fn is_stripped(&self) -> bool { + self.object_id == 0 + } +} diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 7f3d35f16b80b..fc0fd6ae97b4f 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -31,6 +31,7 @@ use risingwave_hummock_sdk::key::{ gen_key_from_bytes, prefixed_range_with_vnode, FullKey, TableKey, UserKey, TABLE_PREFIX_LEN, }; use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::table_stats::TableStats; use risingwave_hummock_sdk::table_watermark::{ TableWatermarksIndex, VnodeWatermark, WatermarkDirection, }; @@ -2510,8 +2511,20 @@ async fn test_commit_multi_epoch() { new_table_watermarks: Default::default(), sst_to_context: context_id_map(&[sst.object_id]), sstables: vec![LocalSstableInfo { + table_stats: sst + .table_ids + .iter() + .map(|&table_id| { + ( + table_id, + TableStats { + total_compressed_size: 10, + ..Default::default() + }, + ) + }) + .collect(), sst_info: sst, - table_stats: Default::default(), }], new_table_fragment_info, change_log_delta: Default::default(), diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 35f3d08a9ed8a..67da2150735af 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -24,7 +24,6 @@ use futures::{pin_mut, StreamExt}; use itertools::Itertools; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::{TableId, TableOption}; -use risingwave_common::hash::table_distribution::TableDistribution; use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::{test_epoch, EpochExt, MAX_EPOCH}; use risingwave_hummock_sdk::key::{prefixed_range_with_vnode, TableKeyRange}; @@ -1565,7 +1564,7 @@ async fn test_iter_log() { }, table_option: Default::default(), is_replicated: false, - vnodes: TableDistribution::all_vnodes(), + vnodes: Bitmap::ones(VirtualNode::COUNT_FOR_TEST).into(), }) .await; @@ -1580,7 +1579,7 @@ async fn test_iter_log() { }, table_option: Default::default(), is_replicated: false, - vnodes: TableDistribution::all_vnodes(), + vnodes: Bitmap::ones(VirtualNode::COUNT_FOR_TEST).into(), }) .await; // flush for about 10 times per epoch diff --git a/src/storage/src/hummock/iterator/change_log.rs b/src/storage/src/hummock/iterator/change_log.rs index 6fc99f29a80f3..ae8061c37b07d 100644 --- a/src/storage/src/hummock/iterator/change_log.rs +++ b/src/storage/src/hummock/iterator/change_log.rs @@ -527,8 +527,9 @@ mod tests { use bytes::Bytes; use itertools::Itertools; + use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::TableId; - use risingwave_common::hash::table_distribution::TableDistribution; + use risingwave_common::hash::VirtualNode; use risingwave_common::util::epoch::test_epoch; use risingwave_hummock_sdk::key::{TableKey, UserKey}; use risingwave_hummock_sdk::EpochWithGap; @@ -699,7 +700,7 @@ mod tests { }, table_option: Default::default(), is_replicated: false, - vnodes: TableDistribution::all_vnodes(), + vnodes: Bitmap::ones(VirtualNode::COUNT_FOR_TEST).into(), }) .await; let logs = gen_test_data(epoch_count, 10000, 0.05, 0.2); diff --git a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs index 92a3caf4cd2e3..17ab103d758b4 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/serde.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/serde.rs @@ -25,7 +25,7 @@ use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::ColumnDesc; -use risingwave_common::hash::VirtualNode; +use risingwave_common::hash::{VirtualNode, VnodeBitmapExt}; use risingwave_common::row::{OwnedRow, Row, RowExt}; use risingwave_common::types::{DataType, ScalarImpl}; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -42,7 +42,7 @@ use risingwave_storage::error::StorageResult; use risingwave_storage::row_serde::row_serde_util::{serialize_pk, serialize_pk_with_vnode}; use risingwave_storage::row_serde::value_serde::ValueRowSerdeNew; use risingwave_storage::store::{StateStoreIterExt, StateStoreReadIter}; -use risingwave_storage::table::{compute_vnode, TableDistribution, SINGLETON_VNODE}; +use risingwave_storage::table::{compute_vnode, SINGLETON_VNODE}; use rw_futures_util::select_all; use crate::common::log_store_impl::kv_log_store::{ @@ -201,8 +201,7 @@ impl LogStoreRowSerde { let vnodes = match vnodes { Some(vnodes) => vnodes, - - None => TableDistribution::singleton_vnode_bitmap(), + None => Bitmap::singleton().into(), }; // epoch and seq_id. The seq_id of barrier is set null, and therefore the second order type @@ -216,7 +215,7 @@ impl LogStoreRowSerde { ); let dist_key_indices = if dist_key_indices.is_empty() { - if &vnodes != TableDistribution::singleton_vnode_bitmap_ref() { + if !vnodes.is_singleton() { warn!( ?vnodes, "singleton log store gets non-singleton vnode bitmap" @@ -946,7 +945,7 @@ mod tests { use risingwave_storage::store::{ FromStreamStateStoreIter, StateStoreIterItem, StateStoreReadIter, }; - use risingwave_storage::table::DEFAULT_VNODE; + use risingwave_storage::table::SINGLETON_VNODE; use tokio::sync::oneshot; use tokio::sync::oneshot::Sender; @@ -1024,7 +1023,7 @@ mod tests { seq_id += 1; } - let (key, encoded_barrier) = serde.serialize_barrier(epoch, DEFAULT_VNODE, false); + let (key, encoded_barrier) = serde.serialize_barrier(epoch, SINGLETON_VNODE, false); let key = remove_vnode_prefix(&key.0); match serde.deserialize(&encoded_barrier).unwrap() { (decoded_epoch, LogStoreRowOp::Barrier { is_checkpoint }) => { @@ -1062,7 +1061,8 @@ mod tests { seq_id += 1; } - let (key, encoded_checkpoint_barrier) = serde.serialize_barrier(epoch, DEFAULT_VNODE, true); + let (key, encoded_checkpoint_barrier) = + serde.serialize_barrier(epoch, SINGLETON_VNODE, true); let key = remove_vnode_prefix(&key.0); match serde.deserialize(&encoded_checkpoint_barrier).unwrap() { (decoded_epoch, LogStoreRowOp::Barrier { is_checkpoint }) => { @@ -1200,7 +1200,7 @@ mod tests { ) { let (ops, rows) = gen_test_data(base); let first_barrier = { - let (key, value) = serde.serialize_barrier(EPOCH0, DEFAULT_VNODE, true); + let (key, value) = serde.serialize_barrier(EPOCH0, SINGLETON_VNODE, true); Ok((FullKey::new(TEST_TABLE_ID, key, EPOCH0), value)) }; let stream = stream::once(async move { first_barrier }); @@ -1210,7 +1210,7 @@ mod tests { let stream = stream.chain(stream::once({ let serde = serde.clone(); async move { - let (key, value) = serde.serialize_barrier(EPOCH1, DEFAULT_VNODE, false); + let (key, value) = serde.serialize_barrier(EPOCH1, SINGLETON_VNODE, false); Ok((FullKey::new(TEST_TABLE_ID, key, EPOCH1), value)) } })); @@ -1218,7 +1218,7 @@ mod tests { gen_row_stream(serde.clone(), ops.clone(), rows.clone(), EPOCH2, seq_id); let stream = stream.chain(row_stream).chain(stream::once({ async move { - let (key, value) = serde.serialize_barrier(EPOCH2, DEFAULT_VNODE, true); + let (key, value) = serde.serialize_barrier(EPOCH2, SINGLETON_VNODE, true); Ok((FullKey::new(TEST_TABLE_ID, key, EPOCH2), value)) } })); diff --git a/src/stream/src/common/log_store_impl/kv_log_store/test_utils.rs b/src/stream/src/common/log_store_impl/kv_log_store/test_utils.rs index 5fc10cd0cc58a..3114c22e63323 100644 --- a/src/stream/src/common/log_store_impl/kv_log_store/test_utils.rs +++ b/src/stream/src/common/log_store_impl/kv_log_store/test_utils.rs @@ -143,7 +143,7 @@ pub(crate) fn gen_multi_vnode_stream_chunks( .collect_vec(); let (ops, rows) = gen_sized_test_data(base, max_count); for (op, row) in zip_eq(ops, rows) { - let vnode = VirtualNode::compute_row(&row, &[TEST_SCHEMA_DIST_KEY_INDEX]); + let vnode = VirtualNode::compute_row_for_test(&row, &[TEST_SCHEMA_DIST_KEY_INDEX]); let (ops, builder) = &mut data_builder[vnode.to_index() % MOD_COUNT]; ops.push(op); assert!(builder.append_one_row(row).is_none()); @@ -177,9 +177,9 @@ pub(crate) fn gen_test_log_store_table(pk_info: &'static KvLogStorePkInfo) -> Pb pub(crate) fn calculate_vnode_bitmap<'a>( test_data: impl Iterator)>, ) -> Bitmap { - let mut builder = BitmapBuilder::zeroed(VirtualNode::COUNT); - for vnode in - test_data.map(|(_, row)| VirtualNode::compute_row(row, &[TEST_SCHEMA_DIST_KEY_INDEX])) + let mut builder = BitmapBuilder::zeroed(VirtualNode::COUNT_FOR_TEST); + for vnode in test_data + .map(|(_, row)| VirtualNode::compute_row_for_test(row, &[TEST_SCHEMA_DIST_KEY_INDEX])) { builder.set(vnode.to_index(), true); } diff --git a/src/stream/src/common/table/test_state_table.rs b/src/stream/src/common/table/test_state_table.rs index 098548c21ac93..dde0d8a581406 100644 --- a/src/stream/src/common/table/test_state_table.rs +++ b/src/stream/src/common/table/test_state_table.rs @@ -27,7 +27,7 @@ use risingwave_common::util::value_encoding::BasicSerde; use risingwave_hummock_test::test_utils::prepare_hummock_test_env; use risingwave_storage::hummock::HummockStorage; use risingwave_storage::store::PrefetchOptions; -use risingwave_storage::table::DEFAULT_VNODE; +use risingwave_storage::table::SINGLETON_VNODE; use crate::common::table::state_table::{ ReplicatedStateTable, StateTable, WatermarkCacheStateTable, @@ -445,7 +445,7 @@ async fn test_state_table_iter_with_pk_range() { std::ops::Bound::Included(OwnedRow::new(vec![Some(4_i32.into())])), ); let iter = state_table - .iter_with_vnode(DEFAULT_VNODE, &pk_range, Default::default()) + .iter_with_vnode(SINGLETON_VNODE, &pk_range, Default::default()) .await .unwrap(); pin_mut!(iter); @@ -470,7 +470,7 @@ async fn test_state_table_iter_with_pk_range() { std::ops::Bound::::Unbounded, ); let iter = state_table - .iter_with_vnode(DEFAULT_VNODE, &pk_range, Default::default()) + .iter_with_vnode(SINGLETON_VNODE, &pk_range, Default::default()) .await .unwrap(); pin_mut!(iter); @@ -1976,11 +1976,11 @@ async fn test_replicated_state_table_replication() { std::ops::Bound::Included(OwnedRow::new(vec![Some(2_i32.into())])), ); let iter = state_table - .iter_with_vnode(DEFAULT_VNODE, &range_bounds, Default::default()) + .iter_with_vnode(SINGLETON_VNODE, &range_bounds, Default::default()) .await .unwrap(); let replicated_iter = replicated_state_table - .iter_with_vnode_and_output_indices(DEFAULT_VNODE, &range_bounds, Default::default()) + .iter_with_vnode_and_output_indices(SINGLETON_VNODE, &range_bounds, Default::default()) .await .unwrap(); pin_mut!(iter); @@ -2039,7 +2039,7 @@ async fn test_replicated_state_table_replication() { ); let iter = state_table - .iter_with_vnode(DEFAULT_VNODE, &range_bounds, Default::default()) + .iter_with_vnode(SINGLETON_VNODE, &range_bounds, Default::default()) .await .unwrap(); @@ -2048,7 +2048,7 @@ async fn test_replicated_state_table_replication() { std::ops::Bound::Unbounded, ); let replicated_iter = replicated_state_table - .iter_with_vnode_and_output_indices(DEFAULT_VNODE, &range_bounds, Default::default()) + .iter_with_vnode_and_output_indices(SINGLETON_VNODE, &range_bounds, Default::default()) .await .unwrap(); pin_mut!(iter); @@ -2079,7 +2079,7 @@ async fn test_replicated_state_table_replication() { let range_bounds: (Bound, Bound) = (std::ops::Bound::Unbounded, std::ops::Bound::Unbounded); let replicated_iter = replicated_state_table - .iter_with_vnode_and_output_indices(DEFAULT_VNODE, &range_bounds, Default::default()) + .iter_with_vnode_and_output_indices(SINGLETON_VNODE, &range_bounds, Default::default()) .await .unwrap(); pin_mut!(replicated_iter); diff --git a/src/stream/src/executor/dispatch.rs b/src/stream/src/executor/dispatch.rs index 82d11db49513b..0fc9da0e5ab23 100644 --- a/src/stream/src/executor/dispatch.rs +++ b/src/stream/src/executor/dispatch.rs @@ -755,7 +755,8 @@ impl Dispatcher for HashDataDispatcher { let num_outputs = self.outputs.len(); // get hash value of every line by its key - let vnodes = VirtualNode::compute_chunk(chunk.data_chunk(), &self.keys); + let vnode_count = self.hash_mapping.len(); + let vnodes = VirtualNode::compute_chunk(chunk.data_chunk(), &self.keys, vnode_count); tracing::debug!(target: "events::stream::dispatch::hash", "\n{}\n keys {:?} => {:?}", chunk.to_pretty(), self.keys, vnodes); @@ -1225,6 +1226,32 @@ mod tests { ) .unwrap(); + let dispatcher_updates = maplit::hashmap! { + actor_id => vec![PbDispatcherUpdate { + actor_id, + dispatcher_id: broadcast_dispatcher_id, + added_downstream_actor_id: vec![new], + removed_downstream_actor_id: vec![old], + hash_mapping: Default::default(), + }] + }; + let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update( + UpdateMutation { + dispatchers: dispatcher_updates, + merges: Default::default(), + vnode_bitmaps: Default::default(), + dropped_actors: Default::default(), + actor_splits: Default::default(), + actor_new_dispatchers: Default::default(), + }, + )); + barrier_test_env.inject_barrier(&b1, [actor_id]); + barrier_test_env + .shared_context + .local_barrier_manager + .flush_all_events() + .await; + let executor = Box::new(DispatchExecutor::new( input, vec![broadcast_dispatcher, simple_dispatcher], @@ -1253,27 +1280,6 @@ mod tests { .await .unwrap(); - // 4. Send a configuration change barrier for broadcast dispatcher. - let dispatcher_updates = maplit::hashmap! { - actor_id => vec![PbDispatcherUpdate { - actor_id, - dispatcher_id: broadcast_dispatcher_id, - added_downstream_actor_id: vec![new], - removed_downstream_actor_id: vec![old], - hash_mapping: Default::default(), - }] - }; - let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update( - UpdateMutation { - dispatchers: dispatcher_updates, - merges: Default::default(), - vnode_bitmaps: Default::default(), - dropped_actors: Default::default(), - actor_splits: Default::default(), - actor_new_dispatchers: Default::default(), - }, - )); - barrier_test_env.inject_barrier(&b1, [actor_id]); tx.send(Message::Barrier(b1.clone().into_dispatcher())) .await .unwrap(); diff --git a/src/stream/src/executor/error.rs b/src/stream/src/executor/error.rs index fa625d8bb8cec..66070ba81e90c 100644 --- a/src/stream/src/executor/error.rs +++ b/src/stream/src/executor/error.rs @@ -67,7 +67,12 @@ pub enum ErrorKind { ), #[error("Sink error: sink_id={1}, error: {0}")] - SinkError(SinkError, u32), + SinkError( + #[source] + #[backtrace] + SinkError, + u32, + ), #[error(transparent)] RpcError( @@ -90,7 +95,11 @@ pub enum ErrorKind { AlignBarrier(Box, Box), #[error("Connector error: {0}")] - ConnectorError(BoxedError), + ConnectorError( + #[source] + #[backtrace] + BoxedError, + ), #[error(transparent)] DmlError( diff --git a/src/stream/src/executor/integration_tests.rs b/src/stream/src/executor/integration_tests.rs index d65abc5a5ce53..13e9a67d1c525 100644 --- a/src/stream/src/executor/integration_tests.rs +++ b/src/stream/src/executor/integration_tests.rs @@ -14,6 +14,8 @@ use std::sync::Mutex; +use futures::future::BoxFuture; +use futures::FutureExt; use futures_async_stream::try_stream; use multimap::MultiMap; use risingwave_common::array::*; @@ -100,7 +102,7 @@ async fn test_merger_sum_aggr() { }; // join handles of all actors - let mut handles = vec![]; + let mut actor_futures: Vec> = vec![]; // input and output channels of the local aggregation actors let mut inputs = vec![]; @@ -113,7 +115,7 @@ async fn test_merger_sum_aggr() { let (tx, rx) = channel_for_test(); let (actor, channel) = make_actor(rx); outputs.push(channel); - handles.push(tokio::spawn(actor.run())); + actor_futures.push(actor.run().boxed()); inputs.push(Box::new(LocalOutput::new(233, tx)) as BoxedOutput); } @@ -154,7 +156,7 @@ async fn test_merger_sum_aggr() { .local_barrier_manager .clone(), ); - handles.push(tokio::spawn(actor.run())); + actor_futures.push(actor.run().boxed()); let actor_ctx = ActorContext::for_test(gen_next_actor_id()); @@ -225,11 +227,21 @@ async fn test_merger_sum_aggr() { .local_barrier_manager .clone(), ); - handles.push(tokio::spawn(actor.run())); + actor_futures.push(actor.run().boxed()); let mut epoch = test_epoch(1); let b1 = Barrier::new_test_barrier(epoch); barrier_test_env.inject_barrier(&b1, actors.clone()); + barrier_test_env + .shared_context + .local_barrier_manager + .flush_all_events() + .await; + let handles = actor_futures + .into_iter() + .map(|actor_future| tokio::spawn(actor_future)) + .collect_vec(); + input .send(Message::Barrier(b1.into_dispatcher())) .await diff --git a/src/stream/src/executor/merge.rs b/src/stream/src/executor/merge.rs index 769c3c5bf8715..1f33830e2a576 100644 --- a/src/stream/src/executor/merge.rs +++ b/src/stream/src/executor/merge.rs @@ -627,6 +627,11 @@ mod tests { let b2 = Barrier::with_prev_epoch_for_test(test_epoch(1000), *prev_epoch) .with_mutation(Mutation::Stop(HashSet::default())); barrier_test_env.inject_barrier(&b2, [actor_id]); + barrier_test_env + .shared_context + .local_barrier_manager + .flush_all_events() + .await; for (tx_id, tx) in txs.into_iter().enumerate() { let epochs = epochs.clone(); @@ -730,6 +735,33 @@ mod tests { .try_collect() .unwrap(); + let merge_updates = maplit::hashmap! { + (actor_id, upstream_fragment_id) => MergeUpdate { + actor_id, + upstream_fragment_id, + new_upstream_fragment_id: None, + added_upstream_actor_id: vec![new], + removed_upstream_actor_id: vec![old], + } + }; + + let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update( + UpdateMutation { + dispatchers: Default::default(), + merges: merge_updates, + vnode_bitmaps: Default::default(), + dropped_actors: Default::default(), + actor_splits: Default::default(), + actor_new_dispatchers: Default::default(), + }, + )); + barrier_test_env.inject_barrier(&b1, [actor_id]); + barrier_test_env + .shared_context + .local_barrier_manager + .flush_all_events() + .await; + let barrier_rx = ctx.local_barrier_manager.subscribe_barrier(actor_id); let actor_ctx = ActorContext::for_test(actor_id); let upstream = MergeExecutor::new_select_receiver(inputs, &metrics, &actor_ctx); @@ -782,28 +814,6 @@ mod tests { recv!().unwrap().as_chunk().unwrap(); assert_recv_pending!(); - // 4. Send a configuration change barrier. - let merge_updates = maplit::hashmap! { - (actor_id, upstream_fragment_id) => MergeUpdate { - actor_id, - upstream_fragment_id, - new_upstream_fragment_id: None, - added_upstream_actor_id: vec![new], - removed_upstream_actor_id: vec![old], - } - }; - - let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update( - UpdateMutation { - dispatchers: Default::default(), - merges: merge_updates, - vnode_bitmaps: Default::default(), - dropped_actors: Default::default(), - actor_splits: Default::default(), - actor_new_dispatchers: Default::default(), - }, - )); - barrier_test_env.inject_barrier(&b1, [actor_id]); send!( [untouched, old], Message::Barrier(b1.clone().into_dispatcher()) diff --git a/src/stream/src/executor/receiver.rs b/src/stream/src/executor/receiver.rs index bd1ebaffb1f3f..c3fd4f9f7e7e2 100644 --- a/src/stream/src/executor/receiver.rs +++ b/src/stream/src/executor/receiver.rs @@ -237,6 +237,35 @@ mod tests { let (upstream_fragment_id, fragment_id) = (10, 18); + // 4. Send a configuration change barrier. + let merge_updates = maplit::hashmap! { + (actor_id, upstream_fragment_id) => MergeUpdate { + actor_id, + upstream_fragment_id, + new_upstream_fragment_id: None, + added_upstream_actor_id: vec![new], + removed_upstream_actor_id: vec![old], + } + }; + + let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update( + UpdateMutation { + dispatchers: Default::default(), + merges: merge_updates, + vnode_bitmaps: Default::default(), + dropped_actors: Default::default(), + actor_splits: Default::default(), + actor_new_dispatchers: Default::default(), + }, + )); + + barrier_test_env.inject_barrier(&b1, [actor_id]); + barrier_test_env + .shared_context + .local_barrier_manager + .flush_all_events() + .await; + let input = new_input( &ctx, metrics.clone(), @@ -303,30 +332,6 @@ mod tests { recv!().unwrap().as_chunk().unwrap(); // We should be able to receive the chunk. assert_recv_pending!(); - // 4. Send a configuration change barrier. - let merge_updates = maplit::hashmap! { - (actor_id, upstream_fragment_id) => MergeUpdate { - actor_id, - upstream_fragment_id, - new_upstream_fragment_id: None, - added_upstream_actor_id: vec![new], - removed_upstream_actor_id: vec![old], - } - }; - - let b1 = Barrier::new_test_barrier(test_epoch(1)).with_mutation(Mutation::Update( - UpdateMutation { - dispatchers: Default::default(), - merges: merge_updates, - vnode_bitmaps: Default::default(), - dropped_actors: Default::default(), - actor_splits: Default::default(), - actor_new_dispatchers: Default::default(), - }, - )); - - barrier_test_env.inject_barrier(&b1, [actor_id]); - send!([new], Message::Barrier(b1.clone().into_dispatcher())); assert_recv_pending!(); // We should not receive the barrier, as new is not the upstream. diff --git a/src/stream/src/executor/source/source_backfill_executor.rs b/src/stream/src/executor/source/source_backfill_executor.rs index b28c707bdedd0..09a4d0a40f1cb 100644 --- a/src/stream/src/executor/source/source_backfill_executor.rs +++ b/src/stream/src/executor/source/source_backfill_executor.rs @@ -14,6 +14,7 @@ use std::cmp::Ordering; use std::collections::{HashMap, HashSet}; +use std::sync::Once; use std::time::Instant; use anyhow::anyhow; @@ -30,6 +31,7 @@ use risingwave_connector::source::{ BackfillInfo, BoxChunkSourceStream, SourceContext, SourceCtrlOpts, SplitId, SplitImpl, SplitMetaData, }; +use risingwave_hummock_sdk::HummockReadEpoch; use serde::{Deserialize, Serialize}; use thiserror_ext::AsReport; @@ -40,6 +42,7 @@ use crate::common::rate_limit::limited_chunk_size; use crate::executor::prelude::*; use crate::executor::source::source_executor::WAIT_BARRIER_MULTIPLE_TIMES; use crate::executor::UpdateMutation; +use crate::task::CreateMviewProgress; #[derive(Clone, Debug, Deserialize, Serialize, PartialEq)] pub enum BackfillState { @@ -88,6 +91,8 @@ pub struct SourceBackfillExecutorInner { /// Rate limit in rows/s. rate_limit_rps: Option, + + progress: CreateMviewProgress, } /// Local variables used in the backfill stage. @@ -230,6 +235,7 @@ impl BackfillStage { } impl SourceBackfillExecutorInner { + #[expect(clippy::too_many_arguments)] pub fn new( actor_ctx: ActorContextRef, info: ExecutorInfo, @@ -238,6 +244,7 @@ impl SourceBackfillExecutorInner { system_params: SystemParamsReaderRef, backfill_state_store: BackfillStateTableHandler, rate_limit_rps: Option, + progress: CreateMviewProgress, ) -> Self { let source_split_change_count = metrics .source_split_change_count @@ -247,6 +254,7 @@ impl SourceBackfillExecutorInner { &actor_ctx.id.to_string(), &actor_ctx.fragment_id.to_string(), ]); + Self { actor_ctx, info, @@ -256,6 +264,7 @@ impl SourceBackfillExecutorInner { source_split_change_count, system_params, rate_limit_rps, + progress, } } @@ -346,7 +355,6 @@ impl SourceBackfillExecutorInner { splits: owned_splits, }; backfill_stage.debug_assert_consistent(); - tracing::debug!(?backfill_stage, "source backfill started"); // Return the ownership of `stream_source_core` to the source executor. self.stream_source_core = core; @@ -370,6 +378,7 @@ impl SourceBackfillExecutorInner { } } } + tracing::debug!(?backfill_stage, "source backfill started"); fn select_strategy(_: &mut ()) -> PollNext { futures::stream::PollNext::Left @@ -407,9 +416,23 @@ impl SourceBackfillExecutorInner { pause_reader!(); } + let state_store = self.backfill_state_store.state_store.state_store().clone(); + static STATE_TABLE_INITIALIZED: Once = Once::new(); + tokio::spawn(async move { + // This is for self.backfill_finished() to be safe. + // We wait for 1st epoch's curr, i.e., the 2nd epoch's prev. + let epoch = barrier.epoch.curr; + tracing::info!("waiting for epoch: {}", epoch); + state_store + .try_wait_epoch(HummockReadEpoch::Committed(epoch)) + .await + .expect("failed to wait epoch"); + STATE_TABLE_INITIALIZED.call_once(|| ()); + tracing::info!("finished waiting for epoch: {}", epoch); + }); yield Message::Barrier(barrier); - if !self.backfill_finished(&backfill_stage.states).await? { + { let source_backfill_row_count = self .metrics .source_backfill_row_count @@ -552,10 +575,26 @@ impl SourceBackfillExecutorInner { .commit(barrier.epoch) .await?; - yield Message::Barrier(barrier); - - if self.backfill_finished(&backfill_stage.states).await? { - break 'backfill_loop; + if self.should_report_finished(&backfill_stage.states) { + // TODO: use a specialized progress for source + // Currently, `CreateMviewProgress` is designed for MV backfill, and rw_ddl_progress calculates + // 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::info!("progress finish"); + let epoch = barrier.epoch; + self.progress.finish(epoch, 114514); + // yield barrier after reporting progress + yield Message::Barrier(barrier); + + // After we reported finished, we still don't exit the loop. + // Because we need to handle split migration. + if STATE_TABLE_INITIALIZED.is_completed() + && self.backfill_finished(&backfill_stage.states).await? + { + break 'backfill_loop; + } + } else { + yield Message::Barrier(barrier); } } Message::Chunk(chunk) => { @@ -665,7 +704,7 @@ impl SourceBackfillExecutorInner { self.apply_split_change_forward_stage( actor_splits, &mut splits, - true, + false, ) .await?; } @@ -688,11 +727,34 @@ impl SourceBackfillExecutorInner { } } - /// All splits finished backfilling. + /// When we should call `progress.finish()` to let blocking DDL return. + /// We report as soon as `SourceCachingUp`. Otherwise the DDL might be blocked forever until upstream messages come. + /// + /// Note: split migration (online scaling) is related with progress tracking. + /// - For foreground DDL, scaling is not allowed before progress is finished. + /// - For background DDL, scaling is skipped when progress is not finished, and can be triggered by recreating actors during recovery. + /// + /// See for more details. + fn should_report_finished(&self, states: &BackfillStates) -> bool { + states.values().all(|state| { + matches!( + state, + BackfillState::Finished | BackfillState::SourceCachingUp(_) + ) + }) + } + + /// All splits entered `Finished` state. /// /// We check all splits for the source, including other actors' splits here, before going to the forward stage. - /// Otherwise if we break early, but after rescheduling, an unfinished split is migrated to + /// Otherwise if we `break` early, but after rescheduling, an unfinished split is migrated to /// this actor, we still need to backfill it. + /// + /// Note: at the beginning, the actor will only read the state written by itself. + /// It needs to _wait until it can read all actors' written data_. + /// i.e., wait for the first checkpoint has been available. + /// + /// See for more details. async fn backfill_finished(&self, states: &BackfillStates) -> StreamExecutorResult { Ok(states .values() @@ -761,7 +823,6 @@ impl SourceBackfillExecutorInner { } Some(backfill_state) => { // Migrated split. Backfill if unfinished. - // TODO: disallow online scaling during backfilling. target_state.insert(split_id, backfill_state); } } 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 be9abe8490e63..3579aff2ec4fb 100644 --- a/src/stream/src/executor/source/source_backfill_state_table.rs +++ b/src/stream/src/executor/source/source_backfill_state_table.rs @@ -76,6 +76,7 @@ impl BackfillStateTableHandler { }; ret.push(state); } + tracing::trace!("scan SourceBackfill state table: {:?}", ret); Ok(ret) } diff --git a/src/stream/src/executor/watermark_filter.rs b/src/stream/src/executor/watermark_filter.rs index 8f8b166626d21..01497c37fdab5 100644 --- a/src/stream/src/executor/watermark_filter.rs +++ b/src/stream/src/executor/watermark_filter.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::cmp; -use std::ops::Deref; use futures::future::{try_join, try_join_all}; use risingwave_common::hash::VnodeBitmapExt; @@ -27,7 +26,6 @@ use risingwave_expr::Result as ExprResult; use risingwave_hummock_sdk::HummockReadEpoch; use risingwave_pb::expr::expr_node::Type; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::TableDistribution; use super::filter::FilterExecutor; use crate::executor::prelude::*; @@ -219,10 +217,7 @@ impl WatermarkFilterExecutor { let mut need_update_global_max_watermark = false; // Update the vnode bitmap for state tables of all agg calls if asked. if let Some(vnode_bitmap) = barrier.as_update_vnode_bitmap(ctx.id) { - let other_vnodes_bitmap = Arc::new( - (!(*vnode_bitmap).clone()) - & TableDistribution::all_vnodes_ref().deref(), - ); + let other_vnodes_bitmap = Arc::new(!(*vnode_bitmap).clone()); let _ = global_watermark_table.update_vnode_bitmap(other_vnodes_bitmap); let (previous_vnode_bitmap, _cache_may_stale) = table.update_vnode_bitmap(vnode_bitmap.clone()); @@ -373,7 +368,9 @@ impl WatermarkFilterExecutor { #[cfg(test)] mod tests { use itertools::Itertools; + use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::{ColumnDesc, ColumnId, Field, TableDesc}; + use risingwave_common::hash::VirtualNode; use risingwave_common::test_prelude::StreamChunkTestExt; use risingwave_common::types::Date; use risingwave_common::util::epoch::test_epoch; @@ -431,7 +428,7 @@ mod tests { let state_table = StateTable::from_table_catalog_inconsistent_op( &table, mem_state.clone(), - Some(TableDistribution::all_vnodes()), + Some(Bitmap::ones(VirtualNode::COUNT_FOR_TEST).into()), ) .await; @@ -440,7 +437,7 @@ mod tests { let storage_table = StorageTable::new_partial( mem_state, val_indices.iter().map(|i| ColumnId::new(*i as _)).collect(), - Some(TableDistribution::all_vnodes()), + Some(Bitmap::ones(VirtualNode::COUNT_FOR_TEST).into()), &desc, ); (storage_table, state_table) diff --git a/src/stream/src/from_proto/mview.rs b/src/stream/src/from_proto/mview.rs index 41fc47609fba7..43fc929edf455 100644 --- a/src/stream/src/from_proto/mview.rs +++ b/src/stream/src/from_proto/mview.rs @@ -100,7 +100,7 @@ impl ExecutorBuilder for ArrangeExecutorBuilder { let table = node.get_table()?; // FIXME: Lookup is now implemented without cell-based table API and relies on all vnodes - // being `DEFAULT_VNODE`, so we need to make the Arrange a singleton. + // being `SINGLETON_VNODE`, so we need to make the Arrange a singleton. let vnodes = params.vnode_bitmap.map(Arc::new); let conflict_behavior = ConflictBehavior::from_protobuf(&table.handle_pk_conflict_behavior()); diff --git a/src/stream/src/from_proto/source_backfill.rs b/src/stream/src/from_proto/source_backfill.rs index ba3ab599af700..65329a26bd40b 100644 --- a/src/stream/src/from_proto/source_backfill.rs +++ b/src/stream/src/from_proto/source_backfill.rs @@ -72,6 +72,9 @@ impl ExecutorBuilder for SourceBackfillExecutorBuilder { source_desc_builder, state_table_handler, ); + let progress = params + .local_barrier_manager + .register_create_mview_progress(params.actor_context.id); let exec = SourceBackfillExecutorInner::new( params.actor_context.clone(), @@ -81,6 +84,7 @@ impl ExecutorBuilder for SourceBackfillExecutorBuilder { params.env.system_params_manager_ref().get_params(), backfill_state_table, node.rate_limit, + progress, ); let [input]: [_; 1] = params.input.try_into().unwrap(); diff --git a/src/stream/src/from_proto/watermark_filter.rs b/src/stream/src/from_proto/watermark_filter.rs index 0081f00cc39e6..4e3147d10853e 100644 --- a/src/stream/src/from_proto/watermark_filter.rs +++ b/src/stream/src/from_proto/watermark_filter.rs @@ -12,14 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::ops::Deref; use std::sync::Arc; use risingwave_common::catalog::{ColumnId, TableDesc}; use risingwave_expr::expr::build_non_strict_from_prost; use risingwave_pb::stream_plan::WatermarkFilterNode; use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::table::TableDistribution; use super::*; use crate::common::table::state_table::StateTable; @@ -57,8 +55,7 @@ impl ExecutorBuilder for WatermarkFilterBuilder { .iter() .map(|i| ColumnId::new(*i as _)) .collect_vec(); - let other_vnodes = - Arc::new((!(*vnodes).clone()) & TableDistribution::all_vnodes_ref().deref()); + let other_vnodes = Arc::new(!(*vnodes).clone()); let global_watermark_table = StorageTable::new_partial(store.clone(), column_ids, Some(other_vnodes), &desc); diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 699da24f0b6be..2af674738dcf2 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -37,8 +37,7 @@ use tonic::{Code, Status}; use self::managed_state::ManagedBarrierState; use crate::error::{IntoUnexpectedExit, StreamError, StreamResult}; use crate::task::{ - ActorHandle, ActorId, AtomicU64Ref, PartialGraphId, SharedContext, StreamEnvironment, - UpDownActorIds, + ActorId, AtomicU64Ref, PartialGraphId, SharedContext, StreamEnvironment, UpDownActorIds, }; mod managed_state; @@ -203,10 +202,6 @@ pub(super) enum LocalActorOperation { handle: ControlStreamHandle, init_request: InitRequest, }, - DropActors { - actors: Vec, - result_sender: oneshot::Sender<()>, - }, TakeReceiver { ids: UpDownActorIds, result_sender: oneshot::Sender>, @@ -221,29 +216,6 @@ pub(super) enum LocalActorOperation { }, } -pub(crate) struct StreamActorManagerState { - /// Each processor runs in a future. Upon receiving a `Terminate` message, they will exit. - /// `handles` store join handles of these futures, and therefore we could wait their - /// termination. - pub(super) handles: HashMap, - - /// Stores all actor information, taken after actor built. - pub(super) actors: HashMap, - - /// Stores all actor tokio runtime monitoring tasks. - pub(super) actor_monitor_tasks: HashMap, -} - -impl StreamActorManagerState { - fn new() -> Self { - Self { - handles: HashMap::new(), - actors: HashMap::new(), - actor_monitor_tasks: HashMap::new(), - } - } -} - pub(crate) struct StreamActorManager { pub(super) env: StreamEnvironment, pub(super) streaming_metrics: Arc, @@ -287,7 +259,7 @@ impl Display for LocalBarrierWorkerDebugInfo<'_> { /// barriers to and collect them from all actors, and finally report the progress. pub(super) struct LocalBarrierWorker { /// Current barrier collection state. - state: ManagedBarrierState, + pub(super) state: ManagedBarrierState, /// Record all unexpected exited actors. failure_actors: HashMap, @@ -296,8 +268,6 @@ pub(super) struct LocalBarrierWorker { pub(super) actor_manager: Arc, - pub(super) actor_manager_state: StreamActorManagerState, - pub(super) current_shared_context: Arc, barrier_event_rx: UnboundedReceiver, @@ -321,14 +291,9 @@ impl LocalBarrierWorker { )); Self { failure_actors: HashMap::default(), - state: ManagedBarrierState::new( - actor_manager.env.state_store(), - actor_manager.streaming_metrics.clone(), - actor_manager.await_tree_reg.clone(), - ), + state: ManagedBarrierState::new(actor_manager.clone(), shared_context.clone()), control_stream_handle: ControlStreamHandle::empty(), actor_manager, - actor_manager_state: StreamActorManagerState::new(), current_shared_context: shared_context, barrier_event_rx: event_rx, actor_failure_rx: failure_rx, @@ -338,7 +303,7 @@ impl LocalBarrierWorker { fn to_debug_info(&self) -> LocalBarrierWorkerDebugInfo<'_> { LocalBarrierWorkerDebugInfo { - running_actors: self.actor_manager_state.handles.keys().cloned().collect(), + running_actors: self.state.actor_states.keys().cloned().collect(), managed_barrier_state: self.state.to_debug_info(), has_control_stream_connected: self.control_stream_handle.connected(), } @@ -377,7 +342,7 @@ impl LocalBarrierWorker { }); } LocalActorOperation::Shutdown { result_sender } => { - if !self.actor_manager_state.handles.is_empty() { + if !self.state.actor_states.is_empty() { tracing::warn!( "shutdown with running actors, scaling or migration will be triggered" ); @@ -412,15 +377,9 @@ impl LocalBarrierWorker { Request::InjectBarrier(req) => { let barrier = Barrier::from_protobuf(req.get_barrier().unwrap())?; self.update_actor_info(req.broadcast_info)?; - let actors = req - .actors_to_build - .iter() - .map(|actor| actor.actor.as_ref().unwrap().actor_id) - .collect_vec(); - self.update_actors(req.actors_to_build)?; - self.start_create_actors(&actors)?; self.send_barrier( &barrier, + req.actors_to_build, req.actor_ids_to_collect.into_iter().collect(), req.table_ids_to_sync .into_iter() @@ -466,7 +425,13 @@ impl LocalBarrierWorker { .map_err(|e| (actor_id, e))?; } #[cfg(test)] - LocalBarrierEvent::Flush(sender) => sender.send(()).unwrap(), + LocalBarrierEvent::Flush(sender) => { + use futures::FutureExt; + while let Some(request) = self.control_stream_handle.next_request().now_or_never() { + self.handle_streaming_control_request(request).unwrap(); + } + sender.send(()).unwrap() + } } Ok(()) } @@ -476,13 +441,6 @@ impl LocalBarrierWorker { LocalActorOperation::NewControlStream { .. } | LocalActorOperation::Shutdown { .. } => { unreachable!("event {actor_op} should be handled separately in async context") } - LocalActorOperation::DropActors { - actors, - result_sender, - } => { - self.drop_actors(&actors); - let _ = result_sender.send(()); - } LocalActorOperation::TakeReceiver { ids, result_sender } => { let _ = result_sender.send(self.current_shared_context.take_receiver(ids)); } @@ -578,29 +536,11 @@ impl LocalBarrierWorker { fn send_barrier( &mut self, barrier: &Barrier, + to_build: Vec, to_collect: HashSet, table_ids: HashSet, partial_graph_id: PartialGraphId, ) -> StreamResult<()> { - if !cfg!(test) { - // The barrier might be outdated and been injected after recovery in some certain extreme - // scenarios. So some newly creating actors in the barrier are possibly not rebuilt during - // recovery. Check it here and return an error here if some actors are not found to - // avoid collection hang. We need some refine in meta side to remove this workaround since - // it will cause another round of unnecessary recovery. - let missing_actor_ids = to_collect - .iter() - .filter(|id| !self.actor_manager_state.handles.contains_key(id)) - .collect_vec(); - if !missing_actor_ids.is_empty() { - tracing::warn!( - "to collect actors not found, they should be cleaned when recovering: {:?}", - missing_actor_ids - ); - return Err(anyhow!("to collect actors not found: {:?}", to_collect).into()); - } - } - if barrier.kind == BarrierKind::Initial { self.actor_manager .watermark_epoch @@ -626,17 +566,13 @@ impl LocalBarrierWorker { } } - self.state - .transform_to_issued(barrier, to_collect, table_ids, partial_graph_id)?; - - // Actors to stop should still accept this barrier, but won't get sent to in next times. - if let Some(actors) = barrier.all_stop_actors() { - debug!( - target: "events::stream::barrier::manager", - "remove actors {:?} from senders", - actors - ); - } + self.state.transform_to_issued( + barrier, + to_build, + to_collect, + table_ids, + partial_graph_id, + )?; Ok(()) } diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 44764d87523ae..2528eb713af8a 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -15,7 +15,7 @@ use std::assert_matches::assert_matches; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::fmt::{Debug, Display, Formatter}; -use std::future::{poll_fn, Future}; +use std::future::{pending, poll_fn, Future}; use std::mem::replace; use std::sync::Arc; use std::task::{ready, Context, Poll}; @@ -32,16 +32,18 @@ use risingwave_common::util::epoch::EpochPair; use risingwave_hummock_sdk::SyncResult; use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; +use risingwave_pb::stream_service::BuildActorInfo; use risingwave_storage::{dispatch_state_store, StateStore, StateStoreImpl}; use thiserror_ext::AsReport; use tokio::sync::mpsc; +use tokio::task::JoinHandle; use super::progress::BackfillState; use super::BarrierCompleteResult; use crate::error::{StreamError, StreamResult}; use crate::executor::monitor::StreamingMetrics; use crate::executor::{Barrier, Mutation}; -use crate::task::{await_tree_key, ActorId, PartialGraphId}; +use crate::task::{await_tree_key, ActorId, PartialGraphId, SharedContext, StreamActorManager}; struct IssuedState { pub mutation: Option>, @@ -83,12 +85,12 @@ enum ManagedBarrierStateInner { #[derive(Debug)] pub(super) struct BarrierState { - curr_epoch: u64, + barrier: Barrier, inner: ManagedBarrierStateInner, } type AwaitEpochCompletedFuture = - impl Future)> + 'static; + impl Future)> + 'static; fn sync_epoch( state_store: &S, @@ -192,8 +194,6 @@ impl Display for &'_ PartialGraphManagedBarrierState { } enum InflightActorStatus { - /// The actor is just spawned and not issued any barrier yet - NotStarted, /// The actor has been issued some barriers, but has not collected the first barrier IssuedFirst(Vec), /// The actor has been issued some barriers, and has collected the first barrier @@ -201,12 +201,11 @@ enum InflightActorStatus { } impl InflightActorStatus { - fn max_issued_epoch(&self) -> Option { + fn max_issued_epoch(&self) -> u64 { match self { - InflightActorStatus::NotStarted => None, - InflightActorStatus::Running(epoch) => Some(*epoch), + InflightActorStatus::Running(epoch) => *epoch, InflightActorStatus::IssuedFirst(issued_barriers) => { - Some(issued_barriers.last().expect("non-empty").epoch.prev) + issued_barriers.last().expect("non-empty").epoch.prev } } } @@ -220,16 +219,30 @@ pub(crate) struct InflightActorState { status: InflightActorStatus, /// Whether the actor has been issued a stop barrier is_stopping: bool, + + join_handle: JoinHandle<()>, + monitor_task_handle: Option>, } impl InflightActorState { - pub(super) fn not_started(actor_id: ActorId) -> Self { + pub(super) fn start( + actor_id: ActorId, + initial_partial_graph_id: PartialGraphId, + initial_barrier: &Barrier, + join_handle: JoinHandle<()>, + monitor_task_handle: Option>, + ) -> Self { Self { actor_id, barrier_senders: vec![], - inflight_barriers: BTreeMap::default(), - status: InflightActorStatus::NotStarted, + inflight_barriers: BTreeMap::from_iter([( + initial_barrier.epoch.prev, + initial_partial_graph_id, + )]), + status: InflightActorStatus::IssuedFirst(vec![initial_barrier.clone()]), is_stopping: false, + join_handle, + monitor_task_handle, } } @@ -239,9 +252,7 @@ impl InflightActorState { barrier: &Barrier, is_stop: bool, ) -> StreamResult<()> { - if let Some(max_issued_epoch) = self.status.max_issued_epoch() { - assert!(barrier.epoch.prev > max_issued_epoch); - } + assert!(barrier.epoch.prev > self.status.max_issued_epoch()); for barrier_sender in &self.barrier_senders { barrier_sender.send(barrier.clone()).map_err(|_| { @@ -259,9 +270,6 @@ impl InflightActorState { .is_none()); match &mut self.status { - InflightActorStatus::NotStarted => { - self.status = InflightActorStatus::IssuedFirst(vec![barrier.clone()]); - } InflightActorStatus::IssuedFirst(pending_barriers) => { pending_barriers.push(barrier.clone()); } @@ -282,9 +290,6 @@ impl InflightActorState { self.inflight_barriers.pop_first().expect("should exist"); assert_eq!(prev_epoch, epoch.prev); match &self.status { - InflightActorStatus::NotStarted => { - unreachable!("should have issued a barrier when collect") - } InflightActorStatus::IssuedFirst(pending_barriers) => { assert_eq!( prev_epoch, @@ -316,6 +321,9 @@ pub(super) struct PartialGraphManagedBarrierState { prev_barrier_table_ids: Option<(EpochPair, HashSet)>, /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. + /// + /// This is updated by [`super::CreateMviewProgress::update`] and will be reported to meta + /// in [`BarrierCompleteResult`]. pub(super) create_mview_progress: HashMap>, pub(super) state_store: StateStoreImpl, @@ -360,32 +368,27 @@ impl PartialGraphManagedBarrierState { } } -pub(super) struct ManagedBarrierState { +pub(crate) struct ManagedBarrierState { pub(super) actor_states: HashMap, pub(super) graph_states: HashMap, - pub(super) state_store: StateStoreImpl, + actor_manager: Arc, - pub(super) streaming_metrics: Arc, - - /// Manages the await-trees of all barriers. - barrier_await_tree_reg: Option, + current_shared_context: Arc, } impl ManagedBarrierState { /// Create a barrier manager state. This will be called only once. pub(super) fn new( - state_store: StateStoreImpl, - streaming_metrics: Arc, - barrier_await_tree_reg: Option, + actor_manager: Arc, + current_shared_context: Arc, ) -> Self { Self { actor_states: Default::default(), graph_states: Default::default(), - state_store, - streaming_metrics, - barrier_await_tree_reg, + actor_manager, + current_shared_context, } } @@ -394,6 +397,21 @@ impl ManagedBarrierState { graph_states: &self.graph_states, } } + + pub(crate) async fn abort_actors(&mut self) { + for (actor_id, state) in &self.actor_states { + tracing::debug!("force stopping actor {}", actor_id); + state.join_handle.abort(); + if let Some(monitor_task_handle) = &state.monitor_task_handle { + monitor_task_handle.abort(); + } + } + for (actor_id, state) in self.actor_states.drain() { + tracing::debug!("join actor {}", actor_id); + let result = state.join_handle.await; + assert!(result.is_ok() || result.unwrap_err().is_cancelled()); + } + } } impl InflightActorState { @@ -402,9 +420,6 @@ impl InflightActorState { tx: mpsc::UnboundedSender, ) -> StreamResult<()> { match &self.status { - InflightActorStatus::NotStarted => { - self.barrier_senders.push(tx); - } InflightActorStatus::IssuedFirst(pending_barriers) => { for barrier in pending_barriers { tx.send(barrier.clone()).map_err(|_| { @@ -432,45 +447,97 @@ impl ManagedBarrierState { tx: mpsc::UnboundedSender, ) -> StreamResult<()> { self.actor_states - .entry(actor_id) - .or_insert_with(|| InflightActorState::not_started(actor_id)) + .get_mut(&actor_id) + .expect("should exist") .register_barrier_sender(tx) } pub(super) fn transform_to_issued( &mut self, barrier: &Barrier, + actors_to_build: Vec, actor_ids_to_collect: HashSet, table_ids: HashSet, partial_graph_id: PartialGraphId, ) -> StreamResult<()> { let actor_to_stop = barrier.all_stop_actors(); + let is_stop_actor = |actor_id| { + actor_to_stop + .map(|actors| actors.contains(&actor_id)) + .unwrap_or(false) + }; let graph_state = self .graph_states .entry(partial_graph_id) .or_insert_with(|| { PartialGraphManagedBarrierState::new( - self.state_store.clone(), - self.streaming_metrics.clone(), - self.barrier_await_tree_reg.clone(), + self.actor_manager.env.state_store(), + self.actor_manager.streaming_metrics.clone(), + self.actor_manager.await_tree_reg.clone(), ) }); graph_state.transform_to_issued(barrier, actor_ids_to_collect.clone(), table_ids); + let mut new_actors = HashSet::new(); + for actor in actors_to_build { + let actor_id = actor.actor.as_ref().unwrap().actor_id; + assert!(!is_stop_actor(actor_id)); + assert!(new_actors.insert(actor_id)); + assert!(actor_ids_to_collect.contains(&actor_id)); + let (join_handle, monitor_join_handle) = self + .actor_manager + .spawn_actor(actor, self.current_shared_context.clone()); + assert!(self + .actor_states + .try_insert( + actor_id, + InflightActorState::start( + actor_id, + partial_graph_id, + barrier, + join_handle, + monitor_join_handle + ) + ) + .is_ok()); + } + + // Spawn a trivial join handle to be compatible with the unit test + if cfg!(test) { + for actor_id in &actor_ids_to_collect { + if !self.actor_states.contains_key(actor_id) { + let join_handle = self.actor_manager.runtime.spawn(async { pending().await }); + assert!(self + .actor_states + .try_insert( + *actor_id, + InflightActorState::start( + *actor_id, + partial_graph_id, + barrier, + join_handle, + None, + ) + ) + .is_ok()); + new_actors.insert(*actor_id); + } + } + } + // Note: it's important to issue barrier to actor after issuing to graph to ensure that // we call `start_epoch` on the graph before the actors receive the barrier - for actor_id in actor_ids_to_collect { + for actor_id in &actor_ids_to_collect { + if new_actors.contains(actor_id) { + continue; + } self.actor_states - .entry(actor_id) - .or_insert_with(|| InflightActorState::not_started(actor_id)) - .issue_barrier( - partial_graph_id, - barrier, - actor_to_stop - .map(|actors| actors.contains(&actor_id)) - .unwrap_or(false), - )?; + .get_mut(actor_id) + .unwrap_or_else(|| { + panic!("should exist: {} {:?}", actor_id, actor_ids_to_collect); + }) + .issue_barrier(partial_graph_id, barrier, is_stop_actor(*actor_id))?; } Ok(()) @@ -481,9 +548,12 @@ impl ManagedBarrierState { ) -> impl Future + '_ { poll_fn(|cx| { for (partial_graph_id, graph_state) in &mut self.graph_states { - if let Poll::Ready(epoch) = graph_state.poll_next_completed_epoch(cx) { + if let Poll::Ready(barrier) = graph_state.poll_next_completed_barrier(cx) { + if let Some(actors_to_stop) = barrier.all_stop_actors() { + self.current_shared_context.drop_actors(actors_to_stop); + } let partial_graph_id = *partial_graph_id; - return Poll::Ready((partial_graph_id, epoch)); + return Poll::Ready((partial_graph_id, barrier.epoch.prev)); } } Poll::Pending @@ -497,7 +567,10 @@ impl ManagedBarrierState { .expect("should exist") .collect(epoch); if is_finished { - self.actor_states.remove(&actor_id); + let state = self.actor_states.remove(&actor_id).expect("should exist"); + if let Some(monitor_task_handle) = state.monitor_task_handle { + monitor_task_handle.abort(); + } } let prev_graph_state = self .graph_states @@ -548,7 +621,7 @@ impl PartialGraphManagedBarrierState { let create_mview_progress = self .create_mview_progress - .remove(&barrier_state.curr_epoch) + .remove(&barrier_state.barrier.epoch.curr) .unwrap_or_default() .into_iter() .map(|(actor, state)| CreateMviewProgress { @@ -556,7 +629,7 @@ impl PartialGraphManagedBarrierState { done: matches!(state, BackfillState::Done(_)), consumed_epoch: match state { BackfillState::ConsumingUpstream(consumed_epoch, _) => consumed_epoch, - BackfillState::Done(_) => barrier_state.curr_epoch, + BackfillState::Done(_) => barrier_state.barrier.epoch.curr, }, consumed_rows: match state { BackfillState::ConsumingUpstream(_, consumed_rows) => consumed_rows, @@ -595,6 +668,8 @@ impl PartialGraphManagedBarrierState { } }; + let barrier = barrier_state.barrier.clone(); + self.await_epoch_completed_futures.push_back({ let future = async move { if let Some(future) = complete_barrier_future { @@ -606,7 +681,7 @@ impl PartialGraphManagedBarrierState { } .map(move |result| { ( - prev_epoch, + barrier, result.map(|sync_result| BarrierCompleteResult { sync_result, create_mview_progress, @@ -646,7 +721,7 @@ impl PartialGraphManagedBarrierState { ) } Some(&mut BarrierState { - curr_epoch, + ref barrier, inner: ManagedBarrierStateInner::Issued(IssuedState { ref mut remaining_actors, @@ -660,7 +735,7 @@ impl PartialGraphManagedBarrierState { "the actor doesn't exist. actor_id: {:?}, curr_epoch: {:?}", actor_id, epoch.curr ); - assert_eq!(curr_epoch, epoch.curr); + assert_eq!(barrier.epoch.curr, epoch.curr); self.may_have_collected_all(epoch.prev); } Some(BarrierState { inner, .. }) => { @@ -742,7 +817,7 @@ impl PartialGraphManagedBarrierState { self.epoch_barrier_state_map.insert( barrier.epoch.prev, BarrierState { - curr_epoch: barrier.epoch.curr, + barrier: barrier.clone(), inner: ManagedBarrierStateInner::Issued(IssuedState { remaining_actors: BTreeSet::from_iter(actor_ids_to_collect), mutation: barrier.mutation.clone(), @@ -756,17 +831,17 @@ impl PartialGraphManagedBarrierState { } /// Return a future that yields the next completed epoch. The future is cancellation safe. - pub(crate) fn poll_next_completed_epoch(&mut self, cx: &mut Context<'_>) -> Poll { + pub(crate) fn poll_next_completed_barrier(&mut self, cx: &mut Context<'_>) -> Poll { ready!(self.await_epoch_completed_futures.next().poll_unpin(cx)) - .map(|(prev_epoch, result)| { + .map(|(barrier, result)| { let state = self .epoch_barrier_state_map - .get_mut(&prev_epoch) + .get_mut(&barrier.epoch.prev) .expect("should exist"); // sanity check on barrier state assert_matches!(&state.inner, ManagedBarrierStateInner::AllCollected); state.inner = ManagedBarrierStateInner::Completed(result); - prev_epoch + barrier }) .map(Poll::Ready) .unwrap_or(Poll::Pending) @@ -812,9 +887,12 @@ impl PartialGraphManagedBarrierState { #[cfg(test)] async fn pop_next_completed_epoch(&mut self) -> u64 { - let epoch = poll_fn(|cx| self.poll_next_completed_epoch(cx)).await; - let _ = self.pop_completed_epoch(epoch).unwrap().unwrap(); - epoch + let barrier = poll_fn(|cx| self.poll_next_completed_barrier(cx)).await; + let _ = self + .pop_completed_epoch(barrier.epoch.prev) + .unwrap() + .unwrap(); + barrier.epoch.prev } } diff --git a/src/stream/src/task/barrier_manager/tests.rs b/src/stream/src/task/barrier_manager/tests.rs index bd530e7daffc4..a9ba0b4b7ed01 100644 --- a/src/stream/src/task/barrier_manager/tests.rs +++ b/src/stream/src/task/barrier_manager/tests.rs @@ -40,19 +40,22 @@ async fn test_managed_barrier_collection() -> StreamResult<()> { // Register actors let actor_ids = vec![233, 234, 235]; - let count = actor_ids.len(); - let mut rxs = actor_ids - .clone() - .into_iter() - .map(register_sender) - .collect_vec(); // Send a barrier to all actors let curr_epoch = test_epoch(2); let barrier = Barrier::new_test_barrier(curr_epoch); let epoch = barrier.epoch.prev; - test_env.inject_barrier(&barrier, actor_ids); + test_env.inject_barrier(&barrier, actor_ids.clone()); + + manager.flush_all_events().await; + + let count = actor_ids.len(); + let mut rxs = actor_ids + .clone() + .into_iter() + .map(register_sender) + .collect_vec(); // Collect barriers from actors let collected_barriers = join_all(rxs.iter_mut().map(|(actor_id, rx)| async move { @@ -105,6 +108,14 @@ async fn test_managed_barrier_collection_separately() -> StreamResult<()> { .chain(once(extra_actor_id)) .collect_vec(); + // Prepare the barrier + let curr_epoch = test_epoch(2); + let barrier = Barrier::new_test_barrier(curr_epoch).with_stop(); + + test_env.inject_barrier(&barrier, actor_ids_to_collect.clone()); + + manager.flush_all_events().await; + // Register actors let count = actor_ids_to_send.len(); let mut rxs = actor_ids_to_send @@ -113,18 +124,12 @@ async fn test_managed_barrier_collection_separately() -> StreamResult<()> { .map(register_sender) .collect_vec(); - // Prepare the barrier - let curr_epoch = test_epoch(2); - let barrier = Barrier::new_test_barrier(curr_epoch).with_stop(); - let mut barrier_subscriber = manager.subscribe_barrier(extra_actor_id); // Read the mutation after receiving the barrier from remote input. let mut mutation_reader = pin!(barrier_subscriber.recv()); assert!(poll_fn(|cx| Poll::Ready(mutation_reader.as_mut().poll(cx).is_pending())).await); - test_env.inject_barrier(&barrier, actor_ids_to_collect); - let recv_barrier = mutation_reader.await.unwrap(); assert_eq!( (recv_barrier.epoch, &recv_barrier.mutation), @@ -198,6 +203,8 @@ async fn test_late_register_barrier_sender() -> StreamResult<()> { test_env.inject_barrier(&barrier1, actor_ids_to_collect.clone()); test_env.inject_barrier(&barrier2, actor_ids_to_collect.clone()); + manager.flush_all_events().await; + // register sender after inject barrier let mut rxs = actor_ids_to_send .clone() diff --git a/src/stream/src/task/mod.rs b/src/stream/src/task/mod.rs index f8ffaf76cb0f9..889495fbefed7 100644 --- a/src/stream/src/task/mod.rs +++ b/src/stream/src/task/mod.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::{HashMap, HashSet}; use anyhow::anyhow; use parking_lot::{MappedMutexGuard, Mutex, MutexGuard, RwLock}; @@ -189,7 +189,7 @@ impl SharedContext { &self.config } - pub fn drop_actors(&self, actors: &[ActorId]) { + pub(super) fn drop_actors(&self, actors: &HashSet) { self.channel_map .lock() .retain(|(up_id, _), _| !actors.contains(up_id)); diff --git a/src/stream/src/task/stream_manager.rs b/src/stream/src/task/stream_manager.rs index f15847750481f..08acb353c739c 100644 --- a/src/stream/src/task/stream_manager.rs +++ b/src/stream/src/task/stream_manager.rs @@ -19,7 +19,6 @@ use std::sync::atomic::AtomicU64; use std::sync::Arc; use std::time::Instant; -use anyhow::anyhow; use async_recursion::async_recursion; use await_tree::InstrumentAwait; use futures::stream::BoxStream; @@ -61,8 +60,8 @@ use crate::task::barrier_manager::{ ControlStreamHandle, EventSender, LocalActorOperation, LocalBarrierWorker, }; use crate::task::{ - ActorId, FragmentId, LocalBarrierManager, SharedContext, StreamActorManager, - StreamActorManagerState, StreamEnvironment, UpDownActorIds, + ActorId, FragmentId, LocalBarrierManager, SharedContext, StreamActorManager, StreamEnvironment, + UpDownActorIds, }; #[cfg(test)] @@ -216,16 +215,6 @@ impl LocalStreamManager { }) } - /// Drop the resources of the given actors. - pub async fn drop_actors(&self, actors: Vec) -> StreamResult<()> { - self.actor_op_tx - .send_and_await(|result_sender| LocalActorOperation::DropActors { - actors, - result_sender, - }) - .await - } - pub async fn take_receiver(&self, ids: UpDownActorIds) -> StreamResult { self.actor_op_tx .send_and_await(|result_sender| LocalActorOperation::TakeReceiver { @@ -258,28 +247,9 @@ impl LocalStreamManager { } impl LocalBarrierWorker { - /// Drop the resources of the given actors. - pub(super) fn drop_actors(&mut self, actors: &[ActorId]) { - self.current_shared_context.drop_actors(actors); - for &id in actors { - self.actor_manager_state.drop_actor(id); - } - tracing::debug!(actors = ?actors, "drop actors"); - } - /// Force stop all actors on this worker, and then drop their resources. pub(super) async fn reset(&mut self, version_id: HummockVersionId) { - let actor_handles = self.actor_manager_state.drain_actor_handles(); - for (actor_id, handle) in &actor_handles { - tracing::debug!("force stopping actor {}", actor_id); - handle.abort(); - } - for (actor_id, handle) in actor_handles { - tracing::debug!("join actor {}", actor_id); - let result = handle.await; - assert!(result.is_ok() || result.unwrap_err().is_cancelled()); - } - self.actor_manager_state.clear_state(); + self.state.abort_actors().await; if let Some(m) = self.actor_manager.await_tree_reg.as_ref() { m.clear(); } @@ -293,26 +263,6 @@ impl LocalBarrierWorker { self.reset_state(); self.actor_manager.env.dml_manager_ref().clear(); } - - pub(super) fn update_actors(&mut self, actors: Vec) -> StreamResult<()> { - self.actor_manager_state.update_actors(actors) - } - - /// This function could only be called once during the lifecycle of `LocalStreamManager` for - /// now. - pub(super) fn start_create_actors(&mut self, actors: &[ActorId]) -> StreamResult<()> { - let actors: Vec<_> = actors - .iter() - .map(|actor_id| { - self.actor_manager_state - .actors - .remove(actor_id) - .ok_or_else(|| anyhow!("No such actor with actor id:{}", actor_id)) - }) - .try_collect()?; - self.spawn_actors(actors); - Ok(()) - } } impl StreamActorManager { @@ -705,18 +655,22 @@ impl StreamActorManager { } } -impl LocalBarrierWorker { - pub(super) fn spawn_actors(&mut self, actors: Vec) { - for actor in actors { +impl StreamActorManager { + pub(super) fn spawn_actor( + self: &Arc, + actor: BuildActorInfo, + current_shared_context: Arc, + ) -> (JoinHandle<()>, Option>) { + { let monitor = tokio_metrics::TaskMonitor::new(); let stream_actor_ref = actor.actor.as_ref().unwrap(); let actor_id = stream_actor_ref.actor_id; let handle = { let trace_span = format!("Actor {actor_id}: `{}`", stream_actor_ref.mview_definition); - let barrier_manager = self.current_shared_context.local_barrier_manager.clone(); + let barrier_manager = current_shared_context.local_barrier_manager.clone(); // wrap the future of `create_actor` with `boxed` to avoid stack overflow - let actor = self.actor_manager.clone().create_actor(actor, self.current_shared_context.clone()).boxed().and_then(|actor| actor.run()).map(move |result| { + let actor = self.clone().create_actor(actor, current_shared_context).boxed().and_then(|actor| actor.run()).map(move |result| { if let Err(err) = result { // TODO: check error type and panic if it's unexpected. // Intentionally use `?` on the report to also include the backtrace. @@ -724,7 +678,7 @@ impl LocalBarrierWorker { barrier_manager.notify_failure(actor_id, err); } }); - let traced = match &self.actor_manager.await_tree_reg { + let traced = match &self.await_tree_reg { Some(m) => m .register(await_tree_key::Actor(actor_id), trace_span) .instrument(actor) @@ -732,24 +686,17 @@ impl LocalBarrierWorker { None => actor.right_future(), }; let instrumented = monitor.instrument(traced); - let with_config = - crate::CONFIG.scope(self.actor_manager.env.config().clone(), instrumented); + let with_config = crate::CONFIG.scope(self.env.config().clone(), instrumented); - self.actor_manager.runtime.spawn(with_config) + self.runtime.spawn(with_config) }; - self.actor_manager_state.handles.insert(actor_id, handle); - - if self.actor_manager.streaming_metrics.level >= MetricLevel::Debug - || self - .actor_manager - .env - .config() - .developer - .enable_actor_tokio_metrics + + let monitor_handle = if self.streaming_metrics.level >= MetricLevel::Debug + || self.env.config().developer.enable_actor_tokio_metrics { tracing::info!("Tokio metrics are enabled."); - let streaming_metrics = self.actor_manager.streaming_metrics.clone(); - let actor_monitor_task = self.actor_manager.runtime.spawn(async move { + let streaming_metrics = self.streaming_metrics.clone(); + let actor_monitor_task = self.runtime.spawn(async move { let metrics = streaming_metrics.new_actor_metrics(actor_id); loop { let task_metrics = monitor.cumulative(); @@ -789,10 +736,11 @@ impl LocalBarrierWorker { tokio::time::sleep(Duration::from_secs(1)).await; } }); - self.actor_manager_state - .actor_monitor_tasks - .insert(actor_id, actor_monitor_task); - } + Some(actor_monitor_task) + } else { + None + }; + (handle, monitor_handle) } } } @@ -817,44 +765,6 @@ impl LocalBarrierWorker { } } -impl StreamActorManagerState { - /// `drop_actor` is invoked by meta node via RPC once the stop barrier arrives at the - /// sink. All the actors in the actors should stop themselves before this method is invoked. - fn drop_actor(&mut self, actor_id: ActorId) { - self.actor_monitor_tasks - .remove(&actor_id) - .inspect(|handle| handle.abort()); - self.actors.remove(&actor_id); - - // Task should have already stopped when this method is invoked. There might be some - // clean-up work left (like dropping in-memory data structures), but we don't have to wait - // for them to finish, in order to make this request non-blocking. - self.handles.remove(&actor_id); - } - - fn drain_actor_handles(&mut self) -> Vec<(ActorId, ActorHandle)> { - self.handles.drain().collect() - } - - /// `stop_all_actors` is invoked by meta node via RPC for recovery purpose. Different from the - /// `drop_actor`, the execution of the actors will be aborted. - fn clear_state(&mut self) { - self.actors.clear(); - self.actor_monitor_tasks.clear(); - } - - fn update_actors(&mut self, actors: Vec) -> StreamResult<()> { - for actor in actors { - let actor_id = actor.actor.as_ref().unwrap().get_actor_id(); - self.actors - .try_insert(actor_id, actor) - .map_err(|_| anyhow!("duplicated actor {}", actor_id))?; - } - - Ok(()) - } -} - #[cfg(test)] pub mod test_utils { use risingwave_pb::common::HostAddress; diff --git a/src/tests/simulation/Cargo.toml b/src/tests/simulation/Cargo.toml index 8729207c0d025..c82f2b7d5911e 100644 --- a/src/tests/simulation/Cargo.toml +++ b/src/tests/simulation/Cargo.toml @@ -25,6 +25,7 @@ glob = "0.3" itertools = { workspace = true } lru = { workspace = true } madsim = "0.2.30" +maplit = "1" paste = "1" pin-project = "1.1" pretty_assertions = "1" diff --git a/src/tests/simulation/src/cluster.rs b/src/tests/simulation/src/cluster.rs index 26fdc3a8757e1..a9ffba0063562 100644 --- a/src/tests/simulation/src/cluster.rs +++ b/src/tests/simulation/src/cluster.rs @@ -158,27 +158,16 @@ impl Configuration { /// Provides a configuration for scale test which ensures that the arrangement backfill is disabled, /// so table scan will use `no_shuffle`. pub fn for_scale_no_shuffle() -> Self { - // Embed the config file and create a temporary file at runtime. The file will be deleted - // automatically when it's dropped. - let config_path = { - let mut file = - tempfile::NamedTempFile::new().expect("failed to create temp config file"); - file.write_all(include_bytes!("risingwave-scale.toml")) - .expect("failed to write config file"); - file.into_temp_path() - }; + let mut conf = Self::for_scale(); + conf.per_session_queries = + vec!["SET STREAMING_USE_ARRANGEMENT_BACKFILL = false;".into()].into(); + conf + } - Configuration { - config_path: ConfigPath::Temp(config_path.into()), - frontend_nodes: 2, - compute_nodes: 3, - meta_nodes: 3, - compactor_nodes: 2, - compute_node_cores: 2, - per_session_queries: vec!["SET STREAMING_USE_ARRANGEMENT_BACKFILL = false;".into()] - .into(), - ..Default::default() - } + pub fn for_scale_shared_source() -> Self { + let mut conf = Self::for_scale(); + conf.per_session_queries = vec!["SET RW_ENABLE_SHARED_SOURCE = true;".into()].into(); + conf } pub fn for_auto_parallelism( diff --git a/src/tests/simulation/src/ctl_ext.rs b/src/tests/simulation/src/ctl_ext.rs index 9b57673e49c16..3986a826e21e7 100644 --- a/src/tests/simulation/src/ctl_ext.rs +++ b/src/tests/simulation/src/ctl_ext.rs @@ -12,9 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -#![cfg_attr(not(madsim), expect(unused_imports))] - -use std::collections::{HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap, HashSet}; use std::ffi::OsString; use std::fmt::Write; use std::sync::Arc; @@ -23,17 +21,17 @@ use anyhow::{anyhow, Result}; use cfg_or_panic::cfg_or_panic; use clap::Parser; use itertools::Itertools; -use rand::seq::{IteratorRandom, SliceRandom}; +use rand::seq::IteratorRandom; use rand::{thread_rng, Rng}; use risingwave_common::catalog::TableId; use risingwave_common::hash::WorkerSlotId; +use risingwave_connector::source::{SplitImpl, SplitMetaData}; use risingwave_hummock_sdk::{CompactionGroupId, HummockSstableId}; use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; use risingwave_pb::meta::table_fragments::PbFragment; use risingwave_pb::meta::update_worker_node_schedulability_request::Schedulability; use risingwave_pb::meta::GetClusterInfoResponse; use risingwave_pb::stream_plan::StreamNode; -use serde::de::IntoDeserializer; use self::predicate::BoxedPredicate; use crate::cluster::Cluster; @@ -76,7 +74,7 @@ pub mod predicate { Box::new(p) } - /// There exists operators whose identity contains `s` in the fragment. + /// There exists operators whose identity contains `s` in the fragment (case insensitive). pub fn identity_contains(s: impl Into) -> BoxedPredicate { let s: String = s.into(); let p = move |f: &PbFragment| { @@ -363,6 +361,30 @@ impl Cluster { Ok(response) } + /// `table_id -> actor_id -> splits` + pub async fn list_source_splits(&self) -> Result>> { + let info = self.get_cluster_info().await?; + let mut res = BTreeMap::new(); + + for table in info.table_fragments { + let mut table_actor_splits = BTreeMap::new(); + + for (actor_id, splits) in table.actor_splits { + let splits = splits + .splits + .iter() + .map(|split| SplitImpl::try_from(split).unwrap()) + .map(|split| split.id()) + .collect_vec() + .join(","); + table_actor_splits.insert(actor_id, splits); + } + res.insert(table.table_id, table_actor_splits); + } + + Ok(res) + } + // update node schedulability #[cfg_or_panic(madsim)] async fn update_worker_node_schedulability( diff --git a/src/tests/simulation/tests/integration_tests/scale/mod.rs b/src/tests/simulation/tests/integration_tests/scale/mod.rs index f6940f072409e..3c7a702dc6290 100644 --- a/src/tests/simulation/tests/integration_tests/scale/mod.rs +++ b/src/tests/simulation/tests/integration_tests/scale/mod.rs @@ -20,6 +20,7 @@ mod nexmark_q4; mod nexmark_source; mod no_shuffle; mod schedulability; +mod shared_source; mod singleton_migration; mod sink; mod streaming_parallelism; diff --git a/src/tests/simulation/tests/integration_tests/scale/shared_source.rs b/src/tests/simulation/tests/integration_tests/scale/shared_source.rs new file mode 100644 index 0000000000000..175b3a043100c --- /dev/null +++ b/src/tests/simulation/tests/integration_tests/scale/shared_source.rs @@ -0,0 +1,192 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::BTreeMap; + +use anyhow::Result; +use itertools::Itertools; +use maplit::{convert_args, hashmap}; +use risingwave_common::hash::WorkerSlotId; +use risingwave_pb::meta::table_fragments::Fragment; +use risingwave_simulation::cluster::{Cluster, Configuration}; +use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; + +const CREATE_SOURCE: &str = r#" +CREATE SOURCE s(v1 int, v2 varchar) WITH ( + connector='kafka', + properties.bootstrap.server='192.168.11.1:29092', + topic='shared_source' +) FORMAT PLAIN ENCODE JSON;"#; + +fn actor_upstream(fragment: &Fragment) -> Vec<(u32, Vec)> { + fragment + .actors + .iter() + .map(|actor| (actor.actor_id, actor.upstream_actor_id.clone())) + .collect_vec() +} + +async fn validate_splits_aligned(cluster: &mut Cluster) -> Result<()> { + let source_backfill_fragment = cluster + .locate_one_fragment([identity_contains("StreamSourceScan")]) + .await?; + // The result of scaling is non-deterministic. + // So we just print the result here, instead of asserting with a fixed value. + let actor_upstream = actor_upstream(&source_backfill_fragment.inner); + tracing::info!( + "{}", + actor_upstream + .iter() + .format_with("\n", |(actor_id, upstream), f| f(&format_args!( + "{} <- {:?}", + actor_id, upstream + ))) + ); + let splits = cluster.list_source_splits().await?; + tracing::info!("{:#?}", splits); + let actor_splits: BTreeMap = splits + .values() + .flat_map(|m| m.clone().into_iter()) + .collect(); + for (actor, upstream) in actor_upstream { + assert!(upstream.len() == 1, "invalid upstream: {:?}", upstream); + let upstream_actor = upstream[0]; + assert_eq!( + actor_splits.get(&actor).unwrap(), + actor_splits.get(&upstream_actor).unwrap() + ); + } + Ok(()) +} + +#[tokio::test] +async fn test_shared_source() -> Result<()> { + tracing_subscriber::fmt::Subscriber::builder() + .with_max_level(tracing::Level::ERROR) + .with_env_filter("risingwave_stream::executor::source::source_backfill_executor=DEBUG,integration_tests=DEBUG") + .init(); + + let mut cluster = Cluster::start(Configuration::for_scale_shared_source()).await?; + cluster.create_kafka_topics(convert_args!(hashmap!( + "shared_source" => 4, + ))); + let mut session = cluster.start_session(); + + session.run("set rw_implicit_flush = true;").await?; + + session.run(CREATE_SOURCE).await?; + session + .run("create materialized view mv as select count(*) from s group by v1;") + .await?; + let source_fragment = cluster + .locate_one_fragment([ + identity_contains("Source"), + no_identity_contains("StreamSourceScan"), + ]) + .await?; + let source_workers = source_fragment.all_worker_count().into_keys().collect_vec(); + let source_backfill_fragment = cluster + .locate_one_fragment([identity_contains("StreamSourceScan")]) + .await?; + let source_backfill_workers = source_backfill_fragment + .all_worker_count() + .into_keys() + .collect_vec(); + let hash_agg_fragment = cluster + .locate_one_fragment([identity_contains("hashagg")]) + .await?; + let hash_agg_workers = hash_agg_fragment + .all_worker_count() + .into_keys() + .collect_vec(); + validate_splits_aligned(&mut cluster).await?; + expect_test::expect![[r#" + 1 1 HASH {2} {} {SOURCE} 6 + 2 3 HASH {4,3} {3} {MVIEW} 6 + 3 3 HASH {5} {1} {SOURCE_SCAN} 6"#]] + .assert_eq(&cluster.run("select * from rw_fragments;").await?); + expect_test::expect![[r#" + 1 CREATED ADAPTIVE + 3 CREATED ADAPTIVE"#]] + .assert_eq(&cluster.run("select * from rw_table_fragments;").await?); + + // SourceBackfill cannot be scaled because of NoShuffle. + assert!( + &cluster + .reschedule( + source_backfill_fragment + .reschedule([WorkerSlotId::new(source_backfill_workers[0], 0)], []), + ) + .await.unwrap_err().to_string().contains("rescheduling NoShuffle downstream fragment (maybe Chain fragment) is forbidden, please use NoShuffle upstream fragment (like Materialized fragment) to scale"), + ); + + // hash agg can be scaled independently + cluster + .reschedule(hash_agg_fragment.reschedule([WorkerSlotId::new(hash_agg_workers[0], 0)], [])) + .await + .unwrap(); + expect_test::expect![[r#" + 1 1 HASH {2} {} {SOURCE} 6 + 2 3 HASH {4,3} {3} {MVIEW} 5 + 3 3 HASH {5} {1} {SOURCE_SCAN} 6"#]] + .assert_eq(&cluster.run("select * from rw_fragments;").await?); + + // source is the NoShuffle upstream. It can be scaled, and the downstream SourceBackfill will be scaled together. + cluster + .reschedule(source_fragment.reschedule( + [ + WorkerSlotId::new(source_workers[0], 0), + WorkerSlotId::new(source_workers[0], 1), + WorkerSlotId::new(source_workers[2], 0), + ], + [], + )) + .await + .unwrap(); + validate_splits_aligned(&mut cluster).await?; + expect_test::expect![[r#" + 1 1 HASH {2} {} {SOURCE} 3 + 2 3 HASH {4,3} {3} {MVIEW} 5 + 3 3 HASH {5} {1} {SOURCE_SCAN} 3"#]] + .assert_eq(&cluster.run("select * from rw_fragments;").await?); + expect_test::expect![[r#" + 1 CREATED CUSTOM + 3 CREATED CUSTOM"#]] + .assert_eq(&cluster.run("select * from rw_table_fragments;").await?); + + // resolve_no_shuffle for backfill fragment is OK, which will scale the upstream together. + cluster + .reschedule_resolve_no_shuffle(source_backfill_fragment.reschedule( + [], + [ + WorkerSlotId::new(source_workers[0], 0), + WorkerSlotId::new(source_workers[0], 1), + WorkerSlotId::new(source_workers[2], 0), + WorkerSlotId::new(source_workers[2], 1), + ], + )) + .await + .unwrap(); + validate_splits_aligned(&mut cluster).await?; + expect_test::expect![[r#" + 1 1 HASH {2} {} {SOURCE} 7 + 2 3 HASH {4,3} {3} {MVIEW} 5 + 3 3 HASH {5} {1} {SOURCE_SCAN} 7"#]] + .assert_eq(&cluster.run("select * from rw_fragments;").await?); + expect_test::expect![[r#" +1 CREATED CUSTOM +3 CREATED CUSTOM"#]] + .assert_eq(&cluster.run("select * from rw_table_fragments;").await?); + Ok(()) +}