From a48ef13cb6b1b6e11a5d466259e22ed1d5a8369e Mon Sep 17 00:00:00 2001 From: Peng Chen Date: Mon, 1 Jul 2024 22:04:34 +0800 Subject: [PATCH] Refactor parallelism handling: Shift from parallel units to parallelism fields & simplify logic --- proto/common.proto | 8 +- proto/meta.proto | 55 +- .../src/executor/join/local_lookup_join.rs | 2 +- .../src/worker_manager/worker_node_manager.rs | 8 +- .../src/hash/consistent_hash/mapping.rs | 39 +- .../src/vnode_mapping/vnode_placement.rs | 31 +- src/ctl/src/cmd_impl/meta/cluster_info.rs | 47 +- src/ctl/src/cmd_impl/meta/migration.rs | 8 +- src/ctl/src/cmd_impl/meta/reschedule.rs | 167 ++- src/ctl/src/cmd_impl/scale/resize.rs | 354 +----- src/ctl/src/lib.rs | 91 -- .../system_catalog/rw_catalog/rw_actors.rs | 4 +- .../rw_catalog/rw_parallel_units.rs | 18 +- .../rw_catalog/rw_worker_nodes.rs | 2 +- src/frontend/src/handler/show.rs | 4 +- .../src/scheduler/distributed/query.rs | 18 +- .../src/scheduler/distributed/stage.rs | 4 +- src/meta/model_v2/migration/src/lib.rs | 2 + .../m20240630_131430_remove_parallel_unit.rs | 110 ++ src/meta/model_v2/src/actor.rs | 1 - src/meta/model_v2/src/fragment.rs | 3 +- src/meta/model_v2/src/worker_property.rs | 4 +- src/meta/service/src/notification_service.rs | 8 +- src/meta/service/src/scale_service.rs | 110 +- src/meta/service/src/stream_service.rs | 4 +- src/meta/src/barrier/mod.rs | 2 +- src/meta/src/barrier/recovery.rs | 155 +-- src/meta/src/controller/catalog.rs | 38 +- src/meta/src/controller/cluster.rs | 112 +- src/meta/src/controller/fragment.rs | 306 ++--- src/meta/src/controller/streaming_job.rs | 37 +- src/meta/src/controller/utils.rs | 161 ++- src/meta/src/manager/catalog/fragment.rs | 297 +++-- src/meta/src/manager/cluster.rs | 91 +- src/meta/src/manager/diagnose.rs | 2 +- src/meta/src/manager/metadata.rs | 33 +- src/meta/src/model/migration_plan.rs | 18 +- src/meta/src/model/stream.rs | 53 +- src/meta/src/rpc/ddl_controller.rs | 72 +- src/meta/src/rpc/metrics.rs | 11 +- src/meta/src/stream/scale.rs | 1024 ++++++----------- src/meta/src/stream/stream_graph/actor.rs | 53 +- src/meta/src/stream/stream_graph/fragment.rs | 29 +- src/meta/src/stream/stream_graph/schedule.rs | 140 +-- src/meta/src/stream/stream_manager.rs | 31 +- src/meta/src/stream/test_fragmenter.rs | 21 +- src/rpc_client/src/meta_client.rs | 19 +- src/tests/simulation/src/ctl_ext.rs | 142 +-- .../integration_tests/recovery/backfill.rs | 64 +- .../scale/auto_parallelism.rs | 197 ++-- .../scale/cascade_materialized_view.rs | 42 +- .../integration_tests/scale/dynamic_filter.rs | 34 +- .../tests/integration_tests/scale/mod.rs | 1 - .../integration_tests/scale/nexmark_q4.rs | 73 +- .../integration_tests/scale/no_shuffle.rs | 59 +- .../tests/integration_tests/scale/plan.rs | 332 ------ .../integration_tests/scale/schedulability.rs | 22 +- .../scale/singleton_migration.rs | 31 +- .../tests/integration_tests/scale/sink.rs | 68 +- .../tests/integration_tests/scale/table.rs | 64 +- .../tests/integration_tests/sink/scale.rs | 47 +- 61 files changed, 2076 insertions(+), 2907 deletions(-) create mode 100644 src/meta/model_v2/migration/src/m20240630_131430_remove_parallel_unit.rs delete mode 100644 src/tests/simulation/tests/integration_tests/scale/plan.rs diff --git a/proto/common.proto b/proto/common.proto index 164150379c484..b93932a831814 100644 --- a/proto/common.proto +++ b/proto/common.proto @@ -61,8 +61,10 @@ message WorkerNode { WorkerType type = 2; HostAddress host = 3; State state = 4; - // TODO #8940 `parallel_units` should be moved into `Property` - repeated ParallelUnit parallel_units = 5; + + reserved 5; + reserved "parallel_units"; + Property property = 6; // Ranges from 0 to 1023, used to generate the machine ID field in the global unique ID. @@ -75,6 +77,8 @@ message WorkerNode { // It's populated by meta node, when the worker node is added by meta node. // It's not persistent in meta store. optional uint64 started_at = 9; + + uint32 parallelism = 10; } message Buffer { diff --git a/proto/meta.proto b/proto/meta.proto index e4068a0b8cd58..fc6ca87b0d298 100644 --- a/proto/meta.proto +++ b/proto/meta.proto @@ -83,7 +83,9 @@ message TableFragments { repeated stream_plan.StreamActor actors = 4; // Vnode mapping (which should be set in upstream dispatcher) of the fragment. // This field is always set to `Some`. For singleton, the parallel unit for all vnodes will be the same. - common.ParallelUnitMapping vnode_mapping = 5; + reserved 5; + reserved "vnode_mapping"; + repeated uint32 state_table_ids = 6; // Note that this can be derived backwards from the upstream actors of the Actor held by the Fragment, // but in some scenarios (e.g. Scaling) it will lead to a lot of duplicate code, @@ -129,7 +131,10 @@ message ActorLocation { message MigrationPlan { // map, the plan indicates that the actors will be migrated from old parallel unit to the new one. - map parallel_unit_migration_plan = 1; + reserved 1; + reserved "parallel_unit_migration_plan"; + + map worker_slot_migration_plan = 2; } message FlushRequest { @@ -244,8 +249,10 @@ message ListActorStatesResponse { message ActorState { uint32 actor_id = 1; uint32 fragment_id = 2; - uint32 parallel_unit_id = 3; + reserved 3; + reserved "parallel_unit_id"; TableFragments.ActorStatus.ActorState state = 4; + uint32 worker_id = 5; } repeated ActorState states = 1; } @@ -492,16 +499,17 @@ message GetClusterInfoResponse { uint64 revision = 5; } -message Reschedule { - repeated uint32 added_parallel_units = 1; - repeated uint32 removed_parallel_units = 2; +message WorkerReschedule { + map increased_actor_count = 1; + map decreased_actor_count = 2; } message RescheduleRequest { - // reschedule plan for each fragment - map reschedules = 1; + reserved "reschedules"; + reserved 1; uint64 revision = 2; bool resolve_no_shuffle_upstream = 3; + map worker_reschedules = 4; } message RescheduleResponse { @@ -529,40 +537,9 @@ message TableParallelism { } } -message GetReschedulePlanRequest { - uint64 revision = 1; - - message WorkerChanges { - repeated uint32 include_worker_ids = 1; - repeated uint32 exclude_worker_ids = 2; - optional uint32 target_parallelism = 3; - optional uint32 target_parallelism_per_worker = 4; - } - - message StableResizePolicy { - map fragment_worker_changes = 1; - } - - oneof policy { - // The StableResizePolicy will generate a stable ReschedulePlan, without altering the distribution on WorkerId that's not involved. - // Note that this "Stable" doesn't refer to the "determinacy" of the algorithm. - // Multiple repeated calls may yield different ReschedulePlan results. - StableResizePolicy stable_resize_policy = 2; - } -} - -message GetReschedulePlanResponse { - uint64 revision = 1; - // reschedule plan for each fragment - map reschedules = 2; - // todo, refactor needed - bool success = 3; -} - service ScaleService { rpc GetClusterInfo(GetClusterInfoRequest) returns (GetClusterInfoResponse); rpc Reschedule(RescheduleRequest) returns (RescheduleResponse); - rpc GetReschedulePlan(GetReschedulePlanRequest) returns (GetReschedulePlanResponse); } message MembersRequest {} diff --git a/src/batch/src/executor/join/local_lookup_join.rs b/src/batch/src/executor/join/local_lookup_join.rs index a00092b36d11c..a9710b1003afc 100644 --- a/src/batch/src/executor/join/local_lookup_join.rs +++ b/src/batch/src/executor/join/local_lookup_join.rs @@ -383,7 +383,7 @@ impl BoxedExecutorBuilder for LocalLookupJoinExecutorBuilder { let worker_slot_mapping: HashMap = worker_nodes .iter() .flat_map(|worker| { - (0..(worker.parallel_units.len())) + (0..(worker.parallelism as usize)) .map(|i| (WorkerSlotId::new(worker.id, i), worker.clone())) }) .collect(); diff --git a/src/batch/src/worker_manager/worker_node_manager.rs b/src/batch/src/worker_manager/worker_node_manager.rs index 8b7dcc42b565a..68d2859d107ef 100644 --- a/src/batch/src/worker_manager/worker_node_manager.rs +++ b/src/batch/src/worker_manager/worker_node_manager.rs @@ -160,7 +160,7 @@ impl WorkerNodeManager { .worker_nodes .iter() .flat_map(|worker| { - (0..worker.parallel_units.len()) + (0..worker.parallelism as usize) .map(move |i| (WorkerSlotId::new(worker.id, i), worker)) }) .collect(); @@ -337,7 +337,7 @@ impl WorkerNodeSelector { }; worker_nodes .iter() - .map(|node| node.parallel_units.len()) + .map(|node| node.parallelism as usize) .sum() } @@ -424,7 +424,7 @@ mod tests { r#type: WorkerType::ComputeNode as i32, host: Some(HostAddr::try_from("127.0.0.1:1234").unwrap().to_protobuf()), state: worker_node::State::Running as i32, - parallel_units: vec![], + parallelism: 0, property: Some(Property { is_unschedulable: false, is_serving: true, @@ -438,7 +438,7 @@ mod tests { r#type: WorkerType::ComputeNode as i32, host: Some(HostAddr::try_from("127.0.0.1:1235").unwrap().to_protobuf()), state: worker_node::State::Running as i32, - parallel_units: vec![], + parallelism: 0, property: Some(Property { is_unschedulable: false, is_serving: true, diff --git a/src/common/src/hash/consistent_hash/mapping.rs b/src/common/src/hash/consistent_hash/mapping.rs index 59460e7966220..54be33103be32 100644 --- a/src/common/src/hash/consistent_hash/mapping.rs +++ b/src/common/src/hash/consistent_hash/mapping.rs @@ -33,7 +33,7 @@ use crate::util::iter_util::ZipEqDebug; // TODO: find a better place for this. pub type ActorId = u32; -#[derive(Debug, Clone, Copy, PartialEq, Eq, Hash, Ord, PartialOrd)] +#[derive(Clone, Copy, PartialEq, Eq, Hash, Ord, PartialOrd)] pub struct WorkerSlotId(u64); impl WorkerSlotId { @@ -68,6 +68,12 @@ impl Display for WorkerSlotId { } } +impl Debug for WorkerSlotId { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + f.write_fmt(format_args!("[{}:{}]", self.worker_id(), self.slot_idx())) + } +} + /// Trait for items that can be used as keys in [`VnodeMapping`]. pub trait VnodeMappingItem { /// The type of the item. @@ -282,12 +288,14 @@ pub mod marker { /// A marker type for items of [`ActorId`]. pub struct Actor; + impl VnodeMappingItem for Actor { type Item = ActorId; } /// A marker type for items of [`ParallelUnitId`]. pub struct ParallelUnit; + impl VnodeMappingItem for ParallelUnit { type Item = ParallelUnitId; } @@ -323,6 +331,26 @@ impl ActorMapping { self.transform(to_map) } + /// Transform the actor mapping to the worker slot mapping. Note that the parameter is a mapping from actor to worker. + pub fn to_worker_slot(&self, actor_to_worker: &HashMap) -> WorkerSlotMapping { + let mut worker_actors = HashMap::new(); + for (actor_id, worker_id) in actor_to_worker { + worker_actors + .entry(worker_id) + .or_insert(BTreeSet::new()) + .insert(actor_id); + } + + let mut actor_location = HashMap::new(); + for (worker, actors) in worker_actors { + for (idx, &actor) in actors.iter().enumerate() { + actor_location.insert(*actor, WorkerSlotId::new(*worker, idx)); + } + } + + self.transform(&actor_location) + } + /// Create an actor mapping from the protobuf representation. pub fn from_protobuf(proto: &ActorMappingProto) -> Self { assert_eq!(proto.original_indices.len(), proto.data.len()); @@ -441,6 +469,13 @@ impl ParallelUnitMapping { } } +impl WorkerSlotMapping { + /// Transform this parallel unit mapping to an actor mapping, essentially `transform`. + pub fn to_actor(&self, to_map: &HashMap) -> ActorMapping { + self.transform(to_map) + } +} + #[cfg(test)] mod tests { use std::iter::repeat_with; @@ -450,11 +485,13 @@ mod tests { use super::*; struct Test; + impl VnodeMappingItem for Test { type Item = u32; } struct Test2; + impl VnodeMappingItem for Test2 { type Item = u32; } diff --git a/src/common/src/vnode_mapping/vnode_placement.rs b/src/common/src/vnode_mapping/vnode_placement.rs index 036cfebe792bb..13cc7b45f864e 100644 --- a/src/common/src/vnode_mapping/vnode_placement.rs +++ b/src/common/src/vnode_mapping/vnode_placement.rs @@ -37,7 +37,7 @@ pub fn place_vnode( .iter() .filter(|w| w.property.as_ref().map_or(false, |p| p.is_serving)) .sorted_by_key(|w| w.id) - .map(|w| (0..w.parallel_units.len()).map(|idx| WorkerSlotId::new(w.id, idx))) + .map(|w| (0..w.parallelism as usize).map(|idx| WorkerSlotId::new(w.id, idx))) .collect(); // Set serving parallelism to the minimum of total number of worker slots, specified @@ -198,42 +198,23 @@ pub fn place_vnode( #[cfg(test)] mod tests { - use std::collections::HashMap; use risingwave_common::hash::WorkerSlotMapping; use risingwave_pb::common::worker_node::Property; - use risingwave_pb::common::{ParallelUnit, WorkerNode}; + use risingwave_pb::common::WorkerNode; - use crate::hash::{ParallelUnitId, VirtualNode}; + use crate::hash::VirtualNode; use crate::vnode_mapping::vnode_placement::place_vnode; #[test] fn test_place_vnode() { assert_eq!(VirtualNode::COUNT, 256); - let mut pu_id_counter: ParallelUnitId = 0; - let mut pu_to_worker: HashMap = Default::default(); let serving_property = Property { is_unschedulable: false, is_serving: true, is_streaming: false, }; - let mut gen_pus_for_worker = - |worker_node_id: u32, number: u32, pu_to_worker: &mut HashMap| { - let mut results = vec![]; - for i in 0..number { - results.push(ParallelUnit { - id: pu_id_counter + i, - worker_node_id, - }) - } - pu_id_counter += number; - for pu in &results { - pu_to_worker.insert(pu.id, pu.worker_node_id); - } - results - }; - let count_same_vnode_mapping = |wm1: &WorkerSlotMapping, wm2: &WorkerSlotMapping| { assert_eq!(wm1.len(), 256); assert_eq!(wm2.len(), 256); @@ -249,7 +230,7 @@ mod tests { let worker_1 = WorkerNode { id: 1, - parallel_units: gen_pus_for_worker(1, 1, &mut pu_to_worker), + parallelism: 1, property: Some(serving_property.clone()), ..Default::default() }; @@ -264,7 +245,7 @@ mod tests { let worker_2 = WorkerNode { id: 2, - parallel_units: gen_pus_for_worker(2, 50, &mut pu_to_worker), + parallelism: 50, property: Some(serving_property.clone()), ..Default::default() }; @@ -283,7 +264,7 @@ mod tests { let worker_3 = WorkerNode { id: 3, - parallel_units: gen_pus_for_worker(3, 60, &mut pu_to_worker), + parallelism: 60, property: Some(serving_property), ..Default::default() }; diff --git a/src/ctl/src/cmd_impl/meta/cluster_info.rs b/src/ctl/src/cmd_impl/meta/cluster_info.rs index 7d38fe8f3ae4e..387746f106cb8 100644 --- a/src/ctl/src/cmd_impl/meta/cluster_info.rs +++ b/src/ctl/src/cmd_impl/meta/cluster_info.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap}; +use std::collections::{BTreeMap, BTreeSet, HashMap}; use comfy_table::{Attribute, Cell, Row, Table}; use itertools::Itertools; @@ -88,7 +88,7 @@ pub async fn cluster_info(context: &CtlContext) -> anyhow::Result<()> { revision, } = get_cluster_info(context).await?; - // Fragment ID -> [Parallel Unit ID -> (Parallel Unit, Actor)] + // Fragment ID -> [Worker ID -> [Actor ID]] let mut fragments = BTreeMap::new(); // Fragment ID -> Table Fragments' State let mut fragment_states = HashMap::new(); @@ -96,32 +96,25 @@ pub async fn cluster_info(context: &CtlContext) -> anyhow::Result<()> { for table_fragment in &table_fragments { for (&id, fragment) in &table_fragment.fragments { for actor in &fragment.actors { - let parallel_unit = table_fragment + let worker_id = table_fragment .actor_status .get(&actor.actor_id) .unwrap() .get_parallel_unit() - .unwrap(); + .unwrap() + .get_worker_node_id(); + fragments .entry(id) - .or_insert_with(HashMap::new) - .insert(parallel_unit.id, (parallel_unit, actor)); + .or_insert_with(BTreeMap::new) + .entry(worker_id) + .or_insert(BTreeSet::new()) + .insert(actor.actor_id); } fragment_states.insert(id, table_fragment.state()); } } - // Parallel Unit ID -> Worker Node - let all_parallel_units: BTreeMap<_, _> = worker_nodes - .iter() - .flat_map(|worker_node| { - worker_node - .parallel_units - .iter() - .map(|parallel_unit| (parallel_unit.id, worker_node.clone())) - }) - .collect(); - let mut table = Table::new(); let cross_out_if_creating = |cell: Cell, fid: u32| -> Cell { @@ -132,11 +125,10 @@ pub async fn cluster_info(context: &CtlContext) -> anyhow::Result<()> { } }; - // Compute Node, Parallel Unit, Frag 1, Frag 2, ..., Frag N + // Compute Node, Frag 1, Frag 2, ..., Frag N table.set_header({ let mut row = Row::new(); row.add_cell("Compute Node".into()); - row.add_cell("Parallel Unit".into()); for &fid in fragments.keys() { let cell = Cell::new(format!("Frag {fid}")); let cell = cross_out_if_creating(cell, fid); @@ -146,8 +138,8 @@ pub async fn cluster_info(context: &CtlContext) -> anyhow::Result<()> { }); let mut last_worker_id = None; - for (pu, worker) in all_parallel_units { - // Compute Node, Parallel Unit, Actor 1, Actor 11, -, ..., Actor N + for worker in worker_nodes { + // Compute Node, Actor 1, Actor 11, -, ..., Actor N let mut row = Row::new(); row.add_cell(if last_worker_id == Some(worker.id) { "".into() @@ -166,14 +158,17 @@ pub async fn cluster_info(context: &CtlContext) -> anyhow::Result<()> { )) .add_attribute(Attribute::Bold) }); - row.add_cell(pu.into()); - for (&fid, f) in &fragments { - let cell = if let Some((_pu, actor)) = f.get(&pu) { - actor.actor_id.into() + for (&fragment_id, worker_actors) in &fragments { + let cell = if let Some(actors) = worker_actors.get(&worker.id) { + actors + .iter() + .map(|actor| format!("{}", actor)) + .join(",") + .into() } else { "-".into() }; - let cell = cross_out_if_creating(cell, fid); + let cell = cross_out_if_creating(cell, fragment_id); row.add_cell(cell); } table.add_row(row); diff --git a/src/ctl/src/cmd_impl/meta/migration.rs b/src/ctl/src/cmd_impl/meta/migration.rs index 93be066d4e727..16b7dedea59e3 100644 --- a/src/ctl/src/cmd_impl/meta/migration.rs +++ b/src/ctl/src/cmd_impl/meta/migration.rs @@ -156,18 +156,12 @@ pub async fn migrate(from: EtcdBackend, target: String, force_clean: bool) -> an .await?; if worker.worker_type() == WorkerType::ComputeNode { let pb_property = worker.worker_node.property.as_ref().unwrap(); - let parallel_unit_ids = worker - .worker_node - .parallel_units - .iter() - .map(|pu| pu.id as i32) - .collect_vec(); let property = worker_property::ActiveModel { worker_id: Set(worker.worker_id() as _), - parallel_unit_ids: Set(parallel_unit_ids.into()), is_streaming: Set(pb_property.is_streaming), is_serving: Set(pb_property.is_serving), is_unschedulable: Set(pb_property.is_unschedulable), + parallelism: Set(worker.worker_node.parallelism as _), }; WorkerProperty::insert(property) .exec(&meta_store_sql.conn) diff --git a/src/ctl/src/cmd_impl/meta/reschedule.rs b/src/ctl/src/cmd_impl/meta/reschedule.rs index 8d0a45be842fb..999767d49c777 100644 --- a/src/ctl/src/cmd_impl/meta/reschedule.rs +++ b/src/ctl/src/cmd_impl/meta/reschedule.rs @@ -15,14 +15,14 @@ use std::collections::{HashMap, HashSet}; use std::process::exit; -use anyhow::{anyhow, Error, Result}; +use anyhow::{anyhow, Result}; use inquire::Confirm; use itertools::Itertools; use regex::{Match, Regex}; +use risingwave_meta::manager::WorkerId; use risingwave_pb::common::WorkerNode; -use risingwave_pb::meta::get_reschedule_plan_request::PbPolicy; use risingwave_pb::meta::table_fragments::ActorStatus; -use risingwave_pb::meta::{GetClusterInfoResponse, GetReschedulePlanResponse, Reschedule}; +use risingwave_pb::meta::{GetClusterInfoResponse, PbWorkerReschedule}; use serde::{Deserialize, Serialize}; use thiserror_ext::AsReport; @@ -34,16 +34,16 @@ pub struct ReschedulePayload { pub reschedule_revision: u64, #[serde(rename = "reschedule_plan")] - pub reschedule_plan: HashMap, + pub worker_reschedule_plan: HashMap, } #[derive(Serialize, Deserialize, Debug)] -pub struct FragmentReschedulePlan { - #[serde(rename = "added_parallel_units")] - pub added_parallel_units: Vec, +pub struct WorkerReschedulePlan { + #[serde(rename = "increased_actor_count")] + pub increased_actor_count: HashMap, - #[serde(rename = "removed_parallel_units")] - pub removed_parallel_units: Vec, + #[serde(rename = "decreased_actor_count")] + pub decreased_actor_count: HashMap, } #[derive(Debug)] @@ -52,66 +52,50 @@ pub enum RescheduleInput { FilePath(String), } -impl From for Reschedule { - fn from(value: FragmentReschedulePlan) -> Self { - let FragmentReschedulePlan { - added_parallel_units, - removed_parallel_units, +impl From for PbWorkerReschedule { + fn from(value: WorkerReschedulePlan) -> Self { + let WorkerReschedulePlan { + increased_actor_count, + decreased_actor_count, } = value; - Reschedule { - added_parallel_units, - removed_parallel_units, + PbWorkerReschedule { + increased_actor_count: increased_actor_count + .into_iter() + .map(|(k, v)| (k as _, v as _)) + .collect(), + decreased_actor_count: decreased_actor_count + .into_iter() + .map(|(k, v)| (k as _, v as _)) + .collect(), } } } -impl From for FragmentReschedulePlan { - fn from(value: Reschedule) -> Self { - let Reschedule { - added_parallel_units, - removed_parallel_units, +impl From for WorkerReschedulePlan { + fn from(value: PbWorkerReschedule) -> Self { + let PbWorkerReschedule { + increased_actor_count, + decreased_actor_count, } = value; - FragmentReschedulePlan { - added_parallel_units, - removed_parallel_units, + WorkerReschedulePlan { + increased_actor_count: increased_actor_count + .into_iter() + .map(|(k, v)| (k as _, v as _)) + .collect(), + decreased_actor_count: decreased_actor_count + .into_iter() + .map(|(k, v)| (k as _, v as _)) + .collect(), } } } -const RESCHEDULE_MATCH_REGEXP: &str = - r"^(?P\d+)(?:-\[(?P\d+(?:,\d+)*)])?(?:\+\[(?P\d+(?:,\d+)*)])?$"; +const RESCHEDULE_MATCH_REGEXP: &str = r"^(?P\d+)(?:-\[(?P(?:\d+:\d+,?)+)])?(?:\+\[(?P(?:\d+:\d+,?)+)])?$"; const RESCHEDULE_FRAGMENT_KEY: &str = "fragment"; -const RESCHEDULE_REMOVED_KEY: &str = "removed"; -const RESCHEDULE_ADDED_KEY: &str = "added"; - -// For plan `100-[1,2,3]+[4,5];101-[1];102+[3]`, the following reschedule request will be generated -// { -// 100: Reschedule { -// added_parallel_units: [ -// 4, -// 5, -// ], -// removed_parallel_units: [ -// 1, -// 2, -// 3, -// ], -// }, -// 101: Reschedule { -// added_parallel_units: [], -// removed_parallel_units: [ -// 1, -// ], -// }, -// 102: Reschedule { -// added_parallel_units: [ -// 3, -// ], -// removed_parallel_units: [], -// }, -// } +const RESCHEDULE_DECREASED_KEY: &str = "decreased"; +const RESCHEDULE_INCREASED_KEY: &str = "increased"; pub async fn reschedule( context: &CtlContext, plan: Option, @@ -128,13 +112,13 @@ pub async fn reschedule( let file = std::fs::File::open(path)?; let ReschedulePayload { reschedule_revision, - reschedule_plan, + worker_reschedule_plan, } = serde_yaml::from_reader(file)?; ( - reschedule_plan + worker_reschedule_plan .into_iter() - .map(|(fragment_id, fragment_reschedule_plan)| { - (fragment_id, fragment_reschedule_plan.into()) + .map(|(fragment_id, worker_reschedule_plan)| { + (fragment_id, worker_reschedule_plan.into()) }) .collect(), reschedule_revision, @@ -145,12 +129,12 @@ pub async fn reschedule( for (fragment_id, reschedule) in &reschedules { println!("For fragment #{}", fragment_id); - if !reschedule.removed_parallel_units.is_empty() { - println!("\tRemove: {:?}", reschedule.removed_parallel_units); + if !reschedule.decreased_actor_count.is_empty() { + println!("\tDecreased: {:?}", reschedule.decreased_actor_count); } - if !reschedule.added_parallel_units.is_empty() { - println!("\tAdd: {:?}", reschedule.added_parallel_units); + if !reschedule.increased_actor_count.is_empty() { + println!("\tIncreased: {:?}", reschedule.increased_actor_count); } println!(); @@ -177,11 +161,13 @@ pub async fn reschedule( Ok(()) } -fn parse_plan(mut plan: String) -> Result, Error> { +fn parse_plan(plan: String) -> Result> { let mut reschedules = HashMap::new(); let regex = Regex::new(RESCHEDULE_MATCH_REGEXP)?; + let mut plan = plan; + plan.retain(|c| !c.is_whitespace()); for fragment_reschedule_plan in plan.split(';') { @@ -195,29 +181,40 @@ fn parse_plan(mut plan: String) -> Result, Error> { .ok_or_else(|| anyhow!("plan \"{}\" does not have a valid fragment id", plan))?; let split_fn = |mat: Match<'_>| { - mat.as_str() - .split(',') - .map(|id_str| id_str.parse::().map_err(Error::msg)) - .collect::>>() + let mut result = HashMap::new(); + for id_str in mat.as_str().split(',') { + let (worker_id, count) = id_str + .split(':') + .map(|v| v.parse::().unwrap()) + .collect_tuple::<(_, _)>() + .unwrap(); + + if let Some(dup_count) = result.insert(worker_id, count) { + println!( + "duplicate worker id {} in plan, prev {} -> {}", + worker_id, worker_id, dup_count + ); + exit(1); + } + } + + result }; - let removed_parallel_units = captures - .name(RESCHEDULE_REMOVED_KEY) + let decreased_actor_count = captures + .name(RESCHEDULE_DECREASED_KEY) .map(split_fn) - .transpose()? .unwrap_or_default(); - let added_parallel_units = captures - .name(RESCHEDULE_ADDED_KEY) + let increased_actor_count = captures + .name(RESCHEDULE_INCREASED_KEY) .map(split_fn) - .transpose()? .unwrap_or_default(); - - if !(removed_parallel_units.is_empty() && added_parallel_units.is_empty()) { + if !(decreased_actor_count.is_empty() && increased_actor_count.is_empty()) { reschedules.insert( fragment_id, - Reschedule { - added_parallel_units, - removed_parallel_units, + PbWorkerReschedule { + increased_actor_count, + decreased_actor_count, }, ); } @@ -225,16 +222,6 @@ fn parse_plan(mut plan: String) -> Result, Error> { Ok(reschedules) } -pub async fn get_reschedule_plan( - context: &CtlContext, - policy: PbPolicy, - revision: u64, -) -> Result { - let meta_client = context.meta_client().await?; - let response = meta_client.get_reschedule_plan(policy, revision).await?; - Ok(response) -} - pub async fn unregister_workers( context: &CtlContext, workers: Vec, diff --git a/src/ctl/src/cmd_impl/scale/resize.rs b/src/ctl/src/cmd_impl/scale/resize.rs index 6b990c7865519..364426e7996eb 100644 --- a/src/ctl/src/cmd_impl/scale/resize.rs +++ b/src/ctl/src/cmd_impl/scale/resize.rs @@ -13,22 +13,14 @@ // limitations under the License. use std::collections::{HashMap, HashSet}; -use std::ops::Sub; use std::process::exit; -use inquire::Confirm; use itertools::Itertools; -use risingwave_pb::meta::get_reschedule_plan_request::{ - PbPolicy, StableResizePolicy, WorkerChanges, -}; use risingwave_pb::meta::update_worker_node_schedulability_request::Schedulability; -use risingwave_pb::meta::{GetClusterInfoResponse, GetReschedulePlanResponse}; -use risingwave_stream::task::FragmentId; +use risingwave_pb::meta::GetClusterInfoResponse; use thiserror_ext::AsReport; -use crate::cmd_impl::meta::ReschedulePayload; use crate::common::CtlContext; -use crate::{ScaleCommon, ScaleHorizonCommands, ScaleVerticalCommands}; macro_rules! fail { ($($arg:tt)*) => {{ @@ -37,350 +29,6 @@ macro_rules! fail { }}; } -impl From for ScaleCommandContext { - fn from(value: ScaleHorizonCommands) -> Self { - let ScaleHorizonCommands { - exclude_workers, - include_workers, - target_parallelism, - common: - ScaleCommon { - generate, - output, - yes, - fragments, - }, - } = value; - - Self { - exclude_workers, - include_workers, - target_parallelism, - generate, - output, - yes, - fragments, - target_parallelism_per_worker: None, - exclusive_for_vertical: false, - } - } -} - -impl From for ScaleCommandContext { - fn from(value: ScaleVerticalCommands) -> Self { - let ScaleVerticalCommands { - workers, - target_parallelism_per_worker, - common: - ScaleCommon { - generate, - output, - yes, - fragments, - }, - exclusive, - } = value; - - Self { - exclude_workers: None, - include_workers: workers, - target_parallelism: None, - generate, - output, - yes, - fragments, - target_parallelism_per_worker, - exclusive_for_vertical: exclusive, - } - } -} - -pub struct ScaleCommandContext { - exclude_workers: Option>, - include_workers: Option>, - target_parallelism: Option, - generate: bool, - output: Option, - yes: bool, - fragments: Option>, - target_parallelism_per_worker: Option, - exclusive_for_vertical: bool, -} - -pub async fn resize(ctl_ctx: &CtlContext, scale_ctx: ScaleCommandContext) -> anyhow::Result<()> { - let meta_client = ctl_ctx.meta_client().await?; - - let GetClusterInfoResponse { - worker_nodes, - table_fragments, - actor_splits: _actor_splits, - source_infos: _source_infos, - revision, - } = match meta_client.get_cluster_info().await { - Ok(resp) => resp, - Err(e) => { - fail!("Failed to fetch cluster info: {}", e.as_report()); - } - }; - - if worker_nodes.is_empty() { - println!("No worker nodes found"); - return Ok(()); - } - - if table_fragments.is_empty() { - println!("No tables found"); - return Ok(()); - } - - println!("Cluster info fetched, revision: {}", revision); - println!("Worker nodes: {}", worker_nodes.len()); - - let streaming_workers_index_by_id = worker_nodes - .into_iter() - .filter(|worker| { - worker - .property - .as_ref() - .map(|property| property.is_streaming) - .unwrap_or(false) - }) - .map(|worker| (worker.id, worker)) - .collect::>(); - - let streaming_workers_index_by_host = streaming_workers_index_by_id - .values() - .map(|worker| { - let host = worker.get_host().expect("worker host must be set"); - (format!("{}:{}", host.host, host.port), worker.clone()) - }) - .collect::>(); - - let worker_input_to_worker_ids = |inputs: Vec, support_all: bool| -> Vec { - let mut result: HashSet<_> = HashSet::new(); - - if inputs.len() == 1 && inputs[0].to_lowercase() == "all" && support_all { - return streaming_workers_index_by_id.keys().cloned().collect(); - } - - for input in inputs { - let worker_id = input.parse::().ok().or_else(|| { - streaming_workers_index_by_host - .get(&input) - .map(|worker| worker.id) - }); - - if let Some(worker_id) = worker_id { - if !result.insert(worker_id) { - println!("warn: {} and {} are the same worker", input, worker_id); - } - } else { - fail!("Invalid worker input: {}", input); - } - } - - result.into_iter().collect() - }; - - println!( - "Streaming workers found: {}", - streaming_workers_index_by_id.len() - ); - - let ScaleCommandContext { - exclude_workers, - include_workers, - target_parallelism, - target_parallelism_per_worker, - generate, - output, - yes, - fragments, - exclusive_for_vertical, - } = scale_ctx; - - let worker_changes = { - let mut exclude_worker_ids = - worker_input_to_worker_ids(exclude_workers.unwrap_or_default(), false); - let include_worker_ids = - worker_input_to_worker_ids(include_workers.unwrap_or_default(), true); - - match (target_parallelism, target_parallelism_per_worker) { - (Some(_), Some(_)) => { - fail!("Cannot specify both target parallelism and target parallelism per worker") - } - (_, Some(_)) if include_worker_ids.is_empty() => { - fail!("Cannot specify target parallelism per worker without including any worker") - } - (Some(0), _) => fail!("Target parallelism must be greater than 0"), - _ => {} - } - - for worker_id in exclude_worker_ids.iter().chain(include_worker_ids.iter()) { - if !streaming_workers_index_by_id.contains_key(worker_id) { - fail!("Invalid worker id: {}", worker_id); - } - } - - for include_worker_id in &include_worker_ids { - let worker_is_unschedulable = streaming_workers_index_by_id - .get(include_worker_id) - .and_then(|worker| worker.property.as_ref()) - .map(|property| property.is_unschedulable) - .unwrap_or(false); - - if worker_is_unschedulable { - fail!( - "Worker {} is unschedulable, should not be included", - include_worker_id - ); - } - } - - if exclusive_for_vertical { - let all_worker_ids: HashSet<_> = - streaming_workers_index_by_id.keys().cloned().collect(); - - let include_worker_id_set: HashSet<_> = include_worker_ids.iter().cloned().collect(); - let generated_exclude_worker_ids = all_worker_ids.sub(&include_worker_id_set); - - exclude_worker_ids = exclude_worker_ids - .into_iter() - .chain(generated_exclude_worker_ids) - .unique() - .collect(); - } - - WorkerChanges { - include_worker_ids, - exclude_worker_ids, - target_parallelism, - target_parallelism_per_worker, - } - }; - - let all_fragment_ids: HashSet<_> = table_fragments - .iter() - .flat_map(|table_fragments| table_fragments.fragments.keys().cloned()) - .collect(); - - let target_fragment_ids = match fragments { - None => all_fragment_ids.into_iter().collect_vec(), - Some(fragment_ids) => { - let provide_fragment_ids: HashSet<_> = fragment_ids.into_iter().collect(); - if provide_fragment_ids - .iter() - .any(|fragment_id| !all_fragment_ids.contains(fragment_id)) - { - fail!( - "Invalid fragment ids: {:?}", - provide_fragment_ids - .difference(&all_fragment_ids) - .collect_vec() - ); - } - - provide_fragment_ids.into_iter().collect() - } - }; - - let policy = PbPolicy::StableResizePolicy(StableResizePolicy { - fragment_worker_changes: target_fragment_ids - .iter() - .map(|id| (*id as FragmentId, worker_changes.clone())) - .collect(), - }); - - let response = meta_client.get_reschedule_plan(policy, revision).await; - - let GetReschedulePlanResponse { - revision, - reschedules, - success, - } = match response { - Ok(response) => response, - Err(e) => { - fail!("Failed to generate plan: {}", e.as_report()); - } - }; - - if !success { - fail!("Failed to generate plan, current revision is {}", revision); - } - - if reschedules.is_empty() { - println!( - "No reschedule plan generated, no action required, current revision is {}", - revision - ); - return Ok(()); - } - - println!( - "Successfully generated plan, current revision is {}", - revision - ); - - if generate { - let payload = ReschedulePayload { - reschedule_revision: revision, - reschedule_plan: reschedules - .into_iter() - .map(|(fragment_id, reschedule)| (fragment_id, reschedule.into())) - .collect(), - }; - - if let Some(output) = output.as_ref() { - println!("Writing plan to file: {}", output); - let writer = std::fs::File::create(output)?; - serde_yaml::to_writer(writer, &payload)?; - println!("Writing plan to file: {} done", output); - println!("You can use the `risectl meta reschedule --from {}` command to execute the generated plan", output); - } else { - println!("Option `--output` is not provided, the result plan will be output to the current command line."); - println!("#=========== Payload ==============#"); - serde_yaml::to_writer(std::io::stdout(), &payload)?; - println!("#=========== Payload ==============#"); - } - } else { - if !yes { - match Confirm::new("Will perform actions on the cluster, are you sure?") - .with_default(false) - .with_help_message("Use the --generate flag to view the generated plan. Use the --yes or -y option to skip this prompt") - .with_placeholder("no") - .prompt() - { - Ok(true) => println!("Processing..."), - Ok(false) => { - fail!("Abort."); - } - Err(_) => { - fail!("Error with questionnaire, try again later"); - } - } - } - - let (success, next_revision) = - match meta_client.reschedule(reschedules, revision, false).await { - Ok(response) => response, - Err(e) => { - fail!("Failed to execute plan: {}", e.as_report()); - } - }; - - if !success { - fail!("Failed to execute plan, current revision is {}", revision); - } - - println!( - "Successfully executed plan, current revision is {}", - next_revision - ); - } - - Ok(()) -} - pub async fn update_schedulability( context: &CtlContext, workers: Vec, diff --git a/src/ctl/src/lib.rs b/src/ctl/src/lib.rs index 002817993d859..60f376a369c56 100644 --- a/src/ctl/src/lib.rs +++ b/src/ctl/src/lib.rs @@ -348,92 +348,8 @@ enum TableCommands { List, } -#[derive(clap::Args, Debug, Clone)] -pub struct ScaleHorizonCommands { - /// The worker that needs to be excluded during scheduling, `worker_id` and `worker_host:worker_port` are both - /// supported - #[clap( - long, - value_delimiter = ',', - value_name = "worker_id or worker_host:worker_port, ..." - )] - exclude_workers: Option>, - - /// The worker that needs to be included during scheduling, `worker_id` and `worker_host:worker_port` are both - /// supported - #[clap( - long, - value_delimiter = ',', - value_name = "all or worker_id or worker_host:worker_port, ..." - )] - include_workers: Option>, - - /// The target parallelism, currently, it is used to limit the target parallelism and only - /// takes effect when the actual parallelism exceeds this value. Can be used in conjunction - /// with `exclude/include_workers`. - #[clap(long)] - target_parallelism: Option, - - #[command(flatten)] - common: ScaleCommon, -} - -#[derive(clap::Args, Debug, Clone)] -pub struct ScaleCommon { - /// Will generate a plan supported by the `reschedule` command and save it to the provided path - /// by the `--output`. - #[clap(long, default_value_t = false)] - generate: bool, - - /// The output file to write the generated plan to, standard output by default - #[clap(long)] - output: Option, - - /// Automatic yes to prompts - #[clap(short = 'y', long, default_value_t = false)] - yes: bool, - - /// Specify the fragment ids that need to be scheduled. - /// empty by default, which means all fragments will be scheduled - #[clap(long, value_delimiter = ',')] - fragments: Option>, -} - -#[derive(clap::Args, Debug, Clone)] -pub struct ScaleVerticalCommands { - #[command(flatten)] - common: ScaleCommon, - - /// The worker that needs to be scheduled, `worker_id` and `worker_host:worker_port` are both - /// supported - #[clap( - long, - required = true, - value_delimiter = ',', - value_name = "all or worker_id or worker_host:worker_port, ..." - )] - workers: Option>, - - /// The target parallelism per worker, requires `workers` to be set. - #[clap(long, required = true)] - target_parallelism_per_worker: Option, - - /// It will exclude all other workers to maintain the target parallelism only for the target workers. - #[clap(long, default_value_t = false)] - exclusive: bool, -} - #[derive(Subcommand, Debug)] enum ScaleCommands { - /// Scale the compute nodes horizontally, alias of `horizon` - Resize(ScaleHorizonCommands), - - /// Scale the compute nodes horizontally - Horizon(ScaleHorizonCommands), - - /// Scale the compute nodes vertically - Vertical(ScaleVerticalCommands), - /// Mark a compute node as unschedulable #[clap(verbatim_doc_comment)] Cordon { @@ -908,13 +824,6 @@ async fn start_impl(opts: CliOpts, context: &CtlContext) -> Result<()> { Commands::Profile(ProfileCommands::Heap { dir }) => { cmd_impl::profile::heap_profile(context, dir).await? } - Commands::Scale(ScaleCommands::Horizon(resize)) - | Commands::Scale(ScaleCommands::Resize(resize)) => { - cmd_impl::scale::resize(context, resize.into()).await? - } - Commands::Scale(ScaleCommands::Vertical(resize)) => { - cmd_impl::scale::resize(context, resize.into()).await? - } Commands::Scale(ScaleCommands::Cordon { workers }) => { cmd_impl::scale::update_schedulability(context, workers, Schedulability::Unschedulable) .await? diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs index 9769c7cafbf6c..b915a1f9dde97 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_actors.rs @@ -23,7 +23,7 @@ struct RwActor { #[primary_key] actor_id: i32, fragment_id: i32, - parallel_unit_id: i32, + worker_id: i32, state: String, } @@ -36,7 +36,7 @@ async fn read_rw_actors(reader: &SysCatalogReaderImpl) -> Result> { .map(|state| RwActor { actor_id: state.actor_id as i32, fragment_id: state.fragment_id as i32, - parallel_unit_id: state.parallel_unit_id as i32, + worker_id: state.worker_id as i32, state: state.state().as_str_name().into(), }) .collect()) diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs index 0a46e10ec1754..c846263deee34 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_parallel_units.rs @@ -19,26 +19,24 @@ use crate::catalog::system_catalog::SysCatalogReaderImpl; use crate::error::Result; #[derive(Fields)] -struct RwParallelUnit { +struct RwWorkerSlot { + #[primary_key] + slot_id: i32, #[primary_key] - id: i32, worker_id: i32, } #[system_catalog(table, "rw_catalog.rw_parallel_units")] -fn read_rw_parallel_units(reader: &SysCatalogReaderImpl) -> Result> { +fn read_rw_parallel_units(reader: &SysCatalogReaderImpl) -> Result> { let workers = reader.worker_node_manager.list_worker_nodes(); Ok(workers .into_iter() .flat_map(|worker| { - worker - .parallel_units - .into_iter() - .map(move |unit| RwParallelUnit { - id: unit.id as i32, - worker_id: worker.id as i32, - }) + (0..worker.parallelism).map(move |slot_id| RwWorkerSlot { + slot_id: slot_id as _, + worker_id: worker.id as _, + }) }) .collect()) } diff --git a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs index 226b0230e3f21..ebd1ba751dd94 100644 --- a/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs +++ b/src/frontend/src/catalog/system_catalog/rw_catalog/rw_worker_nodes.rs @@ -55,7 +55,7 @@ async fn read_rw_worker_nodes_info(reader: &SysCatalogReaderImpl) -> Result, is_serving: Option, is_unschedulable: Option, @@ -435,7 +435,7 @@ pub async fn handle_show_object( addr: addr.to_string(), r#type: worker.get_type().unwrap().as_str_name().into(), state: worker.get_state().unwrap().as_str_name().to_string(), - parallel_units: worker.parallel_units.into_iter().map(|pu| pu.id).join(", "), + parallelism: worker.get_parallelism() as _, is_streaming: property.map(|p| p.is_streaming), is_serving: property.map(|p| p.is_serving), is_unschedulable: property.map(|p| p.is_unschedulable), diff --git a/src/frontend/src/scheduler/distributed/query.rs b/src/frontend/src/scheduler/distributed/query.rs index 165bdcee6476b..c5aa87ca1e3ba 100644 --- a/src/frontend/src/scheduler/distributed/query.rs +++ b/src/frontend/src/scheduler/distributed/query.rs @@ -479,7 +479,7 @@ pub(crate) mod tests { use risingwave_common::hash::{WorkerSlotId, WorkerSlotMapping}; use risingwave_common::types::DataType; use risingwave_pb::common::worker_node::Property; - use risingwave_pb::common::{HostAddress, ParallelUnit, WorkerNode, WorkerType}; + use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::plan_common::JoinType; use risingwave_rpc_client::ComputeClientPool; @@ -675,7 +675,7 @@ pub(crate) mod tests { port: 5687, }), state: risingwave_pb::common::worker_node::State::Running as i32, - parallel_units: generate_parallel_units(0, 0), + parallelism: 8, property: Some(Property { is_unschedulable: false, is_serving: true, @@ -692,7 +692,7 @@ pub(crate) mod tests { port: 5688, }), state: risingwave_pb::common::worker_node::State::Running as i32, - parallel_units: generate_parallel_units(8, 1), + parallelism: 8, property: Some(Property { is_unschedulable: false, is_serving: true, @@ -709,7 +709,7 @@ pub(crate) mod tests { port: 5689, }), state: risingwave_pb::common::worker_node::State::Running as i32, - parallel_units: generate_parallel_units(16, 2), + parallelism: 8, property: Some(Property { is_unschedulable: false, is_serving: true, @@ -743,14 +743,4 @@ pub(crate) mod tests { .unwrap(); fragmenter.generate_complete_query().await.unwrap() } - - fn generate_parallel_units(start_id: u32, node_id: u32) -> Vec { - let parallel_degree = 8; - (start_id..start_id + parallel_degree) - .map(|id| ParallelUnit { - id, - worker_node_id: node_id, - }) - .collect() - } } diff --git a/src/frontend/src/scheduler/distributed/stage.rs b/src/frontend/src/scheduler/distributed/stage.rs index 8b7e07a0aefcd..1ae292a6dde28 100644 --- a/src/frontend/src/scheduler/distributed/stage.rs +++ b/src/frontend/src/scheduler/distributed/stage.rs @@ -712,11 +712,11 @@ impl StageRunner { if let Some(table_id) = dml_table_id { let vnode_mapping = self.get_table_dml_vnode_mapping(&table_id)?; - let worker_ids = vnode_mapping.iter_unique().collect_vec(); + let worker_slot_ids = vnode_mapping.iter_unique().collect_vec(); let candidates = self .worker_node_manager .manager - .get_workers_by_worker_slot_ids(&worker_ids)?; + .get_workers_by_worker_slot_ids(&worker_slot_ids)?; if candidates.is_empty() { return Err(BatchError::EmptyWorkerNodes.into()); } diff --git a/src/meta/model_v2/migration/src/lib.rs b/src/meta/model_v2/migration/src/lib.rs index 83645c479649d..4acefa2812640 100644 --- a/src/meta/model_v2/migration/src/lib.rs +++ b/src/meta/model_v2/migration/src/lib.rs @@ -13,6 +13,7 @@ mod m20240506_112555_subscription_partial_ckpt; mod m20240525_090457_secret; mod m20240617_070131_index_column_properties; mod m20240618_072634_function_compressed_binary; +mod m20240630_131430_remove_parallel_unit; pub struct Migrator; @@ -31,6 +32,7 @@ impl MigratorTrait for Migrator { Box::new(m20240525_090457_secret::Migration), Box::new(m20240618_072634_function_compressed_binary::Migration), Box::new(m20240617_070131_index_column_properties::Migration), + Box::new(m20240630_131430_remove_parallel_unit::Migration), ] } } diff --git a/src/meta/model_v2/migration/src/m20240630_131430_remove_parallel_unit.rs b/src/meta/model_v2/migration/src/m20240630_131430_remove_parallel_unit.rs new file mode 100644 index 0000000000000..bd0fb500c9069 --- /dev/null +++ b/src/meta/model_v2/migration/src/m20240630_131430_remove_parallel_unit.rs @@ -0,0 +1,110 @@ +use sea_orm_migration::prelude::*; + +#[derive(DeriveMigrationName)] +pub struct Migration; + +#[async_trait::async_trait] +impl MigrationTrait for Migration { + async fn up(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(WorkerProperty::Table) + .add_column( + ColumnDef::new(WorkerProperty::Parallelism) + .integer() + .not_null(), + ) + .to_owned(), + ) + .await?; + + manager + .alter_table( + Table::alter() + .table(WorkerProperty::Table) + .drop_column(WorkerProperty::ParallelUnitIds) + .to_owned(), + ) + .await?; + + manager + .alter_table( + Table::alter() + .table(Fragment::Table) + .drop_column(Fragment::VnodeMapping) + .to_owned(), + ) + .await?; + + manager + .alter_table( + Table::alter() + .table(Actor::Table) + .drop_column(Actor::ParallelUnitId) + .to_owned(), + ) + .await + } + + async fn down(&self, manager: &SchemaManager) -> Result<(), DbErr> { + manager + .alter_table( + Table::alter() + .table(WorkerProperty::Table) + .drop_column(WorkerProperty::Parallelism) + .to_owned(), + ) + .await?; + + manager + .alter_table( + Table::alter() + .table(WorkerProperty::Table) + .add_column( + ColumnDef::new(WorkerProperty::ParallelUnitIds) + .json_binary() + .not_null(), + ) + .to_owned(), + ) + .await?; + + manager + .alter_table( + Table::alter() + .table(Fragment::Table) + .add_column(ColumnDef::new(Fragment::VnodeMapping).binary().not_null()) + .to_owned(), + ) + .await?; + + manager + .alter_table( + Table::alter() + .table(Actor::Table) + .add_column(ColumnDef::new(Actor::ParallelUnitId).integer().not_null()) + .to_owned(), + ) + .await + } +} + +#[derive(DeriveIden)] +enum WorkerProperty { + Table, + Parallelism, + ParallelUnitIds, +} + +#[derive(DeriveIden)] +enum Fragment { + Table, + VnodeMapping, +} + +#[derive(DeriveIden)] +enum Actor { + Table, + ParallelUnitId, +} diff --git a/src/meta/model_v2/src/actor.rs b/src/meta/model_v2/src/actor.rs index c75eac7dbc4cf..cbbbca543679a 100644 --- a/src/meta/model_v2/src/actor.rs +++ b/src/meta/model_v2/src/actor.rs @@ -56,7 +56,6 @@ pub struct Model { pub fragment_id: FragmentId, pub status: ActorStatus, pub splits: Option, - pub parallel_unit_id: i32, pub worker_id: WorkerId, pub upstream_actor_ids: ActorUpstreamActors, pub vnode_bitmap: Option, diff --git a/src/meta/model_v2/src/fragment.rs b/src/meta/model_v2/src/fragment.rs index 7f69584538593..a189cb03c7747 100644 --- a/src/meta/model_v2/src/fragment.rs +++ b/src/meta/model_v2/src/fragment.rs @@ -16,7 +16,7 @@ use risingwave_pb::meta::table_fragments::fragment::PbFragmentDistributionType; use sea_orm::entity::prelude::*; use serde::{Deserialize, Serialize}; -use crate::{FragmentId, FragmentVnodeMapping, I32Array, ObjectId, StreamNode}; +use crate::{FragmentId, I32Array, ObjectId, StreamNode}; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize)] #[sea_orm(table_name = "fragment")] @@ -27,7 +27,6 @@ pub struct Model { pub fragment_type_mask: i32, pub distribution_type: DistributionType, pub stream_node: StreamNode, - pub vnode_mapping: FragmentVnodeMapping, pub state_table_ids: I32Array, pub upstream_fragment_id: I32Array, } diff --git a/src/meta/model_v2/src/worker_property.rs b/src/meta/model_v2/src/worker_property.rs index 3ab8d411c8b5c..64834ae0b13cb 100644 --- a/src/meta/model_v2/src/worker_property.rs +++ b/src/meta/model_v2/src/worker_property.rs @@ -15,14 +15,14 @@ use sea_orm::entity::prelude::*; use serde::{Deserialize, Serialize}; -use crate::{I32Array, WorkerId}; +use crate::WorkerId; #[derive(Clone, Debug, PartialEq, DeriveEntityModel, Eq, Serialize, Deserialize)] #[sea_orm(table_name = "worker_property")] pub struct Model { #[sea_orm(primary_key, auto_increment = false)] pub worker_id: WorkerId, - pub parallel_unit_ids: I32Array, + pub parallelism: i32, pub is_streaming: bool, pub is_serving: bool, pub is_unschedulable: bool, diff --git a/src/meta/service/src/notification_service.rs b/src/meta/service/src/notification_service.rs index e4a8d298e0788..5d68211a71d57 100644 --- a/src/meta/service/src/notification_service.rs +++ b/src/meta/service/src/notification_service.rs @@ -148,19 +148,19 @@ impl NotificationServiceImpl { match &self.metadata_manager { MetadataManager::V1(mgr) => { let fragment_guard = mgr.fragment_manager.get_fragment_read_guard().await; - let parallel_unit_mappings = + let worker_slot_mappings = fragment_guard.all_running_fragment_mappings().collect_vec(); let notification_version = self.env.notification_manager().current_version().await; - Ok((parallel_unit_mappings, notification_version)) + Ok((worker_slot_mappings, notification_version)) } MetadataManager::V2(mgr) => { let fragment_guard = mgr.catalog_controller.get_inner_read_guard().await; - let parallel_unit_mappings = fragment_guard + let worker_slot_mappings = fragment_guard .all_running_fragment_mappings() .await? .collect_vec(); let notification_version = self.env.notification_manager().current_version().await; - Ok((parallel_unit_mappings, notification_version)) + Ok((worker_slot_mappings, notification_version)) } } } diff --git a/src/meta/service/src/scale_service.rs b/src/meta/service/src/scale_service.rs index 273fdbca9bc07..33d328a63f88f 100644 --- a/src/meta/service/src/scale_service.rs +++ b/src/meta/service/src/scale_service.rs @@ -14,32 +14,31 @@ use std::collections::HashMap; -use risingwave_common::catalog; +use risingwave_common::catalog::TableId; use risingwave_meta::manager::MetadataManager; use risingwave_meta::model::TableParallelism; -use risingwave_meta::stream::{ScaleControllerRef, TableRevision}; +use risingwave_meta::stream::{ + RescheduleOptions, ScaleControllerRef, TableRevision, WorkerReschedule, +}; use risingwave_meta_model_v2::FragmentId; use risingwave_pb::common::WorkerType; use risingwave_pb::meta::scale_service_server::ScaleService; use risingwave_pb::meta::{ - GetClusterInfoRequest, GetClusterInfoResponse, GetReschedulePlanRequest, - GetReschedulePlanResponse, Reschedule, RescheduleRequest, RescheduleResponse, + GetClusterInfoRequest, GetClusterInfoResponse, PbWorkerReschedule, RescheduleRequest, + RescheduleResponse, }; use risingwave_pb::source::{ConnectorSplit, ConnectorSplits}; use tonic::{Request, Response, Status}; use crate::barrier::BarrierManagerRef; use crate::model::MetadataModel; -use crate::stream::{ - GlobalStreamManagerRef, ParallelUnitReschedule, RescheduleOptions, SourceManagerRef, -}; +use crate::stream::{GlobalStreamManagerRef, SourceManagerRef}; pub struct ScaleServiceImpl { metadata_manager: MetadataManager, source_manager: SourceManagerRef, stream_manager: GlobalStreamManagerRef, barrier_manager: BarrierManagerRef, - scale_controller: ScaleControllerRef, } impl ScaleServiceImpl { @@ -48,14 +47,13 @@ impl ScaleServiceImpl { source_manager: SourceManagerRef, stream_manager: GlobalStreamManagerRef, barrier_manager: BarrierManagerRef, - scale_controller: ScaleControllerRef, + _scale_controller: ScaleControllerRef, ) -> Self { Self { metadata_manager, source_manager, stream_manager, barrier_manager, - scale_controller, } } @@ -137,7 +135,7 @@ impl ScaleService for ScaleServiceImpl { self.barrier_manager.check_status_running()?; let RescheduleRequest { - reschedules, + worker_reschedules, revision, resolve_no_shuffle_upstream, } = request.into_inner(); @@ -162,7 +160,7 @@ impl ScaleService for ScaleServiceImpl { for (table_id, table) in guard.table_fragments() { if table .fragment_ids() - .any(|fragment_id| reschedules.contains_key(&fragment_id)) + .any(|fragment_id| worker_reschedules.contains_key(&fragment_id)) { table_parallelisms.insert(*table_id, TableParallelism::Custom); } @@ -174,13 +172,16 @@ impl ScaleService for ScaleServiceImpl { let streaming_job_ids = mgr .catalog_controller .get_fragment_job_id( - reschedules.keys().map(|id| *id as FragmentId).collect(), + worker_reschedules + .keys() + .map(|id| *id as FragmentId) + .collect(), ) .await?; streaming_job_ids .into_iter() - .map(|id| (catalog::TableId::new(id as _), TableParallelism::Custom)) + .map(|id| (TableId::new(id as _), TableParallelism::Custom)) .collect() } } @@ -188,22 +189,25 @@ impl ScaleService for ScaleServiceImpl { self.stream_manager .reschedule_actors( - reschedules + worker_reschedules .into_iter() .map(|(fragment_id, reschedule)| { - let Reschedule { - added_parallel_units, - removed_parallel_units, + let PbWorkerReschedule { + increased_actor_count, + decreased_actor_count, } = reschedule; - let added_parallel_units = added_parallel_units.into_iter().collect(); - let removed_parallel_units = removed_parallel_units.into_iter().collect(); - ( fragment_id, - ParallelUnitReschedule { - added_parallel_units, - removed_parallel_units, + WorkerReschedule { + increased_actor_count: increased_actor_count + .into_iter() + .map(|(k, v)| (k as _, v as _)) + .collect(), + decreased_actor_count: decreased_actor_count + .into_iter() + .map(|(k, v)| (k as _, v as _)) + .collect(), }, ) }) @@ -223,62 +227,4 @@ impl ScaleService for ScaleServiceImpl { revision: next_revision.into(), })) } - - #[cfg_attr(coverage, coverage(off))] - async fn get_reschedule_plan( - &self, - request: Request, - ) -> Result, Status> { - self.barrier_manager.check_status_running()?; - - let req = request.into_inner(); - - let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await; - - let current_revision = self.get_revision().await; - - if req.revision != current_revision.inner() { - return Ok(Response::new(GetReschedulePlanResponse { - success: false, - revision: current_revision.inner(), - reschedules: Default::default(), - })); - } - - let policy = req - .policy - .ok_or_else(|| Status::invalid_argument("policy is required"))?; - - let scale_controller = &self.scale_controller; - - let plan = scale_controller.get_reschedule_plan(policy).await?; - - let next_revision = self.get_revision().await; - - // generate reschedule plan will not change the revision - assert_eq!(current_revision, next_revision); - - Ok(Response::new(GetReschedulePlanResponse { - success: true, - revision: next_revision.into(), - reschedules: plan - .into_iter() - .map(|(fragment_id, reschedule)| { - ( - fragment_id, - Reschedule { - added_parallel_units: reschedule - .added_parallel_units - .into_iter() - .collect(), - removed_parallel_units: reschedule - .removed_parallel_units - .into_iter() - .collect(), - }, - ) - }) - .collect(), - })) - } } diff --git a/src/meta/service/src/stream_service.rs b/src/meta/service/src/stream_service.rs index be520132b167f..bd18d617c88e0 100644 --- a/src/meta/service/src/stream_service.rs +++ b/src/meta/service/src/stream_service.rs @@ -374,7 +374,7 @@ impl StreamManagerService for StreamServiceImpl { actor_id, fragment_id: actor_to_fragment[&actor_id], state: status.state, - parallel_unit_id: status.parallel_unit.as_ref().unwrap().id, + worker_id: status.parallel_unit.as_ref().unwrap().worker_node_id, } }) }) @@ -388,7 +388,7 @@ impl StreamManagerService for StreamServiceImpl { actor_id: actor_location.actor_id as _, fragment_id: actor_location.fragment_id as _, state: PbActorState::from(actor_location.status) as _, - parallel_unit_id: actor_location.parallel_unit_id as _, + worker_id: actor_location.worker_id, }) .collect_vec() } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index bb6737735dd44..b0605335a4a26 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -619,7 +619,7 @@ impl GlobalBarrierManager { id: node.id, r#type: node.r#type, host: node.host.clone(), - parallel_units: node.parallel_units.clone(), + parallelism: node.parallelism, property: node.property.clone(), resource: node.resource.clone(), ..Default::default() diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 8f4f54692e02f..f9386a00bb173 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -14,12 +14,13 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use std::sync::Arc; -use std::time::{Duration, Instant}; +use std::time::Duration; use anyhow::{anyhow, Context}; use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_common::config::DefaultParallelism; +use risingwave_common::hash::WorkerSlotId; use risingwave_meta_model_v2::StreamingParallelism; use risingwave_pb::common::ActorInfo; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -29,6 +30,7 @@ use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::AddMutation; use thiserror_ext::AsReport; use tokio::sync::oneshot; +use tokio::time::Instant; use tokio_retry::strategy::{jitter, ExponentialBackoff}; use tracing::{debug, warn, Instrument}; @@ -542,69 +544,73 @@ impl GlobalBarrierManagerContext { ) -> MetaResult { let mgr = self.metadata_manager.as_v2_ref(); - let all_inuse_parallel_units: HashSet<_> = mgr + // all worker slots used by actors + let all_inuse_worker_slots: HashSet<_> = mgr .catalog_controller - .all_inuse_parallel_units() + .all_inuse_worker_slots() .await? .into_iter() .collect(); - let active_parallel_units: HashSet<_> = active_nodes + let active_worker_slots: HashSet<_> = active_nodes .current() .values() - .flat_map(|node| node.parallel_units.iter().map(|pu| pu.id as i32)) + .flat_map(|node| { + (0..node.parallelism).map(|idx| WorkerSlotId::new(node.id, idx as usize)) + }) .collect(); - let expired_parallel_units: BTreeSet<_> = all_inuse_parallel_units - .difference(&active_parallel_units) + let expired_worker_slots: BTreeSet<_> = all_inuse_worker_slots + .difference(&active_worker_slots) .cloned() .collect(); - if expired_parallel_units.is_empty() { + + if expired_worker_slots.is_empty() { debug!("no expired parallel units, skipping."); return self.resolve_actor_info(active_nodes).await; } debug!("start migrate actors."); - let mut to_migrate_parallel_units = expired_parallel_units.into_iter().rev().collect_vec(); + let mut to_migrate_worker_slots = expired_worker_slots.into_iter().rev().collect_vec(); debug!( "got to migrate parallel units {:#?}", - to_migrate_parallel_units + to_migrate_worker_slots ); - let mut inuse_parallel_units: HashSet<_> = all_inuse_parallel_units - .intersection(&active_parallel_units) + + let mut inuse_worker_slots: HashSet<_> = all_inuse_worker_slots + .intersection(&active_worker_slots) .cloned() .collect(); let start = Instant::now(); let mut plan = HashMap::new(); - 'discovery: while !to_migrate_parallel_units.is_empty() { - let new_parallel_units = active_nodes + 'discovery: while !to_migrate_worker_slots.is_empty() { + let new_worker_slots = active_nodes .current() .values() .flat_map(|node| { - node.parallel_units - .iter() - .filter(|pu| !inuse_parallel_units.contains(&(pu.id as _))) + (0..node.parallelism) + .map(|idx| WorkerSlotId::new(node.id, idx as _)) + .filter(|worker_slot| !inuse_worker_slots.contains(worker_slot)) }) - .cloned() .collect_vec(); - if !new_parallel_units.is_empty() { - debug!("new parallel units found: {:#?}", new_parallel_units); - for target_parallel_unit in new_parallel_units { - if let Some(from) = to_migrate_parallel_units.pop() { + if !new_worker_slots.is_empty() { + debug!("new worker slots found: {:#?}", new_worker_slots); + for target_worker_slot in new_worker_slots { + if let Some(from) = to_migrate_worker_slots.pop() { debug!( - "plan to migrate from parallel unit {} to {}", - from, target_parallel_unit.id + "plan to migrate from worker slot {} to {}", + from, target_worker_slot ); - inuse_parallel_units.insert(target_parallel_unit.id as i32); - plan.insert(from, target_parallel_unit); + inuse_worker_slots.insert(target_worker_slot); + plan.insert(from, target_worker_slot); } else { break 'discovery; } } } - if to_migrate_parallel_units.is_empty() { + if to_migrate_worker_slots.is_empty() { break; } @@ -614,7 +620,7 @@ impl GlobalBarrierManagerContext { let current_nodes = active_nodes .current() .values() - .map(|node| (node.id, &node.host, &node.parallel_units)) + .map(|node| (node.id, &node.host, node.parallelism)) .collect_vec(); warn!( current_nodes = ?current_nodes, @@ -686,8 +692,8 @@ impl GlobalBarrierManagerContext { let available_parallelism = active_nodes .current() .values() - .flat_map(|worker_node| worker_node.parallel_units.iter()) - .count(); + .map(|worker_node| worker_node.parallelism as usize) + .sum(); let table_parallelisms: HashMap<_, _> = { let streaming_parallelisms = mgr @@ -854,8 +860,8 @@ impl GlobalBarrierManagerContext { let available_parallelism = info .node_map .values() - .flat_map(|worker_node| worker_node.parallel_units.iter()) - .count(); + .map(|worker_node| worker_node.parallelism as usize) + .sum(); if available_parallelism == 0 { return Err(anyhow!("no available parallel units for auto scaling").into()); @@ -966,79 +972,80 @@ impl GlobalBarrierManagerContext { active_nodes: &mut ActiveStreamingWorkerNodes, ) -> MetaResult { let mgr = self.metadata_manager.as_v1_ref(); - let mut cached_plan = MigrationPlan::get(self.env.meta_store().as_kv()).await?; - let all_worker_parallel_units = mgr.fragment_manager.all_worker_parallel_units().await; + let all_worker_slots = mgr.fragment_manager.all_worker_slots().await; - let (expired_inuse_workers, inuse_workers): (Vec<_>, Vec<_>) = all_worker_parallel_units + let (expired_inuse_workers, inuse_workers): (Vec<_>, Vec<_>) = all_worker_slots .into_iter() .partition(|(worker, _)| expired_workers.contains(worker)); - let mut to_migrate_parallel_units: BTreeSet<_> = expired_inuse_workers + let mut to_migrate_worker_slots: BTreeSet<_> = expired_inuse_workers .into_iter() - .flat_map(|(_, pu)| pu.into_iter()) + .flat_map(|(_, worker_slots)| worker_slots.into_iter()) .collect(); - let mut inuse_parallel_units: HashSet<_> = inuse_workers + let mut inuse_worker_slots: HashSet<_> = inuse_workers .into_iter() - .flat_map(|(_, pu)| pu.into_iter()) + .flat_map(|(_, worker_slots)| worker_slots.into_iter()) .collect(); - cached_plan.parallel_unit_plan.retain(|from, to| { - if to_migrate_parallel_units.contains(from) { - if !to_migrate_parallel_units.contains(&to.id) { + cached_plan.worker_slot_plan.retain(|from, to| { + if to_migrate_worker_slots.contains(from) { + if !to_migrate_worker_slots.contains(to) { // clean up target parallel units in migration plan that are expired and not // used by any actors. - return !expired_workers.contains(&to.worker_node_id); + return !expired_workers.contains(&to.worker_id()); } return true; } false }); - to_migrate_parallel_units.retain(|id| !cached_plan.parallel_unit_plan.contains_key(id)); - inuse_parallel_units.extend(cached_plan.parallel_unit_plan.values().map(|pu| pu.id)); + to_migrate_worker_slots.retain(|id| !cached_plan.worker_slot_plan.contains_key(id)); + inuse_worker_slots.extend(cached_plan.worker_slot_plan.values()); - if to_migrate_parallel_units.is_empty() { + if to_migrate_worker_slots.is_empty() { // all expired parallel units are already in migration plan. debug!("all expired parallel units are already in migration plan."); return Ok(cached_plan); } - let mut to_migrate_parallel_units = - to_migrate_parallel_units.into_iter().rev().collect_vec(); + let mut to_migrate_worker_slots = to_migrate_worker_slots.into_iter().rev().collect_vec(); debug!( "got to migrate parallel units {:#?}", - to_migrate_parallel_units + to_migrate_worker_slots ); let start = Instant::now(); // if in-used expire parallel units are not empty, should wait for newly joined worker. - 'discovery: while !to_migrate_parallel_units.is_empty() { - let mut new_parallel_units = active_nodes + 'discovery: while !to_migrate_worker_slots.is_empty() { + let mut new_worker_slots = active_nodes .current() .values() - .flat_map(|worker| worker.parallel_units.iter().cloned()) + .flat_map(|worker| { + (0..worker.parallelism).map(move |i| WorkerSlotId::new(worker.id, i as _)) + }) .collect_vec(); - new_parallel_units.retain(|pu| !inuse_parallel_units.contains(&pu.id)); - if !new_parallel_units.is_empty() { - debug!("new parallel units found: {:#?}", new_parallel_units); - for target_parallel_unit in new_parallel_units { - if let Some(from) = to_migrate_parallel_units.pop() { + new_worker_slots.retain(|worker_slot| !inuse_worker_slots.contains(worker_slot)); + + if !new_worker_slots.is_empty() { + debug!("new worker slots found: {:#?}", new_worker_slots); + for target_worker_slot in new_worker_slots { + if let Some(from) = to_migrate_worker_slots.pop() { debug!( - "plan to migrate from parallel unit {} to {}", - from, target_parallel_unit.id + "plan to migrate from worker slot {} to {}", + from, target_worker_slot ); - inuse_parallel_units.insert(target_parallel_unit.id); + inuse_worker_slots.insert(target_worker_slot); cached_plan - .parallel_unit_plan - .insert(from, target_parallel_unit); + .worker_slot_plan + .insert(from, target_worker_slot); } else { break 'discovery; } } } - if to_migrate_parallel_units.is_empty() { + if to_migrate_worker_slots.is_empty() { break; } @@ -1048,7 +1055,7 @@ impl GlobalBarrierManagerContext { let current_nodes = active_nodes .current() .values() - .map(|node| (node.id, &node.host, &node.parallel_units)) + .map(|node| (node.id, &node.host, &node.parallelism)) .collect_vec(); warn!( current_nodes = ?current_nodes, @@ -1062,22 +1069,18 @@ impl GlobalBarrierManagerContext { // update migration plan, if there is a chain in the plan, update it. let mut new_plan = MigrationPlan::default(); - for (from, to) in &cached_plan.parallel_unit_plan { - let mut to = to.clone(); - while let Some(target) = cached_plan.parallel_unit_plan.get(&to.id) { - to = target.clone(); + for (from, to) in &cached_plan.worker_slot_plan { + let mut to = *to; + while let Some(target) = cached_plan.worker_slot_plan.get(&to) { + to = *target; } - new_plan.parallel_unit_plan.insert(*from, to); + new_plan.worker_slot_plan.insert(*from, to); } assert!( - new_plan - .parallel_unit_plan - .values() - .map(|pu| pu.id) - .all_unique(), - "target parallel units must be unique: {:?}", - new_plan.parallel_unit_plan + new_plan.worker_slot_plan.values().all_unique(), + "target worker slots must be unique: {:?}", + new_plan.worker_slot_plan ); new_plan.insert(self.env.meta_store().as_kv()).await?; diff --git a/src/meta/src/controller/catalog.rs b/src/meta/src/controller/catalog.rs index 9093505cf4a74..3cf64668dbe6f 100644 --- a/src/meta/src/controller/catalog.rs +++ b/src/meta/src/controller/catalog.rs @@ -45,9 +45,7 @@ use risingwave_pb::meta::relation::PbRelationInfo; use risingwave_pb::meta::subscribe_response::{ Info as NotificationInfo, Info, Operation as NotificationOperation, Operation, }; -use risingwave_pb::meta::{ - FragmentParallelUnitMapping, PbFragmentWorkerSlotMapping, PbRelation, PbRelationGroup, -}; +use risingwave_pb::meta::{PbFragmentWorkerSlotMapping, PbRelation, PbRelationGroup}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::FragmentTypeFlag; use risingwave_pb::user::PbUserInfo; @@ -60,15 +58,14 @@ use sea_orm::{ }; use tokio::sync::{RwLock, RwLockReadGuard}; -use super::utils::check_subscription_name_duplicate; +use super::utils::{check_subscription_name_duplicate, get_fragment_ids_by_jobs}; use crate::controller::rename::{alter_relation_rename, alter_relation_rename_refs}; use crate::controller::utils::{ check_connection_name_duplicate, check_database_name_duplicate, check_function_signature_duplicate, check_relation_name_duplicate, check_schema_name_duplicate, check_secret_name_duplicate, ensure_object_id, ensure_object_not_refer, ensure_schema_empty, - ensure_user_id, get_fragment_mappings_by_jobs, get_referring_objects, - get_referring_objects_cascade, get_user_privilege, list_user_info_by_ids, - resolve_source_register_info_for_jobs, PartialObject, + ensure_user_id, get_referring_objects, get_referring_objects_cascade, get_user_privilege, + list_user_info_by_ids, resolve_source_register_info_for_jobs, PartialObject, }; use crate::controller::ObjectModel; use crate::manager::{Catalog, MetaSrvEnv, NotificationVersion, IGNORED_NOTIFICATION_VERSION}; @@ -280,15 +277,13 @@ impl CatalogController { .all(&txn) .await?; - let fragment_mappings = get_fragment_mappings_by_jobs(&txn, streaming_jobs.clone()) + let fragment_mappings = get_fragment_ids_by_jobs(&txn, streaming_jobs.clone()) .await? .into_iter() - .map( - |FragmentParallelUnitMapping { fragment_id, .. }| PbFragmentWorkerSlotMapping { - fragment_id, - mapping: None, - }, - ) + .map(|fragment_id| PbFragmentWorkerSlotMapping { + fragment_id: fragment_id as _, + mapping: None, + }) .collect(); // The schema and objects in the database will be delete cascade. @@ -2072,8 +2067,7 @@ impl CatalogController { let (source_fragments, removed_actors, removed_fragments) = resolve_source_register_info_for_jobs(&txn, to_drop_streaming_jobs.clone()).await?; - let fragment_mappings = - get_fragment_mappings_by_jobs(&txn, to_drop_streaming_jobs.clone()).await?; + let fragment_ids = get_fragment_ids_by_jobs(&txn, to_drop_streaming_jobs.clone()).await?; // Find affect users with privileges on all this objects. let to_update_user_ids: Vec = UserPrivilege::find() @@ -2173,14 +2167,12 @@ impl CatalogController { ) .await; - let fragment_mappings = fragment_mappings + let fragment_mappings = fragment_ids .into_iter() - .map( - |FragmentParallelUnitMapping { fragment_id, .. }| PbFragmentWorkerSlotMapping { - fragment_id, - mapping: None, - }, - ) + .map(|fragment_id| PbFragmentWorkerSlotMapping { + fragment_id: fragment_id as _, + mapping: None, + }) .collect(); self.notify_fragment_mapping(NotificationOperation::Delete, fragment_mappings) diff --git a/src/meta/src/controller/cluster.rs b/src/meta/src/controller/cluster.rs index 41654192a58c8..9a82943412f71 100644 --- a/src/meta/src/controller/cluster.rs +++ b/src/meta/src/controller/cluster.rs @@ -20,6 +20,7 @@ use std::sync::Arc; use std::time::{Duration, SystemTime}; use itertools::Itertools; +use risingwave_common::hash::WorkerSlotId; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; @@ -27,12 +28,9 @@ use risingwave_common::RW_VERSION; use risingwave_hummock_sdk::HummockSstableObjectId; use risingwave_meta_model_v2::prelude::{Worker, WorkerProperty}; use risingwave_meta_model_v2::worker::{WorkerStatus, WorkerType}; -use risingwave_meta_model_v2::{worker, worker_property, I32Array, TransactionId, WorkerId}; +use risingwave_meta_model_v2::{worker, worker_property, TransactionId, WorkerId}; use risingwave_pb::common::worker_node::{PbProperty, PbResource, PbState}; -use risingwave_pb::common::{ - HostAddress, ParallelUnit, PbHostAddress, PbParallelUnit, PbWorkerNode, PbWorkerType, - WorkerNode, -}; +use risingwave_pb::common::{HostAddress, PbHostAddress, PbWorkerNode, PbWorkerType, WorkerNode}; use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; use risingwave_pb::meta::heartbeat_request; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -80,20 +78,7 @@ impl From for PbWorkerNode { port: info.0.port, }), state: PbState::from(info.0.status) as _, - parallel_units: info - .1 - .as_ref() - .map(|p| { - p.parallel_unit_ids - .0 - .iter() - .map(|&id| PbParallelUnit { - id: id as _, - worker_node_id: info.0.worker_id as _, - }) - .collect_vec() - }) - .unwrap_or_default(), + parallelism: info.1.as_ref().map(|p| p.parallelism).unwrap_or_default() as u32, property: info.1.as_ref().map(|p| PbProperty { is_streaming: p.is_streaming, is_serving: p.is_serving, @@ -367,8 +352,8 @@ impl ClusterController { .await } - pub async fn list_active_parallel_units(&self) -> MetaResult> { - self.inner.read().await.list_active_parallel_units().await + pub async fn list_active_worker_slots(&self) -> MetaResult> { + self.inner.read().await.list_active_worker_slots().await } /// Get the cluster info used for scheduling a streaming job, containing all nodes that are @@ -466,7 +451,7 @@ fn meta_node_info(host: &str, started_at: Option) -> PbWorkerNode { .map(HostAddr::to_protobuf) .ok(), state: PbState::Running as _, - parallel_units: vec![], + parallelism: 0, property: None, transactional_id: None, resource: Some(risingwave_pb::common::worker_node::Resource { @@ -599,13 +584,6 @@ impl ClusterControllerInner { ) -> MetaResult { let txn = self.db.begin().await?; - // TODO: remove this workaround when we deprecate parallel unit ids. - let derive_parallel_units = |txn_id: TransactionId, start: i32, end: i32| { - (start..end) - .map(|idx| (idx << Self::MAX_WORKER_REUSABLE_ID_BITS) + txn_id) - .collect_vec() - }; - let worker = Worker::find() .filter( worker::Column::Host @@ -620,28 +598,26 @@ impl ClusterControllerInner { assert_eq!(worker.worker_type, r#type.into()); return if worker.worker_type == WorkerType::ComputeNode { let property = property.unwrap(); - let txn_id = worker.transaction_id.unwrap(); - let mut current_parallelism = property.parallel_unit_ids.0.clone(); + let mut current_parallelism = property.parallelism as usize; let new_parallelism = add_property.worker_node_parallelism as usize; - - match new_parallelism.cmp(¤t_parallelism.len()) { + match new_parallelism.cmp(¤t_parallelism) { Ordering::Less => { if !self.disable_automatic_parallelism_control { // Handing over to the subsequent recovery loop for a forced reschedule. tracing::info!( "worker {} parallelism reduced from {} to {}", worker.worker_id, - current_parallelism.len(), + current_parallelism, new_parallelism ); - current_parallelism.truncate(new_parallelism); + current_parallelism = new_parallelism; } else { // Warn and keep the original parallelism if the worker registered with a // smaller parallelism. tracing::warn!( "worker {} parallelism is less than current, current is {}, but received {}", worker.worker_id, - current_parallelism.len(), + current_parallelism, new_parallelism ); } @@ -650,14 +626,10 @@ impl ClusterControllerInner { tracing::info!( "worker {} parallelism updated from {} to {}", worker.worker_id, - current_parallelism.len(), + current_parallelism, new_parallelism ); - current_parallelism.extend(derive_parallel_units( - txn_id, - current_parallelism.len() as _, - new_parallelism as _, - )); + current_parallelism = new_parallelism; } Ordering::Equal => {} } @@ -666,7 +638,7 @@ impl ClusterControllerInner { // keep `is_unschedulable` unchanged. property.is_streaming = Set(add_property.is_streaming); property.is_serving = Set(add_property.is_serving); - property.parallel_unit_ids = Set(I32Array(current_parallelism)); + property.parallelism = Set(current_parallelism as _); WorkerProperty::update(property).exec(&txn).await?; txn.commit().await?; @@ -694,11 +666,10 @@ impl ClusterControllerInner { if r#type == PbWorkerType::ComputeNode { let property = worker_property::ActiveModel { worker_id: Set(worker_id), - parallel_unit_ids: Set(I32Array(derive_parallel_units( - *txn_id.as_ref().unwrap(), - 0, - add_property.worker_node_parallelism as _, - ))), + parallelism: Set(add_property + .worker_node_parallelism + .try_into() + .expect("invalid parallelism")), is_streaming: Set(add_property.is_streaming), is_serving: Set(add_property.is_serving), is_unschedulable: Set(add_property.is_unschedulable), @@ -843,23 +814,20 @@ impl ClusterControllerInner { .collect_vec()) } - pub async fn list_active_parallel_units(&self) -> MetaResult> { - let parallel_units: Vec<(WorkerId, I32Array)> = WorkerProperty::find() + pub async fn list_active_worker_slots(&self) -> MetaResult> { + let worker_parallelisms: Vec<(WorkerId, i32)> = WorkerProperty::find() .select_only() .column(worker_property::Column::WorkerId) - .column(worker_property::Column::ParallelUnitIds) + .column(worker_property::Column::Parallelism) .inner_join(Worker) .filter(worker::Column::Status.eq(WorkerStatus::Running)) .into_tuple() .all(&self.db) .await?; - Ok(parallel_units + Ok(worker_parallelisms .into_iter() - .flat_map(|(id, pu)| { - pu.0.into_iter().map(move |parallel_unit_id| ParallelUnit { - id: parallel_unit_id as _, - worker_node_id: id as _, - }) + .flat_map(|(worker_id, parallelism)| { + (0..parallelism).map(move |idx| WorkerSlotId::new(worker_id as u32, idx as usize)) }) .collect_vec()) } @@ -889,32 +857,22 @@ impl ClusterControllerInner { pub async fn get_streaming_cluster_info(&self) -> MetaResult { let mut streaming_workers = self.list_active_streaming_workers().await?; - let unschedulable_worker_node = streaming_workers + let unschedulable_workers = streaming_workers .extract_if(|worker| { worker .property .as_ref() .map_or(false, |p| p.is_unschedulable) }) - .collect_vec(); + .map(|w| w.id) + .collect(); let active_workers: HashMap<_, _> = streaming_workers.into_iter().map(|w| (w.id, w)).collect(); - let active_parallel_units = active_workers - .values() - .flat_map(|worker| worker.parallel_units.iter().map(|p| (p.id, p.clone()))) - .collect(); - - let unschedulable_parallel_units = unschedulable_worker_node - .iter() - .flat_map(|worker| worker.parallel_units.iter().map(|p| (p.id, p.clone()))) - .collect(); - Ok(StreamingClusterInfo { worker_nodes: active_workers, - parallel_units: active_parallel_units, - unschedulable_parallel_units, + unschedulable_workers, }) } @@ -978,7 +936,7 @@ mod tests { } // Since no worker is active, the parallel unit count should be 0. - assert_eq!(cluster_ctl.list_active_parallel_units().await?.len(), 0); + assert_eq!(cluster_ctl.list_active_worker_slots().await?.len(), 0); for id in &worker_ids { cluster_ctl.activate_worker(*id).await?; @@ -997,7 +955,7 @@ mod tests { worker_count ); assert_eq!( - cluster_ctl.list_active_parallel_units().await?.len(), + cluster_ctl.list_active_worker_slots().await?.len(), parallelism_num * worker_count ); @@ -1022,9 +980,9 @@ mod tests { cluster_ctl.list_active_serving_workers().await?.len(), worker_count - 1 ); - let parallel_units = cluster_ctl.list_active_parallel_units().await?; - assert!(parallel_units.iter().map(|pu| pu.id).all_unique()); - assert_eq!(parallel_units.len(), parallelism_num * (worker_count + 1)); + let worker_slots = cluster_ctl.list_active_worker_slots().await?; + assert!(worker_slots.iter().all_unique()); + assert_eq!(worker_slots.len(), parallelism_num * (worker_count + 1)); // delete workers. for host in hosts { @@ -1032,7 +990,7 @@ mod tests { } assert_eq!(cluster_ctl.list_active_streaming_workers().await?.len(), 0); assert_eq!(cluster_ctl.list_active_serving_workers().await?.len(), 0); - assert_eq!(cluster_ctl.list_active_parallel_units().await?.len(), 0); + assert_eq!(cluster_ctl.list_active_worker_slots().await?.len(), 0); Ok(()) } diff --git a/src/meta/src/controller/fragment.rs b/src/meta/src/controller/fragment.rs index e88046ed17467..9e6aefcb88688 100644 --- a/src/meta/src/controller/fragment.rs +++ b/src/meta/src/controller/fragment.rs @@ -19,14 +19,15 @@ use std::mem::swap; use anyhow::Context; use itertools::Itertools; use risingwave_common::bail; -use risingwave_common::hash::ParallelUnitMapping; +use risingwave_common::hash::WorkerSlotId; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; +use risingwave_meta_model_v2::fragment::DistributionType; use risingwave_meta_model_v2::prelude::{Actor, ActorDispatcher, Fragment, Sink, StreamingJob}; use risingwave_meta_model_v2::{ - actor, actor_dispatcher, fragment, object, sink, streaming_job, ActorId, ActorUpstreamActors, - ConnectorSplits, ExprContext, FragmentId, FragmentVnodeMapping, I32Array, JobStatus, ObjectId, - SinkId, SourceId, StreamNode, StreamingParallelism, TableId, VnodeBitmap, WorkerId, + actor, actor_dispatcher, fragment, sink, streaming_job, ActorId, ActorUpstreamActors, + ConnectorSplits, ExprContext, FragmentId, I32Array, JobStatus, ObjectId, SinkId, SourceId, + StreamNode, StreamingParallelism, TableId, VnodeBitmap, WorkerId, }; use risingwave_pb::common::PbParallelUnit; use risingwave_pb::meta::subscribe_response::{ @@ -41,19 +42,19 @@ use risingwave_pb::meta::{ use risingwave_pb::source::PbConnectorSplits; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{ - PbDispatchStrategy, PbFragmentTypeFlag, PbStreamActor, PbStreamContext, + DispatchStrategy, PbDispatchStrategy, PbFragmentTypeFlag, PbStreamActor, PbStreamContext, }; -use sea_orm::sea_query::{Expr, Value}; +use sea_orm::sea_query::Expr; use sea_orm::ActiveValue::Set; use sea_orm::{ - ActiveModelTrait, ColumnTrait, EntityTrait, JoinType, ModelTrait, PaginatorTrait, QueryFilter, - QuerySelect, RelationTrait, TransactionTrait, + ColumnTrait, EntityTrait, JoinType, ModelTrait, PaginatorTrait, QueryFilter, QuerySelect, + RelationTrait, TransactionTrait, Value, }; use crate::controller::catalog::{CatalogController, CatalogControllerInner}; use crate::controller::utils::{ - get_actor_dispatchers, get_parallel_unit_to_worker_map, FragmentDesc, PartialActorLocation, - PartialFragmentStateTables, + get_actor_dispatchers, get_fragment_mappings, rebuild_fragment_mapping_from_actors, + FragmentDesc, PartialActorLocation, PartialFragmentStateTables, }; use crate::manager::{ActorInfos, InflightFragmentInfo, LocalNotification}; use crate::model::{TableFragments, TableParallelism}; @@ -67,24 +68,22 @@ impl CatalogControllerInner { ) -> MetaResult + '_> { let txn = self.db.begin().await?; - let fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() - .join(JoinType::InnerJoin, fragment::Relation::Object.def()) - .join(JoinType::InnerJoin, object::Relation::StreamingJob.def()) + let job_ids: Vec = StreamingJob::find() .select_only() - .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) + .column(streaming_job::Column::JobId) .filter(streaming_job::Column::JobStatus.eq(JobStatus::Created)) .into_tuple() .all(&txn) .await?; - let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; + let mut result = vec![]; + for job_id in job_ids { + let mappings = get_fragment_mappings(&txn, job_id as ObjectId).await?; - let mappings = CatalogController::convert_fragment_mappings( - fragment_mappings, - ¶llel_unit_to_worker, - )?; + result.extend(mappings.into_iter()); + } - Ok(mappings.into_iter()) + Ok(result.into_iter()) } } @@ -183,9 +182,9 @@ impl CatalogController { fragment_type_mask: pb_fragment_type_mask, distribution_type: pb_distribution_type, actors: pb_actors, - vnode_mapping: pb_vnode_mapping, state_table_ids: pb_state_table_ids, upstream_fragment_ids: pb_upstream_fragment_ids, + .. } = pb_fragment; let state_table_ids = pb_state_table_ids.into(); @@ -245,11 +244,7 @@ impl CatalogController { ) })?; - let (worker_id, parallel_unit_id) = status - .parallel_unit - .as_ref() - .map(|pu| (pu.worker_node_id as WorkerId, pu.id as i32)) - .expect("no parallel unit id found in actor_status"); + let worker_id = status.parallel_unit.clone().unwrap().get_worker_node_id() as WorkerId; assert_eq!( pb_upstream_actor_id @@ -269,7 +264,6 @@ impl CatalogController { fragment_id: fragment_id as _, status: status.get_state().unwrap().into(), splits, - parallel_unit_id, worker_id, upstream_actor_ids: upstream_actors.into(), vnode_bitmap: pb_vnode_bitmap.as_ref().map(VnodeBitmap::from), @@ -286,11 +280,6 @@ impl CatalogController { let upstream_fragment_id = pb_upstream_fragment_ids.into(); - let vnode_mapping = pb_vnode_mapping - .as_ref() - .map(FragmentVnodeMapping::from) - .unwrap(); - let stream_node = StreamNode::from(&stream_node); let distribution_type = PbFragmentDistributionType::try_from(pb_distribution_type) @@ -303,7 +292,6 @@ impl CatalogController { fragment_type_mask: pb_fragment_type_mask as _, distribution_type, stream_node, - vnode_mapping, state_table_ids, upstream_fragment_id, }; @@ -373,7 +361,6 @@ impl CatalogController { fragment_type_mask, distribution_type, stream_node, - vnode_mapping, state_table_ids, upstream_fragment_id, } = fragment; @@ -399,7 +386,6 @@ impl CatalogController { actor_id, fragment_id, status, - parallel_unit_id, worker_id, splits, upstream_actor_ids, @@ -445,7 +431,7 @@ impl CatalogController { actor_id as _, PbActorStatus { parallel_unit: Some(PbParallelUnit { - id: parallel_unit_id as _, + id: u32::MAX, worker_node_id: worker_id as _, }), state: PbActorState::from(status) as _, @@ -469,7 +455,6 @@ impl CatalogController { } let pb_upstream_fragment_ids = upstream_fragment_id.into_u32_array(); - let pb_vnode_mapping = vnode_mapping.to_protobuf(); let pb_state_table_ids = state_table_ids.into_u32_array(); let pb_distribution_type = PbFragmentDistributionType::from(distribution_type) as _; let pb_fragment = PbFragment { @@ -477,7 +462,6 @@ impl CatalogController { fragment_type_mask: fragment_type_mask as _, distribution_type: pb_distribution_type, actors: pb_actors, - vnode_mapping: Some(pb_vnode_mapping), state_table_ids: pb_state_table_ids, upstream_fragment_ids: pb_upstream_fragment_ids, }; @@ -907,93 +891,124 @@ impl CatalogController { Ok(ActorInfos::new(fragment_infos)) } - pub async fn migrate_actors(&self, plan: HashMap) -> MetaResult<()> { + pub async fn migrate_actors( + &self, + plan: HashMap, + ) -> MetaResult<()> { let inner = self.inner.read().await; let txn = inner.db.begin().await?; - for (from_pu_id, to_pu_id) in &plan { - Actor::update_many() - .col_expr( - actor::Column::ParallelUnitId, - Expr::value(Value::Int(Some(to_pu_id.id as i32))), - ) - .col_expr( - actor::Column::WorkerId, - Expr::value(Value::Int(Some(to_pu_id.worker_node_id as WorkerId))), - ) - .filter(actor::Column::ParallelUnitId.eq(*from_pu_id)) - .exec(&txn) - .await?; - } - let fragment_mapping: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() + let actors: Vec<( + FragmentId, + DistributionType, + ActorId, + Option, + WorkerId, + ActorStatus, + )> = Actor::find() .select_only() - .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) - .join(JoinType::InnerJoin, fragment::Relation::Actor.def()) - .filter(actor::Column::ParallelUnitId.is_in(plan.keys().cloned().collect::>())) + .columns([ + fragment::Column::FragmentId, + fragment::Column::DistributionType, + ]) + .columns([ + actor::Column::ActorId, + actor::Column::VnodeBitmap, + actor::Column::WorkerId, + actor::Column::Status, + ]) + .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) .into_tuple() .all(&txn) .await?; - // TODO: we'd better not store vnode mapping in fragment table and derive it from actors. - for (fragment_id, vnode_mapping) in &fragment_mapping { - let mut pb_vnode_mapping = vnode_mapping.to_protobuf(); - pb_vnode_mapping.data.iter_mut().for_each(|id| { - if let Some(new_id) = plan.get(&(*id as i32)) { - *id = new_id.id; + let mut actor_locations = HashMap::new(); + + for (fragment_id, _, actor_id, _, worker_id, _) in &actors { + // question: shall we check if the actor is Inactive? + actor_locations + .entry(*worker_id) + .or_insert(HashMap::new()) + .entry(*fragment_id) + .or_insert(BTreeSet::new()) + .insert(*actor_id); + } + + let expired_workers: HashSet<_> = plan.keys().map(|k| k.worker_id() as WorkerId).collect(); + + let mut actor_migration_plan = HashMap::new(); + for (worker, fragment) in actor_locations { + if expired_workers.contains(&worker) { + for (_, actors) in fragment { + let worker_slot_to_actor: HashMap<_, _> = actors + .iter() + .enumerate() + .map(|(idx, actor_id)| { + (WorkerSlotId::new(worker as _, idx as _), *actor_id) + }) + .collect(); + + for (worker_slot, actor) in worker_slot_to_actor { + actor_migration_plan + .insert(actor, plan[&worker_slot].worker_id() as WorkerId); + } } - }); - fragment::ActiveModel { - fragment_id: Set(*fragment_id), - vnode_mapping: Set(FragmentVnodeMapping::from(&pb_vnode_mapping)), - ..Default::default() } - .update(&txn) - .await?; } - let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; - - let fragment_worker_slot_mapping = - Self::convert_fragment_mappings(fragment_mapping, ¶llel_unit_to_worker)?; + for (actor, worker) in actor_migration_plan { + Actor::update_many() + .col_expr( + actor::Column::WorkerId, + Expr::value(Value::Int(Some(worker))), + ) + .filter(actor::Column::ActorId.eq(actor)) + .exec(&txn) + .await?; + } txn.commit().await?; - self.notify_fragment_mapping(NotificationOperation::Update, fragment_worker_slot_mapping) - .await; + self.notify_fragment_mapping( + NotificationOperation::Update, + rebuild_fragment_mapping_from_actors(actors), + ) + .await; Ok(()) } - pub(crate) fn convert_fragment_mappings( - fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)>, - parallel_unit_to_worker: &HashMap, - ) -> MetaResult> { - let mut result = vec![]; - - for (fragment_id, mapping) in fragment_mappings { - result.push(PbFragmentWorkerSlotMapping { - fragment_id: fragment_id as _, - mapping: Some( - ParallelUnitMapping::from_protobuf(&mapping.to_protobuf()) - .to_worker_slot(parallel_unit_to_worker)? - .to_protobuf(), - ), - }) - } - - Ok(result) - } - - pub async fn all_inuse_parallel_units(&self) -> MetaResult> { + pub async fn all_inuse_worker_slots(&self) -> MetaResult> { let inner = self.inner.read().await; - let parallel_units: Vec = Actor::find() + + let actors: Vec<(FragmentId, ActorId, WorkerId)> = Actor::find() .select_only() - .column(actor::Column::ParallelUnitId) - .distinct() + .columns([fragment::Column::FragmentId]) + .columns([actor::Column::ActorId, actor::Column::WorkerId]) + .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) .into_tuple() .all(&inner.db) .await?; - Ok(parallel_units) + + let mut actor_locations = HashMap::new(); + + for (fragment_id, _, worker_id) in actors { + *actor_locations + .entry(worker_id) + .or_insert(HashMap::new()) + .entry(fragment_id) + .or_insert(0_usize) += 1; + } + + let mut result = HashSet::new(); + for (worker_id, mapping) in actor_locations { + let max_fragment_len = mapping.values().max().unwrap(); + + result + .extend((0..*max_fragment_len).map(|idx| WorkerSlotId::new(worker_id as u32, idx))) + } + + Ok(result) } pub async fn all_node_actors( @@ -1159,7 +1174,7 @@ impl CatalogController { pub async fn get_upstream_root_fragments( &self, upstream_job_ids: Vec, - ) -> MetaResult> { + ) -> MetaResult<(HashMap, Vec<(ActorId, WorkerId)>)> { let inner = self.inner.read().await; let all_upstream_fragments = Fragment::find() @@ -1192,14 +1207,24 @@ impl CatalogController { ); } - Ok(root_fragments) + let actors: Vec<(ActorId, WorkerId)> = Actor::find() + .select_only() + .columns([actor::Column::ActorId, actor::Column::WorkerId]) + .into_tuple() + .all(&inner.db) + .await?; + + Ok((root_fragments, actors)) } /// Get the downstream `Chain` fragments of the specified table. pub async fn get_downstream_chain_fragments( &self, job_id: ObjectId, - ) -> MetaResult> { + ) -> MetaResult<( + Vec<(DispatchStrategy, PbFragment)>, + Vec<(ActorId, WorkerId)>, + )> { let mview_fragment = self.get_mview_fragment(job_id).await?; let downstream_dispatches: HashMap<_, _> = mview_fragment.actors[0] .dispatcher @@ -1236,7 +1261,14 @@ impl CatalogController { chain_fragments.push((dispatch_strategy, fragment)); } - Ok(chain_fragments) + let actors: Vec<(ActorId, WorkerId)> = Actor::find() + .select_only() + .columns([actor::Column::ActorId, actor::Column::WorkerId]) + .into_tuple() + .all(&inner.db) + .await?; + + Ok((chain_fragments, actors)) } pub async fn load_source_fragment_ids( @@ -1379,15 +1411,14 @@ mod tests { use std::collections::{BTreeMap, HashMap}; use itertools::Itertools; - use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping}; + use risingwave_common::hash::ActorMapping; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_meta_model_v2::actor::ActorStatus; use risingwave_meta_model_v2::fragment::DistributionType; use risingwave_meta_model_v2::{ actor, actor_dispatcher, fragment, ActorId, ActorUpstreamActors, ConnectorSplits, - ExprContext, FragmentId, FragmentVnodeMapping, I32Array, ObjectId, StreamNode, TableId, - VnodeBitmap, + ExprContext, FragmentId, I32Array, ObjectId, StreamNode, TableId, VnodeBitmap, }; use risingwave_pb::common::ParallelUnit; use risingwave_pb::meta::table_fragments::actor_status::PbActorState; @@ -1412,15 +1443,6 @@ mod tests { const TEST_STATE_TABLE_ID: TableId = 1000; - fn generate_parallel_units(count: u32) -> Vec { - (0..count) - .map(|parallel_unit_id| ParallelUnit { - id: parallel_unit_id, - worker_node_id: 0, - }) - .collect_vec() - } - fn generate_dispatchers_for_actor(actor_id: u32) -> Vec { vec![PbDispatcher { r#type: PbDispatcherType::Hash as _, @@ -1465,10 +1487,6 @@ mod tests { #[tokio::test] async fn test_extract_fragment() -> MetaResult<()> { let actor_count = 3u32; - let parallel_units = generate_parallel_units(actor_count); - let parallel_unit_mapping = ParallelUnitMapping::build(¶llel_units); - let actor_vnode_bitmaps = parallel_unit_mapping.to_bitmaps(); - let upstream_actor_ids: HashMap>> = (0 ..actor_count) .map(|actor_id| { @@ -1479,6 +1497,9 @@ mod tests { }) .collect(); + let actor_bitmaps = + ActorMapping::new_uniform((0..actor_count).map(|i| i as _)).to_bitmaps(); + let pb_actors = (0..actor_count) .map(|actor_id| { let actor_upstream_actor_ids = @@ -1495,7 +1516,7 @@ mod tests { .flatten() .map(|id| *id as _) .collect(), - vnode_bitmap: actor_vnode_bitmaps + vnode_bitmap: actor_bitmaps .get(&actor_id) .cloned() .map(|bitmap| bitmap.to_protobuf()), @@ -1512,7 +1533,6 @@ mod tests { fragment_type_mask: PbFragmentTypeFlag::Source as _, distribution_type: PbFragmentDistributionType::Hash as _, actors: pb_actors.clone(), - vnode_mapping: Some(parallel_unit_mapping.to_protobuf()), state_table_ids: vec![TEST_STATE_TABLE_ID as _], upstream_fragment_ids: upstream_actor_ids .values() @@ -1525,7 +1545,10 @@ mod tests { ( actor_id, PbActorStatus { - parallel_unit: Some(parallel_units[actor_id as usize].clone()), + parallel_unit: Some(ParallelUnit { + id: u32::MAX, + worker_node_id: 0, + }), state: PbActorState::Running as _, }, ) @@ -1579,9 +1602,6 @@ mod tests { #[tokio::test] async fn test_compose_fragment() -> MetaResult<()> { let actor_count = 3u32; - let parallel_units = generate_parallel_units(actor_count); - let parallel_unit_mapping = ParallelUnitMapping::build(¶llel_units); - let mut actor_vnode_bitmaps = parallel_unit_mapping.to_bitmaps(); let upstream_actor_ids: HashMap>> = (0 ..actor_count) @@ -1593,14 +1613,11 @@ mod tests { }) .collect(); + let mut actor_bitmaps = + ActorMapping::new_uniform((0..actor_count).map(|i| i as _)).to_bitmaps(); + let actors = (0..actor_count) .map(|actor_id| { - let parallel_unit_id = actor_id as ParallelUnitId; - - let vnode_bitmap = actor_vnode_bitmaps - .remove(¶llel_unit_id) - .map(|m| VnodeBitmap::from(&m.to_protobuf())); - let actor_splits = Some(ConnectorSplits::from(&PbConnectorSplits { splits: vec![PbConnectorSplit { split_type: "dummy".to_string(), @@ -1616,10 +1633,13 @@ mod tests { fragment_id: TEST_FRAGMENT_ID, status: ActorStatus::Running, splits: actor_splits, - parallel_unit_id: parallel_unit_id as i32, worker_id: 0, upstream_actor_ids: ActorUpstreamActors(actor_upstream_actor_ids), - vnode_bitmap, + vnode_bitmap: actor_bitmaps + .remove(&actor_id) + .map(|bitmap| bitmap.to_protobuf()) + .as_ref() + .map(VnodeBitmap::from), expr_context: ExprContext::from(&PbExprContext { time_zone: String::from("America/New_York"), }), @@ -1659,7 +1679,6 @@ mod tests { fragment_type_mask: 0, distribution_type: DistributionType::Hash, stream_node: StreamNode::from(&stream_node), - vnode_mapping: FragmentVnodeMapping::from(¶llel_unit_mapping.to_protobuf()), state_table_ids: I32Array(vec![TEST_STATE_TABLE_ID]), upstream_fragment_id: I32Array::default(), }; @@ -1674,14 +1693,6 @@ mod tests { assert_eq!(pb_actor_status.len(), actor_count as usize); assert_eq!(pb_actor_splits.len(), actor_count as usize); - for (actor_id, actor_status) in &pb_actor_status { - let parallel_unit_id = parallel_units[*actor_id as usize].id; - assert_eq!( - parallel_unit_id, - actor_status.parallel_unit.clone().unwrap().id - ); - } - let pb_actors = pb_fragment.actors.clone(); check_fragment_template(fragment.clone(), pb_actors.clone(), &upstream_actor_ids); @@ -1703,7 +1714,6 @@ mod tests { fragment_id, status, splits, - parallel_unit_id, worker_id: _, upstream_actor_ids, vnode_bitmap, @@ -1723,7 +1733,6 @@ mod tests { { assert_eq!(actor_id, pb_actor_id as ActorId); assert_eq!(fragment_id, pb_fragment_id as FragmentId); - assert_eq!(parallel_unit_id, pb_actor_id as i32); let upstream_actor_ids = upstream_actor_ids.into_inner(); assert_eq!( @@ -1779,7 +1788,6 @@ mod tests { fragment_type_mask, distribution_type: pb_distribution_type, actors: _, - vnode_mapping: pb_vnode_mapping, state_table_ids: pb_state_table_ids, upstream_fragment_ids: pb_upstream_fragment_ids, } = pb_fragment; @@ -1790,10 +1798,6 @@ mod tests { pb_distribution_type, PbFragmentDistributionType::from(fragment.distribution_type) as i32 ); - assert_eq!( - pb_vnode_mapping.unwrap(), - fragment.vnode_mapping.to_protobuf() - ); assert_eq!( pb_upstream_fragment_ids, diff --git a/src/meta/src/controller/streaming_job.rs b/src/meta/src/controller/streaming_job.rs index 3e2c5db4776db..19e722e61086a 100644 --- a/src/meta/src/controller/streaming_job.rs +++ b/src/meta/src/controller/streaming_job.rs @@ -17,7 +17,6 @@ use std::num::NonZeroUsize; use itertools::Itertools; use risingwave_common::buffer::Bitmap; -use risingwave_common::hash::{ActorMapping, ParallelUnitId, ParallelUnitMapping}; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::stream_graph_visitor::visit_stream_node; use risingwave_common::{bail, current_cluster_version}; @@ -43,8 +42,7 @@ use risingwave_pb::meta::subscribe_response::{ }; use risingwave_pb::meta::table_fragments::PbActorStatus; use risingwave_pb::meta::{ - FragmentWorkerSlotMapping, PbFragmentWorkerSlotMapping, PbRelation, PbRelationGroup, - PbTableFragments, Relation, + PbFragmentWorkerSlotMapping, PbRelation, PbRelationGroup, PbTableFragments, Relation, }; use risingwave_pb::source::{PbConnectorSplit, PbConnectorSplits}; use risingwave_pb::stream_plan::stream_fragment_graph::Parallelism; @@ -66,7 +64,7 @@ use crate::controller::catalog::CatalogController; use crate::controller::rename::ReplaceTableExprRewriter; use crate::controller::utils::{ check_relation_name_duplicate, check_sink_into_table_cycle, ensure_object_id, ensure_user_id, - get_fragment_actor_ids, get_fragment_mappings, get_parallel_unit_to_worker_map, + get_fragment_actor_ids, get_fragment_mappings, }; use crate::controller::ObjectModel; use crate::manager::{NotificationVersion, SinkId, StreamingJob}; @@ -1265,9 +1263,7 @@ impl CatalogController { let txn = inner.db.begin().await?; - let parallel_unit_to_worker = get_parallel_unit_to_worker_map(&txn).await?; - - let mut fragment_mapping_to_notify = vec![]; + let fragment_mapping_to_notify = vec![]; // for assert only let mut assert_dispatcher_update_checker = HashSet::new(); @@ -1359,7 +1355,6 @@ impl CatalogController { fragment_id: Set(fragment_id as _), status: Set(ActorStatus::Running), splits: Set(splits.map(|splits| (&PbConnectorSplits { splits }).into())), - parallel_unit_id: Set(parallel_unit.id as _), worker_id: Set(parallel_unit.worker_node_id as _), upstream_actor_ids: Set(actor_upstreams), vnode_bitmap: Set(vnode_bitmap.as_ref().map(|bitmap| bitmap.into())), @@ -1417,40 +1412,14 @@ impl CatalogController { let fragment_actors = fragment.find_related(Actor).all(&txn).await?; - let mut actor_to_parallel_unit = HashMap::with_capacity(fragment_actors.len()); let mut actor_to_vnode_bitmap = HashMap::with_capacity(fragment_actors.len()); for actor in &fragment_actors { - actor_to_parallel_unit.insert(actor.actor_id as u32, actor.parallel_unit_id as _); if let Some(vnode_bitmap) = &actor.vnode_bitmap { let bitmap = Bitmap::from(&vnode_bitmap.to_protobuf()); actor_to_vnode_bitmap.insert(actor.actor_id as u32, bitmap); } } - let vnode_mapping = if actor_to_vnode_bitmap.is_empty() { - let parallel_unit = *actor_to_parallel_unit.values().exactly_one().unwrap(); - ParallelUnitMapping::new_single(parallel_unit as ParallelUnitId) - } else { - // Generate the parallel unit mapping from the fragment's actor bitmaps. - assert_eq!(actor_to_vnode_bitmap.len(), actor_to_parallel_unit.len()); - ActorMapping::from_bitmaps(&actor_to_vnode_bitmap) - .to_parallel_unit(&actor_to_parallel_unit) - } - .to_protobuf(); - - let mut fragment = fragment.into_active_model(); - fragment.vnode_mapping = Set((&vnode_mapping).into()); - fragment.update(&txn).await?; - - let worker_slot_mapping = ParallelUnitMapping::from_protobuf(&vnode_mapping) - .to_worker_slot(¶llel_unit_to_worker)? - .to_protobuf(); - - fragment_mapping_to_notify.push(FragmentWorkerSlotMapping { - fragment_id: fragment_id as u32, - mapping: Some(worker_slot_mapping), - }); - // for downstream and upstream let removed_actor_ids: HashSet<_> = removed_actors .iter() diff --git a/src/meta/src/controller/utils.rs b/src/meta/src/controller/utils.rs index 8a8f1b2d71cda..1cf56c7c4152f 100644 --- a/src/meta/src/controller/utils.rs +++ b/src/meta/src/controller/utils.rs @@ -16,6 +16,9 @@ use std::collections::{BTreeSet, HashMap, HashSet}; use anyhow::anyhow; use itertools::Itertools; +use risingwave_common::buffer::Bitmap; +use risingwave_common::hash; +use risingwave_common::hash::{ActorMapping, WorkerSlotId, WorkerSlotMapping}; use risingwave_meta_model_migration::WithQuery; use risingwave_meta_model_v2::actor::ActorStatus; use risingwave_meta_model_v2::fragment::DistributionType; @@ -24,11 +27,11 @@ use risingwave_meta_model_v2::prelude::*; use risingwave_meta_model_v2::{ actor, actor_dispatcher, connection, database, fragment, function, index, object, object_dependency, schema, secret, sink, source, subscription, table, user, user_privilege, - view, worker_property, ActorId, DataTypeArray, DatabaseId, FragmentId, FragmentVnodeMapping, - I32Array, ObjectId, PrivilegeId, SchemaId, SourceId, StreamNode, UserId, WorkerId, + view, ActorId, DataTypeArray, DatabaseId, FragmentId, I32Array, ObjectId, PrivilegeId, + SchemaId, SourceId, StreamNode, UserId, VnodeBitmap, WorkerId, }; use risingwave_pb::catalog::{PbConnection, PbFunction, PbSecret, PbSubscription}; -use risingwave_pb::meta::{PbFragmentParallelUnitMapping, PbFragmentWorkerSlotMapping}; +use risingwave_pb::meta::{FragmentWorkerSlotMapping, PbFragmentWorkerSlotMapping}; use risingwave_pb::stream_plan::stream_node::NodeBody; use risingwave_pb::stream_plan::{PbFragmentTypeFlag, PbStreamNode, StreamSource}; use risingwave_pb::user::grant_privilege::{PbAction, PbActionWithGrantOption, PbObject}; @@ -42,7 +45,6 @@ use sea_orm::{ Order, PaginatorTrait, QueryFilter, QuerySelect, RelationTrait, Statement, }; -use crate::controller::catalog::CatalogController; use crate::{MetaError, MetaResult}; /// This function will construct a query using recursive cte to find all objects[(id, `obj_type`)] that are used by the given object. @@ -242,7 +244,7 @@ pub struct PartialFragmentStateTables { pub struct PartialActorLocation { pub actor_id: ActorId, pub fragment_id: FragmentId, - pub parallel_unit_id: i32, + pub worker_id: u32, pub status: ActorStatus, } @@ -842,46 +844,128 @@ pub async fn get_fragment_mappings( where C: ConnectionTrait, { - let parallel_unit_to_worker = get_parallel_unit_to_worker_map(db).await?; - - let fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() + let job_actors: Vec<( + FragmentId, + DistributionType, + ActorId, + Option, + WorkerId, + ActorStatus, + )> = Actor::find() .select_only() - .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) + .columns([ + fragment::Column::FragmentId, + fragment::Column::DistributionType, + ]) + .columns([ + actor::Column::ActorId, + actor::Column::VnodeBitmap, + actor::Column::WorkerId, + actor::Column::Status, + ]) + .join(JoinType::InnerJoin, actor::Relation::Fragment.def()) .filter(fragment::Column::JobId.eq(job_id)) .into_tuple() .all(db) .await?; - CatalogController::convert_fragment_mappings(fragment_mappings, ¶llel_unit_to_worker) + Ok(rebuild_fragment_mapping_from_actors(job_actors)) +} + +pub fn rebuild_fragment_mapping_from_actors( + job_actors: Vec<( + FragmentId, + DistributionType, + ActorId, + Option, + WorkerId, + ActorStatus, + )>, +) -> Vec { + let mut all_actor_locations = HashMap::new(); + let mut actor_bitmaps = HashMap::new(); + let mut fragment_actors = HashMap::new(); + let mut fragment_dist = HashMap::new(); + + for (fragment_id, dist, actor_id, bitmap, worker_id, actor_status) in job_actors { + if actor_status == ActorStatus::Inactive { + continue; + } + + all_actor_locations + .entry(fragment_id) + .or_insert(HashMap::new()) + .insert(actor_id as hash::ActorId, worker_id as u32); + actor_bitmaps.insert(actor_id, bitmap); + fragment_actors + .entry(fragment_id) + .or_insert_with(Vec::new) + .push(actor_id); + fragment_dist.insert(fragment_id, dist); + } + + let mut result = vec![]; + for (fragment_id, dist) in fragment_dist { + let mut actor_locations = all_actor_locations.remove(&fragment_id).unwrap(); + let fragment_worker_slot_mapping = match dist { + DistributionType::Single => { + let actor = fragment_actors + .remove(&fragment_id) + .unwrap() + .into_iter() + .exactly_one() + .unwrap() as hash::ActorId; + let actor_location = actor_locations.remove(&actor).unwrap(); + + WorkerSlotMapping::new_single(WorkerSlotId::new(actor_location, 0)) + } + DistributionType::Hash => { + let actors = fragment_actors.remove(&fragment_id).unwrap(); + + let all_actor_bitmaps: HashMap<_, _> = actors + .iter() + .map(|actor_id| { + let vnode_bitmap = actor_bitmaps + .remove(actor_id) + .flatten() + .expect("actor bitmap shouldn't be none in hash fragment"); + + let bitmap = Bitmap::from(&vnode_bitmap.to_protobuf()); + (*actor_id as hash::ActorId, bitmap) + }) + .collect(); + + let actor_mapping = ActorMapping::from_bitmaps(&all_actor_bitmaps); + + actor_mapping.to_worker_slot(&actor_locations) + } + }; + + result.push(PbFragmentWorkerSlotMapping { + fragment_id: fragment_id as u32, + mapping: Some(fragment_worker_slot_mapping.to_protobuf()), + }) + } + result } /// `get_fragment_mappings_by_jobs` returns the fragment vnode mappings of the given job list. -pub async fn get_fragment_mappings_by_jobs( +pub async fn get_fragment_ids_by_jobs( db: &C, job_ids: Vec, -) -> MetaResult> +) -> MetaResult> where C: ConnectionTrait, { - if job_ids.is_empty() { - return Ok(vec![]); - } - - let fragment_mappings: Vec<(FragmentId, FragmentVnodeMapping)> = Fragment::find() + let fragment_ids: Vec = Fragment::find() .select_only() - .columns([fragment::Column::FragmentId, fragment::Column::VnodeMapping]) + .column(fragment::Column::FragmentId) .filter(fragment::Column::JobId.is_in(job_ids)) .into_tuple() .all(db) .await?; - Ok(fragment_mappings - .into_iter() - .map(|(fragment_id, mapping)| PbFragmentParallelUnitMapping { - fragment_id: fragment_id as _, - mapping: Some(mapping.to_protobuf()), - }) - .collect()) + Ok(fragment_ids) } /// `get_fragment_actor_ids` returns the fragment actor ids of the given fragments. @@ -981,30 +1065,3 @@ where removed_fragments, )) } - -pub(crate) async fn get_parallel_unit_to_worker_map(db: &C) -> MetaResult> -where - C: ConnectionTrait, -{ - let worker_parallel_units = WorkerProperty::find() - .select_only() - .columns([ - worker_property::Column::WorkerId, - worker_property::Column::ParallelUnitIds, - ]) - .into_tuple::<(WorkerId, I32Array)>() - .all(db) - .await?; - - let parallel_unit_to_worker = worker_parallel_units - .into_iter() - .flat_map(|(worker_id, parallel_unit_ids)| { - parallel_unit_ids - .into_inner() - .into_iter() - .map(move |parallel_unit_id| (parallel_unit_id as u32, worker_id as u32)) - }) - .collect::>(); - - Ok(parallel_unit_to_worker) -} diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 537e2fac01f22..8e159e016e057 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap, HashSet}; +use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::sync::Arc; use anyhow::{anyhow, Context}; @@ -20,16 +20,17 @@ use itertools::Itertools; use risingwave_common::bail; use risingwave_common::buffer::Bitmap; use risingwave_common::catalog::TableId; -use risingwave_common::hash::{ActorMapping, ParallelUnitId, ParallelUnitMapping}; +use risingwave_common::hash::{ActorMapping, WorkerSlotId, WorkerSlotMapping}; use risingwave_common::util::stream_graph_visitor::{ visit_stream_node, visit_stream_node_cont, visit_stream_node_cont_mut, }; use risingwave_common::util::worker_util::WorkerNodeId; use risingwave_connector::source::SplitImpl; use risingwave_meta_model_v2::SourceId; -use risingwave_pb::common::{PbParallelUnitMapping, PbWorkerSlotMapping}; +use risingwave_pb::common::ParallelUnit; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; +use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; use risingwave_pb::meta::table_fragments::{ActorStatus, Fragment, State}; use risingwave_pb::meta::FragmentWorkerSlotMapping; use risingwave_pb::stream_plan::stream_node::NodeBody; @@ -61,24 +62,76 @@ impl FragmentManagerCore { pub fn all_running_fragment_mappings( &self, ) -> impl Iterator + '_ { - self.table_fragments + let mut result = vec![]; + + for table_fragment in self + .table_fragments .values() .filter(|tf| tf.state() != State::Initial) - .flat_map(|table_fragments| { - table_fragments - .fragments - .values() - .map(move |fragment| FragmentWorkerSlotMapping { + { + result.extend(Self::extract_fragment_mapping(table_fragment).into_iter()); + } + + result.into_iter() + } + + fn extract_fragment_mapping(table_fragment: &TableFragments) -> Vec { + let mut result = Vec::with_capacity(table_fragment.fragments.len()); + for fragment in table_fragment.fragments.values() { + match fragment.get_distribution_type().unwrap() { + FragmentDistributionType::Unspecified => unreachable!(), + FragmentDistributionType::Single => { + let actor = fragment + .get_actors() + .iter() + .exactly_one() + .expect("single actor"); + let status = table_fragment.actor_status.get(&actor.actor_id).unwrap(); + let worker_id = status.get_parallel_unit().unwrap().get_worker_node_id(); + result.push(FragmentWorkerSlotMapping { fragment_id: fragment.fragment_id, mapping: Some( - FragmentManager::convert_mapping( - &table_fragments.actor_status, - fragment.vnode_mapping.as_ref().unwrap(), - ) - .unwrap(), + WorkerSlotMapping::new_single(WorkerSlotId::new(worker_id, 0)) + .to_protobuf(), ), - }) - }) + }); + } + FragmentDistributionType::Hash => { + let mut actor_bitmaps = HashMap::new(); + let mut actor_to_workers = HashMap::new(); + + for actor in &fragment.actors { + let status = table_fragment.actor_status.get(&actor.actor_id).unwrap(); + + match status.state() { + ActorState::Unspecified => unreachable!(), + // skip inactive actors + ActorState::Inactive => continue, + ActorState::Running => {} + } + + let worker_id = status.get_parallel_unit().unwrap().get_worker_node_id(); + actor_bitmaps.insert( + actor.actor_id as ActorId, + Bitmap::from(actor.vnode_bitmap.as_ref().unwrap()), + ); + + actor_to_workers.insert(actor.actor_id, worker_id); + } + + let actor_mapping = ActorMapping::from_bitmaps(&actor_bitmaps); + + let mapping = actor_mapping.to_worker_slot(&actor_to_workers); + + result.push(FragmentWorkerSlotMapping { + fragment_id: fragment.fragment_id, + mapping: Some(mapping.to_protobuf()), + }); + } + } + } + + result } fn running_fragment_parallelisms( @@ -95,16 +148,13 @@ impl FragmentManagerCore { { return None; } - let parallelism = match fragment.vnode_mapping.as_ref() { - None => { - tracing::warn!( - "vnode mapping for fragment {} not found", - fragment.fragment_id - ); - 1 - } - Some(m) => ParallelUnitMapping::from_protobuf(m).iter_unique().count(), + + let parallelism = match fragment.get_distribution_type().unwrap() { + FragmentDistributionType::Unspecified => unreachable!(), + FragmentDistributionType::Single => 1, + FragmentDistributionType::Hash => fragment.get_actors().len(), }; + Some((fragment.fragment_id, parallelism)) }) }) @@ -209,33 +259,10 @@ impl FragmentManager { } async fn notify_fragment_mapping(&self, table_fragment: &TableFragments, operation: Operation) { - // Notify all fragment mapping to frontend nodes - for fragment in table_fragment.fragments.values() { - let vnode_mapping = fragment - .vnode_mapping - .as_ref() - .expect("no data distribution found"); - - let fragment_mapping = if let Operation::Delete = operation { - FragmentWorkerSlotMapping { - fragment_id: fragment.fragment_id, - mapping: None, - } - } else { - FragmentWorkerSlotMapping { - fragment_id: fragment.fragment_id, - mapping: Some( - Self::convert_mapping(&table_fragment.actor_status, vnode_mapping).unwrap(), - ), - } - }; - + for mapping in FragmentManagerCore::extract_fragment_mapping(table_fragment) { self.env .notification_manager() - .notify_frontend( - operation, - Info::StreamingWorkerSlotMapping(fragment_mapping), - ) + .notify_frontend(operation, Info::StreamingWorkerSlotMapping(mapping)) .await; } @@ -801,14 +828,48 @@ impl FragmentManager { .get_mut(table_id) .with_context(|| format!("table_fragment not exist: id={}", table_id))?; - for status in table_fragment.actor_status.values_mut() { - if let Some(pu) = &status.parallel_unit - && migration_plan.parallel_unit_plan.contains_key(&pu.id) - { - status.parallel_unit = Some(migration_plan.parallel_unit_plan[&pu.id].clone()); + let mut worker_fragment_map = HashMap::new(); + for fragment in table_fragment.fragments() { + for actor in fragment.get_actors() { + let worker = table_fragment + .actor_status + .get(&actor.actor_id) + .unwrap() + .get_parallel_unit() + .unwrap() + .get_worker_node_id(); + + let fragment_ref = worker_fragment_map.entry(worker).or_insert(HashMap::new()); + fragment_ref + .entry(fragment.fragment_id) + .or_insert(BTreeSet::new()) + .insert(actor.actor_id); + } + } + + let mut rebuilt_actor_to_worker_slots = HashMap::new(); + + for (worker, fragment_map) in worker_fragment_map { + for (_, actors) in fragment_map { + for (idx, actor) in actors.into_iter().enumerate() { + rebuilt_actor_to_worker_slots.insert(actor, WorkerSlotId::new(worker, idx)); + } + } + } + + for (actor, location) in rebuilt_actor_to_worker_slots { + if let Some(target) = migration_plan.worker_slot_plan.get(&location) { + let status = table_fragment + .actor_status + .get_mut(&actor) + .expect("should exist"); + status.parallel_unit = Some(ParallelUnit { + id: u32::MAX, + worker_node_id: target.worker_id(), + }) } } - table_fragment.update_vnode_mapping(&migration_plan.parallel_unit_plan); + let table_fragment = table_fragment.clone(); let next_revision = current_revision.next(); @@ -826,6 +887,12 @@ impl FragmentManager { /// Used in [`crate::barrier::GlobalBarrierManager`] /// migrate actors and update fragments one by one according to the migration plan. pub async fn migrate_fragment_actors(&self, migration_plan: &MigrationPlan) -> MetaResult<()> { + let expired_workers: HashSet<_> = migration_plan + .worker_slot_plan + .keys() + .map(|w| w.worker_id()) + .collect(); + let to_migrate_table_fragments = self .core .read() @@ -834,8 +901,8 @@ impl FragmentManager { .values() .filter(|tf| { for status in tf.actor_status.values() { - if let Some(pu) = &status.parallel_unit - && migration_plan.parallel_unit_plan.contains_key(&pu.id) + if expired_workers + .contains(&status.get_parallel_unit().unwrap().get_worker_node_id()) { return true; } @@ -853,21 +920,41 @@ impl FragmentManager { Ok(()) } - pub async fn all_worker_parallel_units(&self) -> HashMap> { - let mut all_worker_parallel_units = HashMap::new(); + pub async fn all_worker_slots(&self) -> HashMap> { + let mut all_worker_slots = HashMap::new(); let map = &self.core.read().await.table_fragments; + + let mut worker_fragment_map = HashMap::new(); + for table_fragment in map.values() { - table_fragment.worker_parallel_units().into_iter().for_each( - |(worker_id, parallel_units)| { - all_worker_parallel_units - .entry(worker_id) - .or_insert_with(HashSet::new) - .extend(parallel_units); - }, + for fragment in table_fragment.fragments() { + for actor in fragment.get_actors() { + let worker = table_fragment + .actor_status + .get(&actor.actor_id) + .unwrap() + .get_parallel_unit() + .unwrap() + .get_worker_node_id(); + + let fragment_ref = worker_fragment_map.entry(worker).or_insert(HashMap::new()); + *fragment_ref.entry(fragment.fragment_id).or_insert(0) += 1; + } + } + } + + for (worker, fragment_map) in worker_fragment_map { + let max_fragment = fragment_map.values().copied().max().unwrap(); + + all_worker_slots.insert( + worker, + (0..max_fragment) + .map(|idx| WorkerSlotId::new(worker, idx as _)) + .collect(), ); } - all_worker_parallel_units + all_worker_slots } pub async fn all_node_actors( @@ -1270,12 +1357,13 @@ impl FragmentManager { } // update fragment's vnode mapping - let mut actor_to_parallel_unit = HashMap::with_capacity(fragment.actors.len()); + let mut actor_to_worker = HashMap::with_capacity(fragment.actors.len()); let mut actor_to_vnode_bitmap = HashMap::with_capacity(fragment.actors.len()); for actor in &fragment.actors { let actor_status = &actor_status[&actor.actor_id]; - let parallel_unit_id = actor_status.parallel_unit.as_ref().unwrap().id; - actor_to_parallel_unit.insert(actor.actor_id, parallel_unit_id); + // let parallel_unit_id = actor_status.parallel_unit.as_ref().unwrap().id; + let worker_id = actor_status.parallel_unit.as_ref().unwrap().worker_node_id; + actor_to_worker.insert(actor.actor_id, worker_id); if let Some(vnode_bitmap) = &actor.vnode_bitmap { let bitmap = Bitmap::from(vnode_bitmap); @@ -1283,25 +1371,22 @@ impl FragmentManager { } } - let vnode_mapping = if actor_to_vnode_bitmap.is_empty() { + let worker_slot_mapping = if actor_to_vnode_bitmap.is_empty() { // If there's no `vnode_bitmap`, then the fragment must be a singleton fragment. // We directly use the single parallel unit to construct the mapping. // TODO: also fill `vnode_bitmap` for the actor of singleton fragment so that we // don't need this branch. - let parallel_unit = *actor_to_parallel_unit.values().exactly_one().unwrap(); - ParallelUnitMapping::new_single(parallel_unit) + + let worker_id = *actor_to_worker.values().exactly_one().unwrap(); + WorkerSlotMapping::new_single(WorkerSlotId::new(worker_id, 0)) } else { // Generate the parallel unit mapping from the fragment's actor bitmaps. - assert_eq!(actor_to_vnode_bitmap.len(), actor_to_parallel_unit.len()); + assert_eq!(actor_to_vnode_bitmap.len(), actor_to_worker.len()); ActorMapping::from_bitmaps(&actor_to_vnode_bitmap) - .to_parallel_unit(&actor_to_parallel_unit) + .to_worker_slot(&actor_to_worker) } .to_protobuf(); - *fragment.vnode_mapping.as_mut().unwrap() = vnode_mapping.clone(); - - let worker_slot_mapping = Self::convert_mapping(&actor_status, &vnode_mapping)?; - // Notify fragment mapping to frontend nodes. let fragment_mapping = FragmentWorkerSlotMapping { fragment_id: *fragment_id as FragmentId, @@ -1434,23 +1519,6 @@ impl FragmentManager { Ok(()) } - fn convert_mapping( - actor_status: &BTreeMap, - vnode_mapping: &PbParallelUnitMapping, - ) -> MetaResult { - let parallel_unit_to_worker = actor_status - .values() - .map(|actor_status| { - let parallel_unit = actor_status.get_parallel_unit().unwrap(); - (parallel_unit.id, parallel_unit.worker_node_id) - }) - .collect(); - - Ok(ParallelUnitMapping::from_protobuf(vnode_mapping) - .to_worker_slot(¶llel_unit_to_worker)? - .to_protobuf()) - } - pub async fn table_node_actors( &self, table_ids: &HashSet, @@ -1503,10 +1571,12 @@ impl FragmentManager { pub async fn get_upstream_root_fragments( &self, upstream_table_ids: &HashSet, - ) -> MetaResult> { + ) -> MetaResult<(HashMap, HashMap)> { let map = &self.core.read().await.table_fragments; let mut fragments = HashMap::new(); + let mut actor_locations = HashMap::new(); + for &table_id in upstream_table_ids { let table_fragments = map .get(&table_id) @@ -1518,16 +1588,27 @@ impl FragmentManager { // look for Source fragment if there's no MView fragment fragments.insert(table_id, fragment); } + + // todo: reduce memory usage + table_fragments + .actor_status + .iter() + .for_each(|(actor_id, status)| { + actor_locations.insert( + *actor_id, + status.get_parallel_unit().unwrap().worker_node_id, + ); + }); } - Ok(fragments) + Ok((fragments, actor_locations)) } /// Get the downstream `StreamTableScan` fragments of the specified MV. pub async fn get_downstream_fragments( &self, table_id: TableId, - ) -> MetaResult> { + ) -> MetaResult<(Vec<(DispatchStrategy, Fragment)>, HashMap)> { let map = &self.core.read().await.table_fragments; let table_fragments = map @@ -1569,7 +1650,21 @@ impl FragmentManager { assert_eq!(downstream_dispatches.len(), fragments.len()); - Ok(fragments) + let mut actor_locations = HashMap::new(); + + map.values().for_each(|table_fragments| { + table_fragments + .actor_status + .iter() + .for_each(|(actor_id, status)| { + actor_locations.insert( + *actor_id, + status.get_parallel_unit().unwrap().worker_node_id, + ); + }); + }); + + Ok((fragments, actor_locations)) } /// Get the `Materialize` fragment of the specified table. diff --git a/src/meta/src/manager/cluster.rs b/src/meta/src/manager/cluster.rs index 876050c36ae6c..13bf0251bab8b 100644 --- a/src/meta/src/manager/cluster.rs +++ b/src/meta/src/manager/cluster.rs @@ -20,13 +20,12 @@ use std::sync::Arc; use std::time::{Duration, SystemTime}; use itertools::Itertools; -use risingwave_common::hash::ParallelUnitId; use risingwave_common::util::addr::HostAddr; use risingwave_common::util::resource_util::cpu::total_cpu_available; use risingwave_common::util::resource_util::memory::system_memory_available_bytes; use risingwave_common::RW_VERSION; use risingwave_pb::common::worker_node::{Property, State}; -use risingwave_pb::common::{HostAddress, ParallelUnit, WorkerNode, WorkerType}; +use risingwave_pb::common::{HostAddress, WorkerNode, WorkerType}; use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty; use risingwave_pb::meta::heartbeat_request; use risingwave_pb::meta::subscribe_response::{Info, Operation}; @@ -127,7 +126,7 @@ impl ClusterManager { .unwrap_or_default(); } - let old_worker_parallelism = worker.worker_node.parallel_units.len(); + let old_worker_parallelism = worker.worker_node.parallelism as usize; if old_worker_parallelism == new_worker_parallelism && worker.worker_node.property == property { @@ -144,13 +143,8 @@ impl ClusterManager { old_worker_parallelism, new_worker_parallelism ); - let parallel_units = self - .generate_cn_parallel_units( - new_worker_parallelism - old_worker_parallelism, - new_worker.worker_id(), - ) - .await?; - new_worker.worker_node.parallel_units.extend(parallel_units); + + new_worker.worker_node.parallelism = new_worker_parallelism as _; } Ordering::Greater => { if !self.env.opts.disable_automatic_parallelism_control { @@ -161,10 +155,8 @@ impl ClusterManager { old_worker_parallelism, new_worker_parallelism ); - new_worker - .worker_node - .parallel_units - .truncate(new_worker_parallelism) + + new_worker.worker_node.parallelism = new_worker_parallelism as _; } else { // Warn and keep the original parallelism if the worker registered with a // smaller parallelism, entering compatibility mode. @@ -210,20 +202,13 @@ impl ClusterManager { _ => None, }; - // Generate parallel units. - let parallel_units = if r#type == WorkerType::ComputeNode { - self.generate_cn_parallel_units(new_worker_parallelism, worker_id) - .await? - } else { - vec![] - }; // Construct worker. let worker_node = WorkerNode { id: worker_id, r#type: r#type as i32, host: Some(host_address.clone()), state: State::Starting as i32, - parallel_units, + parallelism: new_worker_parallelism as _, property, transactional_id, // resource doesn't need persist @@ -514,27 +499,6 @@ impl ClusterManager { } } - /// Generate `parallel_degree` parallel units. - async fn generate_cn_parallel_units( - &self, - parallel_degree: usize, - worker_id: WorkerId, - ) -> MetaResult> { - let start_id = self - .env - .id_gen_manager() - .as_kv() - .generate_interval::<{ IdCategory::ParallelUnit }>(parallel_degree as u64) - .await? as ParallelUnitId; - let parallel_units = (start_id..start_id + parallel_degree as ParallelUnitId) - .map(|id| ParallelUnit { - id, - worker_node_id: worker_id, - }) - .collect(); - Ok(parallel_units) - } - pub async fn get_worker_by_id(&self, worker_id: WorkerId) -> Option { self.core.read().await.get_worker_by_id(worker_id) } @@ -546,11 +510,8 @@ pub struct StreamingClusterInfo { /// All **active** compute nodes in the cluster. pub worker_nodes: HashMap, - /// All parallel units of the **active** compute nodes in the cluster. - pub parallel_units: HashMap, - - /// All unschedulable parallel units of compute nodes in the cluster. - pub unschedulable_parallel_units: HashMap, + /// All unschedulable compute nodes in the cluster. + pub unschedulable_workers: HashSet, } pub struct ClusterManagerCore { @@ -723,34 +684,24 @@ impl ClusterManagerCore { fn get_streaming_cluster_info(&self) -> StreamingClusterInfo { let mut streaming_worker_node = self.list_streaming_worker_node(Some(State::Running)); - let unschedulable_worker_node = streaming_worker_node + let unschedulable_workers = streaming_worker_node .extract_if(|worker| { worker .property .as_ref() .map_or(false, |p| p.is_unschedulable) }) - .collect_vec(); + .map(|w| w.id) + .collect(); let active_workers: HashMap<_, _> = streaming_worker_node .into_iter() .map(|w| (w.id, w)) .collect(); - let active_parallel_units = active_workers - .values() - .flat_map(|worker| worker.parallel_units.iter().map(|p| (p.id, p.clone()))) - .collect(); - - let unschedulable_parallel_units = unschedulable_worker_node - .iter() - .flat_map(|worker| worker.parallel_units.iter().map(|p| (p.id, p.clone()))) - .collect(); - StreamingClusterInfo { worker_nodes: active_workers, - parallel_units: active_parallel_units, - unschedulable_parallel_units, + unschedulable_workers, } } @@ -795,7 +746,6 @@ fn meta_node_info(host: &str, started_at: Option) -> WorkerNode { .map(HostAddr::to_protobuf) .ok(), state: State::Running as _, - parallel_units: vec![], property: None, transactional_id: None, resource: Some(risingwave_pb::common::worker_node::Resource { @@ -804,6 +754,7 @@ fn meta_node_info(host: &str, started_at: Option) -> WorkerNode { total_cpu_cores: total_cpu_available() as _, }), started_at, + parallelism: 0, } } @@ -884,7 +835,7 @@ mod tests { ) .await .unwrap(); - assert_eq!(worker_node.parallel_units.len(), fake_parallelism + 4); + assert_eq!(worker_node.parallelism as usize, fake_parallelism + 4); assert_cluster_manager(&cluster_manager, parallel_count + 4).await; // re-register existing worker node with smaller parallelism. @@ -908,11 +859,11 @@ mod tests { .unwrap(); if !env.opts.disable_automatic_parallelism_control { - assert_eq!(worker_node.parallel_units.len(), fake_parallelism - 2); + assert_eq!(worker_node.parallelism as usize, fake_parallelism - 2); assert_cluster_manager(&cluster_manager, parallel_count - 2).await; } else { // compatibility mode - assert_eq!(worker_node.parallel_units.len(), fake_parallelism + 4); + assert_eq!(worker_node.parallelism as usize, fake_parallelism + 4); assert_cluster_manager(&cluster_manager, parallel_count + 4).await; } @@ -978,13 +929,13 @@ mod tests { } async fn assert_cluster_manager(cluster_manager: &ClusterManager, parallel_count: usize) { - let parallel_units = cluster_manager + let parallel_units: usize = cluster_manager .list_active_serving_compute_nodes() .await .into_iter() - .flat_map(|w| w.parallel_units) - .collect_vec(); - assert_eq!(parallel_units.len(), parallel_count); + .map(|w| w.parallelism as usize) + .sum(); + assert_eq!(parallel_units, parallel_count); } // This test takes seconds because the TTL is measured in seconds. diff --git a/src/meta/src/manager/diagnose.rs b/src/meta/src/manager/diagnose.rs index 06c76c47c5daa..40a1cba7aee6a 100644 --- a/src/meta/src/manager/diagnose.rs +++ b/src/meta/src/manager/diagnose.rs @@ -214,7 +214,7 @@ impl DiagnoseCommand { &mut row, worker_node.get_state().ok().map(|s| s.as_str_name()), ); - row.add_cell(worker_node.parallel_units.len().into()); + row.add_cell(worker_node.parallelism.into()); try_add_cell( &mut row, worker_node.property.as_ref().map(|p| p.is_streaming), diff --git a/src/meta/src/manager/metadata.rs b/src/meta/src/manager/metadata.rs index 241a47941755b..856abd261c8af 100644 --- a/src/meta/src/manager/metadata.rs +++ b/src/meta/src/manager/metadata.rs @@ -455,7 +455,7 @@ impl MetadataManager { pub async fn get_upstream_root_fragments( &self, upstream_table_ids: &HashSet, - ) -> MetaResult> { + ) -> MetaResult<(HashMap, HashMap)> { match self { MetadataManager::V1(mgr) => { mgr.fragment_manager @@ -463,7 +463,7 @@ impl MetadataManager { .await } MetadataManager::V2(mgr) => { - let upstream_root_fragments = mgr + let (upstream_root_fragments, actors) = mgr .catalog_controller .get_upstream_root_fragments( upstream_table_ids @@ -472,10 +472,19 @@ impl MetadataManager { .collect(), ) .await?; - Ok(upstream_root_fragments + + let actors = actors .into_iter() - .map(|(id, fragment)| ((id as u32).into(), fragment)) - .collect()) + .map(|(actor, worker)| (actor as u32, worker as u32)) + .collect(); + + Ok(( + upstream_root_fragments + .into_iter() + .map(|(id, fragment)| ((id as u32).into(), fragment)) + .collect(), + actors, + )) } } } @@ -537,7 +546,7 @@ impl MetadataManager { pub async fn get_downstream_chain_fragments( &self, job_id: u32, - ) -> MetaResult> { + ) -> MetaResult<(Vec<(PbDispatchStrategy, PbFragment)>, HashMap)> { match &self { MetadataManager::V1(mgr) => { mgr.fragment_manager @@ -545,9 +554,17 @@ impl MetadataManager { .await } MetadataManager::V2(mgr) => { - mgr.catalog_controller + let (fragments, actors) = mgr + .catalog_controller .get_downstream_chain_fragments(job_id as _) - .await + .await?; + + let actors = actors + .into_iter() + .map(|(actor, worker)| (actor as u32, worker as u32)) + .collect(); + + Ok((fragments, actors)) } } } diff --git a/src/meta/src/model/migration_plan.rs b/src/meta/src/model/migration_plan.rs index b4703c9284fb4..338fa00911c9a 100644 --- a/src/meta/src/model/migration_plan.rs +++ b/src/meta/src/model/migration_plan.rs @@ -15,7 +15,7 @@ use std::collections::HashMap; use prost::Message; -use risingwave_pb::common::ParallelUnit; +use risingwave_common::hash::WorkerSlotId; use risingwave_pb::meta::PbMigrationPlan; use crate::storage::{MetaStore, MetaStoreError, MetaStoreResult, Snapshot, DEFAULT_COLUMN_FAMILY}; @@ -25,11 +25,9 @@ pub const MIGRATION_PLAN_KEY: &[u8] = &[ 109, 105, 103, 114, 97, 116, 105, 111, 110, 95, 112, 108, 97, 110, ]; -type ParallelUnitId = u32; - #[derive(Debug, Default, Clone)] pub struct MigrationPlan { - pub parallel_unit_plan: HashMap, + pub worker_slot_plan: HashMap, } impl MigrationPlan { @@ -75,7 +73,11 @@ impl MigrationPlan { impl From for MigrationPlan { fn from(plan: PbMigrationPlan) -> Self { MigrationPlan { - parallel_unit_plan: plan.parallel_unit_migration_plan, + worker_slot_plan: plan + .worker_slot_migration_plan + .into_iter() + .map(|(k, v)| (WorkerSlotId::from(k), WorkerSlotId::from(v))) + .collect(), } } } @@ -83,7 +85,11 @@ impl From for MigrationPlan { impl From for PbMigrationPlan { fn from(plan: MigrationPlan) -> Self { PbMigrationPlan { - parallel_unit_migration_plan: plan.parallel_unit_plan, + worker_slot_migration_plan: plan + .worker_slot_plan + .into_iter() + .map(|(k, v)| (k.into(), v.into())) + .collect(), } } } diff --git a/src/meta/src/model/stream.rs b/src/meta/src/model/stream.rs index 144a33ae7b24b..368adaec52ead 100644 --- a/src/meta/src/model/stream.rs +++ b/src/meta/src/model/stream.rs @@ -17,9 +17,9 @@ use std::ops::AddAssign; use itertools::Itertools; use risingwave_common::catalog::TableId; -use risingwave_common::hash::ParallelUnitId; +use risingwave_common::hash::WorkerSlotId; use risingwave_connector::source::SplitImpl; -use risingwave_pb::common::{ParallelUnit, ParallelUnitMapping}; +use risingwave_pb::common::ParallelUnit; use risingwave_pb::meta::table_fragments::actor_status::ActorState; use risingwave_pb::meta::table_fragments::{ActorStatus, Fragment, State}; use risingwave_pb::meta::table_parallelism::{ @@ -207,17 +207,20 @@ impl TableFragments { pub fn new( table_id: TableId, fragments: BTreeMap, - actor_locations: &BTreeMap, + actor_locations: &BTreeMap, ctx: StreamContext, table_parallelism: TableParallelism, ) -> Self { let actor_status = actor_locations .iter() - .map(|(&actor_id, parallel_unit)| { + .map(|(&actor_id, worker_slot_id)| { ( actor_id, ActorStatus { - parallel_unit: Some(parallel_unit.clone()), + parallel_unit: Some(ParallelUnit { + id: u32::MAX, + worker_node_id: worker_slot_id.worker_id(), + }), state: ActorState::Inactive as i32, }, ) @@ -279,24 +282,6 @@ impl TableFragments { self.state = state; } - /// Returns mview fragment vnode mapping. - /// Note that: the sink fragment is also stored as `TableFragments`, it's possible that - /// there's no fragment with `FragmentTypeFlag::Mview` exists. - pub fn mview_vnode_mapping(&self) -> Option<(FragmentId, ParallelUnitMapping)> { - self.fragments - .values() - .find(|fragment| { - (fragment.get_fragment_type_mask() & FragmentTypeFlag::Mview as u32) != 0 - }) - .map(|fragment| { - ( - fragment.fragment_id, - // vnode mapping is always `Some`, even for singletons. - fragment.vnode_mapping.clone().unwrap(), - ) - }) - } - /// Update state of all actors pub fn update_actors_state(&mut self, state: ActorState) { for actor_status in self.actor_status.values_mut() { @@ -511,28 +496,6 @@ impl TableFragments { map } - pub fn worker_parallel_units(&self) -> HashMap> { - let mut map = HashMap::new(); - for actor_status in self.actor_status.values() { - map.entry(actor_status.get_parallel_unit().unwrap().worker_node_id) - .or_insert_with(HashSet::new) - .insert(actor_status.get_parallel_unit().unwrap().id); - } - map - } - - pub fn update_vnode_mapping(&mut self, migrate_map: &HashMap) { - for fragment in self.fragments.values_mut() { - if let Some(mapping) = &mut fragment.vnode_mapping { - mapping.data.iter_mut().for_each(|id| { - if migrate_map.contains_key(id) { - *id = migrate_map.get(id).unwrap().id; - } - }); - } - } - } - /// Returns the status of actors group by worker id. pub fn worker_actors(&self, include_inactive: bool) -> BTreeMap> { let mut actors = BTreeMap::default(); diff --git a/src/meta/src/rpc/ddl_controller.rs b/src/meta/src/rpc/ddl_controller.rs index cc5d7dd8971ad..3a434631e9ae4 100644 --- a/src/meta/src/rpc/ddl_controller.rs +++ b/src/meta/src/rpc/ddl_controller.rs @@ -24,15 +24,16 @@ use aes_siv::{Aes128SivAead, KeyInit}; use anyhow::Context; use itertools::Itertools; use rand::{Rng, RngCore}; +use risingwave_common::buffer::Bitmap; use risingwave_common::config::DefaultParallelism; -use risingwave_common::hash::{ParallelUnitMapping, VirtualNode}; +use risingwave_common::hash::{ActorId, ActorMapping, VirtualNode}; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_common::util::column_index_mapping::ColIndexMapping; use risingwave_common::util::epoch::Epoch; use risingwave_common::util::stream_graph_visitor::{ visit_fragment, visit_stream_node, visit_stream_node_cont_mut, }; -use risingwave_common::{bail, current_cluster_version}; +use risingwave_common::{bail, current_cluster_version, hash}; use risingwave_connector::dispatch_source_prop; use risingwave_connector::error::ConnectorError; use risingwave_connector::source::cdc::CdcSourceType; @@ -54,6 +55,7 @@ use risingwave_pb::ddl_service::alter_owner_request::Object; use risingwave_pb::ddl_service::{ alter_name_request, alter_set_schema_request, DdlProgress, TableJobType, }; +use risingwave_pb::meta::table_fragments::fragment::FragmentDistributionType; use risingwave_pb::meta::table_fragments::PbFragment; use risingwave_pb::meta::PbTableParallelism; use risingwave_pb::stream_plan::stream_node::NodeBody; @@ -1191,19 +1193,40 @@ impl DdlController { let dist_key_indices = table.distribution_key.iter().map(|i| *i as _).collect_vec(); - let mapping = downstream_actor_ids - .iter() - .map(|id| { - let actor_status = table_fragments.actor_status.get(id).unwrap(); - let parallel_unit_id = actor_status.parallel_unit.as_ref().unwrap().id; + let mut actor_location = HashMap::new(); - (parallel_unit_id, *id) - }) - .collect(); + for actor in union_fragment.get_actors() { + let worker_id = table_fragments + .actor_status + .get(&actor.actor_id) + .expect("actor status not found") + .parallel_unit + .as_ref() + .expect("parallel unit not found") + .worker_node_id; + + actor_location.insert(actor.actor_id as ActorId, worker_id); + } + + let mapping = match union_fragment.get_distribution_type().unwrap() { + FragmentDistributionType::Unspecified => unreachable!(), + FragmentDistributionType::Single => None, + FragmentDistributionType::Hash => { + let actor_bitmaps: HashMap<_, _> = union_fragment + .actors + .iter() + .map(|actor| { + ( + actor.actor_id as hash::ActorId, + Bitmap::from(actor.vnode_bitmap.as_ref().unwrap()), + ) + }) + .collect(); - let actor_mapping = - ParallelUnitMapping::from_protobuf(union_fragment.vnode_mapping.as_ref().unwrap()) - .to_actor(&mapping); + let actor_mapping = ActorMapping::from_bitmaps(&actor_bitmaps); + Some(actor_mapping) + } + }; let upstream_actors = sink_fragment.get_actors(); @@ -1214,7 +1237,7 @@ impl DdlController { r#type: DispatcherType::Hash as _, dist_key_indices: dist_key_indices.clone(), output_indices: output_indices.clone(), - hash_mapping: Some(actor_mapping.to_protobuf()), + hash_mapping: mapping.as_ref().map(|m| m.to_protobuf()), dispatcher_id: union_fragment.fragment_id as _, downstream_actor_id: downstream_actor_ids.clone(), }], @@ -1490,14 +1513,19 @@ impl DdlController { ) -> MetaResult { const MAX_PARALLELISM: NonZeroUsize = NonZeroUsize::new(VirtualNode::COUNT).unwrap(); - if cluster_info.parallel_units.is_empty() { + let available_parallelism = cluster_info + .worker_nodes + .values() + .map(|worker| worker.parallelism as usize) + .sum::(); + + if available_parallelism == 0 { return Err(MetaError::unavailable( "No available parallel units to schedule", )); } - let available_parallel_units = - NonZeroUsize::new(cluster_info.parallel_units.len()).unwrap(); + let available_parallel_units = NonZeroUsize::new(available_parallelism).unwrap(); // Use configured parallel units if no default parallelism is specified. let parallelism = @@ -1540,7 +1568,7 @@ impl DdlController { // 1. Resolve the upstream fragments, extend the fragment graph to a complete graph that // contains all information needed for building the actor graph. - let upstream_root_fragments = self + let (upstream_root_fragments, existing_actor_location) = self .metadata_manager .get_upstream_root_fragments(fragment_graph.dependent_table_ids()) .await?; @@ -1558,6 +1586,7 @@ impl DdlController { let complete_graph = CompleteStreamFragmentGraph::with_upstreams( fragment_graph, upstream_root_fragments, + existing_actor_location, stream_job.into(), )?; @@ -1917,7 +1946,11 @@ impl DdlController { .expect("mview fragment not found"); // Map the column indices in the dispatchers with the given mapping. - let downstream_fragments = self.metadata_manager.get_downstream_chain_fragments(id).await? + let (downstream_fragments, downstream_actor_locations) = self + .metadata_manager + .get_downstream_chain_fragments(id) + .await?; + let downstream_fragments = downstream_fragments .into_iter() .map(|(d, f)| if let Some(mapping) = &table_col_index_mapping { @@ -1937,6 +1970,7 @@ impl DdlController { fragment_graph, original_table_fragment.fragment_id, downstream_fragments, + downstream_actor_locations, stream_job.into(), )?; diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index 28520720e98fe..5d4f61482a66e 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -820,17 +820,14 @@ pub async fn refresh_fragment_info_metrics_v2( } }; - let pu_addr_mapping: HashMap = worker_nodes + let worker_addr_mapping: HashMap = worker_nodes .into_iter() - .flat_map(|worker_node| { + .map(|worker_node| { let addr = match worker_node.host { Some(host) => format!("{}:{}", host.host, host.port), None => "".to_owned(), }; - worker_node - .parallel_units - .into_iter() - .map(move |pu| (pu.id, addr.clone())) + (worker_node.id, addr) }) .collect(); let table_compaction_group_id_mapping = hummock_manager @@ -847,7 +844,7 @@ pub async fn refresh_fragment_info_metrics_v2( let fragment_id_str = actor_location.fragment_id.to_string(); // Report a dummy gauge metrics with (fragment id, actor id, node // address) as its label - if let Some(address) = pu_addr_mapping.get(&(actor_location.parallel_unit_id as u32)) { + if let Some(address) = worker_addr_mapping.get(&{ actor_location.worker_id }) { meta_metrics .actor_info .with_label_values(&[&actor_id_str, &fragment_id_str, address]) diff --git a/src/meta/src/stream/scale.rs b/src/meta/src/stream/scale.rs index 0b7a711b38643..70db505010c29 100644 --- a/src/meta/src/stream/scale.rs +++ b/src/meta/src/stream/scale.rs @@ -28,13 +28,10 @@ use num_traits::abs; use risingwave_common::bail; use risingwave_common::buffer::{Bitmap, BitmapBuilder}; use risingwave_common::catalog::TableId; -use risingwave_common::hash::{ActorMapping, ParallelUnitId, VirtualNode}; +use risingwave_common::hash::{ActorMapping, VirtualNode}; use risingwave_common::util::iter_util::ZipEqDebug; use risingwave_meta_model_v2::StreamingParallelism; -use risingwave_pb::common::{ - ActorInfo, Buffer, ParallelUnit, ParallelUnitMapping, WorkerNode, WorkerType, -}; -use risingwave_pb::meta::get_reschedule_plan_request::{Policy, StableResizePolicy}; +use risingwave_pb::common::{ActorInfo, Buffer, ParallelUnit, WorkerNode, WorkerType}; use risingwave_pb::meta::subscribe_response::{Info, Operation}; use risingwave_pb::meta::table_fragments::actor_status::ActorState; use risingwave_pb::meta::table_fragments::fragment::{ @@ -53,7 +50,6 @@ use tokio::sync::oneshot::Receiver; use tokio::sync::{oneshot, RwLock, RwLockReadGuard, RwLockWriteGuard}; use tokio::task::JoinHandle; use tokio::time::{Instant, MissedTickBehavior}; -use tracing::warn; use crate::barrier::{Command, Reschedule, StreamRpcManager}; use crate::manager::{ @@ -110,16 +106,15 @@ impl TableRevision { } #[derive(Debug, Clone, Eq, PartialEq)] -pub struct ParallelUnitReschedule { - pub added_parallel_units: BTreeSet, - pub removed_parallel_units: BTreeSet, +pub struct WorkerReschedule { + pub increased_actor_count: BTreeMap, + pub decreased_actor_count: BTreeMap, } pub struct CustomFragmentInfo { pub fragment_id: u32, pub fragment_type_mask: u32, pub distribution_type: PbFragmentDistributionType, - pub vnode_mapping: Option, pub state_table_ids: Vec, pub upstream_fragment_ids: Vec, pub actor_template: PbStreamActor, @@ -162,7 +157,6 @@ impl From<&PbFragment> for CustomFragmentInfo { fragment_id: fragment.fragment_id, fragment_type_mask: fragment.fragment_type_mask, distribution_type: fragment.distribution_type(), - vnode_mapping: fragment.vnode_mapping.clone(), state_table_ids: fragment.state_table_ids.clone(), upstream_fragment_ids: fragment.upstream_fragment_ids.clone(), actor_template: fragment @@ -186,12 +180,10 @@ impl CustomFragmentInfo { } pub struct RescheduleContext { - /// Index used to map `ParallelUnitId` to `WorkerId` - parallel_unit_id_to_worker_id: BTreeMap, /// Meta information for all Actors actor_map: HashMap, /// Status of all Actors, used to find the location of the `Actor` - actor_status: BTreeMap, + actor_status: BTreeMap, /// Meta information of all `Fragment`, used to find the `Fragment`'s `Actor` fragment_map: HashMap, /// Indexes for all `Worker`s @@ -209,27 +201,11 @@ pub struct RescheduleContext { } impl RescheduleContext { - fn actor_id_to_parallel_unit(&self, actor_id: &ActorId) -> MetaResult<&ParallelUnit> { + fn actor_id_to_worker_id(&self, actor_id: &ActorId) -> MetaResult { self.actor_status .get(actor_id) - .and_then(|actor_status| actor_status.parallel_unit.as_ref()) - .ok_or_else(|| anyhow!("could not found parallel unit for actor {}", actor_id).into()) - } - - fn parallel_unit_id_to_worker( - &self, - parallel_unit_id: &ParallelUnitId, - ) -> MetaResult<&WorkerNode> { - self.parallel_unit_id_to_worker_id - .get(parallel_unit_id) - .and_then(|worker_id| self.worker_nodes.get(worker_id)) - .ok_or_else(|| { - anyhow!( - "could not found Worker for ParallelUint {}", - parallel_unit_id - ) - .into() - }) + .cloned() + .ok_or_else(|| anyhow!("could not find worker for actor {}", actor_id).into()) } } @@ -481,7 +457,7 @@ impl ScaleController { /// Build the context for rescheduling and do some validation for the request. async fn build_reschedule_context( &self, - reschedule: &mut HashMap, + reschedule: &mut HashMap, options: RescheduleOptions, table_parallelisms: Option<&mut HashMap>, ) -> MetaResult { @@ -498,7 +474,7 @@ impl ScaleController { } // Check if we are trying to move a fragment to a node marked as unschedulable - let unschedulable_parallel_unit_ids: HashMap<_, _> = worker_nodes + let unschedulable_worker_ids: HashSet<_> = worker_nodes .values() .filter(|w| { w.property @@ -506,37 +482,21 @@ impl ScaleController { .map(|property| property.is_unschedulable) .unwrap_or(false) }) - .flat_map(|w| { - w.parallel_units - .iter() - .map(|parallel_unit| (parallel_unit.id as ParallelUnitId, w.id as WorkerId)) - }) + .map(|worker| worker.id) .collect(); for (fragment_id, reschedule) in &*reschedule { - for parallel_unit_id in &reschedule.added_parallel_units { - if let Some(worker_id) = unschedulable_parallel_unit_ids.get(parallel_unit_id) { + for worker_id in reschedule.increased_actor_count.keys() { + if unschedulable_worker_ids.contains(worker_id) { bail!( - "unable to move fragment {} to unschedulable parallel unit {} from worker {}", + "unable to move fragment {} to unschedulable worker {}", fragment_id, - parallel_unit_id, worker_id ); } } } - // Associating ParallelUnit with Worker - let parallel_unit_id_to_worker_id: BTreeMap<_, _> = worker_nodes - .iter() - .flat_map(|(worker_id, worker_node)| { - worker_node - .parallel_units - .iter() - .map(move |parallel_unit| (parallel_unit.id as ParallelUnitId, *worker_id)) - }) - .collect(); - // FIXME: the same as anther place calling `list_table_fragments` in scaling. // Index for StreamActor let mut actor_map = HashMap::new(); @@ -551,7 +511,7 @@ impl ScaleController { fn fulfill_index_by_table_fragments_ref( actor_map: &mut HashMap, fragment_map: &mut HashMap, - actor_status: &mut BTreeMap, + actor_status: &mut BTreeMap, fragment_state: &mut HashMap, fragment_to_table: &mut HashMap, table_fragments: &TableFragments, @@ -570,7 +530,12 @@ impl ScaleController { fragment_map.insert(*fragment_id, CustomFragmentInfo::from(fragment)); } - actor_status.extend(table_fragments.actor_status.clone()); + for (actor_id, status) in &table_fragments.actor_status { + actor_status.insert( + *actor_id, + status.get_parallel_unit().unwrap().get_worker_node_id(), + ); + } fragment_to_table.extend( table_fragments @@ -670,9 +635,9 @@ impl ScaleController { let mut no_shuffle_reschedule = HashMap::new(); for ( fragment_id, - ParallelUnitReschedule { - added_parallel_units, - removed_parallel_units, + WorkerReschedule { + increased_actor_count, + decreased_actor_count, }, ) in &*reschedule { @@ -726,9 +691,9 @@ impl ScaleController { no_shuffle_reschedule.insert( downstream_id, - ParallelUnitReschedule { - added_parallel_units: added_parallel_units.clone(), - removed_parallel_units: removed_parallel_units.clone(), + WorkerReschedule { + increased_actor_count: increased_actor_count.clone(), + decreased_actor_count: decreased_actor_count.clone(), }, ); } @@ -742,54 +707,33 @@ impl ScaleController { } // Check if the reschedule plan is valid. - let current_parallel_units = fragment + let current_worker_ids = fragment .actors .iter() - .map(|a| { - actor_status - .get(&a.actor_id) - .unwrap() - .get_parallel_unit() - .unwrap() - .id - }) + .map(|a| actor_status.get(&a.actor_id).cloned().unwrap()) .collect::>(); - for removed in removed_parallel_units { - if !current_parallel_units.contains(removed) { + + for removed in decreased_actor_count.keys() { + if !current_worker_ids.contains(removed) { bail!( - "no actor on the parallel unit {} of fragment {}", + "no actor on the worker {} of fragment {}", removed, fragment_id ); } } - for added in added_parallel_units { - if !parallel_unit_id_to_worker_id.contains_key(added) { - bail!("parallel unit {} not available", added); - } - if current_parallel_units.contains(added) && !removed_parallel_units.contains(added) - { - bail!( - "parallel unit {} of fragment {} is already in use", - added, - fragment_id - ); - } - } + + let added_actor_count: usize = increased_actor_count.values().cloned().sum(); + let removed_actor_count: usize = decreased_actor_count.values().cloned().sum(); match fragment.distribution_type() { FragmentDistributionType::Hash => { - if current_parallel_units.len() + added_parallel_units.len() - <= removed_parallel_units.len() - { - bail!( - "can't remove all parallel units from fragment {}", - fragment_id - ); + if fragment.actors.len() + added_actor_count <= removed_actor_count { + bail!("can't remove all actors from fragment {}", fragment_id); } } FragmentDistributionType::Single => { - if added_parallel_units.len() != removed_parallel_units.len() { + if added_actor_count != removed_actor_count { bail!("single distribution fragment only support migration"); } } @@ -808,7 +752,6 @@ impl ScaleController { reschedule.extend(no_shuffle_reschedule.into_iter()); Ok(RescheduleContext { - parallel_unit_id_to_worker_id, actor_map, actor_status, fragment_map, @@ -821,7 +764,7 @@ impl ScaleController { }) } - async fn create_actors_on_compute_node( + pub(crate) async fn create_actors_on_compute_node( &self, worker_nodes: &HashMap, actor_infos_to_broadcast: BTreeMap, @@ -860,7 +803,7 @@ impl ScaleController { // Results are the generated reschedule plan and the changes that need to be updated to the meta store. pub(crate) async fn prepare_reschedule_command( &self, - mut reschedules: HashMap, + mut reschedules: HashMap, options: RescheduleOptions, table_parallelisms: Option<&mut HashMap>, ) -> MetaResult<( @@ -870,9 +813,9 @@ impl ScaleController { let ctx = self .build_reschedule_context(&mut reschedules, options, table_parallelisms) .await?; - // Index of actors to create/remove - // Fragment Id => ( Actor Id => Parallel Unit Id ) + // Index of actors to create/remove + // Fragment Id => ( Actor Id => Worker Id ) let (fragment_actors_to_remove, fragment_actors_to_create) = self.arrange_reschedules(&reschedules, &ctx).await?; @@ -886,12 +829,12 @@ impl ScaleController { let actors_to_create = fragment_actors_to_create .get(fragment_id) - .map(|map| map.keys().cloned().collect()) + .map(|map| map.iter().map(|(actor_id, _)| *actor_id).collect()) .unwrap_or_default(); let actors_to_remove = fragment_actors_to_remove .get(fragment_id) - .map(|map| map.keys().cloned().collect()) + .map(|map| map.iter().map(|(actor_id, _)| *actor_id).collect()) .unwrap_or_default(); let fragment = ctx.fragment_map.get(fragment_id).unwrap(); @@ -929,16 +872,13 @@ impl ScaleController { continue; } } - let parallel_unit_id = ctx.actor_id_to_parallel_unit(&actor.actor_id)?.id; - new_actor_ids.insert( - actor.actor_id as ActorId, - parallel_unit_id as ParallelUnitId, - ); + let worker_id = ctx.actor_id_to_worker_id(&actor.actor_id)?; + new_actor_ids.insert(actor.actor_id as ActorId, worker_id); } if let Some(actors_to_create) = fragment_actors_to_create.get(fragment_id) { - for (actor_id, parallel_unit_id) in actors_to_create { - new_actor_ids.insert(*actor_id, *parallel_unit_id as ParallelUnitId); + for (actor_id, worker_id) in actors_to_create { + new_actor_ids.insert(*actor_id, *worker_id); } } @@ -964,10 +904,8 @@ impl ScaleController { ctx: &RescheduleContext, fragment_id: &FragmentId, upstream_fragment_id: &FragmentId, - fragment_actors_after_reschedule: &HashMap< - FragmentId, - BTreeMap, - >, + fragment_actors_after_reschedule: &HashMap>, + actor_group_map: &mut HashMap, fragment_updated_bitmap: &mut HashMap>, no_shuffle_upstream_actor_map: &mut HashMap>, no_shuffle_downstream_actors_map: &mut HashMap>, @@ -978,8 +916,39 @@ impl ScaleController { let fragment = ctx.fragment_map.get(fragment_id).unwrap(); + let upstream_fragment = ctx.fragment_map.get(upstream_fragment_id).unwrap(); + + for upstream_actor in &upstream_fragment.actors { + for dispatcher in &upstream_actor.dispatcher { + if let DispatcherType::NoShuffle = dispatcher.get_type().unwrap() { + let downstream_actor_id = + *dispatcher.downstream_actor_id.iter().exactly_one().unwrap(); + + // upstream is root + if !ctx + .no_shuffle_target_fragment_ids + .contains(upstream_fragment_id) + { + actor_group_map.insert( + upstream_actor.actor_id, + (upstream_fragment.fragment_id, upstream_actor.actor_id), + ); + actor_group_map.insert( + downstream_actor_id, + (upstream_fragment.fragment_id, upstream_actor.actor_id), + ); + } else { + let root_actor_id = + *actor_group_map.get(&upstream_actor.actor_id).unwrap(); + + actor_group_map.insert(downstream_actor_id, root_actor_id); + } + } + } + } + // If the upstream is a Singleton Fragment, there will be no Bitmap changes - let mut upstream_fragment_bitmap = fragment_updated_bitmap + let upstream_fragment_bitmap = fragment_updated_bitmap .get(upstream_fragment_id) .cloned() .unwrap_or_default(); @@ -989,30 +958,83 @@ impl ScaleController { .cloned() .unwrap(); - let mut parallel_unit_id_to_actor_id = HashMap::new(); - for (actor_id, parallel_unit_id) in - fragment_actors_after_reschedule.get(fragment_id).unwrap() - { - parallel_unit_id_to_actor_id.insert(*parallel_unit_id, *actor_id); - } + let fragment_actor_map = fragment_actors_after_reschedule + .get(fragment_id) + .cloned() + .unwrap(); + let mut worker_reverse_index: HashMap> = HashMap::new(); + + // first, find existing actor bitmap, copy them let mut fragment_bitmap = HashMap::new(); - for (upstream_actor_id, parallel_unit_id) in upstream_fragment_actor_map { - let actor_id = parallel_unit_id_to_actor_id.get(¶llel_unit_id).unwrap(); - if let Some(bitmap) = upstream_fragment_bitmap.remove(&upstream_actor_id) { + for (actor_id, worker_id) in &fragment_actor_map { + if let Some((root_fragment, root_actor_id)) = actor_group_map.get(actor_id) { + let root_bitmap = fragment_updated_bitmap + .get(root_fragment) + .and_then(|map| map.get(root_actor_id)) + .unwrap() + .clone(); + // Copy the bitmap - fragment_bitmap.insert(*actor_id, bitmap); + fragment_bitmap.insert(*actor_id, root_bitmap); + + no_shuffle_upstream_actor_map + .entry(*actor_id as ActorId) + .or_default() + .insert(*upstream_fragment_id, *root_actor_id); + no_shuffle_downstream_actors_map + .entry(*root_actor_id) + .or_default() + .insert(*fragment_id, *actor_id); + } else { + worker_reverse_index + .entry(*worker_id) + .or_default() + .insert(*actor_id); } + } + + let mut upstream_worker_reverse_index: HashMap> = HashMap::new(); - no_shuffle_upstream_actor_map - .entry(*actor_id as ActorId) - .or_default() - .insert(*upstream_fragment_id, upstream_actor_id); - no_shuffle_downstream_actors_map - .entry(upstream_actor_id) - .or_default() - .insert(*fragment_id, *actor_id); + for (actor_id, worker_id) in &upstream_fragment_actor_map { + if !actor_group_map.contains_key(actor_id) { + upstream_worker_reverse_index + .entry(*worker_id) + .or_default() + .insert(*actor_id); + } + } + + // then, find the rest of the actors and copy the bitmap + for (worker_id, actor_ids) in worker_reverse_index { + let upstream_actor_ids = upstream_worker_reverse_index + .get(&worker_id) + .unwrap() + .clone(); + assert_eq!(actor_ids.len(), upstream_actor_ids.len()); + + for (actor_id, upstream_actor_id) in actor_ids + .into_iter() + .zip_eq_debug(upstream_actor_ids.into_iter()) + { + let bitmap = upstream_fragment_bitmap + .get(&upstream_actor_id) + .cloned() + .unwrap(); + + // Copy the bitmap + fragment_bitmap.insert(actor_id, bitmap); + + no_shuffle_upstream_actor_map + .entry(actor_id as ActorId) + .or_default() + .insert(*upstream_fragment_id, upstream_actor_id); + no_shuffle_downstream_actors_map + .entry(upstream_actor_id) + .or_default() + .insert(*fragment_id, actor_id); + } } match fragment.distribution_type() { @@ -1045,6 +1067,7 @@ impl ScaleController { downstream_fragment_id, fragment_id, fragment_actors_after_reschedule, + actor_group_map, fragment_updated_bitmap, no_shuffle_upstream_actor_map, no_shuffle_downstream_actors_map, @@ -1055,6 +1078,7 @@ impl ScaleController { let mut no_shuffle_upstream_actor_map = HashMap::new(); let mut no_shuffle_downstream_actors_map = HashMap::new(); + let mut actor_group_map = HashMap::new(); // For all roots in the upstream and downstream dependency trees of NoShuffle, recursively // find all correspondences for fragment_id in reschedules.keys() { @@ -1068,6 +1092,7 @@ impl ScaleController { downstream_fragment_id, fragment_id, &fragment_actors_after_reschedule, + &mut actor_group_map, &mut fragment_actor_bitmap, &mut no_shuffle_upstream_actor_map, &mut no_shuffle_downstream_actors_map, @@ -1077,6 +1102,8 @@ impl ScaleController { } } + tracing::debug!("actor group map {:?}", actor_group_map); + let mut new_created_actors = HashMap::new(); for fragment_id in reschedules.keys() { let actors_to_create = fragment_actors_to_create @@ -1125,7 +1152,7 @@ impl ScaleController { // After modification, for newly created actors, both upstream and downstream actor ids // have been modified let mut actor_infos_to_broadcast = BTreeMap::new(); - let mut node_actors_to_create: HashMap> = HashMap::new(); + let mut node_actors_to_create: HashMap> = HashMap::new(); let mut broadcast_worker_ids = HashSet::new(); let subscriptions: HashMap<_, SubscriptionIds> = self @@ -1144,16 +1171,15 @@ impl ScaleController { .collect(); for actors_to_create in fragment_actors_to_create.values() { - for (new_actor_id, new_parallel_unit_id) in actors_to_create { + for (new_actor_id, worker_id) in actors_to_create { let new_actor = new_created_actors.get(new_actor_id).unwrap(); for upstream_actor_id in &new_actor.upstream_actor_id { if new_created_actors.contains_key(upstream_actor_id) { continue; } - let upstream_worker_id = ctx - .actor_id_to_parallel_unit(upstream_actor_id)? - .worker_node_id; + let upstream_worker_id = ctx.actor_id_to_worker_id(upstream_actor_id)?; + let upstream_worker = ctx.worker_nodes.get(&upstream_worker_id).with_context(|| { format!("upstream worker {} not found", upstream_worker_id) @@ -1177,9 +1203,9 @@ impl ScaleController { if new_created_actors.contains_key(downstream_actor_id) { continue; } - let downstream_worker_id = ctx - .actor_id_to_parallel_unit(downstream_actor_id)? - .worker_node_id; + let downstream_worker_id = + ctx.actor_id_to_worker_id(downstream_actor_id)?; + let downstream_worker = ctx .worker_nodes .get(&downstream_worker_id) @@ -1199,7 +1225,7 @@ impl ScaleController { } } - let worker = ctx.parallel_unit_id_to_worker(new_parallel_unit_id)?; + let worker = ctx.worker_nodes.get(worker_id).unwrap(); node_actors_to_create .entry(worker.id) @@ -1273,14 +1299,10 @@ impl ScaleController { .fragment_type_mask; let injectable = TableFragments::is_injectable(fragment_type_mask); - if let Some(actor_pu_maps) = fragment_actors_to_create.get(&fragment_id).cloned() { - for (actor_id, parallel_unit_id) in actor_pu_maps { - let worker_id = ctx - .parallel_unit_id_to_worker_id - .get(¶llel_unit_id) - .with_context(|| format!("parallel unit {} not found", parallel_unit_id))?; + if let Some(actor_worker_maps) = fragment_actors_to_create.get(&fragment_id).cloned() { + for (actor_id, worker_id) in actor_worker_maps { actors_to_create - .entry(*worker_id) + .entry(worker_id) .or_default() .push(actor_id); } @@ -1296,14 +1318,7 @@ impl ScaleController { let actors_after_reschedule = fragment_actors_after_reschedule.get(&fragment_id).unwrap(); - let parallel_unit_to_actor_after_reschedule: BTreeMap<_, _> = actors_after_reschedule - .iter() - .map(|(actor_id, parallel_unit_id)| { - (*parallel_unit_id as ParallelUnitId, *actor_id as ActorId) - }) - .collect(); - - assert!(!parallel_unit_to_actor_after_reschedule.is_empty()); + assert!(!actors_after_reschedule.is_empty()); let fragment = ctx.fragment_map.get(&fragment_id).unwrap(); @@ -1323,11 +1338,8 @@ impl ScaleController { FragmentDistributionType::Hash => { if !in_degree_types.contains(&DispatcherType::Hash) { None - } else if parallel_unit_to_actor_after_reschedule.len() == 1 { - let actor_id = parallel_unit_to_actor_after_reschedule - .into_values() - .next() - .unwrap(); + } else if actors_after_reschedule.len() == 1 { + let actor_id = actors_after_reschedule.keys().next().cloned().unwrap(); Some(ActorMapping::new_single(actor_id)) } else { // Changes of the bitmap must occur in the case of HashDistribution @@ -1434,20 +1446,15 @@ impl ScaleController { let mut fragment_created_actors = HashMap::new(); for (fragment_id, actors_to_create) in &fragment_actors_to_create { let mut created_actors = HashMap::new(); - for (actor_id, parallel_unit_id) in actors_to_create { + for (actor_id, worker_id) in actors_to_create { let actor = new_created_actors.get(actor_id).cloned().unwrap(); - let worker_id = ctx - .parallel_unit_id_to_worker_id - .get(parallel_unit_id) - .with_context(|| format!("parallel unit {} not found", parallel_unit_id))?; - created_actors.insert( *actor_id, ( actor, ActorStatus { parallel_unit: Some(ParallelUnit { - id: *parallel_unit_id, + id: u32::MAX, worker_node_id: *worker_id, }), state: ActorState::Inactive as i32, @@ -1474,20 +1481,20 @@ impl ScaleController { async fn arrange_reschedules( &self, - reschedule: &HashMap, + reschedule: &HashMap, ctx: &RescheduleContext, ) -> MetaResult<( - HashMap>, - HashMap>, + HashMap>, + HashMap>, )> { let mut fragment_actors_to_remove = HashMap::with_capacity(reschedule.len()); let mut fragment_actors_to_create = HashMap::with_capacity(reschedule.len()); for ( fragment_id, - ParallelUnitReschedule { - added_parallel_units, - removed_parallel_units, + WorkerReschedule { + increased_actor_count, + decreased_actor_count, }, ) in reschedule { @@ -1497,39 +1504,46 @@ impl ScaleController { let mut actors_to_remove = BTreeMap::new(); let mut actors_to_create = BTreeMap::new(); - let parallel_unit_to_actor: HashMap<_, _> = fragment - .actors - .iter() - .map(|actor| { - ctx.actor_id_to_parallel_unit(&actor.actor_id) - .map(|parallel_unit| { - ( - parallel_unit.id as ParallelUnitId, - actor.actor_id as ActorId, - ) - }) - }) - .try_collect()?; + let mut worker_to_actors = HashMap::new(); - for removed_parallel_unit_id in removed_parallel_units { - if let Some(removed_actor_id) = parallel_unit_to_actor.get(removed_parallel_unit_id) - { - actors_to_remove.insert(*removed_actor_id, *removed_parallel_unit_id); - } + for actor in &fragment.actors { + let worker_id = ctx.actor_id_to_worker_id(&actor.actor_id).unwrap(); + worker_to_actors + .entry(worker_id) + .or_insert(vec![]) + .push(actor.actor_id); } - for created_parallel_unit_id in added_parallel_units { - let id = match self.env.id_gen_manager() { - IdGenManagerImpl::Kv(mgr) => { - mgr.generate::<{ IdCategory::Actor }>().await? as ActorId - } - IdGenManagerImpl::Sql(mgr) => { - let id = mgr.generate_interval::<{ IdCategory::Actor }>(1); - id as ActorId + for (removed, n) in decreased_actor_count { + if let Some(actor_ids) = worker_to_actors.get(removed) { + assert!(actor_ids.len() >= *n); + + let removed_actors: Vec<_> = actor_ids + .iter() + .skip(actor_ids.len().saturating_sub(*n)) + .cloned() + .collect(); + + for actor in removed_actors { + actors_to_remove.insert(actor, *removed); } - }; + } + } - actors_to_create.insert(id, *created_parallel_unit_id); + for (created_worker, n) in increased_actor_count { + for _ in 0..*n { + let id = match self.env.id_gen_manager() { + IdGenManagerImpl::Kv(mgr) => { + mgr.generate::<{ IdCategory::Actor }>().await? as ActorId + } + IdGenManagerImpl::Sql(mgr) => { + let id = mgr.generate_interval::<{ IdCategory::Actor }>(1); + id as ActorId + } + }; + + actors_to_create.insert(id, *created_worker); + } } if !actors_to_remove.is_empty() { @@ -1548,8 +1562,8 @@ impl ScaleController { /// overall changes, and is used to handle cascading updates fn modify_actor_upstream_and_downstream( ctx: &RescheduleContext, - fragment_actors_to_remove: &HashMap>, - fragment_actors_to_create: &HashMap>, + fragment_actors_to_remove: &HashMap>, + fragment_actors_to_create: &HashMap>, fragment_actor_bitmap: &HashMap>, no_shuffle_upstream_actor_map: &HashMap>, no_shuffle_downstream_actors_map: &HashMap>, @@ -1713,8 +1727,9 @@ impl ScaleController { .metadata_manager .running_fragment_parallelisms(Some(reschedules.keys().cloned().collect())) .await?; - let serving_vnode_mapping = Arc::new(ServingVnodeMapping::default()); - let (upserted, failed) = serving_vnode_mapping.upsert(streaming_parallelisms, &workers); + let serving_worker_slot_mapping = Arc::new(ServingVnodeMapping::default()); + let (upserted, failed) = + serving_worker_slot_mapping.upsert(streaming_parallelisms, &workers); if !upserted.is_empty() { tracing::debug!( "Update serving vnode mapping for fragments {:?}.", @@ -1771,7 +1786,7 @@ impl ScaleController { } // FIXME: should be removed - async fn list_all_table_fragments(&self) -> MetaResult> { + pub(crate) async fn list_all_table_fragments(&self) -> MetaResult> { use crate::model::MetadataModel; let all_table_fragments = match &self.metadata_manager { MetadataManager::V1(mgr) => mgr.fragment_manager.list_table_fragments().await, @@ -1790,7 +1805,7 @@ impl ScaleController { pub async fn generate_table_resize_plan( &self, policy: TableResizePolicy, - ) -> MetaResult> { + ) -> MetaResult> { let TableResizePolicy { worker_ids, table_parallelisms, @@ -1814,18 +1829,14 @@ impl ScaleController { .filter(|worker| worker_ids.contains(&worker.id)) .collect::>(); - let worker_parallel_units = workers - .iter() - .map(|worker| { - ( - worker.id, - worker - .parallel_units - .iter() - .map(|parallel_unit| parallel_unit.id as ParallelUnitId) - .collect::>(), - ) - }) + let workers: HashMap<_, _> = workers + .into_iter() + .map(|worker| (worker.id, worker)) + .collect(); + + let worker_slots = workers + .values() + .map(|worker| (worker.id, worker.parallelism as usize)) .collect::>(); // index for no shuffle relation @@ -1834,8 +1845,8 @@ impl ScaleController { // index for fragment_id -> distribution_type let mut fragment_distribution_map = HashMap::new(); - // index for actor -> parallel_unit - let mut actor_status = HashMap::new(); + // index for actor -> worker id + let mut actor_location = HashMap::new(); // index for table_id -> [fragment_id] let mut table_fragment_id_map = HashMap::new(); // index for fragment_id -> [actor_id] @@ -1846,7 +1857,7 @@ impl ScaleController { no_shuffle_source_fragment_ids: &mut HashSet, no_shuffle_target_fragment_ids: &mut HashSet, fragment_distribution_map: &mut HashMap, - actor_status: &mut HashMap, + actor_location: &mut HashMap, table_fragment_id_map: &mut HashMap>, fragment_actor_id_map: &mut HashMap>, table_fragments: &BTreeMap, @@ -1914,7 +1925,12 @@ impl ScaleController { .insert(*fragment_id); } - actor_status.extend(table_fragments.actor_status.clone()); + for (actor_id, status) in &table_fragments.actor_status { + actor_location.insert( + *actor_id, + status.get_parallel_unit().unwrap().get_worker_node_id(), + ); + } } Ok(()) @@ -1927,7 +1943,7 @@ impl ScaleController { &mut no_shuffle_source_fragment_ids, &mut no_shuffle_target_fragment_ids, &mut fragment_distribution_map, - &mut actor_status, + &mut actor_location, &mut table_fragment_id_map, &mut fragment_actor_id_map, guard.table_fragments(), @@ -1944,7 +1960,7 @@ impl ScaleController { &mut no_shuffle_source_fragment_ids, &mut no_shuffle_target_fragment_ids, &mut fragment_distribution_map, - &mut actor_status, + &mut actor_location, &mut table_fragment_id_map, &mut fragment_actor_id_map, &all_table_fragments, @@ -1963,25 +1979,19 @@ impl ScaleController { continue; } - let fragment_parallel_unit_ids: BTreeSet = fragment_actor_id_map - .get(&fragment_id) - .unwrap() - .iter() - .map(|actor_id| { - actor_status - .get(actor_id) - .and_then(|status| status.parallel_unit.clone()) - .unwrap() - .id as ParallelUnitId - }) - .collect(); + let mut fragment_slots: BTreeMap = BTreeMap::new(); - let all_available_parallel_unit_ids: BTreeSet<_> = - worker_parallel_units.values().flatten().cloned().collect(); + for actor_id in fragment_actor_id_map.get(&fragment_id).unwrap() { + let worker_id = actor_location.get(actor_id).unwrap(); - if all_available_parallel_unit_ids.is_empty() { + *fragment_slots.entry(*worker_id).or_default() += 1; + } + + let all_available_slots: usize = worker_slots.values().cloned().sum(); + + if all_available_slots == 0 { bail!( - "No schedulable ParallelUnits available for fragment {}", + "No schedulable slots available for fragment {}", fragment_id ); } @@ -1989,31 +1999,38 @@ impl ScaleController { match fragment_distribution_map.get(&fragment_id).unwrap() { FragmentDistributionType::Unspecified => unreachable!(), FragmentDistributionType::Single => { - let single_parallel_unit_id = - fragment_parallel_unit_ids.iter().exactly_one().unwrap(); + let (single_worker_id, should_be_one) = + fragment_slots.iter().exactly_one().unwrap(); - if all_available_parallel_unit_ids.contains(single_parallel_unit_id) { + assert_eq!(*should_be_one, 1); + + if worker_slots.contains_key(single_worker_id) { // NOTE: shall we continue? continue; } - let units = schedule_units_for_slots(&worker_parallel_units, 1, table_id)?; + let units = schedule_units_for_slots(&worker_slots, 1, table_id)?; + + let (chosen_target_worker_id, should_be_one) = + units.iter().exactly_one().ok().with_context(|| { + format!( + "Cannot find a single target worker for fragment {fragment_id}" + ) + })?; - let chosen_target_parallel_unit_id = units - .values() - .flatten() - .cloned() - .exactly_one() - .ok() - .with_context(|| format!("Cannot find a single target ParallelUnit for fragment {fragment_id}"))?; + assert_eq!(*should_be_one, 1); target_plan.insert( fragment_id, - ParallelUnitReschedule { - added_parallel_units: BTreeSet::from([ - chosen_target_parallel_unit_id, - ]), - removed_parallel_units: BTreeSet::from([*single_parallel_unit_id]), + WorkerReschedule { + increased_actor_count: BTreeMap::from_iter(vec![( + *chosen_target_worker_id, + 1, + )]), + decreased_actor_count: BTreeMap::from_iter(vec![( + *single_worker_id, + 1, + )]), }, ); } @@ -2021,38 +2038,18 @@ impl ScaleController { TableParallelism::Adaptive => { target_plan.insert( fragment_id, - Self::diff_parallel_unit_change( - &fragment_parallel_unit_ids, - &all_available_parallel_unit_ids, - ), + Self::diff_worker_slot_changes(&fragment_slots, &worker_slots), ); } - TableParallelism::Fixed(mut n) => { - let available_parallelism = all_available_parallel_unit_ids.len(); - - if n > available_parallelism { - warn!( - "not enough parallel units available for job {} fragment {}, required {}, resetting to {}", - table_id, - fragment_id, - n, - available_parallelism, - ); - - n = available_parallelism; - } - - let rebalance_result = - schedule_units_for_slots(&worker_parallel_units, n, table_id)?; - - let target_parallel_unit_ids = - rebalance_result.into_values().flatten().collect(); + TableParallelism::Fixed(n) => { + let target_worker_slots = + schedule_units_for_slots(&worker_slots, n, table_id)?; target_plan.insert( fragment_id, - Self::diff_parallel_unit_change( - &fragment_parallel_unit_ids, - &target_parallel_unit_ids, + Self::diff_worker_slot_changes( + &fragment_slots, + &target_worker_slots, ), ); } @@ -2065,358 +2062,13 @@ impl ScaleController { } target_plan.retain(|_, plan| { - !(plan.added_parallel_units.is_empty() && plan.removed_parallel_units.is_empty()) - }); - - Ok(target_plan) - } - - pub async fn generate_stable_resize_plan( - &self, - policy: StableResizePolicy, - parallel_unit_hints: Option>>, - ) -> MetaResult> { - let StableResizePolicy { - fragment_worker_changes, - } = policy; - - let mut target_plan = HashMap::with_capacity(fragment_worker_changes.len()); - - let workers = self - .metadata_manager - .list_active_streaming_compute_nodes() - .await?; - - let unschedulable_worker_ids = Self::filter_unschedulable_workers(&workers); - - for changes in fragment_worker_changes.values() { - for worker_id in &changes.include_worker_ids { - if unschedulable_worker_ids.contains(worker_id) { - bail!("Cannot include unscheduable worker {}", worker_id) - } - } - } - - let worker_parallel_units = workers - .iter() - .map(|worker| { - ( - worker.id, - worker - .parallel_units - .iter() - .map(|parallel_unit| parallel_unit.id as ParallelUnitId) - .collect::>(), - ) - }) - .collect::>(); - - // FIXME: only need actor id and dispatcher info, avoid clone it. - let mut actor_map = HashMap::new(); - let mut actor_status = HashMap::new(); - // FIXME: only need fragment distribution info, should avoid clone it. - let mut fragment_map = HashMap::new(); - let mut fragment_parallelism = HashMap::new(); - - // We are reusing code for the metadata manager of both V1 and V2, which will be deprecated in the future. - fn fulfill_index_by_table_fragments_ref( - actor_map: &mut HashMap, - actor_status: &mut HashMap, - fragment_map: &mut HashMap, - fragment_parallelism: &mut HashMap, - table_fragments: &TableFragments, - ) { - for (fragment_id, fragment) in &table_fragments.fragments { - for actor in &fragment.actors { - actor_map.insert(actor.actor_id, CustomActorInfo::from(actor)); - } - - fragment_map.insert(*fragment_id, CustomFragmentInfo::from(fragment)); - - fragment_parallelism.insert(*fragment_id, table_fragments.assigned_parallelism); - } - - actor_status.extend(table_fragments.actor_status.clone()); - } - - match &self.metadata_manager { - MetadataManager::V1(mgr) => { - let guard = mgr.fragment_manager.get_fragment_read_guard().await; - - for table_fragments in guard.table_fragments().values() { - fulfill_index_by_table_fragments_ref( - &mut actor_map, - &mut actor_status, - &mut fragment_map, - &mut fragment_parallelism, - table_fragments, - ); - } - } - MetadataManager::V2(_) => { - let all_table_fragments = self.list_all_table_fragments().await?; - - for table_fragments in &all_table_fragments { - fulfill_index_by_table_fragments_ref( - &mut actor_map, - &mut actor_status, - &mut fragment_map, - &mut fragment_parallelism, - table_fragments, - ); - } - } - }; - - let mut no_shuffle_source_fragment_ids = HashSet::new(); - let mut no_shuffle_target_fragment_ids = HashSet::new(); - - Self::build_no_shuffle_relation_index( - &actor_map, - &mut no_shuffle_source_fragment_ids, - &mut no_shuffle_target_fragment_ids, - ); - - let mut fragment_dispatcher_map = HashMap::new(); - Self::build_fragment_dispatcher_index(&actor_map, &mut fragment_dispatcher_map); - - #[derive(PartialEq, Eq, Clone)] - struct WorkerChanges { - include_worker_ids: BTreeSet, - exclude_worker_ids: BTreeSet, - target_parallelism: Option, - target_parallelism_per_worker: Option, - } - - let mut fragment_worker_changes: HashMap<_, _> = fragment_worker_changes - .into_iter() - .map(|(fragment_id, changes)| { - ( - fragment_id as FragmentId, - WorkerChanges { - include_worker_ids: changes.include_worker_ids.into_iter().collect(), - exclude_worker_ids: changes.exclude_worker_ids.into_iter().collect(), - target_parallelism: changes.target_parallelism.map(|p| p as usize), - target_parallelism_per_worker: changes - .target_parallelism_per_worker - .map(|p| p as usize), - }, - ) - }) - .collect(); - - Self::resolve_no_shuffle_upstream_fragments( - &mut fragment_worker_changes, - &fragment_map, - &no_shuffle_source_fragment_ids, - &no_shuffle_target_fragment_ids, - )?; - - for ( - fragment_id, - WorkerChanges { - include_worker_ids, - exclude_worker_ids, - target_parallelism, - target_parallelism_per_worker, - }, - ) in fragment_worker_changes - { - let fragment = match fragment_map.get(&fragment_id) { - None => bail!("Fragment id {} not found", fragment_id), - Some(fragment) => fragment, - }; - - let intersection_ids = include_worker_ids - .intersection(&exclude_worker_ids) - .collect_vec(); - - if !intersection_ids.is_empty() { - bail!( - "Include worker ids {:?} and exclude worker ids {:?} have intersection {:?}", - include_worker_ids, - exclude_worker_ids, - intersection_ids - ); - } - - for worker_id in include_worker_ids.iter().chain(exclude_worker_ids.iter()) { - if !worker_parallel_units.contains_key(worker_id) - && !parallel_unit_hints - .as_ref() - .map(|hints| hints.contains_key(worker_id)) - .unwrap_or(false) - { - bail!("Worker id {} not found", worker_id); - } - } - - let fragment_parallel_unit_ids: BTreeSet<_> = fragment - .actors - .iter() - .map(|actor| { - actor_status - .get(&actor.actor_id) - .and_then(|status| status.parallel_unit.clone()) - .unwrap() - .id as ParallelUnitId - }) - .collect(); - - let worker_to_parallel_unit_ids = |worker_ids: &BTreeSet| { - worker_ids - .iter() - .flat_map(|worker_id| { - worker_parallel_units - .get(worker_id) - .or_else(|| { - parallel_unit_hints - .as_ref() - .and_then(|hints| hints.get(worker_id)) - }) - .expect("worker id should be valid") - }) - .cloned() - .collect_vec() - }; - - let include_worker_parallel_unit_ids = worker_to_parallel_unit_ids(&include_worker_ids); - let exclude_worker_parallel_unit_ids = worker_to_parallel_unit_ids(&exclude_worker_ids); - - fn refilter_parallel_unit_id_by_target_parallelism( - worker_parallel_units: &HashMap>, - include_worker_ids: &BTreeSet, - include_worker_parallel_unit_ids: &[ParallelUnitId], - target_parallel_unit_ids: &mut BTreeSet, - target_parallelism_per_worker: usize, - ) { - let limited_worker_parallel_unit_ids = include_worker_ids - .iter() - .flat_map(|worker_id| { - worker_parallel_units - .get(worker_id) - .cloned() - .unwrap() - .into_iter() - .sorted() - .take(target_parallelism_per_worker) - }) - .collect_vec(); - - // remove all the parallel units in the limited workers - target_parallel_unit_ids - .retain(|id| !include_worker_parallel_unit_ids.contains(id)); - - // then we re-add the limited parallel units from the limited workers - target_parallel_unit_ids.extend(limited_worker_parallel_unit_ids.into_iter()); - } - match fragment.distribution_type() { - FragmentDistributionType::Unspecified => unreachable!(), - FragmentDistributionType::Single => { - let single_parallel_unit_id = - fragment_parallel_unit_ids.iter().exactly_one().unwrap(); - - let mut target_parallel_unit_ids: BTreeSet<_> = worker_parallel_units - .keys() - .filter(|id| !unschedulable_worker_ids.contains(*id)) - .filter(|id| !exclude_worker_ids.contains(*id)) - .flat_map(|id| worker_parallel_units.get(id).cloned().unwrap()) - .collect(); - - if let Some(target_parallelism_per_worker) = target_parallelism_per_worker { - refilter_parallel_unit_id_by_target_parallelism( - &worker_parallel_units, - &include_worker_ids, - &include_worker_parallel_unit_ids, - &mut target_parallel_unit_ids, - target_parallelism_per_worker, - ); - } - - if target_parallel_unit_ids.is_empty() { - bail!( - "No schedulable ParallelUnits available for single distribution fragment {}", - fragment_id - ); - } - - if !target_parallel_unit_ids.contains(single_parallel_unit_id) { - let sorted_target_parallel_unit_ids = - target_parallel_unit_ids.into_iter().sorted().collect_vec(); - - let chosen_target_parallel_unit_id = sorted_target_parallel_unit_ids - [fragment_id as usize % sorted_target_parallel_unit_ids.len()]; - - target_plan.insert( - fragment_id, - ParallelUnitReschedule { - added_parallel_units: BTreeSet::from([ - chosen_target_parallel_unit_id, - ]), - removed_parallel_units: BTreeSet::from([*single_parallel_unit_id]), - }, - ); - } - } - FragmentDistributionType::Hash => { - let mut target_parallel_unit_ids: BTreeSet<_> = - fragment_parallel_unit_ids.clone(); - target_parallel_unit_ids.extend(include_worker_parallel_unit_ids.iter()); - target_parallel_unit_ids - .retain(|id| !exclude_worker_parallel_unit_ids.contains(id)); - - if target_parallel_unit_ids.is_empty() { - bail!( - "No schedulable ParallelUnits available for fragment {}", - fragment_id - ); - } - - match (target_parallelism, target_parallelism_per_worker) { - (Some(_), Some(_)) => { - bail!("Cannot specify both target parallelism and target parallelism per worker"); - } - (Some(target_parallelism), _) => { - if target_parallel_unit_ids.len() < target_parallelism { - bail!("Target parallelism {} is greater than schedulable ParallelUnits {}", target_parallelism, target_parallel_unit_ids.len()); - } - - target_parallel_unit_ids = target_parallel_unit_ids - .into_iter() - .take(target_parallelism) - .collect(); - } - (_, Some(target_parallelism_per_worker)) => { - refilter_parallel_unit_id_by_target_parallelism( - &worker_parallel_units, - &include_worker_ids, - &include_worker_parallel_unit_ids, - &mut target_parallel_unit_ids, - target_parallelism_per_worker, - ); - } - _ => {} - } - - target_plan.insert( - fragment_id, - Self::diff_parallel_unit_change( - &fragment_parallel_unit_ids, - &target_parallel_unit_ids, - ), - ); - } - } - } - - target_plan.retain(|_, plan| { - !(plan.added_parallel_units.is_empty() && plan.removed_parallel_units.is_empty()) + !(plan.increased_actor_count.is_empty() && plan.decreased_actor_count.is_empty()) }); Ok(target_plan) } - fn filter_unschedulable_workers(workers: &[WorkerNode]) -> HashSet { + pub(crate) fn filter_unschedulable_workers(workers: &[WorkerNode]) -> HashSet { workers .iter() .filter(|worker| { @@ -2430,35 +2082,33 @@ impl ScaleController { .collect() } - fn diff_parallel_unit_change( - fragment_parallel_unit_ids: &BTreeSet, - target_parallel_unit_ids: &BTreeSet, - ) -> ParallelUnitReschedule { - let to_expand_parallel_units = target_parallel_unit_ids - .difference(fragment_parallel_unit_ids) - .cloned() - .collect(); + fn diff_worker_slot_changes( + fragment_worker_slots: &BTreeMap, + target_worker_slots: &BTreeMap, + ) -> WorkerReschedule { + let mut increased_actor_count: BTreeMap = BTreeMap::new(); + let mut decreased_actor_count: BTreeMap = BTreeMap::new(); - let to_shrink_parallel_units = fragment_parallel_unit_ids - .difference(target_parallel_unit_ids) - .cloned() - .collect(); + for (&worker_id, &target_slots) in target_worker_slots { + let ¤t_slots = fragment_worker_slots.get(&worker_id).unwrap_or(&0); - ParallelUnitReschedule { - added_parallel_units: to_expand_parallel_units, - removed_parallel_units: to_shrink_parallel_units, + if target_slots > current_slots { + increased_actor_count.insert(worker_id, target_slots - current_slots); + } } - } - pub async fn get_reschedule_plan( - &self, - policy: Policy, - ) -> MetaResult> { - match policy { - Policy::StableResizePolicy(resize) => { - self.generate_stable_resize_plan(resize, None).await + for (&worker_id, ¤t_slots) in fragment_worker_slots { + let &target_slots = target_worker_slots.get(&worker_id).unwrap_or(&0); + + if current_slots > target_slots { + decreased_actor_count.insert(worker_id, current_slots - target_slots); } } + + WorkerReschedule { + increased_actor_count, + decreased_actor_count, + } } pub fn build_no_shuffle_relation_index( @@ -2646,7 +2296,7 @@ impl GlobalStreamManager { pub async fn reschedule_actors( &self, - reschedules: HashMap, + reschedules: HashMap, options: RescheduleOptions, table_parallelism: Option>, ) -> MetaResult<()> { @@ -2667,7 +2317,7 @@ impl GlobalStreamManager { async fn reschedule_actors_impl( &self, revert_funcs: &mut Vec>, - reschedules: HashMap, + reschedules: HashMap, options: RescheduleOptions, table_parallelism: Option>, ) -> MetaResult<()> { @@ -2975,7 +2625,7 @@ impl GlobalStreamManager { let prev_worker = worker_cache.insert(worker.id, worker.clone()); match prev_worker { - Some(prev_worker) if prev_worker.parallel_units != worker.parallel_units => { + Some(prev_worker) if prev_worker.get_parallelism() != worker.get_parallelism() => { tracing::info!(worker = worker.id, "worker parallelism changed"); should_trigger = true; } @@ -3021,45 +2671,28 @@ impl GlobalStreamManager { } } -// We redistribute parallel units (which will be ensembles in the future) through a simple consistent hashing ring. -// Note that we have added some simple logic here to ensure the consistency of the ratio between each slot, -// especially when equal division is needed. pub fn schedule_units_for_slots( - slots: &BTreeMap>, + slots: &BTreeMap, total_unit_size: usize, salt: u32, -) -> MetaResult>> { +) -> MetaResult> { let mut ch = ConsistentHashRing::new(salt); - for (worker_id, parallel_unit_ids) in slots { - ch.add_worker(*worker_id, parallel_unit_ids.len() as u32); + for (worker_id, parallelism) in slots { + ch.add_worker(*worker_id, *parallelism as u32); } let target_distribution = ch.distribute_tasks(total_unit_size as u32)?; - Ok(slots - .iter() - .map(|(worker_id, parallel_unit_ids)| { - ( - *worker_id, - parallel_unit_ids - .iter() - .take( - target_distribution - .get(worker_id) - .cloned() - .unwrap_or_default() as usize, - ) - .cloned() - .collect::>(), - ) - }) + Ok(target_distribution + .into_iter() + .map(|(worker_id, task_count)| (worker_id as WorkerId, task_count as usize)) .collect()) } pub struct ConsistentHashRing { ring: BTreeMap, - capacities: BTreeMap, + weights: BTreeMap, virtual_nodes: u32, salt: u32, } @@ -3068,7 +2701,7 @@ impl ConsistentHashRing { fn new(salt: u32) -> Self { ConsistentHashRing { ring: BTreeMap::new(), - capacities: BTreeMap::new(), + weights: BTreeMap::new(), virtual_nodes: 1024, salt, } @@ -3081,7 +2714,7 @@ impl ConsistentHashRing { hasher.finish() } - fn add_worker(&mut self, id: u32, capacity: u32) { + fn add_worker(&mut self, id: u32, weight: u32) { let virtual_nodes_count = self.virtual_nodes; for i in 0..virtual_nodes_count { @@ -3090,21 +2723,17 @@ impl ConsistentHashRing { self.ring.insert(hash, id); } - self.capacities.insert(id, capacity); + self.weights.insert(id, weight); } fn distribute_tasks(&self, total_tasks: u32) -> MetaResult> { - let total_capacity = self.capacities.values().sum::(); - - if total_capacity < total_tasks { - bail!("Total tasks exceed the total weight of all workers."); - } + let total_weight = self.weights.values().sum::(); let mut soft_limits = HashMap::new(); - for (worker_id, worker_capacity) in &self.capacities { + for (worker_id, worker_capacity) in &self.weights { soft_limits.insert( *worker_id, - (total_tasks as f64 * (*worker_capacity as f64 / total_capacity as f64)).ceil() + (total_tasks as f64 * (*worker_capacity as f64 / total_weight as f64)).ceil() as u32, ); } @@ -3124,10 +2753,7 @@ impl ConsistentHashRing { let ring_range = self.ring.range(task_hash..).chain(self.ring.iter()); for (_, &worker_id) in ring_range { - let worker_capacity = self.capacities.get(&worker_id).unwrap(); - let worker_soft_limit = soft_limits.get(&worker_id).unwrap(); - - let task_limit = min(*worker_capacity, *worker_soft_limit); + let task_limit = *soft_limits.get(&worker_id).unwrap(); let worker_task_count = task_distribution.entry(worker_id).or_insert(0); @@ -3207,7 +2833,7 @@ mod tests { let total_tasks = 10; // More tasks than the total weight let task_distribution = ch.distribute_tasks(total_tasks); - assert!(task_distribution.is_err()); + assert!(task_distribution.is_ok()); } #[test] diff --git a/src/meta/src/stream/stream_graph/actor.rs b/src/meta/src/stream/stream_graph/actor.rs index 7b02920ac3c4d..19d460c00d3e9 100644 --- a/src/meta/src/stream/stream_graph/actor.rs +++ b/src/meta/src/stream/stream_graph/actor.rs @@ -20,7 +20,7 @@ use assert_matches::assert_matches; use itertools::Itertools; use risingwave_common::bail; use risingwave_common::buffer::Bitmap; -use risingwave_common::hash::{ActorId, ActorMapping, ParallelUnitId}; +use risingwave_common::hash::{ActorId, ActorMapping, WorkerSlotId}; use risingwave_common::util::iter_util::ZipEqFast; use risingwave_pb::meta::table_fragments::Fragment; use risingwave_pb::plan_common::ExprContext; @@ -331,7 +331,7 @@ impl ExternalChange { } /// The parallel unit location of actors. -type ActorLocations = BTreeMap; +type ActorLocations = BTreeMap; /// The actual mutable state of building an actor graph. /// @@ -369,7 +369,7 @@ impl ActorGraphBuildStateInner { &mut self, actor_id: GlobalActorId, fragment_id: GlobalFragmentId, - parallel_unit_id: ParallelUnitId, + worker_slot_id: WorkerSlotId, vnode_bitmap: Option, node: Arc, ) { @@ -381,18 +381,14 @@ impl ActorGraphBuildStateInner { .unwrap(); self.building_locations - .try_insert(actor_id, parallel_unit_id) + .try_insert(actor_id, worker_slot_id) .unwrap(); } /// Record the location of an external actor. - fn record_external_location( - &mut self, - actor_id: GlobalActorId, - parallel_unit_id: ParallelUnitId, - ) { + fn record_external_location(&mut self, actor_id: GlobalActorId, worker_slot_id: WorkerSlotId) { self.external_locations - .try_insert(actor_id, parallel_unit_id) + .try_insert(actor_id, worker_slot_id) .unwrap(); } @@ -466,7 +462,7 @@ impl ActorGraphBuildStateInner { /// Get the location of an actor. Will look up the location map of both the actors to be built /// and the external actors. - fn get_location(&self, actor_id: GlobalActorId) -> ParallelUnitId { + fn get_location(&self, actor_id: GlobalActorId) -> WorkerSlotId { self.building_locations .get(&actor_id) .copied() @@ -536,7 +532,7 @@ impl ActorGraphBuildStateInner { let dispatcher = if let DispatcherType::Hash = dt { // Transform the `ParallelUnitMapping` from the downstream distribution to the // `ActorMapping`, used for the `HashDispatcher` for the upstream actors. - let downstream_locations: HashMap = downstream + let downstream_locations: HashMap = downstream .actor_ids .iter() .map(|&actor_id| (self.get_location(actor_id), actor_id.as_global_id())) @@ -673,7 +669,7 @@ impl ActorGraphBuilder { // Schedule the distribution of all building fragments. let scheduler = schedule::Scheduler::new( streaming_job_id, - cluster_info.parallel_units.values().cloned(), + &cluster_info.worker_nodes, default_parallelism, )?; let distributions = scheduler.schedule(&fragment_graph)?; @@ -699,12 +695,7 @@ impl ActorGraphBuilder { fn build_locations(&self, actor_locations: ActorLocations) -> Locations { let actor_locations = actor_locations .into_iter() - .map(|(id, p)| { - ( - id.as_global_id(), - self.cluster_info.parallel_units[&p].clone(), - ) - }) + .map(|(id, worker_slot_id)| (id.as_global_id(), worker_slot_id)) .collect(); let worker_locations = self.cluster_info.worker_nodes.clone(); @@ -742,15 +733,15 @@ impl ActorGraphBuilder { external_locations, } = self.build_actor_graph(id_gen)?; - for parallel_unit_id in external_locations.values() { - if let Some(parallel_unit) = self + for worker_slot_id in external_locations.values() { + if self .cluster_info - .unschedulable_parallel_units - .get(parallel_unit_id) + .unschedulable_workers + .contains(&worker_slot_id.worker_id()) { bail!( "The worker {} where the associated upstream is located is unscheduable", - parallel_unit.worker_node_id + worker_slot_id.worker_id(), ); } } @@ -859,15 +850,15 @@ impl ActorGraphBuilder { let bitmaps = distribution.as_hash().map(|m| m.to_bitmaps()); distribution - .parallel_units() - .map(|parallel_unit_id| { + .worker_slots() + .map(|worker_slot| { let actor_id = state.next_actor_id(); - let vnode_bitmap = bitmaps.as_ref().map(|m| &m[¶llel_unit_id]).cloned(); + let vnode_bitmap = bitmaps.as_ref().map(|m| &m[&worker_slot]).cloned(); state.inner.add_actor( actor_id, fragment_id, - parallel_unit_id, + worker_slot, vnode_bitmap, node.clone(), ); @@ -883,8 +874,8 @@ impl ActorGraphBuilder { .iter() .map(|a| { let actor_id = GlobalActorId::new(a.actor_id); - let parallel_unit_id = match &distribution { - Distribution::Singleton(parallel_unit_id) => *parallel_unit_id, + let worker_slot_id = match &distribution { + Distribution::Singleton(worker_slot_id) => *worker_slot_id, Distribution::Hash(mapping) => mapping .get_matched(&Bitmap::from(a.get_vnode_bitmap().unwrap())) .unwrap(), @@ -892,7 +883,7 @@ impl ActorGraphBuilder { state .inner - .record_external_location(actor_id, parallel_unit_id); + .record_external_location(actor_id, worker_slot_id); actor_id }) diff --git a/src/meta/src/stream/stream_graph/fragment.rs b/src/meta/src/stream/stream_graph/fragment.rs index e347dd0287f36..63f1b7eee7650 100644 --- a/src/meta/src/stream/stream_graph/fragment.rs +++ b/src/meta/src/stream/stream_graph/fragment.rs @@ -39,7 +39,7 @@ use risingwave_pb::stream_plan::{ }; use crate::manager::{DdlType, IdGenManagerImpl, MetaSrvEnv, StreamingJob}; -use crate::model::FragmentId; +use crate::model::{ActorId, FragmentId}; use crate::stream::stream_graph::id::{GlobalFragmentId, GlobalFragmentIdGen, GlobalTableIdGen}; use crate::stream::stream_graph::schedule::Distribution; use crate::MetaResult; @@ -539,6 +539,9 @@ pub struct CompleteStreamFragmentGraph { /// The required information of existing fragments. existing_fragments: HashMap, + /// The location of the actors in the existing fragments. + existing_actor_location: HashMap, + /// Extra edges between existing fragments and the building fragments. extra_downstreams: HashMap>, @@ -550,11 +553,13 @@ pub struct FragmentGraphUpstreamContext { /// Root fragment is the root of upstream stream graph, which can be a /// mview fragment or source fragment for cdc source job upstream_root_fragments: HashMap, + upstream_actor_location: HashMap, } pub struct FragmentGraphDownstreamContext { original_table_fragment_id: FragmentId, downstream_fragments: Vec<(DispatchStrategy, Fragment)>, + downstream_actor_location: HashMap, } impl CompleteStreamFragmentGraph { @@ -565,6 +570,7 @@ impl CompleteStreamFragmentGraph { Self { building_graph: graph, existing_fragments: Default::default(), + existing_actor_location: Default::default(), extra_downstreams: Default::default(), extra_upstreams: Default::default(), } @@ -575,12 +581,14 @@ impl CompleteStreamFragmentGraph { pub fn with_upstreams( graph: StreamFragmentGraph, upstream_root_fragments: HashMap, + existing_actor_location: HashMap, ddl_type: DdlType, ) -> MetaResult { Self::build_helper( graph, Some(FragmentGraphUpstreamContext { upstream_root_fragments, + upstream_actor_location: existing_actor_location, }), None, ddl_type, @@ -593,6 +601,7 @@ impl CompleteStreamFragmentGraph { graph: StreamFragmentGraph, original_table_fragment_id: FragmentId, downstream_fragments: Vec<(DispatchStrategy, Fragment)>, + existing_actor_location: HashMap, ddl_type: DdlType, ) -> MetaResult { Self::build_helper( @@ -601,6 +610,7 @@ impl CompleteStreamFragmentGraph { Some(FragmentGraphDownstreamContext { original_table_fragment_id, downstream_fragments, + downstream_actor_location: existing_actor_location, }), ddl_type, ) @@ -617,8 +627,11 @@ impl CompleteStreamFragmentGraph { let mut extra_upstreams = HashMap::new(); let mut existing_fragments = HashMap::new(); + let mut existing_actor_location = HashMap::new(); + if let Some(FragmentGraphUpstreamContext { upstream_root_fragments, + upstream_actor_location, }) = upstream_ctx { for (&id, fragment) in &mut graph.fragments { @@ -781,11 +794,14 @@ impl CompleteStreamFragmentGraph { .into_values() .map(|f| (GlobalFragmentId::new(f.fragment_id), f)), ); + + existing_actor_location.extend(upstream_actor_location); } if let Some(FragmentGraphDownstreamContext { original_table_fragment_id, downstream_fragments, + downstream_actor_location, }) = downstream_ctx { let original_table_fragment_id = GlobalFragmentId::new(original_table_fragment_id); @@ -821,11 +837,14 @@ impl CompleteStreamFragmentGraph { .into_iter() .map(|(_, f)| (GlobalFragmentId::new(f.fragment_id), f)), ); + + existing_actor_location.extend(downstream_actor_location); } Ok(Self { building_graph: graph, existing_fragments, + existing_actor_location, extra_downstreams, extra_upstreams, }) @@ -886,7 +905,12 @@ impl CompleteStreamFragmentGraph { pub(super) fn existing_distribution(&self) -> HashMap { self.existing_fragments .iter() - .map(|(&id, f)| (id, Distribution::from_fragment(f))) + .map(|(&id, f)| { + ( + id, + Distribution::from_fragment(f, &self.existing_actor_location), + ) + }) .collect() } @@ -973,7 +997,6 @@ impl CompleteStreamFragmentGraph { fragment_type_mask: inner.fragment_type_mask, distribution_type, actors, - vnode_mapping: Some(distribution.into_mapping().to_protobuf()), state_table_ids, upstream_fragment_ids, } diff --git a/src/meta/src/stream/stream_graph/schedule.rs b/src/meta/src/stream/stream_graph/schedule.rs index ed2dac5be0e06..bde92ef2ce078 100644 --- a/src/meta/src/stream/stream_graph/schedule.rs +++ b/src/meta/src/stream/stream_graph/schedule.rs @@ -18,15 +18,16 @@ reason = "generated by crepe" )] -use std::collections::{BTreeMap, BTreeSet, HashMap}; +use std::collections::{BTreeMap, HashMap}; use std::num::NonZeroUsize; use either::Either; use enum_as_inner::EnumAsInner; use itertools::Itertools; -use risingwave_common::bail; -use risingwave_common::hash::{ParallelUnitId, ParallelUnitMapping}; -use risingwave_pb::common::{ActorInfo, ParallelUnit}; +use risingwave_common::buffer::Bitmap; +use risingwave_common::hash::{ActorMapping, WorkerSlotId, WorkerSlotMapping}; +use risingwave_common::{bail, hash}; +use risingwave_pb::common::{ActorInfo, WorkerNode}; use risingwave_pb::meta::table_fragments::fragment::{ FragmentDistributionType, PbFragmentDistributionType, }; @@ -46,7 +47,7 @@ type HashMappingId = usize; /// See [`Distribution`] for the public interface. #[derive(Debug, Clone, Copy, PartialEq, Eq, Hash)] enum DistId { - Singleton(ParallelUnitId), + Singleton(WorkerSlotId), Hash(HashMappingId), } @@ -64,7 +65,7 @@ enum Fact { /// A distribution requirement for an external(existing) fragment. ExternalReq { id: Id, dist: DistId }, /// A singleton requirement for a building fragment. - /// Note that the physical parallel unit is not determined yet. + /// Note that the physical worker slot is not determined yet. SingletonReq(Id), } @@ -73,7 +74,7 @@ enum Fact { enum Result { /// This fragment is required to be distributed by the given [`DistId`]. Required(DistId), - /// This fragment is singleton, and should be scheduled to the default parallel unit. + /// This fragment is singleton, and should be scheduled to the default worker slot. DefaultSingleton, /// This fragment is hash-distributed, and should be scheduled by the default hash mapping. DefaultHash, @@ -129,49 +130,57 @@ crepe::crepe! { /// The distribution of a fragment. #[derive(Debug, Clone, EnumAsInner)] pub(super) enum Distribution { - /// The fragment is singleton and is scheduled to the given parallel unit. - Singleton(ParallelUnitId), + /// The fragment is singleton and is scheduled to the given worker slot. + Singleton(WorkerSlotId), /// The fragment is hash-distributed and is scheduled by the given hash mapping. - Hash(ParallelUnitMapping), + Hash(WorkerSlotMapping), } impl Distribution { /// The parallelism required by the distribution. pub fn parallelism(&self) -> usize { - self.parallel_units().count() + self.worker_slots().count() } - /// All parallel units required by the distribution. - pub fn parallel_units(&self) -> impl Iterator + '_ { + /// All worker slots required by the distribution. + pub fn worker_slots(&self) -> impl Iterator + '_ { match self { Distribution::Singleton(p) => Either::Left(std::iter::once(*p)), Distribution::Hash(mapping) => Either::Right(mapping.iter_unique()), } } - /// Convert the distribution to a [`ParallelUnitMapping`]. - /// - /// - For singleton distribution, all of the virtual nodes are mapped to the same parallel unit. - /// - For hash distribution, the mapping is returned as is. - pub fn into_mapping(self) -> ParallelUnitMapping { - match self { - Distribution::Singleton(p) => ParallelUnitMapping::new_single(p), - Distribution::Hash(mapping) => mapping, - } - } - /// Create a distribution from a persisted protobuf `Fragment`. - pub fn from_fragment(fragment: &risingwave_pb::meta::table_fragments::Fragment) -> Self { - let mapping = ParallelUnitMapping::from_protobuf(fragment.get_vnode_mapping().unwrap()); - + pub fn from_fragment( + fragment: &risingwave_pb::meta::table_fragments::Fragment, + actor_location: &HashMap, + ) -> Self { match fragment.get_distribution_type().unwrap() { FragmentDistributionType::Unspecified => unreachable!(), FragmentDistributionType::Single => { - let parallel_unit = mapping.to_single().unwrap(); - Distribution::Singleton(parallel_unit) + let actor_id = fragment.actors.iter().exactly_one().unwrap().actor_id; + let location = actor_location.get(&actor_id).unwrap(); + let worker_slot_id = WorkerSlotId::new(*location, 0); + Distribution::Singleton(worker_slot_id) + } + FragmentDistributionType::Hash => { + let actor_bitmaps: HashMap<_, _> = fragment + .actors + .iter() + .map(|actor| { + ( + actor.actor_id as hash::ActorId, + Bitmap::from(actor.vnode_bitmap.as_ref().unwrap()), + ) + }) + .collect(); + + let actor_mapping = ActorMapping::from_bitmaps(&actor_bitmaps); + let mapping = actor_mapping.to_worker_slot(actor_location); + + Distribution::Hash(mapping) } - FragmentDistributionType::Hash => Distribution::Hash(mapping), } } @@ -187,54 +196,55 @@ impl Distribution { /// [`Scheduler`] schedules the distribution of fragments in a stream graph. pub(super) struct Scheduler { /// The default hash mapping for hash-distributed fragments, if there's no requirement derived. - default_hash_mapping: ParallelUnitMapping, + default_hash_mapping: WorkerSlotMapping, - /// The default parallel unit for singleton fragments, if there's no requirement derived. - default_singleton_parallel_unit: ParallelUnitId, + /// The default worker slot for singleton fragments, if there's no requirement derived. + default_singleton_worker_slot: WorkerSlotId, } impl Scheduler { - /// Create a new [`Scheduler`] with the given parallel units and the default parallelism. + /// Create a new [`Scheduler`] with the given worker slots and the default parallelism. /// /// Each hash-distributed fragment will be scheduled to at most `default_parallelism` parallel /// units, in a round-robin fashion on all compute nodes. If the `default_parallelism` is - /// `None`, all parallel units will be used. + /// `None`, all worker slots will be used. /// /// For different streaming jobs, we even out possible scheduling skew by using the streaming job id as the salt for the scheduling algorithm. pub fn new( streaming_job_id: u32, - parallel_units: impl IntoIterator, + workers: &HashMap, default_parallelism: NonZeroUsize, ) -> MetaResult { - // Group parallel units with worker node. - let mut slots = BTreeMap::new(); - for parallel_unit in parallel_units { - slots - .entry(parallel_unit.worker_node_id as WorkerId) - .or_insert_with(BTreeSet::new) - .insert(parallel_unit.id as ParallelUnitId); - } + // Group worker slots with worker node. + + let slots = workers + .iter() + .map(|(worker_id, worker)| (*worker_id, worker.parallelism as usize)) + .collect(); let parallelism = default_parallelism.get(); let scheduled = schedule_units_for_slots(&slots, parallelism, streaming_job_id)?; - let scheduled_parallel_units = scheduled.values().flatten().cloned().sorted().collect_vec(); - assert_eq!(scheduled_parallel_units.len(), parallelism); + let scheduled_worker_slots = scheduled + .into_iter() + .flat_map(|(worker_id, size)| { + (0..size).map(move |slot| WorkerSlotId::new(worker_id, slot)) + }) + .collect_vec(); + + assert_eq!(scheduled_worker_slots.len(), parallelism); // Build the default hash mapping uniformly. - let default_hash_mapping = ParallelUnitMapping::build_from_ids(&scheduled_parallel_units); + let default_hash_mapping = WorkerSlotMapping::build_from_ids(&scheduled_worker_slots); let single_scheduled = schedule_units_for_slots(&slots, 1, streaming_job_id)?; - let default_singleton_parallel_unit = single_scheduled - .values() - .flatten() - .exactly_one() - .cloned() - .unwrap(); + let default_single_worker_id = single_scheduled.keys().exactly_one().cloned().unwrap(); + + let default_singleton_worker_slot = WorkerSlotId::new(default_single_worker_id, 0); Ok(Self { default_hash_mapping, - default_singleton_parallel_unit, + default_singleton_worker_slot, }) } @@ -271,7 +281,7 @@ impl Scheduler { // External for (id, req) in existing_distribution { let dist = match req { - Distribution::Singleton(parallel_unit) => DistId::Singleton(parallel_unit), + Distribution::Singleton(worker_slot_id) => DistId::Singleton(worker_slot_id), Distribution::Hash(mapping) => DistId::Hash(hash_mapping_id[&mapping]), }; facts.push(Fact::ExternalReq { id, dist }); @@ -301,8 +311,8 @@ impl Scheduler { .map(|Success(id, result)| { let distribution = match result { // Required - Result::Required(DistId::Singleton(parallel_unit)) => { - Distribution::Singleton(parallel_unit) + Result::Required(DistId::Singleton(worker_slot)) => { + Distribution::Singleton(worker_slot) } Result::Required(DistId::Hash(mapping)) => { Distribution::Hash(all_hash_mappings[mapping].clone()) @@ -310,7 +320,7 @@ impl Scheduler { // Default Result::DefaultSingleton => { - Distribution::Singleton(self.default_singleton_parallel_unit) + Distribution::Singleton(self.default_singleton_worker_slot) } Result::DefaultHash => Distribution::Hash(self.default_hash_mapping.clone()), }; @@ -324,11 +334,11 @@ impl Scheduler { } } -/// [`Locations`] represents the parallel unit and worker locations of the actors. +/// [`Locations`] represents the worker slot and worker locations of the actors. #[cfg_attr(test, derive(Default))] pub struct Locations { /// actor location map. - pub actor_locations: BTreeMap, + pub actor_locations: BTreeMap, /// worker location map. pub worker_locations: WorkerLocations, } @@ -338,7 +348,7 @@ impl Locations { pub fn worker_actors(&self) -> HashMap> { self.actor_locations .iter() - .map(|(actor_id, parallel_unit)| (parallel_unit.worker_node_id, *actor_id)) + .map(|(actor_id, worker_slot_id)| (worker_slot_id.worker_id(), *actor_id)) .into_group_map() } @@ -346,9 +356,9 @@ impl Locations { pub fn actor_infos(&self) -> impl Iterator + '_ { self.actor_locations .iter() - .map(|(actor_id, parallel_unit)| ActorInfo { + .map(|(actor_id, worker_slot_id)| ActorInfo { actor_id: *actor_id, - host: self.worker_locations[¶llel_unit.worker_node_id] + host: self.worker_locations[&worker_slot_id.worker_id()] .host .clone(), }) @@ -425,7 +435,7 @@ mod tests { Fact::Fragment(103.into()), Fact::Fragment(104.into()), Fact::ExternalReq { id: 1.into(), dist: DistId::Hash(1) }, - Fact::ExternalReq { id: 2.into(), dist: DistId::Singleton(2) }, + Fact::ExternalReq { id: 2.into(), dist: DistId::Singleton(WorkerSlotId::new(0, 2)) }, Fact::Edge { from: 1.into(), to: 101.into(), dt: NoShuffle }, Fact::Edge { from: 2.into(), to: 102.into(), dt: NoShuffle }, Fact::Edge { from: 101.into(), to: 103.into(), dt: Hash }, @@ -435,7 +445,7 @@ mod tests { let expected = maplit::hashmap! { 101.into() => Result::Required(DistId::Hash(1)), - 102.into() => Result::Required(DistId::Singleton(2)), + 102.into() => Result::Required(DistId::Singleton(WorkerSlotId::new(0, 2))), 103.into() => Result::DefaultHash, 104.into() => Result::DefaultSingleton, }; diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index d6a257575fbd2..62e0d97691922 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -785,7 +785,8 @@ mod tests { use std::time::Duration; use futures::{Stream, TryStreamExt}; - use risingwave_common::hash::ParallelUnitMapping; + use risingwave_common::hash; + use risingwave_common::hash::{ActorMapping, WorkerSlotId}; use risingwave_common::system_param::reader::SystemParamsRead; use risingwave_pb::common::{HostAddress, WorkerType}; use risingwave_pb::meta::add_worker_node_request::Property; @@ -1087,22 +1088,20 @@ mod tests { ) -> MetaResult<()> { // Create fake locations where all actors are scheduled to the same parallel unit. let locations = { - let StreamingClusterInfo { - worker_nodes, - parallel_units, - unschedulable_parallel_units: _, - }: StreamingClusterInfo = self + let StreamingClusterInfo { worker_nodes, .. }: StreamingClusterInfo = self .global_stream_manager .metadata_manager .get_streaming_cluster_info() .await?; + let (worker_id, _worker_node) = worker_nodes.iter().exactly_one().unwrap(); + let actor_locations = fragments .values() .flat_map(|f| &f.actors) .sorted_by(|a, b| a.actor_id.cmp(&b.actor_id)) .enumerate() - .map(|(idx, a)| (a.actor_id, parallel_units[&(idx as u32)].clone())) + .map(|(idx, a)| (a.actor_id, WorkerSlotId::new(*worker_id, idx))) .collect(); Locations { @@ -1167,9 +1166,17 @@ 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(); + (0..count) .map(|i| StreamActor { actor_id: i as u32, + vnode_bitmap: actor_bitmaps.remove(&(i as u32)), // A dummy node to avoid panic. nodes: Some(StreamNode { node_body: Some(NodeBody::Materialize(MaterializeNode { @@ -1191,15 +1198,14 @@ mod tests { let table_id = TableId::new(0); let actors = make_mview_stream_actors(&table_id, 4); - let StreamingClusterInfo { parallel_units, .. } = services + let StreamingClusterInfo { .. } = services .global_stream_manager .metadata_manager .get_streaming_cluster_info() .await?; - let parallel_unit_ids = parallel_units.keys().cloned().sorted().collect_vec(); - let mut fragments = BTreeMap::default(); + fragments.insert( 0, Fragment { @@ -1208,12 +1214,10 @@ mod tests { distribution_type: FragmentDistributionType::Hash as i32, actors: actors.clone(), state_table_ids: vec![0], - vnode_mapping: Some( - ParallelUnitMapping::new_uniform(parallel_unit_ids.into_iter()).to_protobuf(), - ), ..Default::default() }, ); + services .create_materialized_view(table_id, fragments) .await?; @@ -1268,7 +1272,6 @@ mod tests { distribution_type: FragmentDistributionType::Hash as i32, actors: actors.clone(), state_table_ids: vec![0], - vnode_mapping: Some(ParallelUnitMapping::new_single(0).to_protobuf()), ..Default::default() }, ); diff --git a/src/meta/src/stream/test_fragmenter.rs b/src/meta/src/stream/test_fragmenter.rs index 510011e69eb88..f03c9041e6c3d 100644 --- a/src/meta/src/stream/test_fragmenter.rs +++ b/src/meta/src/stream/test_fragmenter.rs @@ -19,9 +19,7 @@ use std::vec; use itertools::Itertools; use risingwave_common::catalog::{DatabaseId, SchemaId, TableId}; use risingwave_pb::catalog::PbTable; -use risingwave_pb::common::{ - ParallelUnit, PbColumnOrder, PbDirection, PbNullsAre, PbOrderType, WorkerNode, -}; +use risingwave_pb::common::{PbColumnOrder, PbDirection, PbNullsAre, PbOrderType, WorkerNode}; use risingwave_pb::data::data_type::TypeName; use risingwave_pb::data::DataType; use risingwave_pb::ddl_service::TableJobType; @@ -420,31 +418,18 @@ fn make_stream_graph() -> StreamFragmentGraphProto { } fn make_cluster_info() -> StreamingClusterInfo { - let parallel_units = (0..8) - .map(|id| { - ( - id, - ParallelUnit { - id, - worker_node_id: 0, - }, - ) - }) - .collect(); - let worker_nodes = std::iter::once(( 0, WorkerNode { id: 0, + parallelism: 8, ..Default::default() }, )) .collect(); - let unschedulable_parallel_units = Default::default(); StreamingClusterInfo { worker_nodes, - parallel_units, - unschedulable_parallel_units, + unschedulable_workers: Default::default(), } } diff --git a/src/rpc_client/src/meta_client.rs b/src/rpc_client/src/meta_client.rs index 296f8de4d888f..42d9258ad8f5e 100644 --- a/src/rpc_client/src/meta_client.rs +++ b/src/rpc_client/src/meta_client.rs @@ -65,7 +65,6 @@ use risingwave_pb::meta::add_worker_node_request::Property; use risingwave_pb::meta::cancel_creating_jobs_request::PbJobs; use risingwave_pb::meta::cluster_service_client::ClusterServiceClient; use risingwave_pb::meta::event_log_service_client::EventLogServiceClient; -use risingwave_pb::meta::get_reschedule_plan_request::PbPolicy; use risingwave_pb::meta::heartbeat_request::{extra_info, ExtraInfo}; use risingwave_pb::meta::heartbeat_service_client::HeartbeatServiceClient; use risingwave_pb::meta::list_actor_states_response::ActorState; @@ -928,32 +927,19 @@ impl MetaClient { pub async fn reschedule( &self, - reschedules: HashMap, + worker_reschedules: HashMap, revision: u64, resolve_no_shuffle_upstream: bool, ) -> Result<(bool, u64)> { let request = RescheduleRequest { - reschedules, revision, resolve_no_shuffle_upstream, + worker_reschedules, }; let resp = self.inner.reschedule(request).await?; Ok((resp.success, resp.revision)) } - pub async fn get_reschedule_plan( - &self, - policy: PbPolicy, - revision: u64, - ) -> Result { - let request = GetReschedulePlanRequest { - revision, - policy: Some(policy), - }; - let resp = self.inner.get_reschedule_plan(request).await?; - Ok(resp) - } - pub async fn risectl_get_pinned_versions_summary( &self, ) -> Result { @@ -2026,7 +2012,6 @@ macro_rules! for_all_meta_rpc { ,{ user_client, revoke_privilege, RevokePrivilegeRequest, RevokePrivilegeResponse } ,{ scale_client, get_cluster_info, GetClusterInfoRequest, GetClusterInfoResponse } ,{ scale_client, reschedule, RescheduleRequest, RescheduleResponse } - ,{ scale_client, get_reschedule_plan, GetReschedulePlanRequest, GetReschedulePlanResponse } ,{ notification_client, subscribe, SubscribeRequest, Streaming } ,{ backup_client, backup_meta, BackupMetaRequest, BackupMetaResponse } ,{ backup_client, get_backup_job_status, GetBackupJobStatusRequest, GetBackupJobStatusResponse } diff --git a/src/tests/simulation/src/ctl_ext.rs b/src/tests/simulation/src/ctl_ext.rs index 77ae0ee414e49..6502b696917e1 100644 --- a/src/tests/simulation/src/ctl_ext.rs +++ b/src/tests/simulation/src/ctl_ext.rs @@ -26,12 +26,11 @@ use itertools::Itertools; use rand::seq::{IteratorRandom, SliceRandom}; use rand::{thread_rng, Rng}; use risingwave_common::catalog::TableId; -use risingwave_common::hash::ParallelUnitId; -use risingwave_pb::meta::get_reschedule_plan_request::PbPolicy; +use risingwave_common::hash::{ParallelUnitId, WorkerSlotId}; 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, GetReschedulePlanResponse}; +use risingwave_pb::meta::GetClusterInfoResponse; use risingwave_pb::stream_plan::StreamNode; use serde::de::IntoDeserializer; @@ -142,19 +141,40 @@ impl Fragment { /// Generate a reschedule plan for the fragment. pub fn reschedule( &self, - remove: impl AsRef<[ParallelUnitId]>, - add: impl AsRef<[ParallelUnitId]>, + remove: impl AsRef<[WorkerSlotId]>, + add: impl AsRef<[WorkerSlotId]>, ) -> String { let remove = remove.as_ref(); let add = add.as_ref(); + let mut worker_decreased = HashMap::new(); + for worker_slot in remove { + let worker_id = worker_slot.worker_id(); + *worker_decreased.entry(worker_id).or_insert(0) += 1; + } + + let mut worker_increased = HashMap::new(); + for worker_slot in add { + let worker_id = worker_slot.worker_id(); + *worker_increased.entry(worker_id).or_insert(0) += 1; + } + + let worker_decr_str = worker_decreased + .iter() + .map(|(work, count)| format!("{}:{}", work, count)) + .join(","); + let worker_incr_str = worker_increased + .iter() + .map(|(work, count)| format!("{}:{}", work, count)) + .join(","); + let mut f = String::new(); write!(f, "{}", self.id()).unwrap(); - if !remove.is_empty() { - write!(f, " -{:?}", remove).unwrap(); + if !worker_decr_str.is_empty() { + write!(f, " -[{}]", worker_decr_str).unwrap(); } - if !add.is_empty() { - write!(f, " +{:?}", add).unwrap(); + if !worker_incr_str.is_empty() { + write!(f, " +[{}]", worker_incr_str).unwrap(); } f } @@ -163,33 +183,56 @@ impl Fragment { /// /// Consumes `self` as the actor info will be stale after rescheduling. pub fn random_reschedule(self) -> String { - let (all_parallel_units, current_parallel_units) = self.parallel_unit_usage(); + let all_worker_slots = self.all_worker_slots(); + let used_worker_slots = self.used_worker_slots(); let rng = &mut thread_rng(); - let target_parallel_unit_count = match self.inner.distribution_type() { + let target_worker_slot_count = match self.inner.distribution_type() { FragmentDistributionType::Unspecified => unreachable!(), FragmentDistributionType::Single => 1, - FragmentDistributionType::Hash => rng.gen_range(1..=all_parallel_units.len()), + FragmentDistributionType::Hash => rng.gen_range(1..=all_worker_slots.len()), }; - let target_parallel_units: HashSet<_> = all_parallel_units - .choose_multiple(rng, target_parallel_unit_count) - .copied() + + let target_worker_slots: HashSet<_> = all_worker_slots + .into_iter() + .choose_multiple(rng, target_worker_slot_count) + .into_iter() .collect(); - let remove = current_parallel_units - .difference(&target_parallel_units) + let remove = used_worker_slots + .difference(&target_worker_slots) .copied() .collect_vec(); - let add = target_parallel_units - .difference(¤t_parallel_units) + + let add = target_worker_slots + .difference(&used_worker_slots) .copied() .collect_vec(); self.reschedule(remove, add) } - pub fn parallel_unit_usage(&self) -> (Vec, HashSet) { - let actor_to_parallel_unit: HashMap<_, _> = self + pub fn all_worker_count(&self) -> HashMap { + self.r + .worker_nodes + .iter() + .map(|w| (w.id, w.parallelism as usize)) + .collect() + } + + pub fn all_worker_slots(&self) -> HashSet { + self.all_worker_count() + .into_iter() + .flat_map(|(k, v)| (0..v).map(move |idx| WorkerSlotId::new(k, idx as _))) + .collect() + } + + pub fn parallelism(&self) -> usize { + self.inner.actors.len() + } + + pub fn used_worker_count(&self) -> HashMap { + let actor_to_worker: HashMap<_, _> = self .r .table_fragments .iter() @@ -197,27 +240,27 @@ impl Fragment { tf.actor_status.iter().map(|(&actor_id, status)| { ( actor_id, - status.get_parallel_unit().unwrap().id as ParallelUnitId, + status.get_parallel_unit().unwrap().get_worker_node_id(), ) }) }) .collect(); - let all_parallel_units = self - .r - .worker_nodes - .iter() - .flat_map(|n| n.parallel_units.iter()) - .map(|p| p.id as ParallelUnitId) - .collect_vec(); - let current_parallel_units: HashSet<_> = self - .inner + self.inner .actors .iter() - .map(|a| actor_to_parallel_unit[&a.actor_id] as ParallelUnitId) - .collect(); + .map(|a| actor_to_worker[&a.actor_id]) + .fold(HashMap::::new(), |mut acc, num| { + *acc.entry(num).or_insert(0) += 1; + acc + }) + } - (all_parallel_units, current_parallel_units) + pub fn used_worker_slots(&self) -> HashSet { + self.used_worker_count() + .into_iter() + .flat_map(|(k, v)| (0..v).map(move |idx| WorkerSlotId::new(k, idx as _))) + .collect() } } @@ -432,37 +475,6 @@ impl Cluster { .await??; Ok(()) } - - #[cfg_or_panic(madsim)] - pub async fn get_reschedule_plan(&self, policy: PbPolicy) -> Result { - let revision = self - .ctl - .spawn(async move { - let r = risingwave_ctl::cmd_impl::meta::get_cluster_info( - &risingwave_ctl::common::CtlContext::default(), - ) - .await?; - - Ok::<_, anyhow::Error>(r.revision) - }) - .await??; - - let resp = self - .ctl - .spawn(async move { - let r = risingwave_ctl::cmd_impl::meta::get_reschedule_plan( - &risingwave_ctl::common::CtlContext::default(), - policy, - revision, - ) - .await?; - - Ok::<_, anyhow::Error>(r) - }) - .await??; - - Ok(resp) - } } #[cfg_attr(not(madsim), allow(dead_code))] diff --git a/src/tests/simulation/tests/integration_tests/recovery/backfill.rs b/src/tests/simulation/tests/integration_tests/recovery/backfill.rs index f2e913b3a4903..5cfd1394c1211 100644 --- a/src/tests/simulation/tests/integration_tests/recovery/backfill.rs +++ b/src/tests/simulation/tests/integration_tests/recovery/backfill.rs @@ -83,13 +83,31 @@ async fn test_snapshot_mv() -> Result<()> { let id = fragment.id(); - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + let workers = fragment.all_worker_count().into_keys().collect_vec(); + + // prev cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}-[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; + sleep(Duration::from_secs(3)).await; // Before complete recovery should be NO_BACKFILL state test_no_backfill_state(&mut session).await?; - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + // prev cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}+[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; + sleep(Duration::from_secs(3)).await; // After recovery should be NO_BACKFILL state @@ -130,7 +148,17 @@ async fn test_backfill_mv() -> Result<()> { let id = fragment.id(); - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + let workers = fragment.all_worker_count().into_keys().collect_vec(); + + // prev cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}-[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; + sleep(Duration::from_secs(3)).await; let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; @@ -139,7 +167,15 @@ async fn test_backfill_mv() -> Result<()> { .await?; assert_eq!(results.lines().collect_vec().len(), 256); - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + // prev cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}+[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; + sleep(Duration::from_secs(3)).await; let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; @@ -183,7 +219,16 @@ async fn test_index_backfill() -> Result<()> { let id = fragment.id(); - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + let workers = fragment.all_worker_count().into_keys().collect_vec(); + + // prev cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}-[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; sleep(Duration::from_secs(3)).await; let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; @@ -192,7 +237,14 @@ async fn test_index_backfill() -> Result<()> { .await?; assert_eq!(results.lines().collect_vec().len(), 256); - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + // prev cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}+[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; sleep(Duration::from_secs(3)).await; let internal_table = session.run(SHOW_INTERNAL_TABLES).await?; diff --git a/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs b/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs index 982b85d48f4cf..e28cdf61594c3 100644 --- a/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs +++ b/src/tests/simulation/tests/integration_tests/scale/auto_parallelism.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet}; +use std::collections::HashMap; use std::time::Duration; use anyhow::Result; @@ -62,37 +62,33 @@ async fn test_passive_online_and_offline() -> Result<()> { ]) .await?; - let (_, single_used_parallel_unit_ids) = single_agg_fragment.parallel_unit_usage(); + assert_eq!(single_agg_fragment.parallelism(), 1); - let used_parallel_unit_id = single_used_parallel_unit_ids.iter().next().unwrap(); + let used_worker_slots = single_agg_fragment.used_worker_count(); - let mut workers: Vec = cluster + let (single_used_worker_id, should_be_one) = + used_worker_slots.into_iter().exactly_one().unwrap(); + + assert_eq!(should_be_one, 1); + + let worker_map: HashMap<_, _> = cluster .get_cluster_info() .await? .worker_nodes .into_iter() .filter(|worker| worker.r#type() == WorkerType::ComputeNode) + .map(|worker| (worker.id, worker)) .collect(); - let prev_workers = workers - .extract_if(|worker| { - worker - .parallel_units - .iter() - .map(|parallel_unit| parallel_unit.id) - .contains(used_parallel_unit_id) - }) - .collect_vec(); - - let prev_worker = prev_workers.into_iter().exactly_one().unwrap(); - let host = prev_worker.host.unwrap().host; + let prev_worker = worker_map.get(&single_used_worker_id).unwrap(); + let host = prev_worker.clone().host.unwrap().host; let host_name = format!("compute-{}", host.split('.').last().unwrap()); - let (all_parallel_units, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); - - assert_eq!(all_parallel_units.len(), used_parallel_units.len()); + let all_worker_slots = table_mat_fragment.all_worker_count(); + let used_worker_slots = table_mat_fragment.used_worker_count(); + assert_eq!(all_worker_slots, used_worker_slots); - let initialized_parallelism = used_parallel_units.len(); + let initialized_parallelism = table_mat_fragment.parallelism(); assert_eq!( initialized_parallelism, @@ -100,13 +96,11 @@ async fn test_passive_online_and_offline() -> Result<()> { ); cluster.simple_kill_nodes(vec![host_name.clone()]).await; - // wait for a while sleep(Duration::from_secs( MAX_HEARTBEAT_INTERVAL_SECS_CONFIG_FOR_AUTO_SCALE * 2, )) .await; - let table_mat_fragment = cluster .locate_one_fragment(vec![ identity_contains("materialize"), @@ -114,22 +108,18 @@ async fn test_passive_online_and_offline() -> Result<()> { ]) .await?; - let (_, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); - assert_eq!( initialized_parallelism - config.compute_node_cores, - used_parallel_units.len() + table_mat_fragment.parallelism() ); let stream_scan_fragment = cluster .locate_one_fragment(vec![identity_contains("streamTableScan")]) .await?; - let (_, used_parallel_units) = stream_scan_fragment.parallel_unit_usage(); - assert_eq!( initialized_parallelism - config.compute_node_cores, - used_parallel_units.len() + stream_scan_fragment.parallelism() ); let single_agg_fragment = cluster @@ -139,12 +129,11 @@ async fn test_passive_online_and_offline() -> Result<()> { ]) .await?; - let (_, used_parallel_units_ids) = single_agg_fragment.parallel_unit_usage(); - - assert_eq!(used_parallel_units_ids.len(), 1); - - assert_ne!(single_used_parallel_unit_ids, used_parallel_units_ids); + let used_worker_slots = single_agg_fragment.used_worker_count(); + let (curr_used_worker_id, should_be_one) = used_worker_slots.into_iter().exactly_one().unwrap(); + assert_eq!(should_be_one, 1); + assert_ne!(single_used_worker_id, curr_used_worker_id); session .run("select count(*) from t") .await? @@ -186,17 +175,14 @@ async fn test_passive_online_and_offline() -> Result<()> { ]) .await?; - let (_, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); - - assert_eq!(initialized_parallelism, used_parallel_units.len()); + assert_eq!(initialized_parallelism, table_mat_fragment.parallelism()); let stream_scan_fragment = cluster .locate_one_fragment(vec![identity_contains("streamTableScan")]) .await?; - let (_, used_parallel_units) = stream_scan_fragment.parallel_unit_usage(); + assert_eq!(initialized_parallelism, stream_scan_fragment.parallelism()); - assert_eq!(initialized_parallelism, used_parallel_units.len()); session .run("select count(*) from t") .await? @@ -248,14 +234,13 @@ async fn test_active_online() -> Result<()> { ]) .await?; - let (all_parallel_units, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + let all_worker_slots = table_mat_fragment.all_worker_count(); - assert_eq!(all_parallel_units.len(), used_parallel_units.len()); + let used_worker_slots = table_mat_fragment.used_worker_count(); - assert_eq!( - all_parallel_units.len(), - (config.compute_nodes - 1) * config.compute_node_cores - ); + assert_eq!(all_worker_slots, used_worker_slots); + + assert_eq!(all_worker_slots.len(), config.compute_nodes - 1); cluster .simple_restart_nodes(vec!["compute-2".to_string()]) @@ -273,14 +258,12 @@ async fn test_active_online() -> Result<()> { ]) .await?; - let (all_parallel_units, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + let all_worker_slots = table_mat_fragment.all_worker_count(); - assert_eq!(all_parallel_units.len(), used_parallel_units.len()); + let used_worker_slots = table_mat_fragment.used_worker_count(); - assert_eq!( - all_parallel_units.len(), - config.compute_nodes * config.compute_node_cores - ); + assert_eq!(all_worker_slots, used_worker_slots); + assert_eq!(all_worker_slots.len(), config.compute_nodes); Ok(()) } @@ -334,14 +317,11 @@ async fn test_auto_parallelism_control_with_fixed_and_auto_helper( let table_mat_fragment = locate_table_fragment(&mut cluster).await?; - let (all_parallel_units, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + let all_worker_slots = table_mat_fragment.all_worker_count(); + let used_worker_slots = table_mat_fragment.used_worker_count(); - assert_eq!(all_parallel_units.len(), used_parallel_units.len()); - - assert_eq!( - all_parallel_units.len(), - (config.compute_nodes - 1) * config.compute_node_cores - ); + assert_eq!(all_worker_slots, used_worker_slots); + assert_eq!(all_worker_slots.len(), config.compute_nodes - 1); session.run("alter table t set parallelism = 3").await?; @@ -352,9 +332,7 @@ async fn test_auto_parallelism_control_with_fixed_and_auto_helper( let table_mat_fragment = locate_table_fragment(&mut cluster).await?; - let (_, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); - - assert_eq!(used_parallel_units.len(), 3); + assert_eq!(table_mat_fragment.parallelism(), 3); // Keep one worker reserved for adding later. cluster @@ -377,33 +355,18 @@ async fn test_auto_parallelism_control_with_fixed_and_auto_helper( assert_eq!(workers.len(), 3); - let parallel_unit_to_worker = workers - .into_iter() - .flat_map(|worker| { - worker - .parallel_units - .into_iter() - .map(move |parallel_unit| (parallel_unit.id, worker.id)) - }) - .collect::>(); - let table_mat_fragment = locate_table_fragment(&mut cluster).await?; - let (_, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + let used_worker_slots = table_mat_fragment.used_worker_count(); - assert_eq!(used_parallel_units.len(), 3); - - let worker_ids: HashSet<_> = used_parallel_units - .iter() - .map(|id| parallel_unit_to_worker.get(id).unwrap()) - .collect(); + assert_eq!(table_mat_fragment.parallelism(), 3); // check auto scale out for fixed if enable_auto_parallelism_control { - assert_eq!(worker_ids.len(), config.compute_nodes); + assert_eq!(used_worker_slots.len(), config.compute_nodes); } else { // no rebalance process - assert_eq!(worker_ids.len(), config.compute_nodes - 1); + assert_eq!(used_worker_slots.len(), config.compute_nodes - 1); } // We kill compute-2 again to verify the behavior of auto scale-in @@ -418,16 +381,11 @@ async fn test_auto_parallelism_control_with_fixed_and_auto_helper( let table_mat_fragment = locate_table_fragment(&mut cluster).await?; - let (_, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + let used_worker_slots = table_mat_fragment.used_worker_count(); - assert_eq!(used_parallel_units.len(), 3); + assert_eq!(table_mat_fragment.parallelism(), 3); - let worker_ids: HashSet<_> = used_parallel_units - .iter() - .map(|id| parallel_unit_to_worker.get(id).unwrap()) - .collect(); - - assert_eq!(worker_ids.len(), config.compute_nodes - 1); + assert_eq!(used_worker_slots.len(), config.compute_nodes - 1); // We alter parallelism back to auto @@ -442,14 +400,11 @@ async fn test_auto_parallelism_control_with_fixed_and_auto_helper( let table_mat_fragment = locate_table_fragment(&mut cluster).await?; - let (all_parallel_units, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + let all_worker_slots = table_mat_fragment.all_worker_count(); + let used_worker_slots = table_mat_fragment.used_worker_count(); - assert_eq!(all_parallel_units.len(), used_parallel_units.len()); - - assert_eq!( - all_parallel_units.len(), - (config.compute_nodes - 1) * config.compute_node_cores - ); + assert_eq!(all_worker_slots, used_worker_slots); + assert_eq!(all_worker_slots.len(), config.compute_nodes - 1); // Keep one worker reserved for adding later. cluster @@ -463,21 +418,15 @@ async fn test_auto_parallelism_control_with_fixed_and_auto_helper( let table_mat_fragment = locate_table_fragment(&mut cluster).await?; - let (all_parallel_units, used_parallel_units) = table_mat_fragment.parallel_unit_usage(); + let all_worker_slots = table_mat_fragment.all_worker_count(); + let used_worker_slots = table_mat_fragment.used_worker_count(); // check auto scale out for auto if enable_auto_parallelism_control { - assert_eq!(all_parallel_units.len(), used_parallel_units.len()); - - assert_eq!( - all_parallel_units.len(), - config.compute_nodes * config.compute_node_cores - ); + assert_eq!(all_worker_slots, used_worker_slots); + assert_eq!(all_worker_slots.len(), config.compute_nodes); } else { - assert_eq!( - used_parallel_units.len(), - (config.compute_nodes - 1) * config.compute_node_cores - ); + assert_eq!(used_worker_slots.len(), config.compute_nodes - 1); } Ok(()) @@ -530,18 +479,19 @@ async fn test_compatibility_with_low_level() -> Result<()> { ]) .await?; - let (mut all_parallel_units, _) = table_mat_fragment.parallel_unit_usage(); + let mut all_workers = table_mat_fragment + .all_worker_count() + .into_keys() + .collect_vec(); - let chosen_parallel_unit_a = all_parallel_units.pop().unwrap(); - let chosen_parallel_unit_b = all_parallel_units.pop().unwrap(); + let chosen_worker_a = all_workers.pop().unwrap(); + let chosen_worker_b = all_workers.pop().unwrap(); let table_mat_fragment_id = table_mat_fragment.id(); // manual scale in table materialize fragment cluster - .reschedule(format!( - "{table_mat_fragment_id}-[{chosen_parallel_unit_a}]", - )) + .reschedule(format!("{table_mat_fragment_id}-[{chosen_worker_a}:1]",)) .await?; session @@ -565,9 +515,7 @@ async fn test_compatibility_with_low_level() -> Result<()> { // manual scale in m_simple materialize fragment cluster - .reschedule_resolve_no_shuffle(format!( - "{simple_mv_fragment_id}-[{chosen_parallel_unit_b}]", - )) + .reschedule_resolve_no_shuffle(format!("{simple_mv_fragment_id}-[{chosen_worker_b}:1]",)) .await?; // Since `m_simple` only has 1 fragment, and this fragment is a downstream of NO_SHUFFLE relation, @@ -592,9 +540,7 @@ async fn test_compatibility_with_low_level() -> Result<()> { // manual scale in m_join materialize fragment cluster - .reschedule_resolve_no_shuffle(format!( - "{hash_join_fragment_id}-[{chosen_parallel_unit_a}]" - )) + .reschedule_resolve_no_shuffle(format!("{hash_join_fragment_id}-[{chosen_worker_a}:1]")) .await?; session @@ -667,18 +613,19 @@ async fn test_compatibility_with_low_level_and_arrangement_backfill() -> Result< ]) .await?; - let (mut all_parallel_units, _) = table_mat_fragment.parallel_unit_usage(); + let mut all_workers = table_mat_fragment + .all_worker_count() + .into_keys() + .collect_vec(); - let chosen_parallel_unit_a = all_parallel_units.pop().unwrap(); - let chosen_parallel_unit_b = all_parallel_units.pop().unwrap(); + let chosen_worker_a = all_workers.pop().unwrap(); + let chosen_worker_b = all_workers.pop().unwrap(); let table_mat_fragment_id = table_mat_fragment.id(); // manual scale in table materialize fragment cluster - .reschedule(format!( - "{table_mat_fragment_id}-[{chosen_parallel_unit_a}]", - )) + .reschedule(format!("{table_mat_fragment_id}-[{chosen_worker_a}:1]",)) .await?; session @@ -704,9 +651,7 @@ async fn test_compatibility_with_low_level_and_arrangement_backfill() -> Result< // manual scale in m_simple materialize fragment cluster - .reschedule_resolve_no_shuffle(format!( - "{simple_mv_fragment_id}-[{chosen_parallel_unit_b}]", - )) + .reschedule_resolve_no_shuffle(format!("{simple_mv_fragment_id}-[{chosen_worker_b}:1]",)) .await?; // The downstream table fragment should be separate from the upstream table fragment. diff --git a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs index 981f79103403d..20edb5b4ed262 100644 --- a/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs +++ b/src/tests/simulation/tests/integration_tests/scale/cascade_materialized_view.rs @@ -48,11 +48,18 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { let id = fragment.id(); - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + let all_workers = fragment.all_worker_count().into_keys().collect_vec(); + + cluster + .reschedule(format!( + "{id}-[{}]", + all_workers.iter().map(|w| format!("{w}:1")).join(",") + )) + .await?; sleep(Duration::from_secs(3)).await; let fragment = cluster.locate_fragment_by_id(id).await?; - assert_eq!(fragment.inner.actors.len(), 1); + assert_eq!(fragment.inner.actors.len(), 3); let chain_fragment = cluster .locate_one_fragment([identity_contains("StreamTableScan")]) @@ -60,9 +67,9 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { if arrangement_backfill_is_enabled { // The chain fragment is in a different table fragment. - assert_eq!(chain_fragment.inner.actors.len(), 6,); + assert_eq!(chain_fragment.inner.actors.len(), 6); // The upstream materialized fragment should be scaled in - assert_eq!(fragment.inner.actors.len(), 1,); + assert_eq!(fragment.inner.actors.len(), 3); } else { // No shuffle, so the fragment of upstream materialized node is the same // as stream table scan. @@ -87,7 +94,12 @@ async fn test_simple_cascade_materialized_view() -> Result<()> { .await? .assert_result_eq("5"); - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{id}+[{}]", + all_workers.iter().map(|w| format!("{w}:1")).join(",") + )) + .await?; sleep(Duration::from_secs(3)).await; let fragment = cluster.locate_fragment_by_id(id).await?; @@ -147,11 +159,19 @@ async fn test_diamond_cascade_materialized_view() -> Result<()> { let id = fragment.id(); - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + let all_workers = fragment.all_worker_count().into_keys().collect_vec(); + + cluster + .reschedule(format!( + "{id}-[{}]", + all_workers.iter().map(|w| format!("{w}:1")).join(",") + )) + .await?; + sleep(Duration::from_secs(3)).await; let fragment = cluster.locate_fragment_by_id(id).await?; - assert_eq!(fragment.inner.actors.len(), 1); + assert_eq!(fragment.inner.actors.len(), 3); session .run(&format!( @@ -166,7 +186,13 @@ async fn test_diamond_cascade_materialized_view() -> Result<()> { .await? .assert_result_eq("0"); - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{id}+[{}]", + all_workers.iter().map(|w| format!("{w}:1")).join(",") + )) + .await?; + sleep(Duration::from_secs(3)).await; let fragment = cluster.locate_fragment_by_id(id).await?; diff --git a/src/tests/simulation/tests/integration_tests/scale/dynamic_filter.rs b/src/tests/simulation/tests/integration_tests/scale/dynamic_filter.rs index a6853b3515c1a..68c24a073a87f 100644 --- a/src/tests/simulation/tests/integration_tests/scale/dynamic_filter.rs +++ b/src/tests/simulation/tests/integration_tests/scale/dynamic_filter.rs @@ -16,6 +16,7 @@ use std::collections::HashSet; use std::time::Duration; use anyhow::Result; +use itertools::Itertools; use risingwave_simulation::cluster::{Cluster, Configuration}; use risingwave_simulation::ctl_ext::predicate::identity_contains; use risingwave_simulation::utils::AssertResult; @@ -56,7 +57,16 @@ async fn test_dynamic_filter() -> Result<()> { let id = fragment.id(); - cluster.reschedule(format!("{id}-[1,2,3]")).await?; + let (worker_1, worker_2, worker_3) = fragment + .all_worker_count() + .into_keys() + .collect_tuple::<(_, _, _)>() + .unwrap(); + + // prev -[1,2,3] + cluster + .reschedule(format!("{id}-[{worker_1}:2, {worker_2}:1]")) + .await?; sleep(Duration::from_secs(3)).await; session.run(SELECT).await?.assert_result_eq(""); @@ -68,7 +78,12 @@ async fn test_dynamic_filter() -> Result<()> { // 2 // 3 - cluster.reschedule(format!("{id}-[4,5]+[1,2,3]")).await?; + // prev -[4,5]+[1,2,3] + cluster + .reschedule(format!( + "{id}-[{worker_3}:1, {worker_2}:1]+[{worker_1}:2, {worker_2}:1]" + )) + .await?; sleep(Duration::from_secs(3)).await; session.run(SELECT).await?.assert_result_eq("1\n2\n3"); @@ -78,7 +93,12 @@ async fn test_dynamic_filter() -> Result<()> { session.run(SELECT).await?.assert_result_eq("3"); // 3 - cluster.reschedule(format!("{id}-[1,2,3]+[4,5]")).await?; + // prev -[1,2,3]+[4,5] + cluster + .reschedule(format!( + "{id}-[{worker_1}:2, {worker_2}:1]+[{worker_2}:1, {worker_3}:1]" + )) + .await?; sleep(Duration::from_secs(3)).await; session.run(SELECT).await?.assert_result_eq("3"); @@ -89,7 +109,10 @@ async fn test_dynamic_filter() -> Result<()> { // 2 // 3 // - cluster.reschedule(format!("{id}+[1,2,3]")).await?; + // prev +[1,2,3] + cluster + .reschedule(format!("{id}+[{worker_1}:2, {worker_2}:1]")) + .await?; sleep(Duration::from_secs(3)).await; session.run(SELECT).await?.assert_result_eq("2\n3"); @@ -98,7 +121,8 @@ async fn test_dynamic_filter() -> Result<()> { sleep(Duration::from_secs(5)).await; session.run(SELECT).await?.assert_result_eq(""); - cluster.reschedule(format!("{id}-[1]")).await?; + // prev -[1] + cluster.reschedule(format!("{id}+[{worker_1}:1]")).await?; sleep(Duration::from_secs(3)).await; session.run(SELECT).await?.assert_result_eq(""); diff --git a/src/tests/simulation/tests/integration_tests/scale/mod.rs b/src/tests/simulation/tests/integration_tests/scale/mod.rs index ced22e6875465..f6940f072409e 100644 --- a/src/tests/simulation/tests/integration_tests/scale/mod.rs +++ b/src/tests/simulation/tests/integration_tests/scale/mod.rs @@ -19,7 +19,6 @@ mod nexmark_chaos; mod nexmark_q4; mod nexmark_source; mod no_shuffle; -mod plan; mod schedulability; mod singleton_migration; mod sink; diff --git a/src/tests/simulation/tests/integration_tests/scale/nexmark_q4.rs b/src/tests/simulation/tests/integration_tests/scale/nexmark_q4.rs index 6408a8d47f951..d0a25e1405da4 100644 --- a/src/tests/simulation/tests/integration_tests/scale/nexmark_q4.rs +++ b/src/tests/simulation/tests/integration_tests/scale/nexmark_q4.rs @@ -15,6 +15,8 @@ use std::time::Duration; use anyhow::Result; +use itertools::Itertools; +use risingwave_common::hash::WorkerSlotId; use risingwave_simulation::cluster::Configuration; use risingwave_simulation::ctl_ext::predicate::{ identity_contains, upstream_fragment_count, BoxedPredicate, @@ -69,7 +71,7 @@ async fn nexmark_q4_common(predicates: impl IntoIterator) let mut cluster = init().await?; let fragment = cluster.locate_one_fragment(predicates).await?; - let id = fragment.id(); + let workers = fragment.all_worker_count().into_keys().collect_vec(); // 0s wait_initial_data(&mut cluster) @@ -77,13 +79,32 @@ async fn nexmark_q4_common(predicates: impl IntoIterator) .assert_result_ne(RESULT); // 0~10s - cluster.reschedule(format!("{id}-[0,1]")).await?; + cluster + .reschedule(fragment.reschedule( + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[0], 1), + ], + [], + )) + .await?; sleep(Duration::from_secs(5)).await; // 5~15s cluster.run(SELECT).await?.assert_result_ne(RESULT); - cluster.reschedule(format!("{id}-[2,3]+[0,1]")).await?; + cluster + .reschedule(fragment.reschedule( + [ + WorkerSlotId::new(workers[1], 0), + WorkerSlotId::new(workers[1], 1), + ], + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[0], 1), + ], + )) + .await?; sleep(Duration::from_secs(20)).await; @@ -138,6 +159,9 @@ async fn nexmark_q4_cascade() -> Result<()> { .await?; let id_2 = fragment_2.id(); + // todo, fragment_1's worker + let workers = fragment_1.all_worker_count().into_keys().collect_vec(); + // 0s wait_initial_data(&mut cluster) .await? @@ -145,7 +169,13 @@ async fn nexmark_q4_cascade() -> Result<()> { // 0~10s cluster - .reschedule(format!("{id_1}-[0,1]; {id_2}-[0,2,4]")) + .reschedule(format!( + "{}-[{}];{}-[{}]", + fragment_1.id(), + format_args!("{}:2", workers[0]), + fragment_2.id(), + format_args!("{}:1,{}:1,{}:1", workers[0], workers[1], workers[2]), + )) .await?; sleep(Duration::from_secs(5)).await; @@ -153,7 +183,15 @@ async fn nexmark_q4_cascade() -> Result<()> { // 5~15s cluster.run(SELECT).await?.assert_result_ne(RESULT); cluster - .reschedule(format!("{id_1}-[2,4]+[0,1]; {id_2}-[3]+[0,4]")) + .reschedule(format!( + "{}-[{}]+[{}];{}-[{}]+[{}]", + id_1, + format_args!("{}:1,{}:1", workers[1], workers[2]), + format_args!("{}:2", workers[0]), + id_2, + format_args!("{}:1", workers[1]), + format_args!("{}:1,{}:1", workers[0], workers[2]), + )) .await?; sleep(Duration::from_secs(20)).await; @@ -176,14 +214,27 @@ async fn nexmark_q4_materialize_agg_cache_invalidation() -> Result<()> { ]) .await?; let id = fragment.id(); + let workers = fragment.all_worker_count().into_keys().collect_vec(); // Let parallel unit 0 handle all groups. - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}-[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; sleep(Duration::from_secs(7)).await; let result_1 = cluster.run(SELECT).await?.assert_result_ne(RESULT); // Scale out. - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}+[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; sleep(Duration::from_secs(7)).await; cluster .run(SELECT) @@ -194,7 +245,13 @@ async fn nexmark_q4_materialize_agg_cache_invalidation() -> Result<()> { // Let parallel unit 0 handle all groups again. // Note that there're only 5 groups, so if the parallel unit 0 doesn't invalidate the cache // correctly, it will yield the wrong result. - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + cluster + .reschedule(format!( + "{}-[{}]", + id, + format_args!("{}:1,{}:2,{}:2", workers[0], workers[1], workers[2]), + )) + .await?; sleep(Duration::from_secs(20)).await; cluster.run(SELECT).await?.assert_result_eq(RESULT); diff --git a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs index a409443371ba9..28104f41aeb38 100644 --- a/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs +++ b/src/tests/simulation/tests/integration_tests/scale/no_shuffle.rs @@ -14,6 +14,7 @@ use anyhow::Result; use itertools::Itertools; +use risingwave_common::hash::WorkerSlotId; use risingwave_simulation::cluster::{Cluster, Configuration}; use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; use risingwave_simulation::utils::AssertResult; @@ -86,34 +87,58 @@ async fn test_delta_join() -> Result<()> { test_works!(); + let workers = union_fragment.all_worker_count().into_keys().collect_vec(); // Scale-in one side - cluster.reschedule(format!("{}-[0]", t1.id())).await?; + cluster + .reschedule(format!("{}-[{}:1]", t1.id(), workers[0])) + .await?; + test_works!(); // Scale-in both sides together cluster - .reschedule(format!("{}-[2];{}-[0,2]", t1.id(), t2.id())) + .reschedule(format!( + "{}-[{}];{}-[{}]", + t1.id(), + format_args!("{}:1", workers[1]), + t2.id(), + format_args!("{}:1, {}:1", workers[0], workers[1]) + )) .await?; test_works!(); // Scale-out one side - cluster.reschedule(format!("{}+[0]", t2.id())).await?; + cluster + .reschedule(format!("{}+[{}:1]", t2.id(), workers[0])) + .await?; test_works!(); // Scale-out both sides together cluster - .reschedule(format!("{}+[0,2];{}+[2]", t1.id(), t2.id())) + .reschedule(format!( + "{}+[{}];{}+[{}]", + t1.id(), + format_args!("{}:1,{}:1", workers[0], workers[1]), + t2.id(), + format_args!("{}:1", workers[1]), + )) .await?; test_works!(); // Scale-in join with union cluster - .reschedule(format!("{}-[5];{}-[5]", t1.id(), union_fragment.id())) + .reschedule(format!( + "{}-[{}];{}-[{}]", + t1.id(), + format_args!("{}:1", workers[2]), + t2.id(), + format_args!("{}:1", workers[2]) + )) .await?; test_works!(); let result = cluster - .reschedule(format!("{}-[0]", lookup_fragments[0].id())) + .reschedule(format!("{}-[{}:1]", lookup_fragments[0].id(), workers[0])) .await; assert!( result.is_err(), @@ -137,8 +162,15 @@ async fn test_share_multiple_no_shuffle_upstream() -> Result<()> { .locate_one_fragment([identity_contains("hashagg")]) .await?; - cluster.reschedule(fragment.reschedule([0], [])).await?; - cluster.reschedule(fragment.reschedule([], [0])).await?; + let workers = fragment.all_worker_count().into_keys().collect_vec(); + + cluster + .reschedule(fragment.reschedule([WorkerSlotId::new(workers[0], 0)], [])) + .await?; + + cluster + .reschedule(fragment.reschedule([], [WorkerSlotId::new(workers[0], 0)])) + .await?; Ok(()) } @@ -157,15 +189,20 @@ async fn test_resolve_no_shuffle_upstream() -> Result<()> { .locate_one_fragment([identity_contains("StreamTableScan")]) .await?; - let result = cluster.reschedule(fragment.reschedule([0], [])).await; + let workers = fragment.all_worker_count().into_keys().collect_vec(); + + let result = cluster + .reschedule(fragment.reschedule([WorkerSlotId::new(workers[0], 0)], [])) + .await; assert!(result.is_err()); cluster - .reschedule_resolve_no_shuffle(fragment.reschedule([0], [])) + .reschedule_resolve_no_shuffle(fragment.reschedule([WorkerSlotId::new(workers[0], 0)], [])) .await?; + cluster - .reschedule_resolve_no_shuffle(fragment.reschedule([], [0])) + .reschedule_resolve_no_shuffle(fragment.reschedule([], [WorkerSlotId::new(workers[0], 0)])) .await?; Ok(()) diff --git a/src/tests/simulation/tests/integration_tests/scale/plan.rs b/src/tests/simulation/tests/integration_tests/scale/plan.rs deleted file mode 100644 index d39e159fc61d9..0000000000000 --- a/src/tests/simulation/tests/integration_tests/scale/plan.rs +++ /dev/null @@ -1,332 +0,0 @@ -// 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 anyhow::Result; -use itertools::Itertools; -use rand::seq::SliceRandom; -use risingwave_pb::common::{WorkerNode, WorkerType}; -use risingwave_pb::meta::get_reschedule_plan_request::Policy::StableResizePolicy; -use risingwave_pb::meta::get_reschedule_plan_request::{ - PbPolicy, PbStableResizePolicy, WorkerChanges, -}; -use risingwave_pb::meta::PbReschedule; -use risingwave_simulation::cluster::{Cluster, Configuration}; -use risingwave_simulation::ctl_ext::predicate::{identity_contains, no_identity_contains}; - -#[tokio::test] -async fn test_resize_normal() -> Result<()> { - let mut cluster = Cluster::start(Configuration::for_scale()).await?; - let mut session = cluster.start_session(); - - session.run("create table t1 (v1 int);").await?; - session.run("create table t2 (v2 int);").await?; - session - .run("create materialized view mv as select * from t1 join t2 on t1.v1 = t2.v2;") - .await?; - - let join_fragment = cluster - .locate_one_fragment([identity_contains("hashJoin")]) - .await?; - - let join_fragment_id = join_fragment.inner.fragment_id; - - let mut workers: Vec = cluster - .get_cluster_info() - .await? - .worker_nodes - .into_iter() - .filter(|worker| worker.r#type() == WorkerType::ComputeNode) - .collect(); - - workers.pop(); - - let removed_workers = workers.iter().map(|worker| worker.id).collect_vec(); - - let resp = cluster - .get_reschedule_plan(PbPolicy::StableResizePolicy(PbStableResizePolicy { - fragment_worker_changes: HashMap::from([( - join_fragment_id, - WorkerChanges { - include_worker_ids: vec![], - exclude_worker_ids: removed_workers, - ..Default::default() - }, - )]), - })) - .await?; - - let reschedules = resp.reschedules; - assert_eq!(reschedules.len(), 1); - let target_plan: PbReschedule = reschedules.get(&join_fragment_id).unwrap().clone(); - - assert_eq!(target_plan.added_parallel_units.len(), 0); - - let removed_parallel_unit_id = workers - .iter() - .flat_map(|worker| { - worker - .parallel_units - .iter() - .map(|parallel_unit| parallel_unit.id) - }) - .sorted() - .collect_vec(); - - assert_eq!(target_plan.removed_parallel_units, removed_parallel_unit_id); - - Ok(()) -} -#[tokio::test] -async fn test_resize_single() -> Result<()> { - let mut cluster = Cluster::start(Configuration::for_scale()).await?; - let mut session = cluster.start_session(); - - session.run("create table t (v int);").await?; - session - .run("create materialized view mv1 as select count(*) from t;") - .await?; - - session - .run("create materialized view mv2 as select * from mv1;") - .await?; - - let agg_fragment = cluster - .locate_one_fragment([ - identity_contains("simpleAgg"), - identity_contains("materialize"), - ]) - .await?; - - let agg_fragment_id = agg_fragment.inner.fragment_id; - - let (_, used_parallel_unit_ids) = agg_fragment.parallel_unit_usage(); - - assert_eq!(used_parallel_unit_ids.len(), 1); - - let used_parallel_unit_id = used_parallel_unit_ids.iter().next().unwrap(); - - let mut workers: Vec = cluster - .get_cluster_info() - .await? - .worker_nodes - .into_iter() - .filter(|worker| worker.r#type() == WorkerType::ComputeNode) - .collect(); - - let prev_workers = workers - .extract_if(|worker| { - worker - .parallel_units - .iter() - .map(|parallel_unit| parallel_unit.id) - .contains(used_parallel_unit_id) - }) - .collect_vec(); - - let prev_worker = prev_workers.into_iter().exactly_one().unwrap(); - - let resp = cluster - .get_reschedule_plan(StableResizePolicy(PbStableResizePolicy { - fragment_worker_changes: HashMap::from([( - agg_fragment_id, - WorkerChanges { - include_worker_ids: vec![], - exclude_worker_ids: vec![prev_worker.id], - ..Default::default() - }, - )]), - })) - .await?; - - let reschedules = resp.reschedules; - assert_eq!(reschedules.len(), 1); - let target_plan: PbReschedule = reschedules.get(&agg_fragment_id).unwrap().clone(); - assert_eq!(target_plan.added_parallel_units.len(), 1); - assert_eq!(target_plan.removed_parallel_units.len(), 1); - - let removed_parallel_unit_id = target_plan - .removed_parallel_units - .iter() - .exactly_one() - .unwrap(); - - assert!(prev_worker - .parallel_units - .iter() - .map(|parallel_unit| parallel_unit.id) - .contains(removed_parallel_unit_id)); - - Ok(()) -} - -#[tokio::test] -async fn test_resize_single_failed() -> Result<()> { - let mut cluster = Cluster::start(Configuration::for_scale_no_shuffle()).await?; - let mut session = cluster.start_session(); - - session.run("create table t (v int);").await?; - session - .run("create materialized view mv1 as select count(*) from t;") - .await?; - - session - .run("create materialized view mv2 as select * from mv1;") - .await?; - - let upstream_fragment = cluster - .locate_one_fragment([ - identity_contains("simpleAgg"), - identity_contains("materialize"), - ]) - .await?; - - let upstream_fragment_id = upstream_fragment.inner.fragment_id; - - let downstream_fragment = cluster - .locate_one_fragment([ - identity_contains("StreamTableScan"), - identity_contains("materialize"), - ]) - .await?; - - let downstream_fragment_id = downstream_fragment.inner.fragment_id; - - let mut workers: Vec = cluster - .get_cluster_info() - .await? - .worker_nodes - .into_iter() - .filter(|worker| worker.r#type() == WorkerType::ComputeNode) - .collect(); - - let worker_a = workers.pop().unwrap(); - let worker_b = workers.pop().unwrap(); - - let resp = cluster - .get_reschedule_plan(StableResizePolicy(PbStableResizePolicy { - fragment_worker_changes: HashMap::from([ - ( - upstream_fragment_id, - WorkerChanges { - include_worker_ids: vec![], - exclude_worker_ids: vec![worker_a.id], - ..Default::default() - }, - ), - ( - downstream_fragment_id, - WorkerChanges { - include_worker_ids: vec![], - exclude_worker_ids: vec![worker_b.id], - ..Default::default() - }, - ), - ]), - })) - .await; - - assert!(resp.is_err()); - - Ok(()) -} -#[tokio::test] -async fn test_resize_no_shuffle() -> Result<()> { - let mut cluster = Cluster::start(Configuration::for_scale_no_shuffle()).await?; - let mut session = cluster.start_session(); - - session.run("create table t (v int);").await?; - session - .run("create materialized view mv1 as select * from t;") - .await?; - session - .run("create materialized view mv2 as select * from t;") - .await?; - session - .run("create materialized view mv3 as select * from mv2;") - .await?; - session - .run("create materialized view mv4 as select * from mv2;") - .await?; - session - .run("create materialized view mv5 as select * from mv3;") - .await?; - session - .run("create materialized view mv6 as select * from mv3;") - .await?; - session - .run( - "create materialized view mv7 as select mv1.v as mv1v, mv5.v as mv5v from mv1 -join mv5 on mv1.v = mv5.v limit 1;", - ) - .await?; - - let chain_fragments: [_; 8] = cluster - .locate_fragments([identity_contains("StreamTableScan")]) - .await? - .try_into() - .unwrap(); - - let selected_fragment = chain_fragments.choose(&mut rand::thread_rng()).unwrap(); - - let selected_fragment_id = selected_fragment.inner.fragment_id; - - let mut workers: Vec = cluster - .get_cluster_info() - .await? - .worker_nodes - .into_iter() - .filter(|worker: &WorkerNode| worker.r#type() == WorkerType::ComputeNode) - .collect(); - - workers.pop(); - - let removed_worker_ids = workers.iter().map(|worker| worker.id).collect_vec(); - - let resp = cluster - .get_reschedule_plan(PbPolicy::StableResizePolicy(PbStableResizePolicy { - fragment_worker_changes: HashMap::from([( - selected_fragment_id, - WorkerChanges { - include_worker_ids: vec![], - exclude_worker_ids: removed_worker_ids, - ..Default::default() - }, - )]), - })) - .await?; - - let reschedules = resp.reschedules; - - assert_eq!(reschedules.len(), 1); - - let top_materialize_fragment = cluster - .locate_one_fragment([ - identity_contains("materialize"), - no_identity_contains("topn"), - no_identity_contains("StreamTableScan"), - no_identity_contains("hashJoin"), - ]) - .await?; - - let top_materialize_fragment_id = reschedules.keys().exactly_one().cloned().unwrap(); - - assert_eq!( - top_materialize_fragment_id, - top_materialize_fragment.inner.fragment_id - ); - - Ok(()) -} diff --git a/src/tests/simulation/tests/integration_tests/scale/schedulability.rs b/src/tests/simulation/tests/integration_tests/scale/schedulability.rs index edd0d02864b24..a8a6f73eedd3d 100644 --- a/src/tests/simulation/tests/integration_tests/scale/schedulability.rs +++ b/src/tests/simulation/tests/integration_tests/scale/schedulability.rs @@ -15,7 +15,7 @@ use std::collections::HashSet; use anyhow::Result; -use risingwave_common::hash::ParallelUnitId; +use risingwave_common::hash::WorkerSlotId; use risingwave_pb::common::{WorkerNode, WorkerType}; use risingwave_simulation::cluster::{Cluster, Configuration}; @@ -36,14 +36,10 @@ async fn test_cordon_normal() -> Result<()> { .collect(); let cordoned_worker = workers.pop().unwrap(); - - let rest_parallel_unit_ids: HashSet<_> = workers + let rest_worker_slots: HashSet<_> = workers .iter() .flat_map(|worker| { - worker - .parallel_units - .iter() - .map(|parallel_unit| parallel_unit.id as ParallelUnitId) + (0..worker.parallelism).map(|idx| WorkerSlotId::new(worker.id, idx as _)) }) .collect(); @@ -54,9 +50,9 @@ async fn test_cordon_normal() -> Result<()> { let fragments = cluster.locate_fragments([]).await?; for fragment in fragments { - let (_, used) = fragment.parallel_unit_usage(); + let used_worker_slots = fragment.used_worker_slots(); - assert_eq!(used, rest_parallel_unit_ids); + assert_eq!(used_worker_slots, rest_worker_slots); } session.run("drop table t;").await?; @@ -68,11 +64,9 @@ async fn test_cordon_normal() -> Result<()> { let fragments = cluster.locate_fragments([]).await?; for fragment in fragments { - let (all, used) = fragment.parallel_unit_usage(); - - let all: HashSet<_> = all.into_iter().collect(); - - assert_eq!(used, all); + let all_worker_slots = fragment.all_worker_slots(); + let used_worker_slots = fragment.used_worker_slots(); + assert_eq!(used_worker_slots, all_worker_slots); } Ok(()) diff --git a/src/tests/simulation/tests/integration_tests/scale/singleton_migration.rs b/src/tests/simulation/tests/integration_tests/scale/singleton_migration.rs index 05e05f157913c..8b37cb7731baf 100644 --- a/src/tests/simulation/tests/integration_tests/scale/singleton_migration.rs +++ b/src/tests/simulation/tests/integration_tests/scale/singleton_migration.rs @@ -43,27 +43,24 @@ async fn test_singleton_migration() -> Result<()> { ]) .await?; - let id = fragment.id(); + let mut all_worker_slots = fragment.all_worker_slots().into_iter().collect_vec(); + let used_worker_slots = fragment.used_worker_slots(); - let (mut all, used) = fragment.parallel_unit_usage(); + assert_eq!(used_worker_slots.len(), 1); - assert_eq!(used.len(), 1); + all_worker_slots.shuffle(&mut thread_rng()); - all.shuffle(&mut thread_rng()); - - let mut target_parallel_units = all + let mut target_worker_slots = all_worker_slots .into_iter() - .filter(|parallel_unit_id| !used.contains(parallel_unit_id)); + .filter(|work_slot| !used_worker_slots.contains(work_slot)); - let source_parallel_unit = used.iter().next().cloned().unwrap(); - let target_parallel_unit = target_parallel_units.next().unwrap(); + let source_slot = used_worker_slots.iter().exactly_one().cloned().unwrap(); + let target_slot = target_worker_slots.next().unwrap(); - assert_ne!(target_parallel_unit, source_parallel_unit); + assert_ne!(target_slot, source_slot); cluster - .reschedule(format!( - "{id}-[{source_parallel_unit}]+[{target_parallel_unit}]" - )) + .reschedule(fragment.reschedule([source_slot], [target_slot])) .await?; sleep(Duration::from_secs(3)).await; @@ -82,13 +79,11 @@ async fn test_singleton_migration() -> Result<()> { .await? .assert_result_eq("10"); - let source_parallel_unit = target_parallel_unit; - let target_parallel_unit = target_parallel_units.next().unwrap(); + let source_slot = target_slot; + let target_slot = target_worker_slots.next().unwrap(); cluster - .reschedule(format!( - "{id}-[{source_parallel_unit}]+[{target_parallel_unit}]" - )) + .reschedule(fragment.reschedule([source_slot], [target_slot])) .await?; sleep(Duration::from_secs(3)).await; diff --git a/src/tests/simulation/tests/integration_tests/scale/sink.rs b/src/tests/simulation/tests/integration_tests/scale/sink.rs index a7cfa2ea2de6e..16621064c2464 100644 --- a/src/tests/simulation/tests/integration_tests/scale/sink.rs +++ b/src/tests/simulation/tests/integration_tests/scale/sink.rs @@ -17,6 +17,7 @@ use std::time::Duration; use anyhow::Result; use futures::StreamExt; +use itertools::Itertools; use rand::prelude::SliceRandom; use rand::thread_rng; use rdkafka::consumer::{Consumer, StreamConsumer}; @@ -35,6 +36,7 @@ const DEBEZIUM_SINK_CREATE: &str = "create sink s2 from m with (connector='kafka const APPEND_ONLY_TOPIC: &str = "t_sink_append_only"; const DEBEZIUM_TOPIC: &str = "t_sink_debezium"; +use risingwave_common::hash::WorkerSlotId; use serde_derive::{Deserialize, Serialize}; #[derive(Default, Debug, Clone, PartialEq, Serialize, Deserialize)] @@ -114,11 +116,39 @@ async fn test_sink_append_only() -> Result<()> { ]) .await?; - let id = materialize_fragment.id(); + let workers = materialize_fragment + .all_worker_count() + .into_keys() + .collect_vec(); + check_kafka_after_insert(&mut cluster, &mut stream, &[1, 2, 3]).await?; - cluster.reschedule(format!("{id}-[1,2,3,4,5]")).await?; + cluster + .reschedule(materialize_fragment.reschedule( + [ + WorkerSlotId::new(workers[0], 1), + WorkerSlotId::new(workers[1], 0), + WorkerSlotId::new(workers[1], 1), + WorkerSlotId::new(workers[2], 0), + WorkerSlotId::new(workers[2], 1), + ], + [], + )) + .await?; + check_kafka_after_insert(&mut cluster, &mut stream, &[4, 5, 6]).await?; - cluster.reschedule(format!("{id}+[1,2,3,4,5]")).await?; + cluster + .reschedule(materialize_fragment.reschedule( + [], + [ + WorkerSlotId::new(workers[0], 1), + WorkerSlotId::new(workers[1], 0), + WorkerSlotId::new(workers[1], 1), + WorkerSlotId::new(workers[2], 0), + WorkerSlotId::new(workers[2], 1), + ], + )) + .await?; + check_kafka_after_insert(&mut cluster, &mut stream, &[7, 8, 9]).await?; Ok(()) @@ -168,36 +198,34 @@ async fn test_sink_debezium() -> Result<()> { ]) .await?; - let (mut all, used) = materialize_fragment.parallel_unit_usage(); + let mut all_worker_slots = materialize_fragment + .all_worker_slots() + .into_iter() + .collect_vec(); + let used_worker_slots = materialize_fragment.used_worker_slots(); - assert_eq!(used.len(), 1); + assert_eq!(used_worker_slots.len(), 1); - all.shuffle(&mut thread_rng()); + all_worker_slots.shuffle(&mut thread_rng()); - let mut target_parallel_units = all + let mut target_worker_slots = all_worker_slots .into_iter() - .filter(|parallel_unit_id| !used.contains(parallel_unit_id)); - - let id = materialize_fragment.id(); + .filter(|worker_slot| !used_worker_slots.contains(worker_slot)); check_kafka_after_insert(&mut cluster, &mut stream, &[1, 2, 3]).await?; - let source_parallel_unit = used.iter().next().cloned().unwrap(); - let target_parallel_unit = target_parallel_units.next().unwrap(); + let source_slot = used_worker_slots.iter().next().cloned().unwrap(); + let target_slot = target_worker_slots.next().unwrap(); cluster - .reschedule(format!( - "{id}-[{source_parallel_unit}]+[{target_parallel_unit}]" - )) + .reschedule(materialize_fragment.reschedule([source_slot], [target_slot])) .await?; check_kafka_after_insert(&mut cluster, &mut stream, &[4, 5, 6]).await?; - let source_parallel_unit = target_parallel_unit; - let target_parallel_unit = target_parallel_units.next().unwrap(); + let source_slot = target_slot; + let target_slot = target_worker_slots.next().unwrap(); cluster - .reschedule(format!( - "{id}-[{source_parallel_unit}]+[{target_parallel_unit}]" - )) + .reschedule(materialize_fragment.reschedule([source_slot], [target_slot])) .await?; check_kafka_after_insert(&mut cluster, &mut stream, &[7, 8, 9]).await?; diff --git a/src/tests/simulation/tests/integration_tests/scale/table.rs b/src/tests/simulation/tests/integration_tests/scale/table.rs index 6bceb5c07b536..85a3e64d64108 100644 --- a/src/tests/simulation/tests/integration_tests/scale/table.rs +++ b/src/tests/simulation/tests/integration_tests/scale/table.rs @@ -16,6 +16,7 @@ use std::iter::repeat_with; use anyhow::Result; use itertools::Itertools; +use risingwave_common::hash::WorkerSlotId; use risingwave_simulation::cluster::{Cluster, Configuration}; use risingwave_simulation::ctl_ext::predicate::identity_contains; @@ -47,13 +48,30 @@ async fn test_table() -> Result<()> { insert_and_flush!(cluster); + let workers = fragment.all_worker_count().into_keys().collect_vec(); + cluster - .reschedule(fragment.reschedule([0, 2, 4], [])) + .reschedule(fragment.reschedule( + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[1], 0), + WorkerSlotId::new(workers[2], 0), + ], + [], + )) .await?; insert_and_flush!(cluster); - cluster.reschedule(fragment.reschedule([1], [0, 4])).await?; + cluster + .reschedule(fragment.reschedule( + [WorkerSlotId::new(workers[0], 1)], + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[2], 0), + ], + )) + .await?; insert_and_flush!(cluster); @@ -69,13 +87,30 @@ async fn test_mv_on_scaled_table() -> Result<()> { .locate_one_fragment([identity_contains("materialize")]) .await?; + let workers = fragment.all_worker_count().into_keys().collect_vec(); + cluster - .reschedule(fragment.reschedule([0, 2, 4], [])) + .reschedule(fragment.reschedule( + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[1], 0), + WorkerSlotId::new(workers[2], 0), + ], + [], + )) .await?; insert_and_flush!(cluster); - cluster.reschedule(fragment.reschedule([1], [0, 4])).await?; + cluster + .reschedule(fragment.reschedule( + [WorkerSlotId::new(workers[0], 1)], + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[2], 0), + ], + )) + .await?; insert_and_flush!(cluster); @@ -97,8 +132,17 @@ async fn test_scale_on_schema_change() -> Result<()> { .locate_one_fragment([identity_contains("materialize"), identity_contains("union")]) .await?; + let workers = fragment.all_worker_count().into_keys().collect_vec(); + cluster - .reschedule(fragment.reschedule([0, 2, 4], [])) + .reschedule(fragment.reschedule( + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[1], 0), + WorkerSlotId::new(workers[2], 0), + ], + [], + )) .await?; insert_and_flush!(cluster); @@ -113,13 +157,19 @@ async fn test_scale_on_schema_change() -> Result<()> { .await?; cluster - .reschedule_resolve_no_shuffle(fragment.reschedule([1], [0, 4])) + .reschedule_resolve_no_shuffle(fragment.reschedule( + [WorkerSlotId::new(workers[0], 1)], + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[2], 0), + ], + )) .await?; let fragment = cluster .locate_one_fragment([identity_contains("materialize"), identity_contains("union")]) .await?; - let (_, used) = fragment.parallel_unit_usage(); + let used = fragment.used_worker_slots(); assert_eq!(used.len(), 4); insert_and_flush!(cluster); diff --git a/src/tests/simulation/tests/integration_tests/sink/scale.rs b/src/tests/simulation/tests/integration_tests/sink/scale.rs index 78cc6905eed67..b1c533b21cab3 100644 --- a/src/tests/simulation/tests/integration_tests/sink/scale.rs +++ b/src/tests/simulation/tests/integration_tests/sink/scale.rs @@ -19,6 +19,7 @@ use std::time::Duration; use anyhow::Result; use itertools::Itertools; use rand::{thread_rng, Rng}; +use risingwave_common::hash::WorkerSlotId; use risingwave_simulation::cluster::{Cluster, KillOpts}; use risingwave_simulation::ctl_ext::predicate::identity_contains; use tokio::time::sleep; @@ -79,19 +80,55 @@ async fn scale_test_inner(is_decouple: bool) -> Result<()> { .await?; assert_eq!(sink_fragments.len(), 1); - let framgment = sink_fragments.pop().unwrap(); - let id = framgment.id(); + let fragment = sink_fragments.pop().unwrap(); + let id = fragment.id(); let count = test_source.id_list.len(); + let workers = fragment.all_worker_count().into_keys().collect_vec(); scale_and_check( &mut cluster, &test_sink, count, vec![ - (format!("{id}-[1,2,3]"), 3), - (format!("{id}-[4,5]+[1,2]"), 3), - (format!("{id}+[3,4,5]"), 6), + // (format!("{id}-[1,2,3]"), 3), + ( + fragment.reschedule( + [ + WorkerSlotId::new(workers[0], 0), + WorkerSlotId::new(workers[1], 0), + WorkerSlotId::new(workers[1], 1), + ], + [], + ), + 3, + ), + // (format!("{id}-[4,5]+[1,2]"), 3) + ( + fragment.reschedule( + [ + WorkerSlotId::new(workers[2], 0), + WorkerSlotId::new(workers[2], 1), + ], + [ + WorkerSlotId::new(workers[0], 1), + WorkerSlotId::new(workers[1], 0), + ], + ), + 3, + ), + // (format!("{id}+[3,4,5]"), 6), + ( + fragment.reschedule( + [], + [ + WorkerSlotId::new(workers[1], 1), + WorkerSlotId::new(workers[2], 0), + WorkerSlotId::new(workers[2], 1), + ], + ), + 6, + ), ] .into_iter(), )