From 44afa1571e58928c35e5aba3f6ff954d33b34dce Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 11 Nov 2024 17:56:30 +0800 Subject: [PATCH 1/3] refactor(meta): reorganize code of global barrier manager (part 1) --- src/meta/src/barrier/checkpoint/control.rs | 820 +++++++ .../creating_job/barrier_control.rs | 0 .../{ => checkpoint}/creating_job/mod.rs | 14 +- .../{ => checkpoint}/creating_job/status.rs | 0 src/meta/src/barrier/checkpoint/mod.rs | 20 + .../src/barrier/{ => checkpoint}/state.rs | 2 +- src/meta/src/barrier/command.rs | 245 +-- src/meta/src/barrier/complete_task.rs | 245 +++ src/meta/src/barrier/context/context_impl.rs | 284 +++ src/meta/src/barrier/context/mod.rs | 83 + .../src/barrier/{ => context}/recovery.rs | 294 +-- src/meta/src/barrier/manager.rs | 136 ++ src/meta/src/barrier/mod.rs | 1943 +---------------- src/meta/src/barrier/progress.rs | 42 +- src/meta/src/barrier/rpc.rs | 22 +- src/meta/src/barrier/schedule.rs | 16 +- src/meta/src/barrier/utils.rs | 191 ++ src/meta/src/barrier/worker.rs | 760 +++++++ 18 files changed, 2656 insertions(+), 2461 deletions(-) create mode 100644 src/meta/src/barrier/checkpoint/control.rs rename src/meta/src/barrier/{ => checkpoint}/creating_job/barrier_control.rs (100%) rename src/meta/src/barrier/{ => checkpoint}/creating_job/mod.rs (96%) rename src/meta/src/barrier/{ => checkpoint}/creating_job/status.rs (100%) create mode 100644 src/meta/src/barrier/checkpoint/mod.rs rename src/meta/src/barrier/{ => checkpoint}/state.rs (99%) create mode 100644 src/meta/src/barrier/complete_task.rs create mode 100644 src/meta/src/barrier/context/context_impl.rs create mode 100644 src/meta/src/barrier/context/mod.rs rename src/meta/src/barrier/{ => context}/recovery.rs (66%) create mode 100644 src/meta/src/barrier/manager.rs create mode 100644 src/meta/src/barrier/utils.rs create mode 100644 src/meta/src/barrier/worker.rs diff --git a/src/meta/src/barrier/checkpoint/control.rs b/src/meta/src/barrier/checkpoint/control.rs new file mode 100644 index 0000000000000..a064848dc2675 --- /dev/null +++ b/src/meta/src/barrier/checkpoint/control.rs @@ -0,0 +1,820 @@ +// 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::hash_map::Entry; +use std::collections::{BTreeMap, HashMap, HashSet}; +use std::mem::take; + +use anyhow::anyhow; +use fail::fail_point; +use prometheus::HistogramTimer; +use risingwave_common::catalog::{DatabaseId, TableId}; +use risingwave_meta_model::WorkerId; +use risingwave_pb::hummock::HummockVersionStats; +use risingwave_pb::meta::PausedReason; +use risingwave_pb::stream_service::BarrierCompleteResponse; +use tracing::{debug, warn}; + +use crate::barrier::checkpoint::creating_job::{CompleteJobType, CreatingStreamingJobControl}; +use crate::barrier::checkpoint::state::BarrierWorkerState; +use crate::barrier::command::CommandContext; +use crate::barrier::complete_task::{BarrierCompleteOutput, CompleteBarrierTask}; +use crate::barrier::info::{BarrierInfo, InflightStreamingJobInfo}; +use crate::barrier::notifier::Notifier; +use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob}; +use crate::barrier::rpc::{from_partial_graph_id, ControlStreamManager}; +use crate::barrier::schedule::{NewBarrier, PeriodicBarriers}; +use crate::barrier::utils::{collect_commit_epoch_info, collect_creating_job_commit_epoch_info}; +use crate::barrier::{ + BarrierKind, Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, + SnapshotBackfillInfo, TracedEpoch, +}; +use crate::manager::ActiveStreamingWorkerNodes; +use crate::rpc::metrics::GLOBAL_META_METRICS; +use crate::MetaResult; + +#[derive(Default)] +pub(crate) struct CheckpointControl { + pub(crate) databases: HashMap, + pub(crate) hummock_version_stats: HummockVersionStats, +} + +impl CheckpointControl { + pub(crate) fn ack_completed(&mut self, output: BarrierCompleteOutput) { + self.hummock_version_stats = output.hummock_version_stats; + for (database_id, (command_prev_epoch, creating_job_epochs)) in output.epochs_to_ack { + self.databases + .get_mut(&database_id) + .expect("should exist") + .ack_completed(command_prev_epoch, creating_job_epochs); + } + } + + pub(crate) fn next_complete_barrier_task( + &mut self, + mut context: Option<(&mut PeriodicBarriers, &mut ControlStreamManager)>, + ) -> Option { + let mut task = None; + for database in self.databases.values_mut() { + let context = context.as_mut().map(|(s, c)| (&mut **s, &mut **c)); + database.next_complete_barrier_task(&mut task, context, &self.hummock_version_stats); + } + task + } + + pub(crate) fn barrier_collected( + &mut self, + resp: BarrierCompleteResponse, + control_stream_manager: &mut ControlStreamManager, + ) -> MetaResult<()> { + let database_id = from_partial_graph_id(resp.partial_graph_id).0; + self.databases + .get_mut(&database_id) + .expect("should exist") + .barrier_collected(resp, control_stream_manager) + } + + pub(crate) fn can_inject_barrier(&self, in_flight_barrier_nums: usize) -> bool { + self.databases + .values() + .all(|database| database.can_inject_barrier(in_flight_barrier_nums)) + } + + pub(crate) fn max_prev_epoch(&self) -> Option { + self.databases + .values() + .map(|database| database.state.in_flight_prev_epoch()) + .max_by_key(|epoch| epoch.value()) + .cloned() + } + + pub(crate) fn handle_new_barrier( + &mut self, + new_barrier: NewBarrier, + control_stream_manager: &mut ControlStreamManager, + active_streaming_nodes: &ActiveStreamingWorkerNodes, + ) -> MetaResult<()> { + let NewBarrier { + command, + span, + checkpoint, + } = new_barrier; + + if let Some((database_id, command, notifiers)) = command { + let max_prev_epoch = self.max_prev_epoch(); + let (database, max_prev_epoch) = match self.databases.entry(database_id) { + Entry::Occupied(entry) => ( + entry.into_mut(), + max_prev_epoch.expect("should exist when having some database"), + ), + Entry::Vacant(entry) => match &command { + Command::CreateStreamingJob { + job_type: CreateStreamingJobType::Normal, + .. + } => { + let new_database = DatabaseCheckpointControl::new(database_id); + let max_prev_epoch = if let Some(max_prev_epoch) = max_prev_epoch { + if max_prev_epoch.value() + < new_database.state.in_flight_prev_epoch().value() + { + new_database.state.in_flight_prev_epoch().clone() + } else { + max_prev_epoch + } + } else { + new_database.state.in_flight_prev_epoch().clone() + }; + (entry.insert(new_database), max_prev_epoch) + } + Command::Flush + | Command::Pause(PausedReason::Manual) + | Command::Resume(PausedReason::Manual) => { + for mut notifier in notifiers { + notifier.notify_started(); + notifier.notify_collected(); + } + warn!(?command, "skip command for empty database"); + return Ok(()); + } + _ => { + panic!("new database graph info can only be created for normal creating streaming job, but get command: {} {:?}", database_id, command) + } + }, + }; + + let curr_epoch = max_prev_epoch.next(); + + database.handle_new_barrier( + Some((command, notifiers)), + checkpoint, + span.clone(), + control_stream_manager, + active_streaming_nodes, + &self.hummock_version_stats, + curr_epoch.clone(), + )?; + for database in self.databases.values_mut() { + if database.database_id == database_id { + continue; + } + database.handle_new_barrier( + None, + checkpoint, + span.clone(), + control_stream_manager, + active_streaming_nodes, + &self.hummock_version_stats, + curr_epoch.clone(), + )?; + } + } else { + let Some(max_prev_epoch) = self.max_prev_epoch() else { + assert!(self.databases.is_empty()); + return Ok(()); + }; + let curr_epoch = max_prev_epoch.next(); + for database in self.databases.values_mut() { + database.handle_new_barrier( + None, + checkpoint, + span.clone(), + control_stream_manager, + active_streaming_nodes, + &self.hummock_version_stats, + curr_epoch.clone(), + )?; + } + } + Ok(()) + } + + pub(crate) fn update_barrier_nums_metrics(&self) { + self.databases + .values() + .for_each(|database| database.update_barrier_nums_metrics()); + } +} + +/// Controls the concurrent execution of commands. +pub(crate) struct DatabaseCheckpointControl { + database_id: DatabaseId, + pub(crate) state: BarrierWorkerState, + + /// Save the state and message of barrier in order. + /// Key is the `prev_epoch`. + pub(crate) command_ctx_queue: BTreeMap, + /// The barrier that are completing. + /// Some((`prev_epoch`, `should_pause_inject_barrier`)) + completing_barrier: Option<(u64, bool)>, + + pub(crate) creating_streaming_job_controls: HashMap, + + pub(crate) create_mview_tracker: CreateMviewProgressTracker, +} + +impl DatabaseCheckpointControl { + fn new(database_id: DatabaseId) -> Self { + Self { + database_id, + state: BarrierWorkerState::new(), + command_ctx_queue: Default::default(), + completing_barrier: None, + creating_streaming_job_controls: Default::default(), + create_mview_tracker: Default::default(), + } + } + + pub(crate) fn recovery( + database_id: DatabaseId, + create_mview_tracker: CreateMviewProgressTracker, + state: BarrierWorkerState, + ) -> Self { + Self { + database_id, + state, + command_ctx_queue: Default::default(), + completing_barrier: None, + creating_streaming_job_controls: Default::default(), + create_mview_tracker, + } + } + + fn total_command_num(&self) -> usize { + self.command_ctx_queue.len() + + match &self.completing_barrier { + Some(_) => 1, + None => 0, + } + } + + /// Update the metrics of barrier nums. + fn update_barrier_nums_metrics(&self) { + let database_id_str = self.database_id.database_id.to_string(); + GLOBAL_META_METRICS + .in_flight_barrier_nums + .with_label_values(&[&database_id_str]) + .set( + self.command_ctx_queue + .values() + .filter(|x| x.state.is_inflight()) + .count() as i64, + ); + GLOBAL_META_METRICS + .all_barrier_nums + .with_label_values(&[&database_id_str]) + .set(self.total_command_num() as i64); + } + + fn jobs_to_merge( + &self, + ) -> Option> { + let mut table_ids_to_merge = HashMap::new(); + + for (table_id, creating_streaming_job) in &self.creating_streaming_job_controls { + if let Some(graph_info) = creating_streaming_job.should_merge_to_upstream() { + table_ids_to_merge.insert( + *table_id, + ( + creating_streaming_job.snapshot_backfill_info.clone(), + graph_info, + ), + ); + } + } + if table_ids_to_merge.is_empty() { + None + } else { + Some(table_ids_to_merge) + } + } + + /// Enqueue a barrier command + fn enqueue_command( + &mut self, + command_ctx: CommandContext, + notifiers: Vec, + node_to_collect: HashSet, + creating_jobs_to_wait: HashSet, + ) { + let timer = GLOBAL_META_METRICS.barrier_latency.start_timer(); + + if let Some((_, node)) = self.command_ctx_queue.last_key_value() { + assert_eq!( + command_ctx.barrier_info.prev_epoch.value(), + node.command_ctx.barrier_info.curr_epoch.value() + ); + } + + tracing::trace!( + prev_epoch = command_ctx.barrier_info.prev_epoch(), + ?creating_jobs_to_wait, + "enqueue command" + ); + self.command_ctx_queue.insert( + command_ctx.barrier_info.prev_epoch(), + EpochNode { + enqueue_time: timer, + state: BarrierEpochState { + node_to_collect, + resps: vec![], + creating_jobs_to_wait, + finished_jobs: HashMap::new(), + }, + command_ctx, + notifiers, + }, + ); + } + + /// Change the state of this `prev_epoch` to `Completed`. Return continuous nodes + /// with `Completed` starting from first node [`Completed`..`InFlight`) and remove them. + fn barrier_collected( + &mut self, + resp: BarrierCompleteResponse, + control_stream_manager: &mut ControlStreamManager, + ) -> MetaResult<()> { + let worker_id = resp.worker_id; + let prev_epoch = resp.epoch; + tracing::trace!( + worker_id, + prev_epoch, + partial_graph_id = resp.partial_graph_id, + "barrier collected" + ); + let (database_id, creating_job_id) = from_partial_graph_id(resp.partial_graph_id); + assert_eq!(database_id, self.database_id); + match creating_job_id { + None => { + if let Some(node) = self.command_ctx_queue.get_mut(&prev_epoch) { + assert!(node.state.node_to_collect.remove(&(worker_id as _))); + node.state.resps.push(resp); + } else { + panic!( + "collect barrier on non-existing barrier: {}, {}", + prev_epoch, worker_id + ); + } + } + Some(creating_job_id) => { + self.creating_streaming_job_controls + .get_mut(&creating_job_id) + .expect("should exist") + .collect(prev_epoch, worker_id as _, resp, control_stream_manager)?; + } + } + Ok(()) + } + + /// Pause inject barrier until True. + fn can_inject_barrier(&self, in_flight_barrier_nums: usize) -> bool { + let in_flight_not_full = self + .command_ctx_queue + .values() + .filter(|x| x.state.is_inflight()) + .count() + < in_flight_barrier_nums; + + // Whether some command requires pausing concurrent barrier. If so, it must be the last one. + let should_pause = self + .command_ctx_queue + .last_key_value() + .and_then(|(_, x)| { + x.command_ctx + .command + .as_ref() + .map(Command::should_pause_inject_barrier) + }) + .or(self + .completing_barrier + .map(|(_, should_pause)| should_pause)) + .unwrap_or(false); + debug_assert_eq!( + self.command_ctx_queue + .values() + .filter_map(|node| { + node.command_ctx + .command + .as_ref() + .map(Command::should_pause_inject_barrier) + }) + .chain( + self.completing_barrier + .map(|(_, should_pause)| should_pause) + .into_iter() + ) + .any(|should_pause| should_pause), + should_pause + ); + + in_flight_not_full && !should_pause + } + + /// Return the earliest command waiting on the `worker_id`. + pub(crate) fn barrier_wait_collect_from_worker( + &self, + worker_id: WorkerId, + ) -> Option<&BarrierInfo> { + for epoch_node in self.command_ctx_queue.values() { + if epoch_node.state.node_to_collect.contains(&worker_id) { + return Some(&epoch_node.command_ctx.barrier_info); + } + } + // TODO: include barrier in creating jobs + None + } +} + +impl DatabaseCheckpointControl { + /// return creating job table fragment id -> (backfill progress epoch , {`upstream_mv_table_id`}) + fn collect_backfill_pinned_upstream_log_epoch( + &self, + ) -> HashMap)> { + self.creating_streaming_job_controls + .iter() + .filter_map(|(table_id, creating_job)| { + creating_job + .pinned_upstream_log_epoch() + .map(|progress_epoch| { + ( + *table_id, + ( + progress_epoch, + creating_job + .snapshot_backfill_info + .upstream_mv_table_ids + .clone(), + ), + ) + }) + }) + .collect() + } + + fn next_complete_barrier_task( + &mut self, + task: &mut Option, + mut context: Option<(&mut PeriodicBarriers, &mut ControlStreamManager)>, + hummock_version_stats: &HummockVersionStats, + ) { + // `Vec::new` is a const fn, and do not have memory allocation, and therefore is lightweight enough + let mut creating_jobs_task = vec![]; + { + // `Vec::new` is a const fn, and do not have memory allocation, and therefore is lightweight enough + let mut finished_jobs = Vec::new(); + let min_upstream_inflight_barrier = self + .command_ctx_queue + .first_key_value() + .map(|(epoch, _)| *epoch); + for (table_id, job) in &mut self.creating_streaming_job_controls { + if let Some((epoch, resps, status)) = + job.start_completing(min_upstream_inflight_barrier) + { + let is_first_time = match status { + CompleteJobType::First => true, + CompleteJobType::Normal => false, + CompleteJobType::Finished => { + finished_jobs.push((*table_id, epoch, resps)); + continue; + } + }; + creating_jobs_task.push((*table_id, epoch, resps, is_first_time)); + } + } + if !finished_jobs.is_empty() + && let Some((_, control_stream_manager)) = &mut context + { + control_stream_manager.remove_partial_graph( + self.database_id, + finished_jobs + .iter() + .map(|(table_id, _, _)| *table_id) + .collect(), + ); + } + for (table_id, epoch, resps) in finished_jobs { + let epoch_state = &mut self + .command_ctx_queue + .get_mut(&epoch) + .expect("should exist") + .state; + assert!(epoch_state.creating_jobs_to_wait.remove(&table_id)); + debug!(epoch, ?table_id, "finish creating job"); + // It's safe to remove the creating job, because on CompleteJobType::Finished, + // all previous barriers have been collected and completed. + let creating_streaming_job = self + .creating_streaming_job_controls + .remove(&table_id) + .expect("should exist"); + assert!(creating_streaming_job.is_finished()); + assert!(epoch_state + .finished_jobs + .insert(table_id, (creating_streaming_job.info, resps)) + .is_none()); + } + } + assert!(self.completing_barrier.is_none()); + while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() + && !state.is_inflight() + { + { + let (_, mut node) = self.command_ctx_queue.pop_first().expect("non-empty"); + assert!(node.state.creating_jobs_to_wait.is_empty()); + assert!(node.state.node_to_collect.is_empty()); + let mut finished_jobs = self + .create_mview_tracker + .apply_collected_command(&node, hummock_version_stats); + if !node.command_ctx.barrier_info.kind.is_checkpoint() { + assert!(finished_jobs.is_empty()); + node.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + if let Some((scheduled_barriers, _)) = &mut context + && self.create_mview_tracker.has_pending_finished_jobs() + && self + .command_ctx_queue + .values() + .all(|node| !node.command_ctx.barrier_info.kind.is_checkpoint()) + { + scheduled_barriers.force_checkpoint_in_next_barrier(); + } + continue; + } + node.state + .finished_jobs + .drain() + .for_each(|(_, (info, resps))| { + node.state.resps.extend(resps); + finished_jobs.push(TrackingJob::New(TrackingCommand { + info, + replace_table_info: None, + })); + }); + let task = task.get_or_insert_default(); + collect_commit_epoch_info( + &mut task.commit_info, + take(&mut node.state.resps), + &node.command_ctx, + self.collect_backfill_pinned_upstream_log_epoch(), + ); + self.completing_barrier = Some(( + node.command_ctx.barrier_info.prev_epoch(), + node.command_ctx + .command + .as_ref() + .map(|c| c.should_pause_inject_barrier()) + .unwrap_or(false), + )); + task.finished_jobs.extend(finished_jobs); + task.notifiers.extend(node.notifiers); + task.epoch_infos + .try_insert( + self.database_id, + (Some((node.command_ctx, node.enqueue_time)), vec![]), + ) + .expect("non duplicate"); + break; + } + } + if !creating_jobs_task.is_empty() { + let task = task.get_or_insert_default(); + for (table_id, epoch, resps, is_first_time) in creating_jobs_task { + collect_creating_job_commit_epoch_info( + &mut task.commit_info, + epoch, + resps, + self.creating_streaming_job_controls[&table_id] + .info + .table_fragments + .all_table_ids() + .map(TableId::new), + is_first_time, + ); + let (_, creating_job_epochs) = + task.epoch_infos.entry(self.database_id).or_default(); + creating_job_epochs.push((table_id, epoch)); + } + } + } + + fn ack_completed( + &mut self, + command_prev_epoch: Option, + creating_job_epochs: Vec<(TableId, u64)>, + ) { + { + assert_eq!( + self.completing_barrier + .take() + .map(|(prev_epoch, _)| prev_epoch), + command_prev_epoch + ); + for (table_id, epoch) in creating_job_epochs { + self.creating_streaming_job_controls + .get_mut(&table_id) + .expect("should exist") + .ack_completed(epoch) + } + } + } +} + +/// The state and message of this barrier, a node for concurrent checkpoint. +pub(crate) struct EpochNode { + /// Timer for recording barrier latency, taken after `complete_barriers`. + pub(crate) enqueue_time: HistogramTimer, + + /// Whether this barrier is in-flight or completed. + pub(crate) state: BarrierEpochState, + + /// Context of this command to generate barrier and do some post jobs. + pub(crate) command_ctx: CommandContext, + /// Notifiers of this barrier. + pub(crate) notifiers: Vec, +} + +#[derive(Debug)] +/// The state of barrier. +pub(crate) struct BarrierEpochState { + pub(crate) node_to_collect: HashSet, + + pub(crate) resps: Vec, + + creating_jobs_to_wait: HashSet, + + finished_jobs: HashMap)>, +} + +impl BarrierEpochState { + fn is_inflight(&self) -> bool { + !self.node_to_collect.is_empty() || !self.creating_jobs_to_wait.is_empty() + } +} + +impl DatabaseCheckpointControl { + /// Handle the new barrier from the scheduled queue and inject it. + fn handle_new_barrier( + &mut self, + command: Option<(Command, Vec)>, + checkpoint: bool, + span: tracing::Span, + control_stream_manager: &mut ControlStreamManager, + active_streaming_nodes: &ActiveStreamingWorkerNodes, + hummock_version_stats: &HummockVersionStats, + curr_epoch: TracedEpoch, + ) -> MetaResult<()> { + let (mut command, mut notifiers) = if let Some((command, notifiers)) = command { + (Some(command), notifiers) + } else { + (None, vec![]) + }; + + if let Some(table_to_cancel) = command.as_ref().and_then(Command::table_to_cancel) + && self + .creating_streaming_job_controls + .contains_key(&table_to_cancel) + { + warn!( + table_id = table_to_cancel.table_id, + "ignore cancel command on creating streaming job" + ); + for notifier in notifiers { + notifier + .notify_start_failed(anyhow!("cannot cancel creating streaming job, the job will continue creating until created or recovery").into()); + } + return Ok(()); + } + + if let Some(Command::RescheduleFragment { .. }) = &command { + if !self.creating_streaming_job_controls.is_empty() { + warn!("ignore reschedule when creating streaming job with snapshot backfill"); + for notifier in notifiers { + notifier.notify_start_failed( + anyhow!( + "cannot reschedule when creating streaming job with snapshot backfill", + ) + .into(), + ); + } + return Ok(()); + } + } + + let Some(barrier_info) = + self.state + .next_barrier_info(command.as_ref(), checkpoint, curr_epoch) + else { + // skip the command when there is nothing to do with the barrier + for mut notifier in notifiers { + notifier.notify_started(); + notifier.notify_collected(); + } + return Ok(()); + }; + + // Insert newly added creating job + if let Some(Command::CreateStreamingJob { + job_type: CreateStreamingJobType::SnapshotBackfill(snapshot_backfill_info), + info, + }) = &command + { + if self.state.paused_reason().is_some() { + warn!("cannot create streaming job with snapshot backfill when paused"); + for notifier in notifiers { + notifier.notify_start_failed( + anyhow!("cannot create streaming job with snapshot backfill when paused",) + .into(), + ); + } + return Ok(()); + } + let mutation = command + .as_ref() + .expect("checked Some") + .to_mutation(None) + .expect("should have some mutation in `CreateStreamingJob` command"); + self.creating_streaming_job_controls.insert( + info.table_fragments.table_id(), + CreatingStreamingJobControl::new( + info.clone(), + snapshot_backfill_info.clone(), + barrier_info.prev_epoch(), + hummock_version_stats, + mutation, + ), + ); + } + + // Collect the jobs to finish + if let (BarrierKind::Checkpoint(_), None) = (&barrier_info.kind, &command) + && let Some(jobs_to_merge) = self.jobs_to_merge() + { + command = Some(Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge)); + } + + let command = command; + + let ( + pre_applied_graph_info, + pre_applied_subscription_info, + table_ids_to_commit, + jobs_to_wait, + prev_paused_reason, + ) = self.state.apply_command(command.as_ref()); + + // Tracing related stuff + barrier_info.prev_epoch.span().in_scope(|| { + tracing::info!(target: "rw_tracing", epoch = barrier_info.curr_epoch.value().0, "new barrier enqueued"); + }); + span.record("epoch", barrier_info.curr_epoch.value().0); + + for creating_job in &mut self.creating_streaming_job_controls.values_mut() { + creating_job.on_new_command(control_stream_manager, command.as_ref(), &barrier_info)?; + } + + let node_to_collect = match control_stream_manager.inject_command_ctx_barrier( + self.database_id, + command.as_ref(), + &barrier_info, + prev_paused_reason, + &pre_applied_graph_info, + &self.state.inflight_graph_info, + ) { + Ok(node_to_collect) => node_to_collect, + Err(err) => { + for notifier in notifiers { + notifier.notify_failed(err.clone()); + } + fail_point!("inject_barrier_err_success"); + return Err(err); + } + }; + + // Notify about the injection. + notifiers.iter_mut().for_each(|n| n.notify_started()); + + let command_ctx = CommandContext::new( + active_streaming_nodes.current().clone(), + barrier_info, + pre_applied_subscription_info, + table_ids_to_commit.clone(), + command, + span, + ); + + // Record the in-flight barrier. + self.enqueue_command(command_ctx, notifiers, node_to_collect, jobs_to_wait); + + Ok(()) + } +} diff --git a/src/meta/src/barrier/creating_job/barrier_control.rs b/src/meta/src/barrier/checkpoint/creating_job/barrier_control.rs similarity index 100% rename from src/meta/src/barrier/creating_job/barrier_control.rs rename to src/meta/src/barrier/checkpoint/creating_job/barrier_control.rs diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/checkpoint/creating_job/mod.rs similarity index 96% rename from src/meta/src/barrier/creating_job/mod.rs rename to src/meta/src/barrier/checkpoint/creating_job/mod.rs index 09d46ae1aff1a..db50b9d8335f4 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/checkpoint/creating_job/mod.rs @@ -19,6 +19,7 @@ use std::cmp::max; use std::collections::HashMap; use std::ops::Bound::{Excluded, Unbounded}; +use barrier_control::CreatingStreamingJobBarrierControl; use risingwave_common::catalog::{DatabaseId, TableId}; use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_meta_model::WorkerId; @@ -26,12 +27,9 @@ use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_service::BarrierCompleteResponse; +use status::{CreatingJobInjectBarrierInfo, CreatingStreamingJobStatus}; use tracing::info; -use crate::barrier::creating_job::barrier_control::CreatingStreamingJobBarrierControl; -use crate::barrier::creating_job::status::{ - CreatingJobInjectBarrierInfo, CreatingStreamingJobStatus, -}; use crate::barrier::info::{BarrierInfo, InflightStreamingJobInfo}; use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::rpc::ControlStreamManager; @@ -41,8 +39,8 @@ use crate::rpc::metrics::GLOBAL_META_METRICS; use crate::MetaResult; #[derive(Debug)] -pub(super) struct CreatingStreamingJobControl { - pub(super) info: CreateStreamingJobCommandInfo, +pub(crate) struct CreatingStreamingJobControl { + pub(crate) info: CreateStreamingJobCommandInfo, pub(super) snapshot_backfill_info: SnapshotBackfillInfo, backfill_epoch: u64, @@ -103,7 +101,7 @@ impl CreatingStreamingJobControl { } } - pub(super) fn is_wait_on_worker(&self, worker_id: WorkerId) -> bool { + pub(crate) fn is_wait_on_worker(&self, worker_id: WorkerId) -> bool { self.barrier_control.is_wait_on_worker(worker_id) || (self.status.is_finishing() && InflightFragmentInfo::contains_worker( @@ -112,7 +110,7 @@ impl CreatingStreamingJobControl { )) } - pub(super) fn gen_ddl_progress(&self) -> DdlProgress { + pub(crate) fn gen_ddl_progress(&self) -> DdlProgress { let progress = match &self.status { CreatingStreamingJobStatus::ConsumingSnapshot { create_mview_tracker, diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/checkpoint/creating_job/status.rs similarity index 100% rename from src/meta/src/barrier/creating_job/status.rs rename to src/meta/src/barrier/checkpoint/creating_job/status.rs diff --git a/src/meta/src/barrier/checkpoint/mod.rs b/src/meta/src/barrier/checkpoint/mod.rs new file mode 100644 index 0000000000000..a5144913258c2 --- /dev/null +++ b/src/meta/src/barrier/checkpoint/mod.rs @@ -0,0 +1,20 @@ +// 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. + +mod control; +mod creating_job; +mod state; + +pub(super) use control::{CheckpointControl, DatabaseCheckpointControl, EpochNode}; +pub(super) use state::BarrierWorkerState; diff --git a/src/meta/src/barrier/state.rs b/src/meta/src/barrier/checkpoint/state.rs similarity index 99% rename from src/meta/src/barrier/state.rs rename to src/meta/src/barrier/checkpoint/state.rs index dda00291878c4..871ff7e7faf60 100644 --- a/src/meta/src/barrier/state.rs +++ b/src/meta/src/barrier/checkpoint/state.rs @@ -24,7 +24,7 @@ use crate::barrier::{BarrierKind, Command, CreateStreamingJobType, TracedEpoch}; use crate::controller::fragment::InflightFragmentInfo; /// The latest state of `GlobalBarrierWorker` after injecting the latest barrier. -pub(super) struct BarrierWorkerState { +pub(crate) struct BarrierWorkerState { /// The last sent `prev_epoch` /// /// There's no need to persist this field. On recovery, we will restore this from the latest diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 6c326f5ccb65a..4f78baf26a593 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -15,7 +15,6 @@ use std::collections::{HashMap, HashSet}; use std::fmt::Formatter; -use futures::future::try_join_all; use risingwave_common::bitmap::Bitmap; use risingwave_common::catalog::TableId; use risingwave_common::hash::ActorMapping; @@ -37,17 +36,15 @@ use risingwave_pb::stream_plan::{ DropSubscriptionsMutation, PauseMutation, ResumeMutation, SourceChangeSplitMutation, StopMutation, StreamActor, SubscriptionUpstreamInfo, ThrottleMutation, UpdateMutation, }; -use risingwave_pb::stream_service::WaitEpochCommitRequest; use tracing::warn; use super::info::{CommandFragmentChanges, InflightStreamingJobInfo}; use crate::barrier::info::BarrierInfo; -use crate::barrier::{GlobalBarrierWorkerContextImpl, InflightSubscriptionInfo}; +use crate::barrier::InflightSubscriptionInfo; use crate::controller::fragment::InflightFragmentInfo; use crate::manager::{DdlType, StreamingJob}; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism}; use crate::stream::{build_actor_connector_splits, SplitAssignment, ThrottleConfig}; -use crate::MetaResult; /// [`Reschedule`] is for the [`Command::RescheduleFragment`], which is used for rescheduling actors /// in some fragment, like scaling or migrating. @@ -479,6 +476,21 @@ impl CommandContext { _span: span, } } + + pub fn get_truncate_epoch(&self, retention_second: u64) -> Epoch { + let Some(truncate_timestamptz) = Timestamptz::from_secs( + self.barrier_info + .prev_epoch + .value() + .as_timestamptz() + .timestamp() + - retention_second as i64, + ) else { + warn!(retention_second, prev_epoch = ?self.barrier_info.prev_epoch.value(), "invalid retention second value"); + return self.barrier_info.prev_epoch.value(); + }; + Epoch::from_unix_millis(truncate_timestamptz.timestamp_millis() as u64) + } } impl Command { @@ -922,228 +934,3 @@ impl Command { } } } - -impl CommandContext { - pub async fn wait_epoch_commit( - &self, - barrier_manager_context: &GlobalBarrierWorkerContextImpl, - ) -> MetaResult<()> { - let table_id = self.table_ids_to_commit.iter().next().cloned(); - // try wait epoch on an existing random table id - let Some(table_id) = table_id else { - // no need to wait epoch when there is no table id - return Ok(()); - }; - let futures = self.node_map.values().map(|worker_node| async { - let client = barrier_manager_context - .env - .stream_client_pool() - .get(worker_node) - .await?; - let request = WaitEpochCommitRequest { - epoch: self.barrier_info.prev_epoch(), - table_id: table_id.table_id, - }; - client.wait_epoch_commit(request).await - }); - - try_join_all(futures).await?; - - Ok(()) - } - - /// Do some stuffs after barriers are collected and the new storage version is committed, for - /// the given command. - pub async fn post_collect( - &self, - barrier_manager_context: &GlobalBarrierWorkerContextImpl, - ) -> MetaResult<()> { - let Some(command) = &self.command else { - return Ok(()); - }; - match command { - Command::Flush => {} - - Command::Throttle(_) => {} - - Command::Pause(reason) => { - if let PausedReason::ConfigChange = reason { - // After the `Pause` barrier is collected and committed, we must ensure that the - // storage version with this epoch is synced to all compute nodes before the - // execution of the next command of `Update`, as some newly created operators - // may immediately initialize their states on that barrier. - self.wait_epoch_commit(barrier_manager_context).await?; - } - } - - Command::Resume(_) => {} - - Command::SourceSplitAssignment(split_assignment) => { - barrier_manager_context - .metadata_manager - .update_actor_splits_by_split_assignment(split_assignment) - .await?; - barrier_manager_context - .source_manager - .apply_source_change(None, None, Some(split_assignment.clone()), None) - .await; - } - - Command::DropStreamingJobs { - unregistered_state_table_ids, - .. - } => { - barrier_manager_context - .hummock_manager - .unregister_table_ids(unregistered_state_table_ids.iter().cloned()) - .await?; - } - - Command::CancelStreamingJob(table_fragments) => { - tracing::debug!(id = ?table_fragments.table_id(), "cancelling stream job"); - - // NOTE(kwannoel): At this point, meta has already registered the table ids. - // We should unregister them. - // This is required for background ddl, for foreground ddl this is a no-op. - // Foreground ddl is handled entirely by stream manager, so it will unregister - // the table ids on failure. - // On the other hand background ddl could be handled by barrier manager. - // It won't clean the tables on failure, - // since the failure could be recoverable. - // As such it needs to be handled here. - barrier_manager_context - .hummock_manager - .unregister_table_ids(table_fragments.all_table_ids().map(TableId::new)) - .await?; - } - - Command::CreateStreamingJob { info, job_type } => { - let CreateStreamingJobCommandInfo { - table_fragments, - dispatchers, - init_split_assignment, - .. - } = info; - barrier_manager_context - .metadata_manager - .catalog_controller - .post_collect_table_fragments( - table_fragments.table_id().table_id as _, - table_fragments.actor_ids(), - dispatchers.clone(), - init_split_assignment, - ) - .await?; - - if let CreateStreamingJobType::SinkIntoTable(ReplaceTablePlan { - new_table_fragments, - dispatchers, - init_split_assignment, - .. - }) = job_type - { - barrier_manager_context - .metadata_manager - .catalog_controller - .post_collect_table_fragments( - new_table_fragments.table_id().table_id as _, - new_table_fragments.actor_ids(), - dispatchers.clone(), - init_split_assignment, - ) - .await?; - } - - // Extract the fragments that include source operators. - let source_fragments = table_fragments.stream_source_fragments(); - let backfill_fragments = table_fragments.source_backfill_fragments()?; - barrier_manager_context - .source_manager - .apply_source_change( - Some(source_fragments), - Some(backfill_fragments), - Some(init_split_assignment.clone()), - None, - ) - .await; - } - Command::RescheduleFragment { - reschedules, - table_parallelism, - .. - } => { - barrier_manager_context - .scale_controller - .post_apply_reschedule(reschedules, table_parallelism) - .await?; - } - - Command::ReplaceTable(ReplaceTablePlan { - old_table_fragments, - new_table_fragments, - dispatchers, - init_split_assignment, - .. - }) => { - // Update actors and actor_dispatchers for new table fragments. - barrier_manager_context - .metadata_manager - .catalog_controller - .post_collect_table_fragments( - new_table_fragments.table_id().table_id as _, - new_table_fragments.actor_ids(), - dispatchers.clone(), - init_split_assignment, - ) - .await?; - - // Apply the split changes in source manager. - barrier_manager_context - .source_manager - .drop_source_fragments_vec(std::slice::from_ref(old_table_fragments)) - .await; - let source_fragments = new_table_fragments.stream_source_fragments(); - // XXX: is it possible to have backfill fragments here? - let backfill_fragments = new_table_fragments.source_backfill_fragments()?; - barrier_manager_context - .source_manager - .apply_source_change( - Some(source_fragments), - Some(backfill_fragments), - Some(init_split_assignment.clone()), - None, - ) - .await; - } - - Command::CreateSubscription { - subscription_id, .. - } => { - barrier_manager_context - .metadata_manager - .catalog_controller - .finish_create_subscription_catalog(*subscription_id) - .await? - } - Command::DropSubscription { .. } => {} - Command::MergeSnapshotBackfillStreamingJobs(_) => {} - } - - Ok(()) - } - - pub fn get_truncate_epoch(&self, retention_second: u64) -> Epoch { - let Some(truncate_timestamptz) = Timestamptz::from_secs( - self.barrier_info - .prev_epoch - .value() - .as_timestamptz() - .timestamp() - - retention_second as i64, - ) else { - warn!(retention_second, prev_epoch = ?self.barrier_info.prev_epoch.value(), "invalid retention second value"); - return self.barrier_info.prev_epoch.value(); - }; - Epoch::from_unix_millis(truncate_timestamptz.timestamp_millis() as u64) - } -} diff --git a/src/meta/src/barrier/complete_task.rs b/src/meta/src/barrier/complete_task.rs new file mode 100644 index 0000000000000..85fb76b1941bb --- /dev/null +++ b/src/meta/src/barrier/complete_task.rs @@ -0,0 +1,245 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::future::{pending, Future}; +use std::mem::replace; +use std::sync::Arc; + +use anyhow::Context; +use futures::future::try_join_all; +use prometheus::HistogramTimer; +use risingwave_common::catalog::{DatabaseId, TableId}; +use risingwave_common::must_match; +use risingwave_pb::hummock::HummockVersionStats; +use tokio::task::JoinHandle; + +use crate::barrier::checkpoint::CheckpointControl; +use crate::barrier::command::CommandContext; +use crate::barrier::context::GlobalBarrierWorkerContext; +use crate::barrier::notifier::Notifier; +use crate::barrier::progress::TrackingJob; +use crate::barrier::rpc::ControlStreamManager; +use crate::barrier::schedule::PeriodicBarriers; +use crate::hummock::CommitEpochInfo; +use crate::manager::MetaSrvEnv; +use crate::rpc::metrics::GLOBAL_META_METRICS; +use crate::{MetaError, MetaResult}; + +pub(super) enum CompletingTask { + None, + Completing { + #[expect(clippy::type_complexity)] + /// `database_id` -> (`Some(database_graph_committed_epoch)`, [(`creating_job_id`, `creating_job_committed_epoch`)]) + epochs_to_ack: HashMap, Vec<(TableId, u64)>)>, + + // The join handle of a spawned task that completes the barrier. + // The return value indicate whether there is some create streaming job command + // that has finished but not checkpointed. If there is any, we will force checkpoint on the next barrier + join_handle: JoinHandle>, + }, + #[expect(dead_code)] + Err(MetaError), +} + +#[derive(Default)] +pub(super) struct CompleteBarrierTask { + pub(super) commit_info: CommitEpochInfo, + pub(super) finished_jobs: Vec, + pub(super) notifiers: Vec, + /// `database_id` -> (Some((`command_ctx`, `enqueue_time`)), vec!((`creating_job_id`, `epoch`))) + #[expect(clippy::type_complexity)] + pub(super) epoch_infos: HashMap< + DatabaseId, + ( + Option<(CommandContext, HistogramTimer)>, + Vec<(TableId, u64)>, + ), + >, +} + +impl CompleteBarrierTask { + #[expect(clippy::type_complexity)] + pub(super) fn epochs_to_ack(&self) -> HashMap, Vec<(TableId, u64)>)> { + self.epoch_infos + .iter() + .map(|(database_id, (command_context, creating_job_epochs))| { + ( + *database_id, + ( + command_context + .as_ref() + .map(|(command, _)| command.barrier_info.prev_epoch.value().0), + creating_job_epochs.clone(), + ), + ) + }) + .collect() + } +} + +impl CompleteBarrierTask { + pub(super) async fn complete_barrier( + self, + context: &impl GlobalBarrierWorkerContext, + env: MetaSrvEnv, + ) -> MetaResult { + let result: MetaResult = try { + let wait_commit_timer = GLOBAL_META_METRICS + .barrier_wait_commit_latency + .start_timer(); + let version_stats = context.commit_epoch(self.commit_info).await?; + for command_ctx in self + .epoch_infos + .values() + .flat_map(|(command, _)| command.as_ref().map(|(command, _)| command)) + { + context.post_collect_command(command_ctx).await?; + } + + wait_commit_timer.observe_duration(); + version_stats + }; + + let version_stats = { + let version_stats = match result { + Ok(version_stats) => version_stats, + Err(e) => { + for notifier in self.notifiers { + notifier.notify_collection_failed(e.clone()); + } + return Err(e); + } + }; + self.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + try_join_all( + self.finished_jobs + .into_iter() + .map(|finished_job| context.finish_creating_job(finished_job)), + ) + .await?; + for (command_ctx, enqueue_time) in self + .epoch_infos + .into_values() + .flat_map(|(command_context, _)| command_context) + { + let duration_sec = enqueue_time.stop_and_record(); + Self::report_complete_event(&env, duration_sec, &command_ctx); + GLOBAL_META_METRICS + .last_committed_barrier_time + .set(command_ctx.barrier_info.curr_epoch.value().as_unix_secs() as i64); + } + version_stats + }; + + Ok(version_stats) + } +} + +impl CompleteBarrierTask { + fn report_complete_event(env: &MetaSrvEnv, duration_sec: f64, command_ctx: &CommandContext) { + // Record barrier latency in event log. + use risingwave_pb::meta::event_log; + let event = event_log::EventBarrierComplete { + prev_epoch: command_ctx.barrier_info.prev_epoch(), + cur_epoch: command_ctx.barrier_info.curr_epoch.value().0, + duration_sec, + command: command_ctx + .command + .as_ref() + .map(|command| command.to_string()) + .unwrap_or_else(|| "barrier".to_string()), + barrier_kind: command_ctx.barrier_info.kind.as_str_name().to_string(), + }; + env.event_log_manager_ref() + .add_event_logs(vec![event_log::Event::BarrierComplete(event)]); + } +} + +pub(super) struct BarrierCompleteOutput { + #[expect(clippy::type_complexity)] + /// `database_id` -> (`Some(database_graph_committed_epoch)`, [(`creating_job_id`, `creating_job_committed_epoch`)]) + pub epochs_to_ack: HashMap, Vec<(TableId, u64)>)>, + pub hummock_version_stats: HummockVersionStats, +} + +impl CompletingTask { + pub(super) fn next_completed_barrier<'a>( + &'a mut self, + scheduled_barriers: &mut PeriodicBarriers, + checkpoint_control: &mut CheckpointControl, + control_stream_manager: &mut ControlStreamManager, + context: &Arc, + env: &MetaSrvEnv, + ) -> impl Future> + 'a { + // If there is no completing barrier, try to start completing the earliest barrier if + // it has been collected. + if let CompletingTask::None = self { + if let Some(task) = checkpoint_control + .next_complete_barrier_task(Some((scheduled_barriers, control_stream_manager))) + { + { + let epochs_to_ack = task.epochs_to_ack(); + let context = context.clone(); + let env = env.clone(); + let join_handle = + tokio::spawn(async move { task.complete_barrier(&*context, env).await }); + *self = CompletingTask::Completing { + epochs_to_ack, + join_handle, + }; + } + } + } + + self.next_completed_barrier_inner() + } + + async fn next_completed_barrier_inner(&mut self) -> MetaResult { + let CompletingTask::Completing { join_handle, .. } = self else { + return pending().await; + }; + + { + { + let join_result: MetaResult<_> = try { + join_handle + .await + .context("failed to join completing command")?? + }; + // It's important to reset the completing_command after await no matter the result is err + // or not, and otherwise the join handle will be polled again after ready. + let next_completing_command_status = if let Err(e) = &join_result { + CompletingTask::Err(e.clone()) + } else { + CompletingTask::None + }; + let completed_command = replace(self, next_completing_command_status); + let hummock_version_stats = join_result?; + + must_match!(completed_command, CompletingTask::Completing { + epochs_to_ack, + .. + } => { + Ok(BarrierCompleteOutput { + epochs_to_ack, + hummock_version_stats, + }) + }) + } + } + } +} diff --git a/src/meta/src/barrier/context/context_impl.rs b/src/meta/src/barrier/context/context_impl.rs new file mode 100644 index 0000000000000..25cdee4cd076d --- /dev/null +++ b/src/meta/src/barrier/context/context_impl.rs @@ -0,0 +1,284 @@ +use std::sync::Arc; + +use futures::future::try_join_all; +use risingwave_common::catalog::TableId; +use risingwave_pb::common::WorkerNode; +use risingwave_pb::hummock::HummockVersionStats; +use risingwave_pb::meta::PausedReason; +use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; +use risingwave_pb::stream_service::WaitEpochCommitRequest; +use risingwave_rpc_client::StreamingControlHandle; + +use crate::barrier::command::CommandContext; +use crate::barrier::context::{GlobalBarrierWorkerContext, GlobalBarrierWorkerContextImpl}; +use crate::barrier::progress::TrackingJob; +use crate::barrier::{ + BarrierManagerStatus, BarrierWorkerRuntimeInfoSnapshot, Command, CreateStreamingJobCommandInfo, + CreateStreamingJobType, RecoveryReason, ReplaceTablePlan, Scheduled, +}; +use crate::hummock::CommitEpochInfo; +use crate::{MetaError, MetaResult}; + +impl GlobalBarrierWorkerContext for GlobalBarrierWorkerContextImpl { + async fn commit_epoch(&self, commit_info: CommitEpochInfo) -> MetaResult { + self.hummock_manager.commit_epoch(commit_info).await?; + Ok(self.hummock_manager.get_version_stats().await) + } + + async fn next_scheduled(&self) -> Scheduled { + self.scheduled_barriers.next_scheduled().await + } + + fn abort_and_mark_blocked(&self, recovery_reason: RecoveryReason) { + self.set_status(BarrierManagerStatus::Recovering(recovery_reason)); + + // Mark blocked and abort buffered schedules, they might be dirty already. + self.scheduled_barriers + .abort_and_mark_blocked("cluster is under recovering"); + } + + fn mark_ready(&self) { + self.scheduled_barriers.mark_ready(); + self.set_status(BarrierManagerStatus::Running); + } + + async fn post_collect_command<'a>(&'a self, command: &'a CommandContext) -> MetaResult<()> { + command.post_collect(self).await + } + + async fn notify_creating_job_failed(&self, err: &MetaError) { + self.metadata_manager.notify_finish_failed(err).await + } + + async fn finish_creating_job(&self, job: TrackingJob) -> MetaResult<()> { + job.finish(&self.metadata_manager).await + } + + async fn new_control_stream( + &self, + node: &WorkerNode, + subscriptions: impl Iterator, + ) -> MetaResult { + self.new_control_stream_impl(node, subscriptions).await + } + + async fn reload_runtime_info(&self) -> MetaResult { + self.reload_runtime_info_impl().await + } +} + +impl GlobalBarrierWorkerContextImpl { + fn set_status(&self, new_status: BarrierManagerStatus) { + self.status.store(Arc::new(new_status)); + } +} + +impl CommandContext { + pub async fn wait_epoch_commit( + &self, + barrier_manager_context: &GlobalBarrierWorkerContextImpl, + ) -> MetaResult<()> { + let table_id = self.table_ids_to_commit.iter().next().cloned(); + // try wait epoch on an existing random table id + let Some(table_id) = table_id else { + // no need to wait epoch when there is no table id + return Ok(()); + }; + let futures = self.node_map.values().map(|worker_node| async { + let client = barrier_manager_context + .env + .stream_client_pool() + .get(worker_node) + .await?; + let request = WaitEpochCommitRequest { + epoch: self.barrier_info.prev_epoch(), + table_id: table_id.table_id, + }; + client.wait_epoch_commit(request).await + }); + + try_join_all(futures).await?; + + Ok(()) + } + + /// Do some stuffs after barriers are collected and the new storage version is committed, for + /// the given command. + pub async fn post_collect( + &self, + barrier_manager_context: &GlobalBarrierWorkerContextImpl, + ) -> MetaResult<()> { + let Some(command) = &self.command else { + return Ok(()); + }; + match command { + Command::Flush => {} + + Command::Throttle(_) => {} + + Command::Pause(reason) => { + if let PausedReason::ConfigChange = reason { + // After the `Pause` barrier is collected and committed, we must ensure that the + // storage version with this epoch is synced to all compute nodes before the + // execution of the next command of `Update`, as some newly created operators + // may immediately initialize their states on that barrier. + self.wait_epoch_commit(barrier_manager_context).await?; + } + } + + Command::Resume(_) => {} + + Command::SourceSplitAssignment(split_assignment) => { + barrier_manager_context + .metadata_manager + .update_actor_splits_by_split_assignment(split_assignment) + .await?; + barrier_manager_context + .source_manager + .apply_source_change(None, None, Some(split_assignment.clone()), None) + .await; + } + + Command::DropStreamingJobs { + unregistered_state_table_ids, + .. + } => { + barrier_manager_context + .hummock_manager + .unregister_table_ids(unregistered_state_table_ids.iter().cloned()) + .await?; + } + + Command::CancelStreamingJob(table_fragments) => { + tracing::debug!(id = ?table_fragments.table_id(), "cancelling stream job"); + + // NOTE(kwannoel): At this point, meta has already registered the table ids. + // We should unregister them. + // This is required for background ddl, for foreground ddl this is a no-op. + // Foreground ddl is handled entirely by stream manager, so it will unregister + // the table ids on failure. + // On the other hand background ddl could be handled by barrier manager. + // It won't clean the tables on failure, + // since the failure could be recoverable. + // As such it needs to be handled here. + barrier_manager_context + .hummock_manager + .unregister_table_ids(table_fragments.all_table_ids().map(TableId::new)) + .await?; + } + + Command::CreateStreamingJob { info, job_type } => { + let CreateStreamingJobCommandInfo { + table_fragments, + dispatchers, + init_split_assignment, + .. + } = info; + barrier_manager_context + .metadata_manager + .catalog_controller + .post_collect_table_fragments( + table_fragments.table_id().table_id as _, + table_fragments.actor_ids(), + dispatchers.clone(), + init_split_assignment, + ) + .await?; + + if let CreateStreamingJobType::SinkIntoTable(ReplaceTablePlan { + new_table_fragments, + dispatchers, + init_split_assignment, + .. + }) = job_type + { + barrier_manager_context + .metadata_manager + .catalog_controller + .post_collect_table_fragments( + new_table_fragments.table_id().table_id as _, + new_table_fragments.actor_ids(), + dispatchers.clone(), + init_split_assignment, + ) + .await?; + } + + // Extract the fragments that include source operators. + let source_fragments = table_fragments.stream_source_fragments(); + let backfill_fragments = table_fragments.source_backfill_fragments()?; + barrier_manager_context + .source_manager + .apply_source_change( + Some(source_fragments), + Some(backfill_fragments), + Some(init_split_assignment.clone()), + None, + ) + .await; + } + Command::RescheduleFragment { + reschedules, + table_parallelism, + .. + } => { + barrier_manager_context + .scale_controller + .post_apply_reschedule(reschedules, table_parallelism) + .await?; + } + + Command::ReplaceTable(ReplaceTablePlan { + old_table_fragments, + new_table_fragments, + dispatchers, + init_split_assignment, + .. + }) => { + // Update actors and actor_dispatchers for new table fragments. + barrier_manager_context + .metadata_manager + .catalog_controller + .post_collect_table_fragments( + new_table_fragments.table_id().table_id as _, + new_table_fragments.actor_ids(), + dispatchers.clone(), + init_split_assignment, + ) + .await?; + + // Apply the split changes in source manager. + barrier_manager_context + .source_manager + .drop_source_fragments_vec(std::slice::from_ref(old_table_fragments)) + .await; + let source_fragments = new_table_fragments.stream_source_fragments(); + // XXX: is it possible to have backfill fragments here? + let backfill_fragments = new_table_fragments.source_backfill_fragments()?; + barrier_manager_context + .source_manager + .apply_source_change( + Some(source_fragments), + Some(backfill_fragments), + Some(init_split_assignment.clone()), + None, + ) + .await; + } + + Command::CreateSubscription { + subscription_id, .. + } => { + barrier_manager_context + .metadata_manager + .catalog_controller + .finish_create_subscription_catalog(*subscription_id) + .await? + } + Command::DropSubscription { .. } => {} + Command::MergeSnapshotBackfillStreamingJobs(_) => {} + } + + Ok(()) + } +} diff --git a/src/meta/src/barrier/context/mod.rs b/src/meta/src/barrier/context/mod.rs new file mode 100644 index 0000000000000..f1e0737929518 --- /dev/null +++ b/src/meta/src/barrier/context/mod.rs @@ -0,0 +1,83 @@ +// 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. + +mod context_impl; +mod recovery; + +use std::future::Future; +use std::sync::Arc; + +use arc_swap::ArcSwap; +use risingwave_pb::common::WorkerNode; +use risingwave_pb::hummock::HummockVersionStats; +use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; +use risingwave_rpc_client::StreamingControlHandle; + +use crate::barrier::command::CommandContext; +use crate::barrier::progress::TrackingJob; +use crate::barrier::schedule::ScheduledBarriers; +use crate::barrier::{ + BarrierManagerStatus, BarrierWorkerRuntimeInfoSnapshot, RecoveryReason, Scheduled, +}; +use crate::hummock::{CommitEpochInfo, HummockManagerRef}; +use crate::manager::{MetaSrvEnv, MetadataManager}; +use crate::stream::{ScaleControllerRef, SourceManagerRef}; +use crate::{MetaError, MetaResult}; + +pub(super) trait GlobalBarrierWorkerContext: Send + Sync + 'static { + fn commit_epoch( + &self, + commit_info: CommitEpochInfo, + ) -> impl Future> + Send + '_; + + async fn next_scheduled(&self) -> Scheduled; + fn abort_and_mark_blocked(&self, recovery_reason: RecoveryReason); + fn mark_ready(&self); + + fn post_collect_command<'a>( + &'a self, + command: &'a CommandContext, + ) -> impl Future> + Send + 'a; + + async fn notify_creating_job_failed(&self, err: &MetaError); + + fn finish_creating_job( + &self, + job: TrackingJob, + ) -> impl Future> + Send + '_; + + async fn new_control_stream( + &self, + node: &WorkerNode, + subscriptions: impl Iterator, + ) -> MetaResult; + + async fn reload_runtime_info(&self) -> MetaResult; +} + +pub(crate) struct GlobalBarrierWorkerContextImpl { + pub(crate) scheduled_barriers: ScheduledBarriers, + + pub(crate) status: Arc>, + + pub(crate) metadata_manager: MetadataManager, + + pub(crate) hummock_manager: HummockManagerRef, + + pub(crate) source_manager: SourceManagerRef, + + pub(crate) scale_controller: ScaleControllerRef, + + pub(crate) env: MetaSrvEnv, +} diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/context/recovery.rs similarity index 66% rename from src/meta/src/barrier/recovery.rs rename to src/meta/src/barrier/context/recovery.rs index 6e17736d2fdd4..5c13b923b8fc8 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/context/recovery.rs @@ -21,49 +21,21 @@ use risingwave_common::bail; use risingwave_common::catalog::{DatabaseId, TableId}; use risingwave_common::config::DefaultParallelism; use risingwave_common::hash::WorkerSlotId; -use risingwave_common::util::epoch::Epoch; use risingwave_meta_model::{StreamingParallelism, WorkerId}; -use risingwave_pb::meta::subscribe_response::{Info, Operation}; -use risingwave_pb::meta::{PausedReason, Recovery}; -use risingwave_pb::stream_plan::barrier_mutation::Mutation; -use risingwave_pb::stream_plan::{AddMutation, StreamActor}; +use risingwave_pb::stream_plan::StreamActor; use thiserror_ext::AsReport; use tokio::time::Instant; -use tokio_retry::strategy::{jitter, ExponentialBackoff}; -use tracing::{debug, info, warn, Instrument}; - -use super::{ - BarrierWorkerRuntimeInfoSnapshot, CheckpointControl, DatabaseCheckpointControl, - GlobalBarrierWorker, GlobalBarrierWorkerContext, InflightSubscriptionInfo, RecoveryReason, - TracedEpoch, -}; -use crate::barrier::info::{BarrierInfo, InflightDatabaseInfo}; -use crate::barrier::progress::CreateMviewProgressTracker; -use crate::barrier::rpc::ControlStreamManager; -use crate::barrier::schedule::ScheduledBarriers; -use crate::barrier::state::BarrierWorkerState; -use crate::barrier::{BarrierKind, GlobalBarrierWorkerContextImpl}; +use tracing::{debug, info, warn}; + +use super::BarrierWorkerRuntimeInfoSnapshot; +use crate::barrier::context::GlobalBarrierWorkerContextImpl; +use crate::barrier::info::InflightDatabaseInfo; +use crate::barrier::InflightSubscriptionInfo; use crate::controller::fragment::InflightFragmentInfo; use crate::manager::ActiveStreamingWorkerNodes; use crate::model::{ActorId, TableFragments, TableParallelism}; -use crate::rpc::metrics::GLOBAL_META_METRICS; -use crate::stream::{build_actor_connector_splits, RescheduleOptions, TableResizePolicy}; -use crate::{model, MetaError, MetaResult}; - -impl GlobalBarrierWorker { - // Retry base interval in milliseconds. - const RECOVERY_RETRY_BASE_INTERVAL: u64 = 20; - // Retry max interval. - const RECOVERY_RETRY_MAX_INTERVAL: Duration = Duration::from_secs(5); - - #[inline(always)] - /// Initialize a retry strategy for operation in recovery. - fn get_retry_strategy() -> impl Iterator { - ExponentialBackoff::from_millis(Self::RECOVERY_RETRY_BASE_INTERVAL) - .max_delay(Self::RECOVERY_RETRY_MAX_INTERVAL) - .map(jitter) - } -} +use crate::stream::{RescheduleOptions, TableResizePolicy}; +use crate::{model, MetaResult}; impl GlobalBarrierWorkerContextImpl { /// Clean catalogs for creating streaming jobs that are in foreground mode or table fragments not persisted. @@ -117,41 +89,23 @@ impl GlobalBarrierWorkerContextImpl { Ok(mview_map) } -} - -impl ScheduledBarriers { - /// Pre buffered drop and cancel command, return true if any. - fn pre_apply_drop_cancel(&self) -> bool { - let (dropped_actors, cancelled) = self.pre_apply_drop_cancel_scheduled(); - !dropped_actors.is_empty() || !cancelled.is_empty() - } -} + /// Resolve actor information from cluster, fragment manager and `ChangedTableId`. + /// We use `changed_table_id` to modify the actors to be sent or collected. Because these actor + /// will create or drop before this barrier flow through them. + async fn resolve_graph_info(&self) -> MetaResult> { + let all_actor_infos = self + .metadata_manager + .catalog_controller + .load_all_actors() + .await?; -impl GlobalBarrierWorker { - /// Recovery the whole cluster from the latest epoch. - /// - /// If `paused_reason` is `Some`, all data sources (including connectors and DMLs) will be - /// immediately paused after recovery, until the user manually resume them either by restarting - /// the cluster or `risectl` command. Used for debugging purpose. - /// - /// Returns the new state of the barrier manager after recovery. - pub async fn recovery( - &mut self, - paused_reason: Option, - err: Option, - recovery_reason: RecoveryReason, - ) { - self.context.abort_and_mark_blocked(recovery_reason); - // Clear all control streams to release resources (connections to compute nodes) first. - self.control_stream_manager.clear(); - - self.recovery_inner(paused_reason, err).await; - self.context.mark_ready(); + Ok(all_actor_infos + .into_iter() + .map(|(database_id, actor_infos)| (database_id, InflightDatabaseInfo::new(actor_infos))) + .collect()) } -} -impl GlobalBarrierWorkerContextImpl { pub(super) async fn reload_runtime_info_impl( &self, ) -> MetaResult { @@ -274,210 +228,6 @@ impl GlobalBarrierWorkerContextImpl { } } -impl GlobalBarrierWorker { - async fn recovery_inner( - &mut self, - paused_reason: Option, - err: Option, - ) { - tracing::info!("recovery start!"); - let retry_strategy = Self::get_retry_strategy(); - - // We take retry into consideration because this is the latency user sees for a cluster to - // get recovered. - let recovery_timer = GLOBAL_META_METRICS.recovery_latency.start_timer(); - - let new_state = tokio_retry::Retry::spawn(retry_strategy, || async { - if let Some(err) = &err { - self.context.notify_creating_job_failed(err).await; - }; - let runtime_info_snapshot = self - .context - .reload_runtime_info() - .await?; - runtime_info_snapshot.validate().inspect_err(|e| { - warn!(err = ?e.as_report(), ?runtime_info_snapshot, "reloaded runtime info failed to validate"); - })?; - let BarrierWorkerRuntimeInfoSnapshot { - active_streaming_nodes, - database_fragment_infos, - mut state_table_committed_epochs, - mut subscription_infos, - mut stream_actors, - mut source_splits, - mut background_jobs, - hummock_version_stats, - } = runtime_info_snapshot; - - self.sink_manager.reset().await; - - let mut control_stream_manager = ControlStreamManager::new(self.env.clone()); - let reset_start_time = Instant::now(); - control_stream_manager - .reset( - subscription_infos.values(), - active_streaming_nodes.current(), - &*self.context, - ) - .await - .inspect_err(|err| { - warn!(error = %err.as_report(), "reset compute nodes failed"); - })?; - info!(elapsed=?reset_start_time.elapsed(), "control stream reset"); - - let mut databases = HashMap::new(); - - let recovery_result: MetaResult<_> = try { - for (database_id, info) in database_fragment_infos { - let source_split_assignments = info - .fragment_infos() - .flat_map(|info| info.actors.keys()) - .filter_map(|actor_id| { - let actor_id = *actor_id as ActorId; - source_splits - .remove(&actor_id) - .map(|splits| (actor_id, splits)) - }) - .collect(); - let mutation = Mutation::Add(AddMutation { - // Actors built during recovery is not treated as newly added actors. - actor_dispatchers: Default::default(), - added_actors: Default::default(), - actor_splits: build_actor_connector_splits(&source_split_assignments), - pause: paused_reason.is_some(), - subscriptions_to_add: Default::default(), - }); - - let new_epoch = { - let mut epochs = info.existing_table_ids().map(|table_id| { - ( - table_id, - state_table_committed_epochs - .remove(&table_id) - .expect("should exist"), - ) - }); - let (first_table_id, prev_epoch) = epochs.next().expect("non-empty"); - for (table_id, epoch) in epochs { - assert_eq!( - prev_epoch, epoch, - "{} has different committed epoch to {}", - first_table_id, table_id - ); - } - let prev_epoch = TracedEpoch::new(Epoch(prev_epoch)); - // Use a different `curr_epoch` for each recovery attempt. - let curr_epoch = prev_epoch.next(); - let barrier_info = BarrierInfo { - prev_epoch, - curr_epoch, - kind: BarrierKind::Initial, - }; - - let mut node_actors: HashMap<_, Vec<_>> = HashMap::new(); - for (actor_id, worker_id) in - info.fragment_infos().flat_map(|info| info.actors.iter()) - { - let worker_id = *worker_id as WorkerId; - let actor_id = *actor_id as ActorId; - let stream_actor = - stream_actors.remove(&actor_id).expect("should exist"); - node_actors.entry(worker_id).or_default().push(stream_actor); - } - - let mut node_to_collect = control_stream_manager.inject_barrier( - database_id, - None, - Some(mutation), - &barrier_info, - info.fragment_infos(), - info.fragment_infos(), - Some(node_actors), - vec![], - vec![], - )?; - debug!(?node_to_collect, "inject initial barrier"); - while !node_to_collect.is_empty() { - let (worker_id, result) = - control_stream_manager.next_collect_barrier_response().await; - let resp = result?; - assert_eq!(resp.epoch, barrier_info.prev_epoch()); - assert!(node_to_collect.remove(&worker_id)); - } - debug!("collected initial barrier"); - barrier_info.curr_epoch - }; - - let background_mviews = info - .job_ids() - .filter_map(|job_id| { - background_jobs.remove(&job_id).map(|mview| (job_id, mview)) - }) - .collect(); - let tracker = CreateMviewProgressTracker::recover( - background_mviews, - &hummock_version_stats, - ); - let state = BarrierWorkerState::recovery( - new_epoch, - info, - subscription_infos.remove(&database_id).unwrap_or_default(), - paused_reason, - ); - databases.insert( - database_id, - DatabaseCheckpointControl::recovery(database_id, tracker, state), - ); - } - if !stream_actors.is_empty() { - warn!(actor_ids = ?stream_actors.keys().collect_vec(), "unused stream actors in recovery"); - } - if !source_splits.is_empty() { - warn!(actor_ids = ?source_splits.keys().collect_vec(), "unused actor source splits in recovery"); - } - if !background_jobs.is_empty() { - warn!(job_ids = ?background_jobs.keys().collect_vec(), "unused recovered background mview in recovery"); - } - if !subscription_infos.is_empty() { - warn!(?subscription_infos, "unused subscription infos in recovery"); - } - if !state_table_committed_epochs.is_empty() { - warn!(?state_table_committed_epochs, "unused state table committed epoch in recovery"); - } - ( - active_streaming_nodes, - control_stream_manager, - CheckpointControl { - databases, - hummock_version_stats, - }, - ) - }; - if recovery_result.is_err() { - GLOBAL_META_METRICS.recovery_failure_cnt.inc(); - } - recovery_result - }) - .instrument(tracing::info_span!("recovery_attempt")) - .await - .expect("Retry until recovery success."); - - recovery_timer.observe_duration(); - - ( - self.active_streaming_nodes, - self.control_stream_manager, - self.checkpoint_control, - ) = new_state; - - tracing::info!("recovery success"); - - self.env - .notification_manager() - .notify_frontend_without_version(Operation::Update, Info::Recovery(Recovery {})); - } -} - impl GlobalBarrierWorkerContextImpl { // Migration timeout. const RECOVERY_FORCE_MIGRATION_TIMEOUT: Duration = Duration::from_secs(300); diff --git a/src/meta/src/barrier/manager.rs b/src/meta/src/barrier/manager.rs new file mode 100644 index 0000000000000..4b65733721231 --- /dev/null +++ b/src/meta/src/barrier/manager.rs @@ -0,0 +1,136 @@ +// 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::hash_map::Entry; +use std::sync::Arc; + +use anyhow::Context; +use arc_swap::ArcSwap; +use risingwave_common::bail; +use risingwave_hummock_sdk::HummockVersionId; +use risingwave_pb::ddl_service::DdlProgress; +use risingwave_pb::meta::PbRecoveryStatus; +use tokio::sync::mpsc::unbounded_channel; +use tokio::sync::{mpsc, oneshot}; +use tokio::task::JoinHandle; + +use crate::barrier::worker::GlobalBarrierWorker; +use crate::barrier::{schedule, BarrierManagerRequest, BarrierManagerStatus, RecoveryReason}; +use crate::hummock::HummockManagerRef; +use crate::manager::sink_coordination::SinkCoordinatorManager; +use crate::manager::{MetaSrvEnv, MetadataManager}; +use crate::stream::{ScaleControllerRef, SourceManagerRef}; +use crate::MetaResult; + +pub struct GlobalBarrierManager { + status: Arc>, + hummock_manager: HummockManagerRef, + request_tx: mpsc::UnboundedSender, + metadata_manager: MetadataManager, +} + +pub type BarrierManagerRef = Arc; + +impl GlobalBarrierManager { + /// Serving `SHOW JOBS / SELECT * FROM rw_ddl_progress` + pub async fn get_ddl_progress(&self) -> MetaResult> { + let mut ddl_progress = { + let (tx, rx) = oneshot::channel(); + self.request_tx + .send(BarrierManagerRequest::GetDdlProgress(tx)) + .context("failed to send get ddl progress request")?; + rx.await.context("failed to receive get ddl progress")? + }; + // If not in tracker, means the first barrier not collected yet. + // In that case just return progress 0. + let mviews = self + .metadata_manager + .catalog_controller + .list_background_creating_mviews(true) + .await + .unwrap(); + for mview in mviews { + if let Entry::Vacant(e) = ddl_progress.entry(mview.table_id as _) { + e.insert(DdlProgress { + id: mview.table_id as u64, + statement: mview.definition, + progress: "0.0%".into(), + }); + } + } + + Ok(ddl_progress.into_values().collect()) + } + + pub async fn get_hummock_version_id(&self) -> HummockVersionId { + self.hummock_manager.get_version_id().await + } +} + +impl GlobalBarrierManager { + /// Check the status of barrier manager, return error if it is not `Running`. + pub fn check_status_running(&self) -> MetaResult<()> { + let status = self.status.load(); + match &**status { + BarrierManagerStatus::Starting + | BarrierManagerStatus::Recovering(RecoveryReason::Bootstrap) => { + bail!("The cluster is bootstrapping") + } + BarrierManagerStatus::Recovering(RecoveryReason::Failover(e)) => { + Err(anyhow::anyhow!(e.clone()).context("The cluster is recovering"))? + } + BarrierManagerStatus::Recovering(RecoveryReason::Adhoc) => { + bail!("The cluster is recovering-adhoc") + } + BarrierManagerStatus::Running => Ok(()), + } + } + + pub fn get_recovery_status(&self) -> PbRecoveryStatus { + (&**self.status.load()).into() + } +} + +impl GlobalBarrierManager { + pub async fn start( + scheduled_barriers: schedule::ScheduledBarriers, + env: MetaSrvEnv, + metadata_manager: MetadataManager, + hummock_manager: HummockManagerRef, + source_manager: SourceManagerRef, + sink_manager: SinkCoordinatorManager, + scale_controller: ScaleControllerRef, + ) -> (Arc, JoinHandle<()>, oneshot::Sender<()>) { + let (request_tx, request_rx) = unbounded_channel(); + let barrier_worker = GlobalBarrierWorker::new( + scheduled_barriers, + env, + metadata_manager, + hummock_manager, + source_manager, + sink_manager, + scale_controller, + request_rx, + ) + .await; + let manager = Self { + status: barrier_worker.context.status.clone(), + hummock_manager: barrier_worker.context.hummock_manager.clone(), + request_tx, + metadata_manager: barrier_worker.context.metadata_manager.clone(), + }; + let (join_handle, shutdown_tx) = barrier_worker.start(); + (Arc::new(manager), join_handle, shutdown_tx) + } +} diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 2a3d333f61eca..b8680a0e520c9 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -12,103 +12,46 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::hash_map::Entry; -use std::collections::{BTreeMap, HashMap, HashSet}; -use std::future::{pending, Future}; -use std::mem::{replace, take}; -use std::sync::Arc; -use std::time::Duration; +use std::collections::HashMap; -use anyhow::{anyhow, Context}; -use arc_swap::ArcSwap; -use fail::fail_point; -use futures::future::try_join_all; -use itertools::Itertools; -use prometheus::HistogramTimer; +use anyhow::anyhow; use risingwave_common::catalog::{DatabaseId, TableId}; -use risingwave_common::system_param::reader::SystemParamsRead; -use risingwave_common::system_param::PAUSE_ON_NEXT_BOOTSTRAP_KEY; -use risingwave_common::{bail, must_match}; use risingwave_connector::source::SplitImpl; -use risingwave_hummock_sdk::change_log::build_table_change_log_delta; -use risingwave_hummock_sdk::sstable_info::SstableInfo; -use risingwave_hummock_sdk::table_stats::from_prost_table_stats_map; -use risingwave_hummock_sdk::table_watermark::{ - merge_multiple_new_table_watermarks, TableWatermarks, -}; -use risingwave_hummock_sdk::{HummockSstableObjectId, HummockVersionId, LocalSstableInfo}; -use risingwave_meta_model::WorkerId; -use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; -use risingwave_pb::meta::{PausedReason, PbRecoveryStatus}; -use risingwave_pb::stream_plan::{StreamActor, SubscriptionUpstreamInfo}; -use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; -use risingwave_pb::stream_service::BarrierCompleteResponse; -use risingwave_rpc_client::StreamingControlHandle; -use thiserror_ext::AsReport; -use tokio::sync::mpsc::unbounded_channel; -use tokio::sync::oneshot::{Receiver, Sender}; -use tokio::sync::{mpsc, oneshot}; -use tokio::task::JoinHandle; -use tracing::{debug, error, info, warn, Instrument}; +use risingwave_pb::meta::PbRecoveryStatus; +use risingwave_pb::stream_plan::StreamActor; +use tokio::sync::oneshot::Sender; -use self::command::CommandContext; use self::notifier::Notifier; -use crate::barrier::creating_job::{CompleteJobType, CreatingStreamingJobControl}; -use crate::barrier::info::{BarrierInfo, InflightDatabaseInfo, InflightStreamingJobInfo}; -use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob}; -use crate::barrier::rpc::{from_partial_graph_id, merge_node_rpc_errors, ControlStreamManager}; -use crate::barrier::schedule::{PeriodicBarriers, ScheduledBarriers}; -use crate::barrier::state::BarrierWorkerState; -use crate::error::MetaErrorInner; -use crate::hummock::{CommitEpochInfo, HummockManagerRef, NewTableFragmentInfo}; -use crate::manager::sink_coordination::SinkCoordinatorManager; -use crate::manager::{ - ActiveStreamingWorkerChange, ActiveStreamingWorkerNodes, LocalNotification, MetaSrvEnv, - MetadataManager, -}; +use crate::barrier::info::{BarrierInfo, InflightDatabaseInfo}; +use crate::manager::ActiveStreamingWorkerNodes; use crate::model::{ActorId, TableFragments}; -use crate::rpc::metrics::GLOBAL_META_METRICS; -use crate::stream::{ScaleControllerRef, SourceManagerRef}; use crate::{MetaError, MetaResult}; +mod checkpoint; mod command; -mod creating_job; +mod complete_task; +mod context; mod info; +mod manager; mod notifier; mod progress; -mod recovery; mod rpc; mod schedule; -mod state; mod trace; +mod utils; +mod worker; pub use self::command::{ BarrierKind, Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, ReplaceTablePlan, Reschedule, SnapshotBackfillInfo, }; pub use self::info::InflightSubscriptionInfo; +pub use self::manager::{BarrierManagerRef, GlobalBarrierManager}; pub use self::schedule::BarrierScheduler; pub use self::trace::TracedEpoch; -#[derive(Debug, Default, Clone, PartialEq, Eq)] -pub(crate) struct TableMap { - inner: HashMap, -} - -impl From> for TableMap { - fn from(inner: HashMap) -> Self { - Self { inner } - } -} - -impl From> for HashMap { - fn from(table_map: TableMap) -> Self { - table_map.inner - } -} - /// The reason why the cluster is recovering. enum RecoveryReason { /// After bootstrap. @@ -129,12 +72,6 @@ enum BarrierManagerStatus { Running, } -struct NewBarrier { - command: Option<(DatabaseId, Command, Vec)>, - span: tracing::Span, - checkpoint: bool, -} - /// Scheduled command with its notifiers. struct Scheduled { database_id: DatabaseId, @@ -160,55 +97,6 @@ pub(crate) enum BarrierManagerRequest { GetDdlProgress(Sender>), } -struct GlobalBarrierWorkerContextImpl { - scheduled_barriers: ScheduledBarriers, - - status: Arc>, - - metadata_manager: MetadataManager, - - hummock_manager: HummockManagerRef, - - source_manager: SourceManagerRef, - - scale_controller: ScaleControllerRef, - - env: MetaSrvEnv, -} - -impl GlobalBarrierManager { - pub async fn start( - scheduled_barriers: schedule::ScheduledBarriers, - env: MetaSrvEnv, - metadata_manager: MetadataManager, - hummock_manager: HummockManagerRef, - source_manager: SourceManagerRef, - sink_manager: SinkCoordinatorManager, - scale_controller: ScaleControllerRef, - ) -> (Arc, JoinHandle<()>, oneshot::Sender<()>) { - let (request_tx, request_rx) = unbounded_channel(); - let barrier_worker = GlobalBarrierWorker::new( - scheduled_barriers, - env, - metadata_manager, - hummock_manager, - source_manager, - sink_manager, - scale_controller, - request_rx, - ) - .await; - let manager = Self { - status: barrier_worker.context.status.clone(), - hummock_manager: barrier_worker.context.hummock_manager.clone(), - request_tx, - metadata_manager: barrier_worker.context.metadata_manager.clone(), - }; - let (join_handle, shutdown_tx) = barrier_worker.start(); - (Arc::new(manager), join_handle, shutdown_tx) - } -} - #[derive(Debug)] struct BarrierWorkerRuntimeInfoSnapshot { active_streaming_nodes: ActiveStreamingWorkerNodes, @@ -284,1806 +172,3 @@ impl BarrierWorkerRuntimeInfoSnapshot { Ok(()) } } - -trait GlobalBarrierWorkerContext: Send + Sync + 'static { - fn commit_epoch( - &self, - commit_info: CommitEpochInfo, - ) -> impl Future> + Send + '_; - - async fn next_scheduled(&self) -> Scheduled; - fn abort_and_mark_blocked(&self, recovery_reason: RecoveryReason); - fn mark_ready(&self); - - fn post_collect_command<'a>( - &'a self, - command: &'a CommandContext, - ) -> impl Future> + Send + 'a; - - async fn notify_creating_job_failed(&self, err: &MetaError); - - fn finish_creating_job( - &self, - job: TrackingJob, - ) -> impl Future> + Send + '_; - - async fn new_control_stream( - &self, - node: &WorkerNode, - subscriptions: impl Iterator, - ) -> MetaResult; - - async fn reload_runtime_info(&self) -> MetaResult; -} - -impl GlobalBarrierWorkerContext for GlobalBarrierWorkerContextImpl { - async fn commit_epoch(&self, commit_info: CommitEpochInfo) -> MetaResult { - self.hummock_manager.commit_epoch(commit_info).await?; - Ok(self.hummock_manager.get_version_stats().await) - } - - async fn next_scheduled(&self) -> Scheduled { - self.scheduled_barriers.next_scheduled().await - } - - fn abort_and_mark_blocked(&self, recovery_reason: RecoveryReason) { - self.set_status(BarrierManagerStatus::Recovering(recovery_reason)); - - // Mark blocked and abort buffered schedules, they might be dirty already. - self.scheduled_barriers - .abort_and_mark_blocked("cluster is under recovering"); - } - - fn mark_ready(&self) { - self.scheduled_barriers.mark_ready(); - self.set_status(BarrierManagerStatus::Running); - } - - async fn post_collect_command<'a>(&'a self, command: &'a CommandContext) -> MetaResult<()> { - command.post_collect(self).await - } - - async fn notify_creating_job_failed(&self, err: &MetaError) { - self.metadata_manager.notify_finish_failed(err).await - } - - async fn finish_creating_job(&self, job: TrackingJob) -> MetaResult<()> { - job.finish(&self.metadata_manager).await - } - - async fn new_control_stream( - &self, - node: &WorkerNode, - subscriptions: impl Iterator, - ) -> MetaResult { - self.new_control_stream_impl(node, subscriptions).await - } - - async fn reload_runtime_info(&self) -> MetaResult { - self.reload_runtime_info_impl().await - } -} - -/// [`crate::barrier::GlobalBarrierWorker`] sends barriers to all registered compute nodes and -/// collect them, with monotonic increasing epoch numbers. On compute nodes, `LocalBarrierManager` -/// in `risingwave_stream` crate will serve these requests and dispatch them to source actors. -/// -/// Configuration change in our system is achieved by the mutation in the barrier. Thus, -/// [`crate::barrier::GlobalBarrierWorker`] provides a set of interfaces like a state machine, -/// accepting [`Command`] that carries info to build `Mutation`. To keep the consistency between -/// barrier manager and meta store, some actions like "drop materialized view" or "create mv on mv" -/// must be done in barrier manager transactional using [`Command`]. -struct GlobalBarrierWorker { - /// Enable recovery or not when failover. - enable_recovery: bool, - - /// The queue of scheduled barriers. - periodic_barriers: PeriodicBarriers, - - /// The max barrier nums in flight - in_flight_barrier_nums: usize, - - context: Arc, - - env: MetaSrvEnv, - - checkpoint_control: CheckpointControl, - - /// Command that has been collected but is still completing. - /// The join handle of the completing future is stored. - completing_task: CompletingTask, - - request_rx: mpsc::UnboundedReceiver, - - active_streaming_nodes: ActiveStreamingWorkerNodes, - - sink_manager: SinkCoordinatorManager, - - control_stream_manager: ControlStreamManager, -} - -struct CheckpointControl { - databases: HashMap, - hummock_version_stats: HummockVersionStats, -} - -impl CheckpointControl { - fn ack_completed(&mut self, output: BarrierCompleteOutput) { - self.hummock_version_stats = output.hummock_version_stats; - for (database_id, (command_prev_epoch, creating_job_epochs)) in output.epochs_to_ack { - self.databases - .get_mut(&database_id) - .expect("should exist") - .ack_completed(command_prev_epoch, creating_job_epochs); - } - } - - fn next_complete_barrier_task( - &mut self, - mut context: Option<(&mut PeriodicBarriers, &mut ControlStreamManager)>, - ) -> Option { - let mut task = None; - for database in self.databases.values_mut() { - let context = context.as_mut().map(|(s, c)| (&mut **s, &mut **c)); - database.next_complete_barrier_task(&mut task, context, &self.hummock_version_stats); - } - task - } - - fn barrier_collected( - &mut self, - resp: BarrierCompleteResponse, - control_stream_manager: &mut ControlStreamManager, - ) -> MetaResult<()> { - let database_id = from_partial_graph_id(resp.partial_graph_id).0; - self.databases - .get_mut(&database_id) - .expect("should exist") - .barrier_collected(resp, control_stream_manager) - } - - fn can_inject_barrier(&self, in_flight_barrier_nums: usize) -> bool { - self.databases - .values() - .all(|database| database.can_inject_barrier(in_flight_barrier_nums)) - } - - fn max_prev_epoch(&self) -> Option { - self.databases - .values() - .map(|database| database.state.in_flight_prev_epoch()) - .max_by_key(|epoch| epoch.value()) - .cloned() - } - - fn handle_new_barrier( - &mut self, - new_barrier: NewBarrier, - control_stream_manager: &mut ControlStreamManager, - active_streaming_nodes: &ActiveStreamingWorkerNodes, - ) -> MetaResult<()> { - let NewBarrier { - command, - span, - checkpoint, - } = new_barrier; - - if let Some((database_id, command, notifiers)) = command { - let max_prev_epoch = self.max_prev_epoch(); - let (database, max_prev_epoch) = match self.databases.entry(database_id) { - Entry::Occupied(entry) => ( - entry.into_mut(), - max_prev_epoch.expect("should exist when having some database"), - ), - Entry::Vacant(entry) => match &command { - Command::CreateStreamingJob { - job_type: CreateStreamingJobType::Normal, - .. - } => { - let new_database = DatabaseCheckpointControl::new(database_id); - let max_prev_epoch = if let Some(max_prev_epoch) = max_prev_epoch { - if max_prev_epoch.value() - < new_database.state.in_flight_prev_epoch().value() - { - new_database.state.in_flight_prev_epoch().clone() - } else { - max_prev_epoch - } - } else { - new_database.state.in_flight_prev_epoch().clone() - }; - (entry.insert(new_database), max_prev_epoch) - } - Command::Flush - | Command::Pause(PausedReason::Manual) - | Command::Resume(PausedReason::Manual) => { - for mut notifier in notifiers { - notifier.notify_started(); - notifier.notify_collected(); - } - warn!(?command, "skip command for empty database"); - return Ok(()); - } - _ => { - panic!("new database graph info can only be created for normal creating streaming job, but get command: {} {:?}", database_id, command) - } - }, - }; - - let curr_epoch = max_prev_epoch.next(); - - database.handle_new_barrier( - Some((command, notifiers)), - checkpoint, - span.clone(), - control_stream_manager, - active_streaming_nodes, - &self.hummock_version_stats, - curr_epoch.clone(), - )?; - for database in self.databases.values_mut() { - if database.database_id == database_id { - continue; - } - database.handle_new_barrier( - None, - checkpoint, - span.clone(), - control_stream_manager, - active_streaming_nodes, - &self.hummock_version_stats, - curr_epoch.clone(), - )?; - } - } else { - let Some(max_prev_epoch) = self.max_prev_epoch() else { - assert!(self.databases.is_empty()); - return Ok(()); - }; - let curr_epoch = max_prev_epoch.next(); - for database in self.databases.values_mut() { - database.handle_new_barrier( - None, - checkpoint, - span.clone(), - control_stream_manager, - active_streaming_nodes, - &self.hummock_version_stats, - curr_epoch.clone(), - )?; - } - } - Ok(()) - } - - fn update_barrier_nums_metrics(&self) { - self.databases - .values() - .for_each(|database| database.update_barrier_nums_metrics()); - } -} - -/// Controls the concurrent execution of commands. -struct DatabaseCheckpointControl { - database_id: DatabaseId, - state: BarrierWorkerState, - - /// Save the state and message of barrier in order. - /// Key is the `prev_epoch`. - command_ctx_queue: BTreeMap, - /// The barrier that are completing. - /// Some((`prev_epoch`, `should_pause_inject_barrier`)) - completing_barrier: Option<(u64, bool)>, - - creating_streaming_job_controls: HashMap, - - create_mview_tracker: CreateMviewProgressTracker, -} - -impl DatabaseCheckpointControl { - fn new(database_id: DatabaseId) -> Self { - Self { - database_id, - state: BarrierWorkerState::new(), - command_ctx_queue: Default::default(), - completing_barrier: None, - creating_streaming_job_controls: Default::default(), - create_mview_tracker: Default::default(), - } - } - - fn recovery( - database_id: DatabaseId, - create_mview_tracker: CreateMviewProgressTracker, - state: BarrierWorkerState, - ) -> Self { - Self { - database_id, - state, - command_ctx_queue: Default::default(), - completing_barrier: None, - creating_streaming_job_controls: Default::default(), - create_mview_tracker, - } - } - - fn total_command_num(&self) -> usize { - self.command_ctx_queue.len() - + match &self.completing_barrier { - Some(_) => 1, - None => 0, - } - } - - /// Update the metrics of barrier nums. - fn update_barrier_nums_metrics(&self) { - let database_id_str = self.database_id.database_id.to_string(); - GLOBAL_META_METRICS - .in_flight_barrier_nums - .with_label_values(&[&database_id_str]) - .set( - self.command_ctx_queue - .values() - .filter(|x| x.state.is_inflight()) - .count() as i64, - ); - GLOBAL_META_METRICS - .all_barrier_nums - .with_label_values(&[&database_id_str]) - .set(self.total_command_num() as i64); - } - - fn jobs_to_merge( - &self, - ) -> Option> { - let mut table_ids_to_merge = HashMap::new(); - - for (table_id, creating_streaming_job) in &self.creating_streaming_job_controls { - if let Some(graph_info) = creating_streaming_job.should_merge_to_upstream() { - table_ids_to_merge.insert( - *table_id, - ( - creating_streaming_job.snapshot_backfill_info.clone(), - graph_info, - ), - ); - } - } - if table_ids_to_merge.is_empty() { - None - } else { - Some(table_ids_to_merge) - } - } - - /// Enqueue a barrier command - fn enqueue_command( - &mut self, - command_ctx: CommandContext, - notifiers: Vec, - node_to_collect: HashSet, - creating_jobs_to_wait: HashSet, - ) { - let timer = GLOBAL_META_METRICS.barrier_latency.start_timer(); - - if let Some((_, node)) = self.command_ctx_queue.last_key_value() { - assert_eq!( - command_ctx.barrier_info.prev_epoch.value(), - node.command_ctx.barrier_info.curr_epoch.value() - ); - } - - tracing::trace!( - prev_epoch = command_ctx.barrier_info.prev_epoch(), - ?creating_jobs_to_wait, - "enqueue command" - ); - self.command_ctx_queue.insert( - command_ctx.barrier_info.prev_epoch(), - EpochNode { - enqueue_time: timer, - state: BarrierEpochState { - node_to_collect, - resps: vec![], - creating_jobs_to_wait, - finished_jobs: HashMap::new(), - }, - command_ctx, - notifiers, - }, - ); - } - - /// Change the state of this `prev_epoch` to `Completed`. Return continuous nodes - /// with `Completed` starting from first node [`Completed`..`InFlight`) and remove them. - fn barrier_collected( - &mut self, - resp: BarrierCompleteResponse, - control_stream_manager: &mut ControlStreamManager, - ) -> MetaResult<()> { - let worker_id = resp.worker_id; - let prev_epoch = resp.epoch; - tracing::trace!( - worker_id, - prev_epoch, - partial_graph_id = resp.partial_graph_id, - "barrier collected" - ); - let (database_id, creating_job_id) = from_partial_graph_id(resp.partial_graph_id); - assert_eq!(database_id, self.database_id); - match creating_job_id { - None => { - if let Some(node) = self.command_ctx_queue.get_mut(&prev_epoch) { - assert!(node.state.node_to_collect.remove(&(worker_id as _))); - node.state.resps.push(resp); - } else { - panic!( - "collect barrier on non-existing barrier: {}, {}", - prev_epoch, worker_id - ); - } - } - Some(creating_job_id) => { - self.creating_streaming_job_controls - .get_mut(&creating_job_id) - .expect("should exist") - .collect(prev_epoch, worker_id as _, resp, control_stream_manager)?; - } - } - Ok(()) - } - - /// Pause inject barrier until True. - fn can_inject_barrier(&self, in_flight_barrier_nums: usize) -> bool { - let in_flight_not_full = self - .command_ctx_queue - .values() - .filter(|x| x.state.is_inflight()) - .count() - < in_flight_barrier_nums; - - // Whether some command requires pausing concurrent barrier. If so, it must be the last one. - let should_pause = self - .command_ctx_queue - .last_key_value() - .and_then(|(_, x)| { - x.command_ctx - .command - .as_ref() - .map(Command::should_pause_inject_barrier) - }) - .or(self - .completing_barrier - .map(|(_, should_pause)| should_pause)) - .unwrap_or(false); - debug_assert_eq!( - self.command_ctx_queue - .values() - .filter_map(|node| { - node.command_ctx - .command - .as_ref() - .map(Command::should_pause_inject_barrier) - }) - .chain( - self.completing_barrier - .map(|(_, should_pause)| should_pause) - .into_iter() - ) - .any(|should_pause| should_pause), - should_pause - ); - - in_flight_not_full && !should_pause - } -} - -impl GlobalBarrierWorker { - /// We need to make sure there are no changes when doing recovery - pub async fn clear_on_err(&mut self, err: &MetaError) { - // join spawned completing command to finish no matter it succeeds or not. - let is_err = match replace(&mut self.completing_task, CompletingTask::None) { - CompletingTask::None => false, - CompletingTask::Completing { - epochs_to_ack, - join_handle, - .. - } => { - info!("waiting for completing command to finish in recovery"); - match join_handle.await { - Err(e) => { - warn!(err = ?e.as_report(), "failed to join completing task"); - true - } - Ok(Err(e)) => { - warn!(err = ?e.as_report(), "failed to complete barrier during clear"); - true - } - Ok(Ok(hummock_version_stats)) => { - self.checkpoint_control - .ack_completed(BarrierCompleteOutput { - epochs_to_ack, - hummock_version_stats, - }); - false - } - } - } - CompletingTask::Err(_) => true, - }; - if !is_err { - // continue to finish the pending collected barrier. - while let Some(task) = self.checkpoint_control.next_complete_barrier_task(None) { - let epochs_to_ack = task.epochs_to_ack(); - match task - .complete_barrier(&*self.context, self.env.clone()) - .await - { - Ok(hummock_version_stats) => { - self.checkpoint_control - .ack_completed(BarrierCompleteOutput { - epochs_to_ack, - hummock_version_stats, - }); - } - Err(e) => { - error!( - err = ?e.as_report(), - "failed to complete barrier during recovery" - ); - break; - } - } - } - } - for (_, node) in self - .checkpoint_control - .databases - .values_mut() - .flat_map(|database| take(&mut database.command_ctx_queue)) - { - for notifier in node.notifiers { - notifier.notify_failed(err.clone()); - } - node.enqueue_time.observe_duration(); - } - self.checkpoint_control - .databases - .values_mut() - .for_each(|database| database.create_mview_tracker.abort_all()); - } -} - -impl DatabaseCheckpointControl { - /// Return the earliest command waiting on the `worker_id`. - fn barrier_wait_collect_from_worker(&self, worker_id: WorkerId) -> Option<&BarrierInfo> { - for epoch_node in self.command_ctx_queue.values() { - if epoch_node.state.node_to_collect.contains(&worker_id) { - return Some(&epoch_node.command_ctx.barrier_info); - } - } - // TODO: include barrier in creating jobs - None - } -} - -/// The state and message of this barrier, a node for concurrent checkpoint. -struct EpochNode { - /// Timer for recording barrier latency, taken after `complete_barriers`. - enqueue_time: HistogramTimer, - - /// Whether this barrier is in-flight or completed. - state: BarrierEpochState, - - /// Context of this command to generate barrier and do some post jobs. - command_ctx: CommandContext, - /// Notifiers of this barrier. - notifiers: Vec, -} - -#[derive(Debug)] -/// The state of barrier. -struct BarrierEpochState { - node_to_collect: HashSet, - - resps: Vec, - - creating_jobs_to_wait: HashSet, - - finished_jobs: HashMap)>, -} - -impl BarrierEpochState { - fn is_inflight(&self) -> bool { - !self.node_to_collect.is_empty() || !self.creating_jobs_to_wait.is_empty() - } -} - -enum CompletingTask { - None, - Completing { - #[expect(clippy::type_complexity)] - /// `database_id` -> (`Some(database_graph_committed_epoch)`, [(`creating_job_id`, `creating_job_committed_epoch`)]) - epochs_to_ack: HashMap, Vec<(TableId, u64)>)>, - - // The join handle of a spawned task that completes the barrier. - // The return value indicate whether there is some create streaming job command - // that has finished but not checkpointed. If there is any, we will force checkpoint on the next barrier - join_handle: JoinHandle>, - }, - #[expect(dead_code)] - Err(MetaError), -} - -impl GlobalBarrierWorker { - /// Create a new [`crate::barrier::GlobalBarrierWorker`]. - pub async fn new( - scheduled_barriers: schedule::ScheduledBarriers, - env: MetaSrvEnv, - metadata_manager: MetadataManager, - hummock_manager: HummockManagerRef, - source_manager: SourceManagerRef, - sink_manager: SinkCoordinatorManager, - scale_controller: ScaleControllerRef, - request_rx: mpsc::UnboundedReceiver, - ) -> Self { - let enable_recovery = env.opts.enable_recovery; - let in_flight_barrier_nums = env.opts.in_flight_barrier_nums; - - let active_streaming_nodes = - ActiveStreamingWorkerNodes::uninitialized(metadata_manager.clone()); - - let status = Arc::new(ArcSwap::new(Arc::new(BarrierManagerStatus::Starting))); - - let context = Arc::new(GlobalBarrierWorkerContextImpl { - scheduled_barriers, - status, - metadata_manager, - hummock_manager, - source_manager, - scale_controller, - env: env.clone(), - }); - - let control_stream_manager = ControlStreamManager::new(env.clone()); - - let checkpoint_frequency = env.system_params_reader().await.checkpoint_frequency() as _; - let interval = - Duration::from_millis(env.system_params_reader().await.barrier_interval_ms() as u64); - let periodic_barriers = PeriodicBarriers::new(interval, checkpoint_frequency); - tracing::info!( - "Starting barrier scheduler with: checkpoint_frequency={:?}", - checkpoint_frequency, - ); - - Self { - enable_recovery, - periodic_barriers, - in_flight_barrier_nums, - context, - env, - checkpoint_control: CheckpointControl { - databases: HashMap::new(), - hummock_version_stats: Default::default(), - }, - completing_task: CompletingTask::None, - request_rx, - active_streaming_nodes, - sink_manager, - control_stream_manager, - } - } - - pub fn start(self) -> (JoinHandle<()>, Sender<()>) { - let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); - let join_handle = tokio::spawn(async move { - self.run(shutdown_rx).await; - }); - - (join_handle, shutdown_tx) - } - - /// Check whether we should pause on bootstrap from the system parameter and reset it. - async fn take_pause_on_bootstrap(&mut self) -> MetaResult { - let paused = self - .env - .system_params_reader() - .await - .pause_on_next_bootstrap(); - if paused { - warn!( - "The cluster will bootstrap with all data sources paused as specified by the system parameter `{}`. \ - It will now be reset to `false`. \ - To resume the data sources, either restart the cluster again or use `risectl meta resume`.", - PAUSE_ON_NEXT_BOOTSTRAP_KEY - ); - self.env - .system_params_manager_impl_ref() - .set_param(PAUSE_ON_NEXT_BOOTSTRAP_KEY, Some("false".to_owned())) - .await?; - } - Ok(paused) - } - - /// Start an infinite loop to take scheduled barriers and send them. - async fn run(mut self, shutdown_rx: Receiver<()>) { - tracing::info!( - "Starting barrier manager with: enable_recovery={}, in_flight_barrier_nums={}", - self.enable_recovery, - self.in_flight_barrier_nums, - ); - - if !self.enable_recovery { - let job_exist = self - .context - .metadata_manager - .catalog_controller - .has_any_streaming_jobs() - .await - .unwrap(); - if job_exist { - panic!( - "Some streaming jobs already exist in meta, please start with recovery enabled \ - or clean up the metadata using `./risedev clean-data`" - ); - } - } - - { - // Bootstrap recovery. Here we simply trigger a recovery process to achieve the - // consistency. - // Even if there's no actor to recover, we still go through the recovery process to - // inject the first `Initial` barrier. - let span = tracing::info_span!("bootstrap_recovery"); - crate::telemetry::report_event( - risingwave_pb::telemetry::TelemetryEventStage::Recovery, - "normal_recovery", - 0, - None, - None, - None, - ); - - let paused = self.take_pause_on_bootstrap().await.unwrap_or(false); - let paused_reason = paused.then_some(PausedReason::Manual); - - self.recovery(paused_reason, None, RecoveryReason::Bootstrap) - .instrument(span) - .await; - } - - self.run_inner(shutdown_rx).await - } -} - -impl GlobalBarrierWorker { - async fn run_inner(mut self, mut shutdown_rx: Receiver<()>) { - let (local_notification_tx, mut local_notification_rx) = - tokio::sync::mpsc::unbounded_channel(); - self.env - .notification_manager() - .insert_local_sender(local_notification_tx) - .await; - - // Start the event loop. - loop { - tokio::select! { - biased; - - // Shutdown - _ = &mut shutdown_rx => { - tracing::info!("Barrier manager is stopped"); - break; - } - - request = self.request_rx.recv() => { - if let Some(request) = request { - match request { - BarrierManagerRequest::GetDdlProgress(result_tx) => { - let mut progress = HashMap::new(); - for database_checkpoint_control in self.checkpoint_control.databases.values() { - // Progress of normal backfill - progress.extend(database_checkpoint_control.create_mview_tracker.gen_ddl_progress()); - // Progress of snapshot backfill - for creating_job in database_checkpoint_control.creating_streaming_job_controls.values() { - progress.extend([(creating_job.info.table_fragments.table_id().table_id, creating_job.gen_ddl_progress())]); - } - } - if result_tx.send(progress).is_err() { - error!("failed to send get ddl progress"); - } - } - } - } else { - tracing::info!("end of request stream. meta node may be shutting down. Stop global barrier manager"); - return; - } - } - - changed_worker = self.active_streaming_nodes.changed() => { - #[cfg(debug_assertions)] - { - self.active_streaming_nodes.validate_change().await; - } - - info!(?changed_worker, "worker changed"); - - if let ActiveStreamingWorkerChange::Add(node) | ActiveStreamingWorkerChange::Update(node) = changed_worker { - self.control_stream_manager.add_worker(node, self.checkpoint_control.databases.values().flat_map(|database| &database.state.inflight_subscription_info), &*self.context).await; - } - } - - notification = local_notification_rx.recv() => { - let notification = notification.unwrap(); - match notification { - // Handle barrier interval and checkpoint frequency changes. - LocalNotification::SystemParamsChange(p) => { - self.periodic_barriers.set_min_interval(Duration::from_millis(p.barrier_interval_ms() as u64)); - self.periodic_barriers - .set_checkpoint_frequency(p.checkpoint_frequency() as usize) - }, - // Handle adhoc recovery triggered by user. - LocalNotification::AdhocRecovery => { - self.adhoc_recovery().await; - } - _ => {} - } - } - complete_result = self - .completing_task - .next_completed_barrier( - &mut self.periodic_barriers, - &mut self.checkpoint_control, - &mut self.control_stream_manager, - &self.context, - &self.env, - ) => { - match complete_result { - Ok(output) => { - self.checkpoint_control.ack_completed(output); - } - Err(e) => { - self.failure_recovery(e).await; - } - } - }, - (worker_id, resp_result) = self.control_stream_manager.next_collect_barrier_response() => { - if let Err(e) = resp_result.and_then(|resp| self.checkpoint_control.barrier_collected(resp, &mut self.control_stream_manager)) { - { - let mut err = None; - for database_checkpoint_control in self.checkpoint_control.databases.values() { - let failed_barrier = database_checkpoint_control.barrier_wait_collect_from_worker(worker_id as _); - if failed_barrier.is_some() - || database_checkpoint_control.state.inflight_graph_info.contains_worker(worker_id as _) - || database_checkpoint_control.creating_streaming_job_controls.values().any(|job| job.is_wait_on_worker(worker_id)) { - - err = Some((e, failed_barrier)); - break; - } - } - if let Some((e, failed_barrier)) = err { - let errors = self.control_stream_manager.collect_errors(worker_id, e).await; - let err = merge_node_rpc_errors("get error from control stream", errors); - if let Some(failed_barrier) = failed_barrier { - self.report_collect_failure(failed_barrier, &err); - } - self.failure_recovery(err).await; - } else { - warn!(worker_id, "no barrier to collect from worker, ignore err"); - } - } - } - } - new_barrier = self.periodic_barriers.next_barrier(&*self.context), - if self - .checkpoint_control - .can_inject_barrier(self.in_flight_barrier_nums) => { - if let Err(e) = self.checkpoint_control.handle_new_barrier(new_barrier, &mut self.control_stream_manager, &self.active_streaming_nodes) { - self.failure_recovery(e).await; - } - } - } - self.checkpoint_control.update_barrier_nums_metrics(); - } - } -} - -impl DatabaseCheckpointControl { - /// Handle the new barrier from the scheduled queue and inject it. - fn handle_new_barrier( - &mut self, - command: Option<(Command, Vec)>, - checkpoint: bool, - span: tracing::Span, - control_stream_manager: &mut ControlStreamManager, - active_streaming_nodes: &ActiveStreamingWorkerNodes, - hummock_version_stats: &HummockVersionStats, - curr_epoch: TracedEpoch, - ) -> MetaResult<()> { - let (mut command, mut notifiers) = if let Some((command, notifiers)) = command { - (Some(command), notifiers) - } else { - (None, vec![]) - }; - - if let Some(table_to_cancel) = command.as_ref().and_then(Command::table_to_cancel) - && self - .creating_streaming_job_controls - .contains_key(&table_to_cancel) - { - warn!( - table_id = table_to_cancel.table_id, - "ignore cancel command on creating streaming job" - ); - for notifier in notifiers { - notifier - .notify_start_failed(anyhow!("cannot cancel creating streaming job, the job will continue creating until created or recovery").into()); - } - return Ok(()); - } - - if let Some(Command::RescheduleFragment { .. }) = &command { - if !self.creating_streaming_job_controls.is_empty() { - warn!("ignore reschedule when creating streaming job with snapshot backfill"); - for notifier in notifiers { - notifier.notify_start_failed( - anyhow!( - "cannot reschedule when creating streaming job with snapshot backfill", - ) - .into(), - ); - } - return Ok(()); - } - } - - let Some(barrier_info) = - self.state - .next_barrier_info(command.as_ref(), checkpoint, curr_epoch) - else { - // skip the command when there is nothing to do with the barrier - for mut notifier in notifiers { - notifier.notify_started(); - notifier.notify_collected(); - } - return Ok(()); - }; - - // Insert newly added creating job - if let Some(Command::CreateStreamingJob { - job_type: CreateStreamingJobType::SnapshotBackfill(snapshot_backfill_info), - info, - }) = &command - { - if self.state.paused_reason().is_some() { - warn!("cannot create streaming job with snapshot backfill when paused"); - for notifier in notifiers { - notifier.notify_start_failed( - anyhow!("cannot create streaming job with snapshot backfill when paused",) - .into(), - ); - } - return Ok(()); - } - let mutation = command - .as_ref() - .expect("checked Some") - .to_mutation(None) - .expect("should have some mutation in `CreateStreamingJob` command"); - self.creating_streaming_job_controls.insert( - info.table_fragments.table_id(), - CreatingStreamingJobControl::new( - info.clone(), - snapshot_backfill_info.clone(), - barrier_info.prev_epoch(), - hummock_version_stats, - mutation, - ), - ); - } - - // Collect the jobs to finish - if let (BarrierKind::Checkpoint(_), None) = (&barrier_info.kind, &command) - && let Some(jobs_to_merge) = self.jobs_to_merge() - { - command = Some(Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge)); - } - - let command = command; - - let ( - pre_applied_graph_info, - pre_applied_subscription_info, - table_ids_to_commit, - jobs_to_wait, - prev_paused_reason, - ) = self.state.apply_command(command.as_ref()); - - // Tracing related stuff - barrier_info.prev_epoch.span().in_scope(|| { - tracing::info!(target: "rw_tracing", epoch = barrier_info.curr_epoch.value().0, "new barrier enqueued"); - }); - span.record("epoch", barrier_info.curr_epoch.value().0); - - for creating_job in &mut self.creating_streaming_job_controls.values_mut() { - creating_job.on_new_command(control_stream_manager, command.as_ref(), &barrier_info)?; - } - - let node_to_collect = match control_stream_manager.inject_command_ctx_barrier( - self.database_id, - command.as_ref(), - &barrier_info, - prev_paused_reason, - &pre_applied_graph_info, - &self.state.inflight_graph_info, - ) { - Ok(node_to_collect) => node_to_collect, - Err(err) => { - for notifier in notifiers { - notifier.notify_failed(err.clone()); - } - fail_point!("inject_barrier_err_success"); - return Err(err); - } - }; - - // Notify about the injection. - notifiers.iter_mut().for_each(|n| n.notify_started()); - - let command_ctx = CommandContext::new( - active_streaming_nodes.current().clone(), - barrier_info, - pre_applied_subscription_info, - table_ids_to_commit.clone(), - command, - span, - ); - - // Record the in-flight barrier. - self.enqueue_command(command_ctx, notifiers, node_to_collect, jobs_to_wait); - - Ok(()) - } -} - -impl GlobalBarrierWorker { - /// Set barrier manager status. - async fn failure_recovery(&mut self, err: MetaError) { - self.clear_on_err(&err).await; - - if self.enable_recovery { - let span = tracing::info_span!( - "failure_recovery", - error = %err.as_report(), - ); - - crate::telemetry::report_event( - risingwave_pb::telemetry::TelemetryEventStage::Recovery, - "failure_recovery", - 0, - None, - None, - None, - ); - - let reason = RecoveryReason::Failover(err.clone()); - - // No need to clean dirty tables for barrier recovery, - // The foreground stream job should cleanup their own tables. - self.recovery(None, Some(err), reason) - .instrument(span) - .await; - } else { - panic!( - "a streaming error occurred while recovery is disabled, aborting: {:?}", - err.as_report() - ); - } - } - - async fn adhoc_recovery(&mut self) { - let err = MetaErrorInner::AdhocRecovery.into(); - self.clear_on_err(&err).await; - - let span = tracing::info_span!( - "adhoc_recovery", - error = %err.as_report(), - ); - - crate::telemetry::report_event( - risingwave_pb::telemetry::TelemetryEventStage::Recovery, - "adhoc_recovery", - 0, - None, - None, - None, - ); - - // No need to clean dirty tables for barrier recovery, - // The foreground stream job should cleanup their own tables. - self.recovery(None, Some(err), RecoveryReason::Adhoc) - .instrument(span) - .await; - } -} - -#[derive(Default)] -pub struct CompleteBarrierTask { - commit_info: CommitEpochInfo, - finished_jobs: Vec, - notifiers: Vec, - /// `database_id` -> (Some((`command_ctx`, `enqueue_time`)), vec!((`creating_job_id`, `epoch`))) - #[expect(clippy::type_complexity)] - epoch_infos: HashMap< - DatabaseId, - ( - Option<(CommandContext, HistogramTimer)>, - Vec<(TableId, u64)>, - ), - >, -} - -impl CompleteBarrierTask { - #[expect(clippy::type_complexity)] - fn epochs_to_ack(&self) -> HashMap, Vec<(TableId, u64)>)> { - self.epoch_infos - .iter() - .map(|(database_id, (command_context, creating_job_epochs))| { - ( - *database_id, - ( - command_context - .as_ref() - .map(|(command, _)| command.barrier_info.prev_epoch.value().0), - creating_job_epochs.clone(), - ), - ) - }) - .collect() - } -} - -impl GlobalBarrierWorkerContextImpl { - fn set_status(&self, new_status: BarrierManagerStatus) { - self.status.store(Arc::new(new_status)); - } - - fn collect_creating_job_commit_epoch_info( - commit_info: &mut CommitEpochInfo, - epoch: u64, - resps: Vec, - tables_to_commit: impl Iterator, - is_first_time: bool, - ) { - let (sst_to_context, sstables, new_table_watermarks, old_value_sst) = - collect_resp_info(resps); - assert!(old_value_sst.is_empty()); - commit_info.sst_to_context.extend(sst_to_context); - commit_info.sstables.extend(sstables); - commit_info - .new_table_watermarks - .extend(new_table_watermarks); - let tables_to_commit: HashSet<_> = tables_to_commit.collect(); - tables_to_commit.iter().for_each(|table_id| { - commit_info - .tables_to_commit - .try_insert(*table_id, epoch) - .expect("non duplicate"); - }); - if is_first_time { - commit_info - .new_table_fragment_infos - .push(NewTableFragmentInfo { - table_ids: tables_to_commit, - }); - }; - } -} - -impl CompleteBarrierTask { - async fn complete_barrier( - self, - context: &impl GlobalBarrierWorkerContext, - env: MetaSrvEnv, - ) -> MetaResult { - let result: MetaResult = try { - let wait_commit_timer = GLOBAL_META_METRICS - .barrier_wait_commit_latency - .start_timer(); - let version_stats = context.commit_epoch(self.commit_info).await?; - for command_ctx in self - .epoch_infos - .values() - .flat_map(|(command, _)| command.as_ref().map(|(command, _)| command)) - { - context.post_collect_command(command_ctx).await?; - } - - wait_commit_timer.observe_duration(); - version_stats - }; - - let version_stats = { - let version_stats = match result { - Ok(version_stats) => version_stats, - Err(e) => { - for notifier in self.notifiers { - notifier.notify_collection_failed(e.clone()); - } - return Err(e); - } - }; - self.notifiers.into_iter().for_each(|notifier| { - notifier.notify_collected(); - }); - try_join_all( - self.finished_jobs - .into_iter() - .map(|finished_job| context.finish_creating_job(finished_job)), - ) - .await?; - for (command_ctx, enqueue_time) in self - .epoch_infos - .into_values() - .flat_map(|(command_context, _)| command_context) - { - let duration_sec = enqueue_time.stop_and_record(); - Self::report_complete_event(&env, duration_sec, &command_ctx); - GLOBAL_META_METRICS - .last_committed_barrier_time - .set(command_ctx.barrier_info.curr_epoch.value().as_unix_secs() as i64); - } - version_stats - }; - - Ok(version_stats) - } -} - -impl CreateMviewProgressTracker { - fn update_tracking_jobs<'a>( - &mut self, - info: Option<(&CreateStreamingJobCommandInfo, Option<&ReplaceTablePlan>)>, - create_mview_progress: impl IntoIterator, - version_stats: &HummockVersionStats, - ) { - { - { - // Save `finished_commands` for Create MVs. - let finished_commands = { - let mut commands = vec![]; - // Add the command to tracker. - if let Some((create_job_info, replace_table)) = info - && let Some(command) = - self.add(create_job_info, replace_table, version_stats) - { - // Those with no actors to track can be finished immediately. - commands.push(command); - } - // Update the progress of all commands. - for progress in create_mview_progress { - // Those with actors complete can be finished immediately. - if let Some(command) = self.update(progress, version_stats) { - tracing::trace!(?progress, "finish progress"); - commands.push(command); - } else { - tracing::trace!(?progress, "update progress"); - } - } - commands - }; - - for command in finished_commands { - self.stash_command_to_finish(command); - } - } - } - } -} - -impl CompleteBarrierTask { - fn report_complete_event(env: &MetaSrvEnv, duration_sec: f64, command_ctx: &CommandContext) { - // Record barrier latency in event log. - use risingwave_pb::meta::event_log; - let event = event_log::EventBarrierComplete { - prev_epoch: command_ctx.barrier_info.prev_epoch(), - cur_epoch: command_ctx.barrier_info.curr_epoch.value().0, - duration_sec, - command: command_ctx - .command - .as_ref() - .map(|command| command.to_string()) - .unwrap_or_else(|| "barrier".to_string()), - barrier_kind: command_ctx.barrier_info.kind.as_str_name().to_string(), - }; - env.event_log_manager_ref() - .add_event_logs(vec![event_log::Event::BarrierComplete(event)]); - } -} - -struct BarrierCompleteOutput { - #[expect(clippy::type_complexity)] - /// `database_id` -> (`Some(database_graph_committed_epoch)`, [(`creating_job_id`, `creating_job_committed_epoch`)]) - epochs_to_ack: HashMap, Vec<(TableId, u64)>)>, - hummock_version_stats: HummockVersionStats, -} - -impl DatabaseCheckpointControl { - /// return creating job table fragment id -> (backfill progress epoch , {`upstream_mv_table_id`}) - fn collect_backfill_pinned_upstream_log_epoch( - &self, - ) -> HashMap)> { - self.creating_streaming_job_controls - .iter() - .filter_map(|(table_id, creating_job)| { - creating_job - .pinned_upstream_log_epoch() - .map(|progress_epoch| { - ( - *table_id, - ( - progress_epoch, - creating_job - .snapshot_backfill_info - .upstream_mv_table_ids - .clone(), - ), - ) - }) - }) - .collect() - } - - fn next_complete_barrier_task( - &mut self, - task: &mut Option, - mut context: Option<(&mut PeriodicBarriers, &mut ControlStreamManager)>, - hummock_version_stats: &HummockVersionStats, - ) { - // `Vec::new` is a const fn, and do not have memory allocation, and therefore is lightweight enough - let mut creating_jobs_task = vec![]; - { - // `Vec::new` is a const fn, and do not have memory allocation, and therefore is lightweight enough - let mut finished_jobs = Vec::new(); - let min_upstream_inflight_barrier = self - .command_ctx_queue - .first_key_value() - .map(|(epoch, _)| *epoch); - for (table_id, job) in &mut self.creating_streaming_job_controls { - if let Some((epoch, resps, status)) = - job.start_completing(min_upstream_inflight_barrier) - { - let is_first_time = match status { - CompleteJobType::First => true, - CompleteJobType::Normal => false, - CompleteJobType::Finished => { - finished_jobs.push((*table_id, epoch, resps)); - continue; - } - }; - creating_jobs_task.push((*table_id, epoch, resps, is_first_time)); - } - } - if !finished_jobs.is_empty() - && let Some((_, control_stream_manager)) = &mut context - { - control_stream_manager.remove_partial_graph( - self.database_id, - finished_jobs - .iter() - .map(|(table_id, _, _)| *table_id) - .collect(), - ); - } - for (table_id, epoch, resps) in finished_jobs { - let epoch_state = &mut self - .command_ctx_queue - .get_mut(&epoch) - .expect("should exist") - .state; - assert!(epoch_state.creating_jobs_to_wait.remove(&table_id)); - debug!(epoch, ?table_id, "finish creating job"); - // It's safe to remove the creating job, because on CompleteJobType::Finished, - // all previous barriers have been collected and completed. - let creating_streaming_job = self - .creating_streaming_job_controls - .remove(&table_id) - .expect("should exist"); - assert!(creating_streaming_job.is_finished()); - assert!(epoch_state - .finished_jobs - .insert(table_id, (creating_streaming_job.info, resps)) - .is_none()); - } - } - assert!(self.completing_barrier.is_none()); - while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() - { - { - let (_, mut node) = self.command_ctx_queue.pop_first().expect("non-empty"); - assert!(node.state.creating_jobs_to_wait.is_empty()); - assert!(node.state.node_to_collect.is_empty()); - let mut finished_jobs = self - .create_mview_tracker - .apply_collected_command(&node, hummock_version_stats); - if !node.command_ctx.barrier_info.kind.is_checkpoint() { - assert!(finished_jobs.is_empty()); - node.notifiers.into_iter().for_each(|notifier| { - notifier.notify_collected(); - }); - if let Some((scheduled_barriers, _)) = &mut context - && self.create_mview_tracker.has_pending_finished_jobs() - && self - .command_ctx_queue - .values() - .all(|node| !node.command_ctx.barrier_info.kind.is_checkpoint()) - { - scheduled_barriers.force_checkpoint_in_next_barrier(); - } - continue; - } - node.state - .finished_jobs - .drain() - .for_each(|(_, (info, resps))| { - node.state.resps.extend(resps); - finished_jobs.push(TrackingJob::New(TrackingCommand { - info, - replace_table_info: None, - })); - }); - let task = task.get_or_insert_default(); - collect_commit_epoch_info( - &mut task.commit_info, - take(&mut node.state.resps), - &node.command_ctx, - self.collect_backfill_pinned_upstream_log_epoch(), - ); - self.completing_barrier = Some(( - node.command_ctx.barrier_info.prev_epoch(), - node.command_ctx - .command - .as_ref() - .map(|c| c.should_pause_inject_barrier()) - .unwrap_or(false), - )); - task.finished_jobs.extend(finished_jobs); - task.notifiers.extend(node.notifiers); - task.epoch_infos - .try_insert( - self.database_id, - (Some((node.command_ctx, node.enqueue_time)), vec![]), - ) - .expect("non duplicate"); - break; - } - } - if !creating_jobs_task.is_empty() { - let task = task.get_or_insert_default(); - for (table_id, epoch, resps, is_first_time) in creating_jobs_task { - GlobalBarrierWorkerContextImpl::collect_creating_job_commit_epoch_info( - &mut task.commit_info, - epoch, - resps, - self.creating_streaming_job_controls[&table_id] - .info - .table_fragments - .all_table_ids() - .map(TableId::new), - is_first_time, - ); - let (_, creating_job_epochs) = - task.epoch_infos.entry(self.database_id).or_default(); - creating_job_epochs.push((table_id, epoch)); - } - } - } -} - -impl CompletingTask { - pub(super) fn next_completed_barrier<'a>( - &'a mut self, - scheduled_barriers: &mut PeriodicBarriers, - checkpoint_control: &mut CheckpointControl, - control_stream_manager: &mut ControlStreamManager, - context: &Arc, - env: &MetaSrvEnv, - ) -> impl Future> + 'a { - // If there is no completing barrier, try to start completing the earliest barrier if - // it has been collected. - if let CompletingTask::None = self { - if let Some(task) = checkpoint_control - .next_complete_barrier_task(Some((scheduled_barriers, control_stream_manager))) - { - { - let epochs_to_ack = task.epochs_to_ack(); - let context = context.clone(); - let env = env.clone(); - let join_handle = - tokio::spawn(async move { task.complete_barrier(&*context, env).await }); - *self = CompletingTask::Completing { - epochs_to_ack, - join_handle, - }; - } - } - } - - self.next_completed_barrier_inner() - } - - async fn next_completed_barrier_inner(&mut self) -> MetaResult { - let CompletingTask::Completing { join_handle, .. } = self else { - return pending().await; - }; - - { - { - let join_result: MetaResult<_> = try { - join_handle - .await - .context("failed to join completing command")?? - }; - // It's important to reset the completing_command after await no matter the result is err - // or not, and otherwise the join handle will be polled again after ready. - let next_completing_command_status = if let Err(e) = &join_result { - CompletingTask::Err(e.clone()) - } else { - CompletingTask::None - }; - let completed_command = replace(self, next_completing_command_status); - let hummock_version_stats = join_result?; - - must_match!(completed_command, CompletingTask::Completing { - epochs_to_ack, - .. - } => { - Ok(BarrierCompleteOutput { - epochs_to_ack, - hummock_version_stats, - }) - }) - } - } - } -} - -impl DatabaseCheckpointControl { - fn ack_completed( - &mut self, - command_prev_epoch: Option, - creating_job_epochs: Vec<(TableId, u64)>, - ) { - { - assert_eq!( - self.completing_barrier - .take() - .map(|(prev_epoch, _)| prev_epoch), - command_prev_epoch - ); - for (table_id, epoch) in creating_job_epochs { - self.creating_streaming_job_controls - .get_mut(&table_id) - .expect("should exist") - .ack_completed(epoch) - } - } - } -} - -impl GlobalBarrierManager { - /// Check the status of barrier manager, return error if it is not `Running`. - pub fn check_status_running(&self) -> MetaResult<()> { - let status = self.status.load(); - match &**status { - BarrierManagerStatus::Starting - | BarrierManagerStatus::Recovering(RecoveryReason::Bootstrap) => { - bail!("The cluster is bootstrapping") - } - BarrierManagerStatus::Recovering(RecoveryReason::Failover(e)) => { - Err(anyhow::anyhow!(e.clone()).context("The cluster is recovering"))? - } - BarrierManagerStatus::Recovering(RecoveryReason::Adhoc) => { - bail!("The cluster is recovering-adhoc") - } - BarrierManagerStatus::Running => Ok(()), - } - } - - pub fn get_recovery_status(&self) -> PbRecoveryStatus { - (&**self.status.load()).into() - } -} - -impl GlobalBarrierWorkerContextImpl { - /// Resolve actor information from cluster, fragment manager and `ChangedTableId`. - /// We use `changed_table_id` to modify the actors to be sent or collected. Because these actor - /// will create or drop before this barrier flow through them. - async fn resolve_graph_info(&self) -> MetaResult> { - let all_actor_infos = self - .metadata_manager - .catalog_controller - .load_all_actors() - .await?; - - Ok(all_actor_infos - .into_iter() - .map(|(database_id, actor_infos)| (database_id, InflightDatabaseInfo::new(actor_infos))) - .collect()) - } -} - -pub struct GlobalBarrierManager { - status: Arc>, - hummock_manager: HummockManagerRef, - request_tx: mpsc::UnboundedSender, - metadata_manager: MetadataManager, -} - -pub type BarrierManagerRef = Arc; - -impl GlobalBarrierManager { - /// Serving `SHOW JOBS / SELECT * FROM rw_ddl_progress` - pub async fn get_ddl_progress(&self) -> MetaResult> { - let mut ddl_progress = { - let (tx, rx) = oneshot::channel(); - self.request_tx - .send(BarrierManagerRequest::GetDdlProgress(tx)) - .context("failed to send get ddl progress request")?; - rx.await.context("failed to receive get ddl progress")? - }; - // If not in tracker, means the first barrier not collected yet. - // In that case just return progress 0. - let mviews = self - .metadata_manager - .catalog_controller - .list_background_creating_mviews(true) - .await - .unwrap(); - for mview in mviews { - if let Entry::Vacant(e) = ddl_progress.entry(mview.table_id as _) { - e.insert(DdlProgress { - id: mview.table_id as u64, - statement: mview.definition, - progress: "0.0%".into(), - }); - } - } - - Ok(ddl_progress.into_values().collect()) - } - - pub async fn get_hummock_version_id(&self) -> HummockVersionId { - self.hummock_manager.get_version_id().await - } -} - -#[expect(clippy::type_complexity)] -fn collect_resp_info( - resps: Vec, -) -> ( - HashMap, - Vec, - HashMap, - Vec, -) { - let mut sst_to_worker: HashMap = HashMap::new(); - let mut synced_ssts: Vec = vec![]; - let mut table_watermarks = Vec::with_capacity(resps.len()); - let mut old_value_ssts = Vec::with_capacity(resps.len()); - - for resp in resps { - let ssts_iter = resp.synced_sstables.into_iter().map(|local_sst| { - let sst_info = local_sst.sst.expect("field not None"); - sst_to_worker.insert(sst_info.object_id, resp.worker_id); - LocalSstableInfo::new( - sst_info.into(), - from_prost_table_stats_map(local_sst.table_stats_map), - local_sst.created_at, - ) - }); - synced_ssts.extend(ssts_iter); - table_watermarks.push(resp.table_watermarks); - old_value_ssts.extend(resp.old_value_sstables.into_iter().map(|s| s.into())); - } - - ( - sst_to_worker, - synced_ssts, - merge_multiple_new_table_watermarks( - table_watermarks - .into_iter() - .map(|watermarks| { - watermarks - .into_iter() - .map(|(table_id, watermarks)| { - (TableId::new(table_id), TableWatermarks::from(&watermarks)) - }) - .collect() - }) - .collect_vec(), - ), - old_value_ssts, - ) -} - -fn collect_commit_epoch_info( - info: &mut CommitEpochInfo, - resps: Vec, - command_ctx: &CommandContext, - backfill_pinned_log_epoch: HashMap)>, -) { - let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) = - collect_resp_info(resps); - - let new_table_fragment_infos = if let Some(Command::CreateStreamingJob { info, job_type }) = - &command_ctx.command - && !matches!(job_type, CreateStreamingJobType::SnapshotBackfill(_)) - { - let table_fragments = &info.table_fragments; - let mut table_ids: HashSet<_> = table_fragments - .internal_table_ids() - .into_iter() - .map(TableId::new) - .collect(); - if let Some(mv_table_id) = table_fragments.mv_table_id() { - table_ids.insert(TableId::new(mv_table_id)); - } - - vec![NewTableFragmentInfo { table_ids }] - } else { - vec![] - }; - - let mut mv_log_store_truncate_epoch = HashMap::new(); - let mut update_truncate_epoch = - |table_id: TableId, truncate_epoch| match mv_log_store_truncate_epoch - .entry(table_id.table_id) - { - Entry::Occupied(mut entry) => { - let prev_truncate_epoch = entry.get_mut(); - if truncate_epoch < *prev_truncate_epoch { - *prev_truncate_epoch = truncate_epoch; - } - } - Entry::Vacant(entry) => { - entry.insert(truncate_epoch); - } - }; - for (mv_table_id, subscriptions) in &command_ctx.subscription_info.mv_depended_subscriptions { - if let Some(truncate_epoch) = subscriptions - .values() - .max() - .map(|max_retention| command_ctx.get_truncate_epoch(*max_retention).0) - { - update_truncate_epoch(*mv_table_id, truncate_epoch); - } - } - for (_, (backfill_epoch, upstream_mv_table_ids)) in backfill_pinned_log_epoch { - for mv_table_id in upstream_mv_table_ids { - update_truncate_epoch(mv_table_id, backfill_epoch); - } - } - - let table_new_change_log = build_table_change_log_delta( - old_value_ssts.into_iter(), - synced_ssts.iter().map(|sst| &sst.sst_info), - must_match!(&command_ctx.barrier_info.kind, BarrierKind::Checkpoint(epochs) => epochs), - mv_log_store_truncate_epoch.into_iter(), - ); - - let epoch = command_ctx.barrier_info.prev_epoch(); - for table_id in &command_ctx.table_ids_to_commit { - info.tables_to_commit - .try_insert(*table_id, epoch) - .expect("non duplicate"); - } - - info.sstables.extend(synced_ssts); - info.new_table_watermarks.extend(new_table_watermarks); - info.sst_to_context.extend(sst_to_context); - info.new_table_fragment_infos - .extend(new_table_fragment_infos); - info.change_log_delta.extend(table_new_change_log); -} diff --git a/src/meta/src/barrier/progress.rs b/src/meta/src/barrier/progress.rs index 9cb163d3448b5..17f044142b027 100644 --- a/src/meta/src/barrier/progress.rs +++ b/src/meta/src/barrier/progress.rs @@ -24,8 +24,9 @@ use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; +use crate::barrier::checkpoint::EpochNode; use crate::barrier::{ - Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, EpochNode, ReplaceTablePlan, + Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, ReplaceTablePlan, }; use crate::manager::{DdlType, MetadataManager}; use crate::model::{ActorId, BackfillUpstreamType, TableFragments}; @@ -375,6 +376,45 @@ impl CreateMviewProgressTracker { .collect() } + pub(super) fn update_tracking_jobs<'a>( + &mut self, + info: Option<(&CreateStreamingJobCommandInfo, Option<&ReplaceTablePlan>)>, + create_mview_progress: impl IntoIterator, + version_stats: &HummockVersionStats, + ) { + { + { + // Save `finished_commands` for Create MVs. + let finished_commands = { + let mut commands = vec![]; + // Add the command to tracker. + if let Some((create_job_info, replace_table)) = info + && let Some(command) = + self.add(create_job_info, replace_table, version_stats) + { + // Those with no actors to track can be finished immediately. + commands.push(command); + } + // Update the progress of all commands. + for progress in create_mview_progress { + // Those with actors complete can be finished immediately. + if let Some(command) = self.update(progress, version_stats) { + tracing::trace!(?progress, "finish progress"); + commands.push(command); + } else { + tracing::trace!(?progress, "update progress"); + } + } + commands + }; + + for command in finished_commands { + self.stash_command_to_finish(command); + } + } + } + } + /// Apply a collected epoch node command to the tracker /// Return the finished jobs when the barrier kind is `Checkpoint` pub(super) fn apply_collected_command( diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 7a67e48a8bafe..d1522de6ba0c2 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -43,10 +43,8 @@ use tokio_retry::strategy::ExponentialBackoff; use tracing::{error, info, warn}; use uuid::Uuid; -use super::{ - Command, GlobalBarrierWorker, GlobalBarrierWorkerContext, GlobalBarrierWorkerContextImpl, - InflightSubscriptionInfo, -}; +use super::{Command, InflightSubscriptionInfo}; +use crate::barrier::context::{GlobalBarrierWorkerContext, GlobalBarrierWorkerContextImpl}; use crate::barrier::info::{BarrierInfo, InflightDatabaseInfo}; use crate::controller::fragment::InflightFragmentInfo; use crate::manager::MetaSrvEnv; @@ -491,22 +489,6 @@ impl GlobalBarrierWorkerContextImpl { } } -impl GlobalBarrierWorker { - /// Send barrier-complete-rpc and wait for responses from all CNs - pub(super) fn report_collect_failure(&self, barrier_info: &BarrierInfo, error: &MetaError) { - // Record failure in event log. - use risingwave_pb::meta::event_log; - let event = event_log::EventCollectBarrierFail { - prev_epoch: barrier_info.prev_epoch(), - cur_epoch: barrier_info.curr_epoch.value().0, - error: error.to_report_string(), - }; - self.env - .event_log_manager_ref() - .add_event_logs(vec![event_log::Event::CollectBarrierFail(event)]); - } -} - pub(super) fn merge_node_rpc_errors( message: &str, errors: impl IntoIterator, diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index b3f7c717cf361..dd443ec47a221 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -29,12 +29,19 @@ use tokio::sync::{oneshot, watch}; use tokio::time::Interval; use super::notifier::Notifier; -use super::{Command, GlobalBarrierWorkerContext, NewBarrier, Scheduled}; +use super::{Command, Scheduled}; +use crate::barrier::context::GlobalBarrierWorkerContext; use crate::hummock::HummockManagerRef; use crate::model::ActorId; use crate::rpc::metrics::MetaMetrics; use crate::{MetaError, MetaResult}; +pub(super) struct NewBarrier { + pub command: Option<(DatabaseId, Command, Vec)>, + pub span: tracing::Span, + pub checkpoint: bool, +} + /// A queue for scheduling barriers. /// /// We manually implement one here instead of using channels since we may need to update the front @@ -391,6 +398,13 @@ impl ScheduledBarriers { } impl ScheduledBarriers { + /// Pre buffered drop and cancel command, return true if any. + pub(super) fn pre_apply_drop_cancel(&self) -> bool { + let (dropped_actors, cancelled) = self.pre_apply_drop_cancel_scheduled(); + + !dropped_actors.is_empty() || !cancelled.is_empty() + } + /// Mark command scheduler as blocked and abort all queued scheduled command and notify with /// specific reason. pub(super) fn abort_and_mark_blocked(&self, reason: impl Into + Copy) { diff --git a/src/meta/src/barrier/utils.rs b/src/meta/src/barrier/utils.rs new file mode 100644 index 0000000000000..dfdc2a6ab7e88 --- /dev/null +++ b/src/meta/src/barrier/utils.rs @@ -0,0 +1,191 @@ +// 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::hash_map::Entry; +use std::collections::{HashMap, HashSet}; + +use itertools::Itertools; +use risingwave_common::catalog::TableId; +use risingwave_common::must_match; +use risingwave_hummock_sdk::change_log::build_table_change_log_delta; +use risingwave_hummock_sdk::sstable_info::SstableInfo; +use risingwave_hummock_sdk::table_stats::from_prost_table_stats_map; +use risingwave_hummock_sdk::table_watermark::{ + merge_multiple_new_table_watermarks, TableWatermarks, +}; +use risingwave_hummock_sdk::{HummockSstableObjectId, LocalSstableInfo}; +use risingwave_pb::stream_service::BarrierCompleteResponse; + +use crate::barrier::command::CommandContext; +use crate::barrier::{BarrierKind, Command, CreateStreamingJobType}; +use crate::hummock::{CommitEpochInfo, NewTableFragmentInfo}; + +#[expect(clippy::type_complexity)] +pub(super) fn collect_resp_info( + resps: Vec, +) -> ( + HashMap, + Vec, + HashMap, + Vec, +) { + let mut sst_to_worker: HashMap = HashMap::new(); + let mut synced_ssts: Vec = vec![]; + let mut table_watermarks = Vec::with_capacity(resps.len()); + let mut old_value_ssts = Vec::with_capacity(resps.len()); + + for resp in resps { + let ssts_iter = resp.synced_sstables.into_iter().map(|local_sst| { + let sst_info = local_sst.sst.expect("field not None"); + sst_to_worker.insert(sst_info.object_id, resp.worker_id); + LocalSstableInfo::new( + sst_info.into(), + from_prost_table_stats_map(local_sst.table_stats_map), + local_sst.created_at, + ) + }); + synced_ssts.extend(ssts_iter); + table_watermarks.push(resp.table_watermarks); + old_value_ssts.extend(resp.old_value_sstables.into_iter().map(|s| s.into())); + } + + ( + sst_to_worker, + synced_ssts, + merge_multiple_new_table_watermarks( + table_watermarks + .into_iter() + .map(|watermarks| { + watermarks + .into_iter() + .map(|(table_id, watermarks)| { + (TableId::new(table_id), TableWatermarks::from(&watermarks)) + }) + .collect() + }) + .collect_vec(), + ), + old_value_ssts, + ) +} + +pub(super) fn collect_commit_epoch_info( + info: &mut CommitEpochInfo, + resps: Vec, + command_ctx: &CommandContext, + backfill_pinned_log_epoch: HashMap)>, +) { + let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) = + collect_resp_info(resps); + + let new_table_fragment_infos = if let Some(Command::CreateStreamingJob { info, job_type }) = + &command_ctx.command + && !matches!(job_type, CreateStreamingJobType::SnapshotBackfill(_)) + { + let table_fragments = &info.table_fragments; + let mut table_ids: HashSet<_> = table_fragments + .internal_table_ids() + .into_iter() + .map(TableId::new) + .collect(); + if let Some(mv_table_id) = table_fragments.mv_table_id() { + table_ids.insert(TableId::new(mv_table_id)); + } + + vec![NewTableFragmentInfo { table_ids }] + } else { + vec![] + }; + + let mut mv_log_store_truncate_epoch = HashMap::new(); + let mut update_truncate_epoch = + |table_id: TableId, truncate_epoch| match mv_log_store_truncate_epoch + .entry(table_id.table_id) + { + Entry::Occupied(mut entry) => { + let prev_truncate_epoch = entry.get_mut(); + if truncate_epoch < *prev_truncate_epoch { + *prev_truncate_epoch = truncate_epoch; + } + } + Entry::Vacant(entry) => { + entry.insert(truncate_epoch); + } + }; + for (mv_table_id, subscriptions) in &command_ctx.subscription_info.mv_depended_subscriptions { + if let Some(truncate_epoch) = subscriptions + .values() + .max() + .map(|max_retention| command_ctx.get_truncate_epoch(*max_retention).0) + { + update_truncate_epoch(*mv_table_id, truncate_epoch); + } + } + for (_, (backfill_epoch, upstream_mv_table_ids)) in backfill_pinned_log_epoch { + for mv_table_id in upstream_mv_table_ids { + update_truncate_epoch(mv_table_id, backfill_epoch); + } + } + + let table_new_change_log = build_table_change_log_delta( + old_value_ssts.into_iter(), + synced_ssts.iter().map(|sst| &sst.sst_info), + must_match!(&command_ctx.barrier_info.kind, BarrierKind::Checkpoint(epochs) => epochs), + mv_log_store_truncate_epoch.into_iter(), + ); + + let epoch = command_ctx.barrier_info.prev_epoch(); + for table_id in &command_ctx.table_ids_to_commit { + info.tables_to_commit + .try_insert(*table_id, epoch) + .expect("non duplicate"); + } + + info.sstables.extend(synced_ssts); + info.new_table_watermarks.extend(new_table_watermarks); + info.sst_to_context.extend(sst_to_context); + info.new_table_fragment_infos + .extend(new_table_fragment_infos); + info.change_log_delta.extend(table_new_change_log); +} + +pub(super) fn collect_creating_job_commit_epoch_info( + commit_info: &mut CommitEpochInfo, + epoch: u64, + resps: Vec, + tables_to_commit: impl Iterator, + is_first_time: bool, +) { + let (sst_to_context, sstables, new_table_watermarks, old_value_sst) = collect_resp_info(resps); + assert!(old_value_sst.is_empty()); + commit_info.sst_to_context.extend(sst_to_context); + commit_info.sstables.extend(sstables); + commit_info + .new_table_watermarks + .extend(new_table_watermarks); + let tables_to_commit: HashSet<_> = tables_to_commit.collect(); + tables_to_commit.iter().for_each(|table_id| { + commit_info + .tables_to_commit + .try_insert(*table_id, epoch) + .expect("non duplicate"); + }); + if is_first_time { + commit_info + .new_table_fragment_infos + .push(NewTableFragmentInfo { + table_ids: tables_to_commit, + }); + }; +} diff --git a/src/meta/src/barrier/worker.rs b/src/meta/src/barrier/worker.rs new file mode 100644 index 0000000000000..4df88d92150db --- /dev/null +++ b/src/meta/src/barrier/worker.rs @@ -0,0 +1,760 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::collections::HashMap; +use std::mem::{replace, take}; +use std::sync::Arc; +use std::time::Duration; + +use arc_swap::ArcSwap; +use itertools::Itertools; +use risingwave_common::system_param::reader::SystemParamsRead; +use risingwave_common::system_param::PAUSE_ON_NEXT_BOOTSTRAP_KEY; +use risingwave_common::util::epoch::Epoch; +use risingwave_meta_model::WorkerId; +use risingwave_pb::meta::subscribe_response::{Info, Operation}; +use risingwave_pb::meta::{PausedReason, Recovery}; +use risingwave_pb::stream_plan::barrier_mutation::Mutation; +use risingwave_pb::stream_plan::AddMutation; +use thiserror_ext::AsReport; +use tokio::sync::mpsc; +use tokio::sync::oneshot::{Receiver, Sender}; +use tokio::task::JoinHandle; +use tokio::time::Instant; +use tokio_retry::strategy::{jitter, ExponentialBackoff}; +use tracing::{debug, error, info, warn, Instrument}; + +use crate::barrier::checkpoint::{ + BarrierWorkerState, CheckpointControl, DatabaseCheckpointControl, +}; +use crate::barrier::complete_task::{BarrierCompleteOutput, CompletingTask}; +use crate::barrier::context::{GlobalBarrierWorkerContext, GlobalBarrierWorkerContextImpl}; +use crate::barrier::info::BarrierInfo; +use crate::barrier::progress::CreateMviewProgressTracker; +use crate::barrier::rpc::{merge_node_rpc_errors, ControlStreamManager}; +use crate::barrier::schedule::PeriodicBarriers; +use crate::barrier::{ + schedule, BarrierKind, BarrierManagerRequest, BarrierManagerStatus, + BarrierWorkerRuntimeInfoSnapshot, RecoveryReason, TracedEpoch, +}; +use crate::error::MetaErrorInner; +use crate::hummock::HummockManagerRef; +use crate::manager::sink_coordination::SinkCoordinatorManager; +use crate::manager::{ + ActiveStreamingWorkerChange, ActiveStreamingWorkerNodes, LocalNotification, MetaSrvEnv, + MetadataManager, +}; +use crate::model::ActorId; +use crate::rpc::metrics::GLOBAL_META_METRICS; +use crate::stream::{build_actor_connector_splits, ScaleControllerRef, SourceManagerRef}; +use crate::{MetaError, MetaResult}; + +/// [`crate::barrier::GlobalBarrierWorker`] sends barriers to all registered compute nodes and +/// collect them, with monotonic increasing epoch numbers. On compute nodes, `LocalBarrierManager` +/// in `risingwave_stream` crate will serve these requests and dispatch them to source actors. +/// +/// Configuration change in our system is achieved by the mutation in the barrier. Thus, +/// [`crate::barrier::GlobalBarrierWorker`] provides a set of interfaces like a state machine, +/// accepting [`Command`] that carries info to build `Mutation`. To keep the consistency between +/// barrier manager and meta store, some actions like "drop materialized view" or "create mv on mv" +/// must be done in barrier manager transactional using [`Command`]. +pub(super) struct GlobalBarrierWorker { + /// Enable recovery or not when failover. + enable_recovery: bool, + + /// The queue of scheduled barriers. + periodic_barriers: PeriodicBarriers, + + /// The max barrier nums in flight + in_flight_barrier_nums: usize, + + pub(super) context: Arc, + + env: MetaSrvEnv, + + checkpoint_control: CheckpointControl, + + /// Command that has been collected but is still completing. + /// The join handle of the completing future is stored. + completing_task: CompletingTask, + + request_rx: mpsc::UnboundedReceiver, + + active_streaming_nodes: ActiveStreamingWorkerNodes, + + sink_manager: SinkCoordinatorManager, + + control_stream_manager: ControlStreamManager, +} + +impl GlobalBarrierWorker { + /// Create a new [`crate::barrier::GlobalBarrierWorker`]. + pub async fn new( + scheduled_barriers: schedule::ScheduledBarriers, + env: MetaSrvEnv, + metadata_manager: MetadataManager, + hummock_manager: HummockManagerRef, + source_manager: SourceManagerRef, + sink_manager: SinkCoordinatorManager, + scale_controller: ScaleControllerRef, + request_rx: mpsc::UnboundedReceiver, + ) -> Self { + let enable_recovery = env.opts.enable_recovery; + let in_flight_barrier_nums = env.opts.in_flight_barrier_nums; + + let active_streaming_nodes = + ActiveStreamingWorkerNodes::uninitialized(metadata_manager.clone()); + + let status = Arc::new(ArcSwap::new(Arc::new(BarrierManagerStatus::Starting))); + + let context = Arc::new(GlobalBarrierWorkerContextImpl { + scheduled_barriers, + status, + metadata_manager, + hummock_manager, + source_manager, + scale_controller, + env: env.clone(), + }); + + let control_stream_manager = ControlStreamManager::new(env.clone()); + + let checkpoint_frequency = env.system_params_reader().await.checkpoint_frequency() as _; + let interval = + Duration::from_millis(env.system_params_reader().await.barrier_interval_ms() as u64); + let periodic_barriers = PeriodicBarriers::new(interval, checkpoint_frequency); + tracing::info!( + "Starting barrier scheduler with: checkpoint_frequency={:?}", + checkpoint_frequency, + ); + + Self { + enable_recovery, + periodic_barriers, + in_flight_barrier_nums, + context, + env, + checkpoint_control: CheckpointControl::default(), + completing_task: CompletingTask::None, + request_rx, + active_streaming_nodes, + sink_manager, + control_stream_manager, + } + } + + pub fn start(self) -> (JoinHandle<()>, Sender<()>) { + let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); + let join_handle = tokio::spawn(async move { + self.run(shutdown_rx).await; + }); + + (join_handle, shutdown_tx) + } + + /// Check whether we should pause on bootstrap from the system parameter and reset it. + async fn take_pause_on_bootstrap(&mut self) -> MetaResult { + let paused = self + .env + .system_params_reader() + .await + .pause_on_next_bootstrap(); + if paused { + warn!( + "The cluster will bootstrap with all data sources paused as specified by the system parameter `{}`. \ + It will now be reset to `false`. \ + To resume the data sources, either restart the cluster again or use `risectl meta resume`.", + PAUSE_ON_NEXT_BOOTSTRAP_KEY + ); + self.env + .system_params_manager_impl_ref() + .set_param(PAUSE_ON_NEXT_BOOTSTRAP_KEY, Some("false".to_owned())) + .await?; + } + Ok(paused) + } + + /// Start an infinite loop to take scheduled barriers and send them. + async fn run(mut self, shutdown_rx: Receiver<()>) { + tracing::info!( + "Starting barrier manager with: enable_recovery={}, in_flight_barrier_nums={}", + self.enable_recovery, + self.in_flight_barrier_nums, + ); + + if !self.enable_recovery { + let job_exist = self + .context + .metadata_manager + .catalog_controller + .has_any_streaming_jobs() + .await + .unwrap(); + if job_exist { + panic!( + "Some streaming jobs already exist in meta, please start with recovery enabled \ + or clean up the metadata using `./risedev clean-data`" + ); + } + } + + { + // Bootstrap recovery. Here we simply trigger a recovery process to achieve the + // consistency. + // Even if there's no actor to recover, we still go through the recovery process to + // inject the first `Initial` barrier. + let span = tracing::info_span!("bootstrap_recovery"); + crate::telemetry::report_event( + risingwave_pb::telemetry::TelemetryEventStage::Recovery, + "normal_recovery", + 0, + None, + None, + None, + ); + + let paused = self.take_pause_on_bootstrap().await.unwrap_or(false); + let paused_reason = paused.then_some(PausedReason::Manual); + + self.recovery(paused_reason, None, RecoveryReason::Bootstrap) + .instrument(span) + .await; + } + + self.run_inner(shutdown_rx).await + } + + async fn run_inner(mut self, mut shutdown_rx: Receiver<()>) { + let (local_notification_tx, mut local_notification_rx) = + tokio::sync::mpsc::unbounded_channel(); + self.env + .notification_manager() + .insert_local_sender(local_notification_tx) + .await; + + // Start the event loop. + loop { + tokio::select! { + biased; + + // Shutdown + _ = &mut shutdown_rx => { + tracing::info!("Barrier manager is stopped"); + break; + } + + request = self.request_rx.recv() => { + if let Some(request) = request { + match request { + BarrierManagerRequest::GetDdlProgress(result_tx) => { + let mut progress = HashMap::new(); + for database_checkpoint_control in self.checkpoint_control.databases.values() { + // Progress of normal backfill + progress.extend(database_checkpoint_control.create_mview_tracker.gen_ddl_progress()); + // Progress of snapshot backfill + for creating_job in database_checkpoint_control.creating_streaming_job_controls.values() { + progress.extend([(creating_job.info.table_fragments.table_id().table_id, creating_job.gen_ddl_progress())]); + } + } + if result_tx.send(progress).is_err() { + error!("failed to send get ddl progress"); + } + } + } + } else { + tracing::info!("end of request stream. meta node may be shutting down. Stop global barrier manager"); + return; + } + } + + changed_worker = self.active_streaming_nodes.changed() => { + #[cfg(debug_assertions)] + { + self.active_streaming_nodes.validate_change().await; + } + + info!(?changed_worker, "worker changed"); + + if let ActiveStreamingWorkerChange::Add(node) | ActiveStreamingWorkerChange::Update(node) = changed_worker { + self.control_stream_manager.add_worker(node, self.checkpoint_control.databases.values().flat_map(|database| &database.state.inflight_subscription_info), &*self.context).await; + } + } + + notification = local_notification_rx.recv() => { + let notification = notification.unwrap(); + match notification { + // Handle barrier interval and checkpoint frequency changes. + LocalNotification::SystemParamsChange(p) => { + self.periodic_barriers.set_min_interval(Duration::from_millis(p.barrier_interval_ms() as u64)); + self.periodic_barriers + .set_checkpoint_frequency(p.checkpoint_frequency() as usize) + }, + // Handle adhoc recovery triggered by user. + LocalNotification::AdhocRecovery => { + self.adhoc_recovery().await; + } + _ => {} + } + } + complete_result = self + .completing_task + .next_completed_barrier( + &mut self.periodic_barriers, + &mut self.checkpoint_control, + &mut self.control_stream_manager, + &self.context, + &self.env, + ) => { + match complete_result { + Ok(output) => { + self.checkpoint_control.ack_completed(output); + } + Err(e) => { + self.failure_recovery(e).await; + } + } + }, + (worker_id, resp_result) = self.control_stream_manager.next_collect_barrier_response() => { + if let Err(e) = resp_result.and_then(|resp| self.checkpoint_control.barrier_collected(resp, &mut self.control_stream_manager)) { + { + let mut err = None; + for database_checkpoint_control in self.checkpoint_control.databases.values() { + let failed_barrier = database_checkpoint_control.barrier_wait_collect_from_worker(worker_id as _); + if failed_barrier.is_some() + || database_checkpoint_control.state.inflight_graph_info.contains_worker(worker_id as _) + || database_checkpoint_control.creating_streaming_job_controls.values().any(|job| job.is_wait_on_worker(worker_id)) { + + err = Some((e, failed_barrier)); + break; + } + } + if let Some((e, failed_barrier)) = err { + let errors = self.control_stream_manager.collect_errors(worker_id, e).await; + let err = merge_node_rpc_errors("get error from control stream", errors); + if let Some(failed_barrier) = failed_barrier { + self.report_collect_failure(failed_barrier, &err); + } + self.failure_recovery(err).await; + } else { + warn!(worker_id, "no barrier to collect from worker, ignore err"); + } + } + } + } + new_barrier = self.periodic_barriers.next_barrier(&*self.context), + if self + .checkpoint_control + .can_inject_barrier(self.in_flight_barrier_nums) => { + if let Err(e) = self.checkpoint_control.handle_new_barrier(new_barrier, &mut self.control_stream_manager, &self.active_streaming_nodes) { + self.failure_recovery(e).await; + } + } + } + self.checkpoint_control.update_barrier_nums_metrics(); + } + } +} + +impl GlobalBarrierWorker { + /// We need to make sure there are no changes when doing recovery + pub async fn clear_on_err(&mut self, err: &MetaError) { + // join spawned completing command to finish no matter it succeeds or not. + let is_err = match replace(&mut self.completing_task, CompletingTask::None) { + CompletingTask::None => false, + CompletingTask::Completing { + epochs_to_ack, + join_handle, + .. + } => { + info!("waiting for completing command to finish in recovery"); + match join_handle.await { + Err(e) => { + warn!(err = ?e.as_report(), "failed to join completing task"); + true + } + Ok(Err(e)) => { + warn!(err = ?e.as_report(), "failed to complete barrier during clear"); + true + } + Ok(Ok(hummock_version_stats)) => { + self.checkpoint_control + .ack_completed(BarrierCompleteOutput { + epochs_to_ack, + hummock_version_stats, + }); + false + } + } + } + CompletingTask::Err(_) => true, + }; + if !is_err { + // continue to finish the pending collected barrier. + while let Some(task) = self.checkpoint_control.next_complete_barrier_task(None) { + let epochs_to_ack = task.epochs_to_ack(); + match task + .complete_barrier(&*self.context, self.env.clone()) + .await + { + Ok(hummock_version_stats) => { + self.checkpoint_control + .ack_completed(BarrierCompleteOutput { + epochs_to_ack, + hummock_version_stats, + }); + } + Err(e) => { + error!( + err = ?e.as_report(), + "failed to complete barrier during recovery" + ); + break; + } + } + } + } + for (_, node) in self + .checkpoint_control + .databases + .values_mut() + .flat_map(|database| take(&mut database.command_ctx_queue)) + { + for notifier in node.notifiers { + notifier.notify_failed(err.clone()); + } + node.enqueue_time.observe_duration(); + } + self.checkpoint_control + .databases + .values_mut() + .for_each(|database| database.create_mview_tracker.abort_all()); + } +} + +impl GlobalBarrierWorker { + /// Set barrier manager status. + async fn failure_recovery(&mut self, err: MetaError) { + self.clear_on_err(&err).await; + + if self.enable_recovery { + let span = tracing::info_span!( + "failure_recovery", + error = %err.as_report(), + ); + + crate::telemetry::report_event( + risingwave_pb::telemetry::TelemetryEventStage::Recovery, + "failure_recovery", + 0, + None, + None, + None, + ); + + let reason = RecoveryReason::Failover(err.clone()); + + // No need to clean dirty tables for barrier recovery, + // The foreground stream job should cleanup their own tables. + self.recovery(None, Some(err), reason) + .instrument(span) + .await; + } else { + panic!( + "a streaming error occurred while recovery is disabled, aborting: {:?}", + err.as_report() + ); + } + } + + async fn adhoc_recovery(&mut self) { + let err = MetaErrorInner::AdhocRecovery.into(); + self.clear_on_err(&err).await; + + let span = tracing::info_span!( + "adhoc_recovery", + error = %err.as_report(), + ); + + crate::telemetry::report_event( + risingwave_pb::telemetry::TelemetryEventStage::Recovery, + "adhoc_recovery", + 0, + None, + None, + None, + ); + + // No need to clean dirty tables for barrier recovery, + // The foreground stream job should cleanup their own tables. + self.recovery(None, Some(err), RecoveryReason::Adhoc) + .instrument(span) + .await; + } +} + +impl GlobalBarrierWorker { + /// Send barrier-complete-rpc and wait for responses from all CNs + pub(super) fn report_collect_failure(&self, barrier_info: &BarrierInfo, error: &MetaError) { + // Record failure in event log. + use risingwave_pb::meta::event_log; + let event = event_log::EventCollectBarrierFail { + prev_epoch: barrier_info.prev_epoch(), + cur_epoch: barrier_info.curr_epoch.value().0, + error: error.to_report_string(), + }; + self.env + .event_log_manager_ref() + .add_event_logs(vec![event_log::Event::CollectBarrierFail(event)]); + } +} + +impl GlobalBarrierWorker { + // Retry base interval in milliseconds. + const RECOVERY_RETRY_BASE_INTERVAL: u64 = 20; + // Retry max interval. + const RECOVERY_RETRY_MAX_INTERVAL: Duration = Duration::from_secs(5); + + #[inline(always)] + /// Initialize a retry strategy for operation in recovery. + fn get_retry_strategy() -> impl Iterator { + ExponentialBackoff::from_millis(Self::RECOVERY_RETRY_BASE_INTERVAL) + .max_delay(Self::RECOVERY_RETRY_MAX_INTERVAL) + .map(jitter) + } +} + +impl GlobalBarrierWorker { + /// Recovery the whole cluster from the latest epoch. + /// + /// If `paused_reason` is `Some`, all data sources (including connectors and DMLs) will be + /// immediately paused after recovery, until the user manually resume them either by restarting + /// the cluster or `risectl` command. Used for debugging purpose. + /// + /// Returns the new state of the barrier manager after recovery. + pub async fn recovery( + &mut self, + paused_reason: Option, + err: Option, + recovery_reason: RecoveryReason, + ) { + self.context.abort_and_mark_blocked(recovery_reason); + // Clear all control streams to release resources (connections to compute nodes) first. + self.control_stream_manager.clear(); + + self.recovery_inner(paused_reason, err).await; + self.context.mark_ready(); + } + + async fn recovery_inner( + &mut self, + paused_reason: Option, + err: Option, + ) { + tracing::info!("recovery start!"); + let retry_strategy = Self::get_retry_strategy(); + + // We take retry into consideration because this is the latency user sees for a cluster to + // get recovered. + let recovery_timer = GLOBAL_META_METRICS.recovery_latency.start_timer(); + + let new_state = tokio_retry::Retry::spawn(retry_strategy, || async { + if let Some(err) = &err { + self.context.notify_creating_job_failed(err).await; + }; + let runtime_info_snapshot = self + .context + .reload_runtime_info() + .await?; + runtime_info_snapshot.validate().inspect_err(|e| { + warn!(err = ?e.as_report(), ?runtime_info_snapshot, "reloaded runtime info failed to validate"); + })?; + let BarrierWorkerRuntimeInfoSnapshot { + active_streaming_nodes, + database_fragment_infos, + mut state_table_committed_epochs, + mut subscription_infos, + mut stream_actors, + mut source_splits, + mut background_jobs, + hummock_version_stats, + } = runtime_info_snapshot; + + self.sink_manager.reset().await; + + let mut control_stream_manager = ControlStreamManager::new(self.env.clone()); + let reset_start_time = Instant::now(); + control_stream_manager + .reset( + subscription_infos.values(), + active_streaming_nodes.current(), + &*self.context, + ) + .await + .inspect_err(|err| { + warn!(error = %err.as_report(), "reset compute nodes failed"); + })?; + info!(elapsed=?reset_start_time.elapsed(), "control stream reset"); + + let mut databases = HashMap::new(); + + let recovery_result: MetaResult<_> = try { + for (database_id, info) in database_fragment_infos { + let source_split_assignments = info + .fragment_infos() + .flat_map(|info| info.actors.keys()) + .filter_map(|actor_id| { + let actor_id = *actor_id as ActorId; + source_splits + .remove(&actor_id) + .map(|splits| (actor_id, splits)) + }) + .collect(); + let mutation = Mutation::Add(AddMutation { + // Actors built during recovery is not treated as newly added actors. + actor_dispatchers: Default::default(), + added_actors: Default::default(), + actor_splits: build_actor_connector_splits(&source_split_assignments), + pause: paused_reason.is_some(), + subscriptions_to_add: Default::default(), + }); + + let new_epoch = { + let mut epochs = info.existing_table_ids().map(|table_id| { + ( + table_id, + state_table_committed_epochs + .remove(&table_id) + .expect("should exist"), + ) + }); + let (first_table_id, prev_epoch) = epochs.next().expect("non-empty"); + for (table_id, epoch) in epochs { + assert_eq!( + prev_epoch, epoch, + "{} has different committed epoch to {}", + first_table_id, table_id + ); + } + let prev_epoch = TracedEpoch::new(Epoch(prev_epoch)); + // Use a different `curr_epoch` for each recovery attempt. + let curr_epoch = prev_epoch.next(); + let barrier_info = BarrierInfo { + prev_epoch, + curr_epoch, + kind: BarrierKind::Initial, + }; + + let mut node_actors: HashMap<_, Vec<_>> = HashMap::new(); + for (actor_id, worker_id) in + info.fragment_infos().flat_map(|info| info.actors.iter()) + { + let worker_id = *worker_id as WorkerId; + let actor_id = *actor_id as ActorId; + let stream_actor = + stream_actors.remove(&actor_id).expect("should exist"); + node_actors.entry(worker_id).or_default().push(stream_actor); + } + + let mut node_to_collect = control_stream_manager.inject_barrier( + database_id, + None, + Some(mutation), + &barrier_info, + info.fragment_infos(), + info.fragment_infos(), + Some(node_actors), + vec![], + vec![], + )?; + debug!(?node_to_collect, "inject initial barrier"); + while !node_to_collect.is_empty() { + let (worker_id, result) = + control_stream_manager.next_collect_barrier_response().await; + let resp = result?; + assert_eq!(resp.epoch, barrier_info.prev_epoch()); + assert!(node_to_collect.remove(&worker_id)); + } + debug!("collected initial barrier"); + barrier_info.curr_epoch + }; + + let background_mviews = info + .job_ids() + .filter_map(|job_id| { + background_jobs.remove(&job_id).map(|mview| (job_id, mview)) + }) + .collect(); + let tracker = CreateMviewProgressTracker::recover( + background_mviews, + &hummock_version_stats, + ); + let state = BarrierWorkerState::recovery( + new_epoch, + info, + subscription_infos.remove(&database_id).unwrap_or_default(), + paused_reason, + ); + databases.insert( + database_id, + DatabaseCheckpointControl::recovery(database_id, tracker, state), + ); + } + if !stream_actors.is_empty() { + warn!(actor_ids = ?stream_actors.keys().collect_vec(), "unused stream actors in recovery"); + } + if !source_splits.is_empty() { + warn!(actor_ids = ?source_splits.keys().collect_vec(), "unused actor source splits in recovery"); + } + if !background_jobs.is_empty() { + warn!(job_ids = ?background_jobs.keys().collect_vec(), "unused recovered background mview in recovery"); + } + if !subscription_infos.is_empty() { + warn!(?subscription_infos, "unused subscription infos in recovery"); + } + if !state_table_committed_epochs.is_empty() { + warn!(?state_table_committed_epochs, "unused state table committed epoch in recovery"); + } + ( + active_streaming_nodes, + control_stream_manager, + CheckpointControl { + databases, + hummock_version_stats, + }, + ) + }; + if recovery_result.is_err() { + GLOBAL_META_METRICS.recovery_failure_cnt.inc(); + } + recovery_result + }) + .instrument(tracing::info_span!("recovery_attempt")) + .await + .expect("Retry until recovery success."); + + recovery_timer.observe_duration(); + + ( + self.active_streaming_nodes, + self.control_stream_manager, + self.checkpoint_control, + ) = new_state; + + tracing::info!("recovery success"); + + self.env + .notification_manager() + .notify_frontend_without_version(Operation::Update, Info::Recovery(Recovery {})); + } +} From fd47f2fd2c7feffd9bd2fc8bae738dcc106f6c80 Mon Sep 17 00:00:00 2001 From: William Wen <44139337+wenym1@users.noreply.github.com> Date: Mon, 11 Nov 2024 18:01:29 +0800 Subject: [PATCH 2/3] Update src/meta/src/barrier/context/context_impl.rs Co-authored-by: github-actions[bot] <41898282+github-actions[bot]@users.noreply.github.com> --- src/meta/src/barrier/context/context_impl.rs | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/src/meta/src/barrier/context/context_impl.rs b/src/meta/src/barrier/context/context_impl.rs index 25cdee4cd076d..947c8a08ad3f1 100644 --- a/src/meta/src/barrier/context/context_impl.rs +++ b/src/meta/src/barrier/context/context_impl.rs @@ -1,3 +1,17 @@ +// 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::sync::Arc; use futures::future::try_join_all; From bde74905746399f9751b6b38c5deb89d97a4bc77 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 11 Nov 2024 18:20:19 +0800 Subject: [PATCH 3/3] doc --- src/meta/src/barrier/command.rs | 2 +- src/meta/src/barrier/schedule.rs | 2 +- src/meta/src/barrier/worker.rs | 10 +++++----- 3 files changed, 7 insertions(+), 7 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 4f78baf26a593..23a8857e9ad11 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -200,7 +200,7 @@ pub enum CreateStreamingJobType { SnapshotBackfill(SnapshotBackfillInfo), } -/// [`Command`] is the input of [`crate::barrier::GlobalBarrierWorker`]. For different commands, +/// [`Command`] is the input of [`crate::barrier::worker::GlobalBarrierWorker`]. For different commands, /// it will build different barriers to send, and may do different stuffs after the barrier is /// collected. #[derive(Debug, strum::Display)] diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index dd443ec47a221..ebffb56efe5a4 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -316,7 +316,7 @@ pub struct ScheduledBarriers { inner: Arc, } -/// Held by the [`super::GlobalBarrierWorker`] to execute these commands. +/// Held by the [`crate::barrier::worker::GlobalBarrierWorker`] to execute these commands. pub(super) struct PeriodicBarriers { min_interval: Interval, diff --git a/src/meta/src/barrier/worker.rs b/src/meta/src/barrier/worker.rs index 4df88d92150db..dfc902f31cfce 100644 --- a/src/meta/src/barrier/worker.rs +++ b/src/meta/src/barrier/worker.rs @@ -60,15 +60,15 @@ use crate::rpc::metrics::GLOBAL_META_METRICS; use crate::stream::{build_actor_connector_splits, ScaleControllerRef, SourceManagerRef}; use crate::{MetaError, MetaResult}; -/// [`crate::barrier::GlobalBarrierWorker`] sends barriers to all registered compute nodes and +/// [`crate::barrier::worker::GlobalBarrierWorker`] sends barriers to all registered compute nodes and /// collect them, with monotonic increasing epoch numbers. On compute nodes, `LocalBarrierManager` /// in `risingwave_stream` crate will serve these requests and dispatch them to source actors. /// /// Configuration change in our system is achieved by the mutation in the barrier. Thus, -/// [`crate::barrier::GlobalBarrierWorker`] provides a set of interfaces like a state machine, -/// accepting [`Command`] that carries info to build `Mutation`. To keep the consistency between +/// [`crate::barrier::worker::GlobalBarrierWorker`] provides a set of interfaces like a state machine, +/// accepting [`crate::barrier::command::Command`] that carries info to build `Mutation`. To keep the consistency between /// barrier manager and meta store, some actions like "drop materialized view" or "create mv on mv" -/// must be done in barrier manager transactional using [`Command`]. +/// must be done in barrier manager transactional using [`crate::barrier::command::Command`]. pub(super) struct GlobalBarrierWorker { /// Enable recovery or not when failover. enable_recovery: bool, @@ -99,7 +99,7 @@ pub(super) struct GlobalBarrierWorker { } impl GlobalBarrierWorker { - /// Create a new [`crate::barrier::GlobalBarrierWorker`]. + /// Create a new [`crate::barrier::worker::GlobalBarrierWorker`]. pub async fn new( scheduled_barriers: schedule::ScheduledBarriers, env: MetaSrvEnv,