1use std::collections::hash_map::Entry;
16use std::collections::{HashMap, HashSet};
17use std::mem::replace;
18use std::sync::Arc;
19use std::time::Duration;
20
21use anyhow::anyhow;
22use arc_swap::ArcSwap;
23use futures::TryFutureExt;
24use itertools::Itertools;
25use risingwave_common::system_param::PAUSE_ON_NEXT_BOOTSTRAP_KEY;
26use risingwave_common::system_param::reader::SystemParamsRead;
27use risingwave_pb::meta::Recovery;
28use risingwave_pb::meta::subscribe_response::{Info, Operation};
29use risingwave_pb::stream_service::streaming_control_stream_response::Response;
30use thiserror_ext::AsReport;
31use tokio::sync::mpsc;
32use tokio::sync::oneshot::{Receiver, Sender};
33use tokio::task::JoinHandle;
34use tonic::Status;
35use tracing::{Instrument, debug, error, info, warn};
36use uuid::Uuid;
37
38use crate::barrier::checkpoint::{CheckpointControl, CheckpointControlEvent};
39use crate::barrier::complete_task::{BarrierCompleteOutput, CompletingTask};
40use crate::barrier::context::{GlobalBarrierWorkerContext, GlobalBarrierWorkerContextImpl};
41use crate::barrier::rpc::{ControlStreamManager, WorkerNodeEvent, merge_node_rpc_errors};
42use crate::barrier::schedule::{MarkReadyOptions, PeriodicBarriers};
43use crate::barrier::{
44 BarrierManagerRequest, BarrierManagerStatus, BarrierWorkerRuntimeInfoSnapshot, RecoveryReason,
45 schedule,
46};
47use crate::error::MetaErrorInner;
48use crate::hummock::HummockManagerRef;
49use crate::manager::sink_coordination::SinkCoordinatorManager;
50use crate::manager::{
51 ActiveStreamingWorkerChange, ActiveStreamingWorkerNodes, LocalNotification, MetaSrvEnv,
52 MetadataManager,
53};
54use crate::rpc::metrics::GLOBAL_META_METRICS;
55use crate::stream::{GlobalRefreshManagerRef, ScaleControllerRef, SourceManagerRef};
56use crate::{MetaError, MetaResult};
57
58pub(super) struct GlobalBarrierWorker<C> {
68 enable_recovery: bool,
70
71 periodic_barriers: PeriodicBarriers,
73
74 system_enable_per_database_isolation: bool,
76
77 pub(super) context: Arc<C>,
78
79 env: MetaSrvEnv,
80
81 checkpoint_control: CheckpointControl,
82
83 completing_task: CompletingTask,
86
87 request_rx: mpsc::UnboundedReceiver<BarrierManagerRequest>,
88
89 active_streaming_nodes: ActiveStreamingWorkerNodes,
90
91 control_stream_manager: ControlStreamManager,
92
93 term_id: String,
94}
95
96impl<C: GlobalBarrierWorkerContext> GlobalBarrierWorker<C> {
97 pub(super) async fn new_inner(
98 env: MetaSrvEnv,
99 request_rx: mpsc::UnboundedReceiver<BarrierManagerRequest>,
100 context: Arc<C>,
101 ) -> Self {
102 let enable_recovery = env.opts.enable_recovery;
103
104 let active_streaming_nodes = ActiveStreamingWorkerNodes::uninitialized();
105
106 let control_stream_manager = ControlStreamManager::new(env.clone());
107
108 let reader = env.system_params_reader().await;
109 let system_enable_per_database_isolation = reader.per_database_isolation();
110 let periodic_barriers = PeriodicBarriers::default();
112
113 let checkpoint_control = CheckpointControl::new(env.clone());
114 Self {
115 enable_recovery,
116 periodic_barriers,
117 system_enable_per_database_isolation,
118 context,
119 env,
120 checkpoint_control,
121 completing_task: CompletingTask::None,
122 request_rx,
123 active_streaming_nodes,
124 control_stream_manager,
125 term_id: "uninitialized".into(),
126 }
127 }
128}
129
130impl GlobalBarrierWorker<GlobalBarrierWorkerContextImpl> {
131 pub async fn new(
133 scheduled_barriers: schedule::ScheduledBarriers,
134 env: MetaSrvEnv,
135 metadata_manager: MetadataManager,
136 hummock_manager: HummockManagerRef,
137 source_manager: SourceManagerRef,
138 sink_manager: SinkCoordinatorManager,
139 scale_controller: ScaleControllerRef,
140 request_rx: mpsc::UnboundedReceiver<BarrierManagerRequest>,
141 barrier_scheduler: schedule::BarrierScheduler,
142 refresh_manager: GlobalRefreshManagerRef,
143 ) -> Self {
144 let status = Arc::new(ArcSwap::new(Arc::new(BarrierManagerStatus::Starting)));
145
146 let context = Arc::new(GlobalBarrierWorkerContextImpl::new(
147 scheduled_barriers,
148 status,
149 metadata_manager,
150 hummock_manager,
151 source_manager,
152 scale_controller,
153 env.clone(),
154 barrier_scheduler,
155 refresh_manager,
156 sink_manager,
157 ));
158
159 Self::new_inner(env, request_rx, context).await
160 }
161
162 pub fn start(self) -> (JoinHandle<()>, Sender<()>) {
163 let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel();
164 let fut = (self.env.await_tree_reg())
165 .register_derived_root("Global Barrier Worker")
166 .instrument(self.run(shutdown_rx));
167 let join_handle = tokio::spawn(fut);
168
169 (join_handle, shutdown_tx)
170 }
171
172 async fn take_pause_on_bootstrap(&mut self) -> MetaResult<bool> {
174 let paused = self
175 .env
176 .system_params_reader()
177 .await
178 .pause_on_next_bootstrap()
179 || self.env.opts.pause_on_next_bootstrap_offline;
180
181 if paused {
182 warn!(
183 "The cluster will bootstrap with all data sources paused as specified by the system parameter `{}`. \
184 It will now be reset to `false`. \
185 To resume the data sources, either restart the cluster again or use `risectl meta resume`.",
186 PAUSE_ON_NEXT_BOOTSTRAP_KEY
187 );
188 self.env
189 .system_params_manager_impl_ref()
190 .set_param(PAUSE_ON_NEXT_BOOTSTRAP_KEY, Some("false".to_owned()))
191 .await?;
192 }
193 Ok(paused)
194 }
195
196 async fn run(mut self, shutdown_rx: Receiver<()>) {
198 tracing::info!(
199 "Starting barrier manager with: enable_recovery={}, in_flight_barrier_nums={}",
200 self.enable_recovery,
201 self.checkpoint_control.in_flight_barrier_nums,
202 );
203
204 if !self.enable_recovery {
205 let job_exist = self
206 .context
207 .metadata_manager
208 .catalog_controller
209 .has_any_streaming_jobs()
210 .await
211 .unwrap();
212 if job_exist {
213 panic!(
214 "Some streaming jobs already exist in meta, please start with recovery enabled \
215 or clean up the metadata using `./risedev clean-data`"
216 );
217 }
218 }
219
220 {
221 let span = tracing::info_span!("bootstrap_recovery");
226 crate::telemetry::report_event(
227 risingwave_pb::telemetry::TelemetryEventStage::Recovery,
228 "normal_recovery",
229 0,
230 None,
231 None,
232 None,
233 );
234
235 let paused = self.take_pause_on_bootstrap().await.unwrap_or(false);
236
237 self.recovery(paused, RecoveryReason::Bootstrap)
238 .instrument(span)
239 .await;
240 }
241
242 self.run_inner(shutdown_rx).await
243 }
244}
245
246impl<C: GlobalBarrierWorkerContext> GlobalBarrierWorker<C> {
247 fn enable_per_database_isolation(&self) -> bool {
248 self.system_enable_per_database_isolation && {
249 if let Err(e) =
250 risingwave_common::license::Feature::DatabaseFailureIsolation.check_available()
251 {
252 warn!(error = %e.as_report(), "DatabaseFailureIsolation disabled by license");
253 false
254 } else {
255 true
256 }
257 }
258 }
259
260 pub(super) async fn run_inner(mut self, mut shutdown_rx: Receiver<()>) {
261 let (local_notification_tx, mut local_notification_rx) =
262 tokio::sync::mpsc::unbounded_channel();
263 self.env
264 .notification_manager()
265 .insert_local_sender(local_notification_tx);
266
267 loop {
269 tokio::select! {
270 biased;
271
272 _ = &mut shutdown_rx => {
274 tracing::info!("Barrier manager is stopped");
275 break;
276 }
277
278 request = self.request_rx.recv() => {
279 if let Some(request) = request {
280 match request {
281 BarrierManagerRequest::GetDdlProgress(result_tx) => {
282 let progress = self.checkpoint_control.gen_ddl_progress();
283 if result_tx.send(progress).is_err() {
284 error!("failed to send get ddl progress");
285 }
286 }BarrierManagerRequest::AdhocRecovery(sender) => {
288 self.adhoc_recovery().await;
289 if sender.send(()).is_err() {
290 warn!("failed to notify finish of adhoc recovery");
291 }
292 }
293 BarrierManagerRequest::UpdateDatabaseBarrier {
294 database_id,
295 barrier_interval_ms,
296 checkpoint_frequency,
297 sender,
298 } => {
299 self.periodic_barriers
300 .update_database_barrier(
301 database_id,
302 barrier_interval_ms,
303 checkpoint_frequency,
304 );
305 if sender.send(()).is_err() {
306 warn!("failed to notify finish of update database barrier");
307 }
308 }
309 }
310 } else {
311 tracing::info!("end of request stream. meta node may be shutting down. Stop global barrier manager");
312 return;
313 }
314 }
315
316 changed_worker = self.active_streaming_nodes.changed() => {
317 #[cfg(debug_assertions)]
318 {
319 self.active_streaming_nodes.validate_change().await;
320 }
321
322 info!(?changed_worker, "worker changed");
323
324 match changed_worker {
325 ActiveStreamingWorkerChange::Add(node) | ActiveStreamingWorkerChange::Update(node) => {
326 self.control_stream_manager.add_worker(node, self.checkpoint_control.inflight_infos(), self.term_id.clone(), &*self.context).await;
327 }
328 ActiveStreamingWorkerChange::Remove(node) => {
329 self.control_stream_manager.remove_worker(node);
330 }
331 }
332 }
333
334 notification = local_notification_rx.recv() => {
335 let notification = notification.unwrap();
336 if let LocalNotification::SystemParamsChange(p) = notification {
337 {
338 self.periodic_barriers.set_sys_barrier_interval(Duration::from_millis(p.barrier_interval_ms() as u64));
339 self.periodic_barriers
340 .set_sys_checkpoint_frequency(p.checkpoint_frequency());
341 self.system_enable_per_database_isolation = p.per_database_isolation();
342 }
343 }
344 }
345 complete_result = self
346 .completing_task
347 .next_completed_barrier(
348 &mut self.periodic_barriers,
349 &mut self.checkpoint_control,
350 &mut self.control_stream_manager,
351 &self.context,
352 &self.env,
353 ) => {
354 match complete_result {
355 Ok(output) => {
356 self.checkpoint_control.ack_completed(output);
357 }
358 Err(e) => {
359 self.failure_recovery(e).await;
360 }
361 }
362 },
363 event = self.checkpoint_control.next_event() => {
364 let result: MetaResult<()> = try {
365 match event {
366 CheckpointControlEvent::EnteringInitializing(entering_initializing) => {
367 let database_id = entering_initializing.database_id();
368 let error = merge_node_rpc_errors(&format!("database {} reset", database_id), entering_initializing.action.0.iter().filter_map(|(worker_id, resp)| {
369 resp.root_err.as_ref().map(|root_err| {
370 (*worker_id, ScoredError {
371 error: Status::internal(&root_err.err_msg),
372 score: Score(root_err.score)
373 })
374 })
375 }));
376 Self::report_collect_failure(&self.env, &error);
377 self.context.notify_creating_job_failed(Some(database_id), format!("{}", error.as_report())).await;
378 match self.context.reload_database_runtime_info(database_id).await? { Some(runtime_info) => {
379 runtime_info.validate(database_id, &self.active_streaming_nodes).inspect_err(|e| {
380 warn!(%database_id, err = ?e.as_report(), ?runtime_info, "reloaded database runtime info failed to validate");
381 })?;
382 entering_initializing.enter(runtime_info, &mut self.control_stream_manager);
383 } _ => {
384 info!(%database_id, "database removed after reloading empty runtime info");
385 self.context.mark_ready(MarkReadyOptions::Database(database_id));
387 entering_initializing.remove();
388 }}
389 }
390 CheckpointControlEvent::EnteringRunning(entering_running) => {
391 self.context.mark_ready(MarkReadyOptions::Database(entering_running.database_id()));
392 entering_running.enter();
393 }
394 }
395 };
396 if let Err(e) = result {
397 self.failure_recovery(e).await;
398 }
399 }
400 (worker_id, event) = self.control_stream_manager.next_event(&self.term_id, &self.context) => {
401 let resp_result = match event {
402 WorkerNodeEvent::Response(result) => {
403 result
404 }
405 WorkerNodeEvent::Connected(connected) => {
406 connected.initialize(self.checkpoint_control.inflight_infos());
407 continue;
408 }
409 };
410 let result: MetaResult<()> = try {
411 let resp = match resp_result {
412 Err(err) => {
413 let failed_databases = self.checkpoint_control.databases_failed_at_worker_err(worker_id);
414 if !failed_databases.is_empty() {
415 if !self.enable_recovery {
416 panic!("control stream to worker {} failed but recovery not enabled: {:?}", worker_id, err.as_report());
417 }
418 if !self.enable_per_database_isolation() {
419 Err(err.clone())?;
420 }
421 Self::report_collect_failure(&self.env, &err);
422 for database_id in failed_databases {
423 if let Some(entering_recovery) = self.checkpoint_control.on_report_failure(database_id, &mut self.control_stream_manager) {
424 warn!(%worker_id, %database_id, "database entering recovery on node failure");
425 self.context.abort_and_mark_blocked(Some(database_id), RecoveryReason::Failover(anyhow!("reset database: {}", database_id).into()));
426 self.context.notify_creating_job_failed(Some(database_id), format!("database {} reset due to node {} failure: {}", database_id, worker_id, err.as_report())).await;
427 let output = self.completing_task.wait_completing_task().await?;
429 entering_recovery.enter(output, &mut self.control_stream_manager);
430 }
431 }
432 } else {
433 warn!(%worker_id, "no barrier to collect from worker, ignore err");
434 }
435 continue;
436 }
437 Ok(resp) => resp,
438 };
439 match resp {
440 Response::CompleteBarrier(resp) => {
441 self.checkpoint_control.barrier_collected(resp, &mut self.periodic_barriers)?;
442 },
443 Response::ReportDatabaseFailure(resp) => {
444 if !self.enable_recovery {
445 panic!("database failure reported but recovery not enabled: {:?}", resp)
446 }
447 if !self.enable_per_database_isolation() {
448 Err(anyhow!("database {} reset", resp.database_id))?;
449 }
450 let database_id = resp.database_id;
451 if let Some(entering_recovery) = self.checkpoint_control.on_report_failure(database_id, &mut self.control_stream_manager) {
452 warn!(%database_id, "database entering recovery");
453 self.context.abort_and_mark_blocked(Some(database_id), RecoveryReason::Failover(anyhow!("reset database: {}", database_id).into()));
454 let output = self.completing_task.wait_completing_task().await?;
456 entering_recovery.enter(output, &mut self.control_stream_manager);
457 }
458 }
459 Response::ResetDatabase(resp) => {
460 self.checkpoint_control.on_reset_database_resp(worker_id, resp);
461 }
462 other @ Response::Init(_) | other @ Response::Shutdown(_) => {
463 Err(anyhow!("get expected response: {:?}", other))?;
464 }
465 }
466 };
467 if let Err(e) = result {
468 self.failure_recovery(e).await;
469 }
470 }
471 new_barrier = self.periodic_barriers.next_barrier(&*self.context) => {
472 let database_id = new_barrier.database_id;
473 if let Err(e) = self.checkpoint_control.handle_new_barrier(new_barrier, &mut self.control_stream_manager) {
474 if !self.enable_recovery {
475 panic!(
476 "failed to inject barrier to some databases but recovery not enabled: {:?}", (
477 database_id,
478 e.as_report()
479 )
480 );
481 }
482 let result: MetaResult<_> = try {
483 if !self.enable_per_database_isolation() {
484 let err = anyhow!("failed to inject barrier to databases: {:?}", (database_id, e.as_report()));
485 Err(err)?;
486 } else if let Some(entering_recovery) = self.checkpoint_control.on_report_failure(database_id, &mut self.control_stream_manager) {
487 warn!(%database_id, e = %e.as_report(),"database entering recovery on inject failure");
488 self.context.abort_and_mark_blocked(Some(database_id), RecoveryReason::Failover(anyhow!(e).context("inject barrier failure").into()));
489 let output = self.completing_task.wait_completing_task().await?;
491 entering_recovery.enter(output, &mut self.control_stream_manager);
492 }
493 };
494 if let Err(e) = result {
495 self.failure_recovery(e).await;
496 }
497 }
498 }
499 }
500 self.checkpoint_control.update_barrier_nums_metrics();
501 }
502 }
503}
504
505impl<C: GlobalBarrierWorkerContext> GlobalBarrierWorker<C> {
506 pub async fn clear_on_err(&mut self, err: &MetaError) {
508 let is_err = match replace(&mut self.completing_task, CompletingTask::None) {
510 CompletingTask::None => false,
511 CompletingTask::Completing {
512 epochs_to_ack,
513 join_handle,
514 ..
515 } => {
516 info!("waiting for completing command to finish in recovery");
517 match join_handle.await {
518 Err(e) => {
519 warn!(err = %e.as_report(), "failed to join completing task");
520 true
521 }
522 Ok(Err(e)) => {
523 warn!(
524 err = %e.as_report(),
525 "failed to complete barrier during clear"
526 );
527 true
528 }
529 Ok(Ok(hummock_version_stats)) => {
530 self.checkpoint_control
531 .ack_completed(BarrierCompleteOutput {
532 epochs_to_ack,
533 hummock_version_stats,
534 });
535 false
536 }
537 }
538 }
539 CompletingTask::Err(_) => true,
540 };
541 if !is_err {
542 while let Some(task) = self.checkpoint_control.next_complete_barrier_task(None) {
544 let epochs_to_ack = task.epochs_to_ack();
545 match task
546 .complete_barrier(&*self.context, self.env.clone())
547 .await
548 {
549 Ok(hummock_version_stats) => {
550 self.checkpoint_control
551 .ack_completed(BarrierCompleteOutput {
552 epochs_to_ack,
553 hummock_version_stats,
554 });
555 }
556 Err(e) => {
557 error!(
558 err = %e.as_report(),
559 "failed to complete barrier during recovery"
560 );
561 break;
562 }
563 }
564 }
565 }
566 self.checkpoint_control.clear_on_err(err);
567 }
568}
569
570impl<C: GlobalBarrierWorkerContext> GlobalBarrierWorker<C> {
571 async fn failure_recovery(&mut self, err: MetaError) {
573 self.clear_on_err(&err).await;
574
575 if self.enable_recovery {
576 let span = tracing::info_span!(
577 "failure_recovery",
578 error = %err.as_report(),
579 );
580
581 crate::telemetry::report_event(
582 risingwave_pb::telemetry::TelemetryEventStage::Recovery,
583 "failure_recovery",
584 0,
585 None,
586 None,
587 None,
588 );
589
590 let reason = RecoveryReason::Failover(err);
591
592 self.recovery(false, reason).instrument(span).await;
595 } else {
596 panic!(
597 "a streaming error occurred while recovery is disabled, aborting: {:?}",
598 err.as_report()
599 );
600 }
601 }
602
603 async fn adhoc_recovery(&mut self) {
604 let err = MetaErrorInner::AdhocRecovery.into();
605 self.clear_on_err(&err).await;
606
607 let span = tracing::info_span!(
608 "adhoc_recovery",
609 error = %err.as_report(),
610 );
611
612 crate::telemetry::report_event(
613 risingwave_pb::telemetry::TelemetryEventStage::Recovery,
614 "adhoc_recovery",
615 0,
616 None,
617 None,
618 None,
619 );
620
621 self.recovery(false, RecoveryReason::Adhoc)
624 .instrument(span)
625 .await;
626 }
627}
628
629impl<C> GlobalBarrierWorker<C> {
630 pub(super) fn report_collect_failure(env: &MetaSrvEnv, error: &MetaError) {
632 use risingwave_pb::meta::event_log;
634 let event = event_log::EventCollectBarrierFail {
635 error: error.to_report_string(),
636 };
637 env.event_log_manager_ref()
638 .add_event_logs(vec![event_log::Event::CollectBarrierFail(event)]);
639 }
640}
641
642mod retry_strategy {
643 use std::time::Duration;
644
645 use tokio_retry::strategy::{ExponentialBackoff, jitter};
646
647 const RECOVERY_RETRY_BASE_INTERVAL: u64 = 20;
649 const RECOVERY_RETRY_MAX_INTERVAL: Duration = Duration::from_secs(5);
651
652 pub(crate) type RetryBackoffFuture = std::pin::Pin<Box<tokio::time::Sleep>>;
671
672 pub(crate) fn get_retry_backoff_future(duration: Duration) -> RetryBackoffFuture {
673 Box::pin(tokio::time::sleep(duration))
674 }
675
676 pub(crate) type RetryBackoffStrategy =
677 impl Iterator<Item = RetryBackoffFuture> + Send + 'static;
678
679 #[inline(always)]
681 pub(crate) fn get_retry_strategy() -> impl Iterator<Item = Duration> + Send + 'static {
682 ExponentialBackoff::from_millis(RECOVERY_RETRY_BASE_INTERVAL)
683 .max_delay(RECOVERY_RETRY_MAX_INTERVAL)
684 .map(jitter)
685 }
686
687 #[define_opaque(RetryBackoffStrategy)]
688 pub(crate) fn get_retry_backoff_strategy() -> RetryBackoffStrategy {
689 get_retry_strategy().map(get_retry_backoff_future)
690 }
691}
692
693pub(crate) use retry_strategy::*;
694use risingwave_common::error::tonic::extra::{Score, ScoredError};
695use risingwave_pb::meta::event_log::{Event, EventRecovery};
696
697use crate::barrier::edge_builder::FragmentEdgeBuilder;
698
699impl<C: GlobalBarrierWorkerContext> GlobalBarrierWorker<C> {
700 pub async fn recovery(&mut self, is_paused: bool, recovery_reason: RecoveryReason) {
708 self.control_stream_manager.clear();
710
711 let reason_str = match &recovery_reason {
712 RecoveryReason::Bootstrap => "bootstrap".to_owned(),
713 RecoveryReason::Failover(err) => {
714 format!("failed over: {}", err.as_report())
715 }
716 RecoveryReason::Adhoc => "adhoc recovery".to_owned(),
717 };
718 self.context.abort_and_mark_blocked(None, recovery_reason);
719
720 self.recovery_inner(is_paused, reason_str).await;
721 self.context.mark_ready(MarkReadyOptions::Global {
722 blocked_databases: self.checkpoint_control.recovering_databases().collect(),
723 });
724 }
725
726 #[await_tree::instrument("recovery({recovery_reason})")]
727 async fn recovery_inner(&mut self, is_paused: bool, recovery_reason: String) {
728 let event_log_manager_ref = self.env.event_log_manager_ref();
729
730 tracing::info!("recovery start!");
731 event_log_manager_ref.add_event_logs(vec![Event::Recovery(
732 EventRecovery::global_recovery_start(recovery_reason.clone()),
733 )]);
734
735 let retry_strategy = get_retry_strategy();
736
737 let recovery_timer = GLOBAL_META_METRICS
740 .recovery_latency
741 .with_label_values(&["global"])
742 .start_timer();
743
744 let enable_per_database_isolation = self.enable_per_database_isolation();
745
746 let new_state = tokio_retry::Retry::spawn(retry_strategy, || async {
747 self.env.stream_client_pool().invalidate_all();
748 self.context
753 .notify_creating_job_failed(None, recovery_reason.clone())
754 .await;
755
756 let runtime_info_snapshot = self
757 .context
758 .reload_runtime_info()
759 .await?;
760 runtime_info_snapshot.validate().inspect_err(|e| {
761 warn!(err = ?e.as_report(), ?runtime_info_snapshot, "reloaded runtime info failed to validate");
762 })?;
763 let BarrierWorkerRuntimeInfoSnapshot {
764 active_streaming_nodes,
765 database_job_infos,
766 mut state_table_committed_epochs,
767 mut state_table_log_epochs,
768 mut mv_depended_subscriptions,
769 stream_actors,
770 fragment_relations,
771 mut source_splits,
772 mut background_jobs,
773 hummock_version_stats,
774 database_infos,
775 mut cdc_table_snapshot_split_assignment,
776 } = runtime_info_snapshot;
777
778 let term_id = Uuid::new_v4().to_string();
779
780
781 let mut control_stream_manager = ControlStreamManager::recover(
782 self.env.clone(),
783 active_streaming_nodes.current(),
784 &term_id,
785 self.context.clone(),
786 )
787 .await;
788
789
790 {
791 let mut builder = FragmentEdgeBuilder::new(database_job_infos.values().flat_map(|jobs| jobs.values().flat_map(|fragments| fragments.values())), &control_stream_manager);
792 builder.add_relations(&fragment_relations);
793 let mut edges = builder.build();
794
795 let mut collected_databases = HashMap::new();
796 let mut collecting_databases = HashMap::new();
797 let mut failed_databases = HashSet::new();
798 for (database_id, jobs) in database_job_infos {
799 let result = control_stream_manager.inject_database_initial_barrier(
800 database_id,
801 jobs,
802 &mut state_table_committed_epochs,
803 &mut state_table_log_epochs,
804 &mut edges,
805 &stream_actors,
806 &mut source_splits,
807 &mut background_jobs,
808 &mut mv_depended_subscriptions,
809 is_paused,
810 &hummock_version_stats,
811 &mut cdc_table_snapshot_split_assignment,
812 );
813 let node_to_collect = match result {
814 Ok(info) => {
815 info
816 }
817 Err(e) => {
818 warn!(%database_id, e = %e.as_report(), "failed to inject database initial barrier");
819 assert!(failed_databases.insert(database_id), "non-duplicate");
820 continue;
821 }
822 };
823 if !node_to_collect.is_collected() {
824 assert!(collecting_databases.insert(database_id, node_to_collect).is_none());
825 } else {
826 warn!(%database_id, "database has no node to inject initial barrier");
827 assert!(collected_databases.insert(database_id, node_to_collect.finish()).is_none());
828 }
829 }
830 while !collecting_databases.is_empty() {
831 let (worker_id, result) =
832 control_stream_manager.next_response(&term_id, &self.context).await;
833 let resp = match result {
834 Err(e) => {
835 warn!(%worker_id, err = %e.as_report(), "worker node failure during recovery");
836 for (failed_database_id, _) in collecting_databases.extract_if(|_, node_to_collect| {
837 !node_to_collect.is_valid_after_worker_err(worker_id)
838 }) {
839 warn!(%failed_database_id, %worker_id, "database failed to recovery in global recovery due to worker node err");
840 assert!(failed_databases.insert(failed_database_id));
841 }
842 continue;
843 }
844 Ok(resp) => {
845 match resp {
846 Response::CompleteBarrier(resp) => {
847 resp
848 }
849 Response::ReportDatabaseFailure(resp) => {
850 let database_id = resp.database_id;
851 if collecting_databases.remove(&database_id).is_some() {
852 warn!(%database_id, %worker_id, "database reset during global recovery");
853 assert!(failed_databases.insert(database_id));
854 } else if collected_databases.remove(&database_id).is_some() {
855 warn!(%database_id, %worker_id, "database initialized but later reset during global recovery");
856 assert!(failed_databases.insert(database_id));
857 } else {
858 assert!(failed_databases.contains(&database_id));
859 }
860 continue;
861 }
862 other @ (Response::Init(_) | Response::Shutdown(_) | Response::ResetDatabase(_)) => {
863 return Err(anyhow!("get unexpected resp {:?}", other).into());
864 }
865 }
866 }
867 };
868 assert_eq!(worker_id, resp.worker_id);
869 let database_id = resp.database_id;
870 if failed_databases.contains(&database_id) {
871 assert!(!collecting_databases.contains_key(&database_id));
872 continue;
874 }
875 let Entry::Occupied(mut entry) = collecting_databases.entry(database_id) else {
876 unreachable!("should exist")
877 };
878 let node_to_collect = entry.get_mut();
879 node_to_collect.collect_resp(resp);
880 if node_to_collect.is_collected() {
881 let node_to_collect = entry.remove();
882 assert!(collected_databases.insert(database_id, node_to_collect.finish()).is_none());
883 }
884 }
885 debug!("collected initial barrier");
886 if !stream_actors.is_empty() {
887 warn!(actor_ids = ?stream_actors.keys().collect_vec(), "unused stream actors in recovery");
888 }
889 if !source_splits.is_empty() {
890 warn!(actor_ids = ?source_splits.keys().collect_vec(), "unused actor source splits in recovery");
891 }
892 if !background_jobs.is_empty() {
893 warn!(job_ids = ?background_jobs.keys().collect_vec(), "unused recovered background mview in recovery");
894 }
895 if !mv_depended_subscriptions.is_empty() {
896 warn!(?mv_depended_subscriptions, "unused subscription infos in recovery");
897 }
898 if !state_table_committed_epochs.is_empty() {
899 warn!(?state_table_committed_epochs, "unused state table committed epoch in recovery");
900 }
901 if !enable_per_database_isolation && !failed_databases.is_empty() {
902 return Err(anyhow!(
903 "global recovery failed due to failure of databases {:?}",
904 failed_databases.iter().map(|database_id| database_id.as_raw_id()).collect_vec()).into()
905 );
906 }
907 let checkpoint_control = CheckpointControl::recover(
908 collected_databases,
909 failed_databases,
910 &mut control_stream_manager,
911 hummock_version_stats,
912 self.env.clone(),
913 );
914
915 let reader = self.env.system_params_reader().await;
916 let checkpoint_frequency = reader.checkpoint_frequency();
917 let barrier_interval = Duration::from_millis(reader.barrier_interval_ms() as u64);
918 let periodic_barriers = PeriodicBarriers::new(
919 barrier_interval,
920 checkpoint_frequency,
921 database_infos,
922 );
923
924 Ok((
925 active_streaming_nodes,
926 control_stream_manager,
927 checkpoint_control,
928 term_id,
929 periodic_barriers,
930 ))
931 }
932 }.inspect_err(|err: &MetaError| {
933 tracing::error!(error = %err.as_report(), "recovery failed");
934 event_log_manager_ref.add_event_logs(vec![Event::Recovery(
935 EventRecovery::global_recovery_failure(recovery_reason.clone(), err.to_report_string()),
936 )]);
937 GLOBAL_META_METRICS.recovery_failure_cnt.with_label_values(&["global"]).inc();
938 }))
939 .instrument(tracing::info_span!("recovery_attempt"))
940 .await
941 .expect("Retry until recovery success.");
942
943 let duration = recovery_timer.stop_and_record();
944
945 (
946 self.active_streaming_nodes,
947 self.control_stream_manager,
948 self.checkpoint_control,
949 self.term_id,
950 self.periodic_barriers,
951 ) = new_state;
952
953 tracing::info!("recovery success");
954
955 let recovering_databases = self
956 .checkpoint_control
957 .recovering_databases()
958 .map(|database| database.as_raw_id())
959 .collect_vec();
960 let running_databases = self
961 .checkpoint_control
962 .running_databases()
963 .map(|database| database.as_raw_id())
964 .collect_vec();
965
966 event_log_manager_ref.add_event_logs(vec![Event::Recovery(
967 EventRecovery::global_recovery_success(
968 recovery_reason.clone(),
969 duration as f32,
970 running_databases,
971 recovering_databases,
972 ),
973 )]);
974
975 self.env
976 .notification_manager()
977 .notify_frontend_without_version(Operation::Update, Info::Recovery(Recovery {}));
978 self.env
979 .notification_manager()
980 .notify_compute_without_version(Operation::Update, Info::Recovery(Recovery {}));
981 }
982}