1use std::collections::hash_map::Entry;
16use std::collections::{BTreeMap, HashMap, HashSet};
17use std::future::{Future, poll_fn};
18use std::mem::take;
19use std::task::Poll;
20
21use anyhow::anyhow;
22use fail::fail_point;
23use prometheus::HistogramTimer;
24use risingwave_common::catalog::{DatabaseId, TableId};
25use risingwave_common::metrics::{LabelGuardedHistogram, LabelGuardedIntGauge};
26use risingwave_meta_model::WorkerId;
27use risingwave_pb::ddl_service::DdlProgress;
28use risingwave_pb::hummock::HummockVersionStats;
29use risingwave_pb::stream_service::BarrierCompleteResponse;
30use risingwave_pb::stream_service::streaming_control_stream_response::ResetDatabaseResponse;
31use thiserror_ext::AsReport;
32use tracing::{debug, warn};
33
34use crate::barrier::checkpoint::creating_job::{CompleteJobType, CreatingStreamingJobControl};
35use crate::barrier::checkpoint::recovery::{
36 DatabaseRecoveringState, DatabaseStatusAction, EnterInitializing, EnterRunning,
37 RecoveringStateAction,
38};
39use crate::barrier::checkpoint::state::BarrierWorkerState;
40use crate::barrier::command::CommandContext;
41use crate::barrier::complete_task::{BarrierCompleteOutput, CompleteBarrierTask};
42use crate::barrier::info::InflightStreamingJobInfo;
43use crate::barrier::notifier::Notifier;
44use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob};
45use crate::barrier::rpc::{ControlStreamManager, from_partial_graph_id};
46use crate::barrier::schedule::{NewBarrier, PeriodicBarriers};
47use crate::barrier::utils::{
48 NodeToCollect, collect_creating_job_commit_epoch_info, is_valid_after_worker_err,
49};
50use crate::barrier::{
51 BarrierKind, Command, CreateStreamingJobType, InflightSubscriptionInfo, TracedEpoch,
52};
53use crate::manager::MetaSrvEnv;
54use crate::rpc::metrics::GLOBAL_META_METRICS;
55use crate::stream::fill_snapshot_backfill_epoch;
56use crate::{MetaError, MetaResult};
57
58pub(crate) struct CheckpointControl {
59 pub(crate) env: MetaSrvEnv,
60 pub(super) databases: HashMap<DatabaseId, DatabaseCheckpointControlStatus>,
61 pub(super) hummock_version_stats: HummockVersionStats,
62}
63
64impl CheckpointControl {
65 pub fn new(env: MetaSrvEnv) -> Self {
66 Self {
67 env,
68 databases: Default::default(),
69 hummock_version_stats: Default::default(),
70 }
71 }
72
73 pub(crate) fn recover(
74 databases: impl IntoIterator<Item = (DatabaseId, DatabaseCheckpointControl)>,
75 failed_databases: HashSet<DatabaseId>,
76 control_stream_manager: &mut ControlStreamManager,
77 hummock_version_stats: HummockVersionStats,
78 env: MetaSrvEnv,
79 ) -> Self {
80 Self {
81 env,
82 databases: databases
83 .into_iter()
84 .map(|(database_id, control)| {
85 (
86 database_id,
87 DatabaseCheckpointControlStatus::Running(control),
88 )
89 })
90 .chain(failed_databases.into_iter().map(|database_id| {
91 (
92 database_id,
93 DatabaseCheckpointControlStatus::Recovering(
94 DatabaseRecoveringState::resetting(database_id, control_stream_manager),
95 ),
96 )
97 }))
98 .collect(),
99 hummock_version_stats,
100 }
101 }
102
103 pub(crate) fn ack_completed(&mut self, output: BarrierCompleteOutput) {
104 self.hummock_version_stats = output.hummock_version_stats;
105 for (database_id, (command_prev_epoch, creating_job_epochs)) in output.epochs_to_ack {
106 self.databases
107 .get_mut(&database_id)
108 .expect("should exist")
109 .expect_running("should have wait for completing command before enter recovery")
110 .ack_completed(command_prev_epoch, creating_job_epochs);
111 }
112 }
113
114 pub(crate) fn next_complete_barrier_task(
115 &mut self,
116 mut context: Option<(&mut PeriodicBarriers, &mut ControlStreamManager)>,
117 ) -> Option<CompleteBarrierTask> {
118 let mut task = None;
119 for database in self.databases.values_mut() {
120 let Some(database) = database.running_state_mut() else {
121 continue;
122 };
123 let context = context.as_mut().map(|(s, c)| (&mut **s, &mut **c));
124 database.next_complete_barrier_task(&mut task, context, &self.hummock_version_stats);
125 }
126 task
127 }
128
129 pub(crate) fn barrier_collected(
130 &mut self,
131 resp: BarrierCompleteResponse,
132 periodic_barriers: &mut PeriodicBarriers,
133 ) -> MetaResult<()> {
134 let database_id = DatabaseId::new(resp.database_id);
135 let database_status = self.databases.get_mut(&database_id).expect("should exist");
136 match database_status {
137 DatabaseCheckpointControlStatus::Running(database) => {
138 database.barrier_collected(resp, periodic_barriers)
139 }
140 DatabaseCheckpointControlStatus::Recovering(state) => {
141 state.barrier_collected(database_id, resp);
142 Ok(())
143 }
144 }
145 }
146
147 pub(crate) fn can_inject_barrier(&self, in_flight_barrier_nums: usize) -> bool {
148 self.databases.values().all(|database| {
149 database
150 .running_state()
151 .map(|database| database.can_inject_barrier(in_flight_barrier_nums))
152 .unwrap_or(true)
153 })
154 }
155
156 pub(crate) fn max_prev_epoch(&self) -> Option<TracedEpoch> {
157 self.databases
158 .values()
159 .flat_map(|database| {
160 database
161 .running_state()
162 .map(|database| database.state.in_flight_prev_epoch())
163 })
164 .max_by_key(|epoch| epoch.value())
165 .cloned()
166 }
167
168 pub(crate) fn recovering_databases(&self) -> impl Iterator<Item = DatabaseId> + '_ {
169 self.databases.iter().filter_map(|(database_id, database)| {
170 database.running_state().is_none().then_some(*database_id)
171 })
172 }
173
174 pub(crate) fn running_databases(&self) -> impl Iterator<Item = DatabaseId> + '_ {
175 self.databases.iter().filter_map(|(database_id, database)| {
176 database.running_state().is_some().then_some(*database_id)
177 })
178 }
179
180 pub(crate) fn handle_new_barrier(
182 &mut self,
183 new_barrier: NewBarrier,
184 control_stream_manager: &mut ControlStreamManager,
185 ) -> Option<HashMap<DatabaseId, MetaError>> {
186 let NewBarrier {
187 command,
188 span,
189 checkpoint,
190 } = new_barrier;
191
192 if let Some((database_id, mut command, notifiers)) = command {
193 if let &mut Command::CreateStreamingJob {
194 ref mut cross_db_snapshot_backfill_info,
195 ref info,
196 ..
197 } = &mut command
198 {
199 for (table_id, snapshot_epoch) in
200 &mut cross_db_snapshot_backfill_info.upstream_mv_table_id_to_backfill_epoch
201 {
202 for database in self.databases.values() {
203 if let Some(database) = database.running_state()
204 && database.state.inflight_graph_info.contains_job(*table_id)
205 {
206 if let Some(committed_epoch) = database.committed_epoch {
207 *snapshot_epoch = Some(committed_epoch);
208 }
209 break;
210 }
211 }
212 if snapshot_epoch.is_none() {
213 let table_id = *table_id;
214 warn!(
215 ?cross_db_snapshot_backfill_info,
216 ?table_id,
217 ?info,
218 "database of cross db upstream table not found"
219 );
220 let err: MetaError =
221 anyhow!("database of cross db upstream table {} not found", table_id)
222 .into();
223 for notifier in notifiers {
224 notifier.notify_start_failed(err.clone());
225 }
226
227 return None;
228 }
229 }
230 }
231
232 let max_prev_epoch = self.max_prev_epoch();
233 let (database, max_prev_epoch) = match self.databases.entry(database_id) {
234 Entry::Occupied(entry) => (
235 entry
236 .into_mut()
237 .expect_running("should not have command when not running"),
238 max_prev_epoch.expect("should exist when having some database"),
239 ),
240 Entry::Vacant(entry) => match &command {
241 Command::CreateStreamingJob {
242 job_type: CreateStreamingJobType::Normal,
243 ..
244 } => {
245 let new_database = DatabaseCheckpointControl::new(database_id);
246 let max_prev_epoch = if let Some(max_prev_epoch) = max_prev_epoch {
247 if max_prev_epoch.value()
248 < new_database.state.in_flight_prev_epoch().value()
249 {
250 new_database.state.in_flight_prev_epoch().clone()
251 } else {
252 max_prev_epoch
253 }
254 } else {
255 new_database.state.in_flight_prev_epoch().clone()
256 };
257 control_stream_manager.add_partial_graph(database_id, None);
258 (
259 entry
260 .insert(DatabaseCheckpointControlStatus::Running(new_database))
261 .expect_running("just initialized as running"),
262 max_prev_epoch,
263 )
264 }
265 Command::Flush | Command::Pause | Command::Resume => {
266 for mut notifier in notifiers {
267 notifier.notify_started();
268 notifier.notify_collected();
269 }
270 warn!(?command, "skip command for empty database");
271 return None;
272 }
273 _ => {
274 panic!(
275 "new database graph info can only be created for normal creating streaming job, but get command: {} {:?}",
276 database_id, command
277 )
278 }
279 },
280 };
281
282 let curr_epoch = max_prev_epoch.next();
283
284 let mut failed_databases: Option<HashMap<_, _>> = None;
285
286 if let Err(e) = database.handle_new_barrier(
287 Some((command, notifiers)),
288 checkpoint,
289 span.clone(),
290 control_stream_manager,
291 &self.hummock_version_stats,
292 curr_epoch.clone(),
293 ) {
294 failed_databases
295 .get_or_insert_default()
296 .try_insert(database_id, e)
297 .expect("non-duplicate");
298 }
299 for database in self.databases.values_mut() {
300 let Some(database) = database.running_state_mut() else {
301 continue;
302 };
303 if database.database_id == database_id {
304 continue;
305 }
306 if let Err(e) = database.handle_new_barrier(
307 None,
308 checkpoint,
309 span.clone(),
310 control_stream_manager,
311 &self.hummock_version_stats,
312 curr_epoch.clone(),
313 ) {
314 failed_databases
315 .get_or_insert_default()
316 .try_insert(database.database_id, e)
317 .expect("non-duplicate");
318 }
319 }
320 failed_databases
321 } else {
322 #[expect(clippy::question_mark)]
323 let Some(max_prev_epoch) = self.max_prev_epoch() else {
324 return None;
325 };
326 let curr_epoch = max_prev_epoch.next();
327
328 let mut failed_databases: Option<HashMap<_, _>> = None;
329 for database in self.databases.values_mut() {
330 let Some(database) = database.running_state_mut() else {
331 continue;
332 };
333 if let Err(e) = database.handle_new_barrier(
334 None,
335 checkpoint,
336 span.clone(),
337 control_stream_manager,
338 &self.hummock_version_stats,
339 curr_epoch.clone(),
340 ) {
341 failed_databases
342 .get_or_insert_default()
343 .try_insert(database.database_id, e)
344 .expect("non-duplicate");
345 }
346 }
347
348 failed_databases
349 }
350 }
351
352 pub(crate) fn update_barrier_nums_metrics(&self) {
353 self.databases
354 .values()
355 .flat_map(|database| database.running_state())
356 .for_each(|database| database.update_barrier_nums_metrics());
357 }
358
359 pub(crate) fn gen_ddl_progress(&self) -> HashMap<u32, DdlProgress> {
360 let mut progress = HashMap::new();
361 for status in self.databases.values() {
362 let Some(database_checkpoint_control) = status.running_state() else {
363 continue;
364 };
365 progress.extend(
367 database_checkpoint_control
368 .create_mview_tracker
369 .gen_ddl_progress(),
370 );
371 for creating_job in database_checkpoint_control
373 .creating_streaming_job_controls
374 .values()
375 {
376 progress.extend([(
377 creating_job.job_id.table_id,
378 creating_job.gen_ddl_progress(),
379 )]);
380 }
381 }
382 progress
383 }
384
385 pub(crate) fn databases_failed_at_worker_err(
386 &mut self,
387 worker_id: WorkerId,
388 ) -> Vec<DatabaseId> {
389 let mut failed_databases = Vec::new();
390 for (database_id, database_status) in &mut self.databases {
391 let database_checkpoint_control = match database_status {
392 DatabaseCheckpointControlStatus::Running(control) => control,
393 DatabaseCheckpointControlStatus::Recovering(state) => {
394 if !state.is_valid_after_worker_err(worker_id) {
395 failed_databases.push(*database_id);
396 }
397 continue;
398 }
399 };
400
401 if !database_checkpoint_control.is_valid_after_worker_err(worker_id as _)
402 || database_checkpoint_control
403 .state
404 .inflight_graph_info
405 .contains_worker(worker_id as _)
406 || database_checkpoint_control
407 .creating_streaming_job_controls
408 .values_mut()
409 .any(|job| !job.is_valid_after_worker_err(worker_id))
410 {
411 failed_databases.push(*database_id);
412 }
413 }
414 failed_databases
415 }
416
417 pub(crate) fn clear_on_err(&mut self, err: &MetaError) {
418 for (_, node) in self.databases.values_mut().flat_map(|status| {
419 status
420 .running_state_mut()
421 .map(|database| take(&mut database.command_ctx_queue))
422 .into_iter()
423 .flatten()
424 }) {
425 for notifier in node.notifiers {
426 notifier.notify_collection_failed(err.clone());
427 }
428 node.enqueue_time.observe_duration();
429 }
430 self.databases.values_mut().for_each(|database| {
431 if let Some(database) = database.running_state_mut() {
432 database.create_mview_tracker.abort_all()
433 }
434 });
435 }
436
437 pub(crate) fn inflight_infos(
438 &self,
439 ) -> impl Iterator<
440 Item = (
441 DatabaseId,
442 &InflightSubscriptionInfo,
443 impl Iterator<Item = TableId> + '_,
444 ),
445 > + '_ {
446 self.databases.iter().flat_map(|(database_id, database)| {
447 database
448 .database_state()
449 .map(|(database_state, creating_jobs)| {
450 (
451 *database_id,
452 &database_state.inflight_subscription_info,
453 creating_jobs
454 .values()
455 .filter_map(|job| job.is_consuming().then_some(job.job_id)),
456 )
457 })
458 })
459 }
460}
461
462pub(crate) enum CheckpointControlEvent<'a> {
463 EnteringInitializing(DatabaseStatusAction<'a, EnterInitializing>),
464 EnteringRunning(DatabaseStatusAction<'a, EnterRunning>),
465}
466
467impl CheckpointControl {
468 pub(crate) fn on_reset_database_resp(
469 &mut self,
470 worker_id: WorkerId,
471 resp: ResetDatabaseResponse,
472 ) {
473 let database_id = DatabaseId::new(resp.database_id);
474 match self.databases.get_mut(&database_id).expect("should exist") {
475 DatabaseCheckpointControlStatus::Running(_) => {
476 unreachable!("should not receive reset database resp when running")
477 }
478 DatabaseCheckpointControlStatus::Recovering(state) => {
479 state.on_reset_database_resp(worker_id, resp)
480 }
481 }
482 }
483
484 pub(crate) fn next_event(
485 &mut self,
486 ) -> impl Future<Output = CheckpointControlEvent<'_>> + Send + '_ {
487 let mut this = Some(self);
488 poll_fn(move |cx| {
489 let Some(this_mut) = this.as_mut() else {
490 unreachable!("should not be polled after poll ready")
491 };
492 for (&database_id, database_status) in &mut this_mut.databases {
493 match database_status {
494 DatabaseCheckpointControlStatus::Running(_) => {}
495 DatabaseCheckpointControlStatus::Recovering(state) => {
496 let poll_result = state.poll_next_event(cx);
497 if let Poll::Ready(action) = poll_result {
498 let this = this.take().expect("checked Some");
499 return Poll::Ready(match action {
500 RecoveringStateAction::EnterInitializing(reset_workers) => {
501 CheckpointControlEvent::EnteringInitializing(
502 this.new_database_status_action(
503 database_id,
504 EnterInitializing(reset_workers),
505 ),
506 )
507 }
508 RecoveringStateAction::EnterRunning => {
509 CheckpointControlEvent::EnteringRunning(
510 this.new_database_status_action(database_id, EnterRunning),
511 )
512 }
513 });
514 }
515 }
516 }
517 }
518 Poll::Pending
519 })
520 }
521}
522
523pub(crate) enum DatabaseCheckpointControlStatus {
524 Running(DatabaseCheckpointControl),
525 Recovering(DatabaseRecoveringState),
526}
527
528impl DatabaseCheckpointControlStatus {
529 fn running_state(&self) -> Option<&DatabaseCheckpointControl> {
530 match self {
531 DatabaseCheckpointControlStatus::Running(state) => Some(state),
532 DatabaseCheckpointControlStatus::Recovering(_) => None,
533 }
534 }
535
536 fn running_state_mut(&mut self) -> Option<&mut DatabaseCheckpointControl> {
537 match self {
538 DatabaseCheckpointControlStatus::Running(state) => Some(state),
539 DatabaseCheckpointControlStatus::Recovering(_) => None,
540 }
541 }
542
543 fn database_state(
544 &self,
545 ) -> Option<(
546 &BarrierWorkerState,
547 &HashMap<TableId, CreatingStreamingJobControl>,
548 )> {
549 match self {
550 DatabaseCheckpointControlStatus::Running(control) => {
551 Some((&control.state, &control.creating_streaming_job_controls))
552 }
553 DatabaseCheckpointControlStatus::Recovering(state) => state.database_state(),
554 }
555 }
556
557 fn expect_running(&mut self, reason: &'static str) -> &mut DatabaseCheckpointControl {
558 match self {
559 DatabaseCheckpointControlStatus::Running(state) => state,
560 DatabaseCheckpointControlStatus::Recovering(_) => {
561 panic!("should be at running: {}", reason)
562 }
563 }
564 }
565}
566
567struct DatabaseCheckpointControlMetrics {
568 barrier_latency: LabelGuardedHistogram,
569 in_flight_barrier_nums: LabelGuardedIntGauge,
570 all_barrier_nums: LabelGuardedIntGauge,
571}
572
573impl DatabaseCheckpointControlMetrics {
574 fn new(database_id: DatabaseId) -> Self {
575 let database_id_str = database_id.database_id.to_string();
576 let barrier_latency = GLOBAL_META_METRICS
577 .barrier_latency
578 .with_guarded_label_values(&[&database_id_str]);
579 let in_flight_barrier_nums = GLOBAL_META_METRICS
580 .in_flight_barrier_nums
581 .with_guarded_label_values(&[&database_id_str]);
582 let all_barrier_nums = GLOBAL_META_METRICS
583 .all_barrier_nums
584 .with_guarded_label_values(&[&database_id_str]);
585 Self {
586 barrier_latency,
587 in_flight_barrier_nums,
588 all_barrier_nums,
589 }
590 }
591}
592
593pub(crate) struct DatabaseCheckpointControl {
595 database_id: DatabaseId,
596 state: BarrierWorkerState,
597
598 command_ctx_queue: BTreeMap<u64, EpochNode>,
601 completing_barrier: Option<u64>,
604
605 committed_epoch: Option<u64>,
606 creating_streaming_job_controls: HashMap<TableId, CreatingStreamingJobControl>,
607
608 create_mview_tracker: CreateMviewProgressTracker,
609
610 metrics: DatabaseCheckpointControlMetrics,
611}
612
613impl DatabaseCheckpointControl {
614 fn new(database_id: DatabaseId) -> Self {
615 Self {
616 database_id,
617 state: BarrierWorkerState::new(),
618 command_ctx_queue: Default::default(),
619 completing_barrier: None,
620 committed_epoch: None,
621 creating_streaming_job_controls: Default::default(),
622 create_mview_tracker: Default::default(),
623 metrics: DatabaseCheckpointControlMetrics::new(database_id),
624 }
625 }
626
627 pub(crate) fn recovery(
628 database_id: DatabaseId,
629 create_mview_tracker: CreateMviewProgressTracker,
630 state: BarrierWorkerState,
631 committed_epoch: u64,
632 creating_streaming_job_controls: HashMap<TableId, CreatingStreamingJobControl>,
633 ) -> Self {
634 Self {
635 database_id,
636 state,
637 command_ctx_queue: Default::default(),
638 completing_barrier: None,
639 committed_epoch: Some(committed_epoch),
640 creating_streaming_job_controls,
641 create_mview_tracker,
642 metrics: DatabaseCheckpointControlMetrics::new(database_id),
643 }
644 }
645
646 fn total_command_num(&self) -> usize {
647 self.command_ctx_queue.len()
648 + match &self.completing_barrier {
649 Some(_) => 1,
650 None => 0,
651 }
652 }
653
654 fn update_barrier_nums_metrics(&self) {
656 self.metrics.in_flight_barrier_nums.set(
657 self.command_ctx_queue
658 .values()
659 .filter(|x| x.state.is_inflight())
660 .count() as i64,
661 );
662 self.metrics
663 .all_barrier_nums
664 .set(self.total_command_num() as i64);
665 }
666
667 fn jobs_to_merge(
668 &self,
669 ) -> Option<HashMap<TableId, (HashSet<TableId>, InflightStreamingJobInfo)>> {
670 let mut table_ids_to_merge = HashMap::new();
671
672 for (table_id, creating_streaming_job) in &self.creating_streaming_job_controls {
673 if let Some(graph_info) = creating_streaming_job.should_merge_to_upstream() {
674 table_ids_to_merge.insert(
675 *table_id,
676 (
677 creating_streaming_job
678 .snapshot_backfill_upstream_tables
679 .clone(),
680 graph_info.clone(),
681 ),
682 );
683 }
684 }
685 if table_ids_to_merge.is_empty() {
686 None
687 } else {
688 Some(table_ids_to_merge)
689 }
690 }
691
692 fn enqueue_command(
694 &mut self,
695 command_ctx: CommandContext,
696 notifiers: Vec<Notifier>,
697 node_to_collect: NodeToCollect,
698 creating_jobs_to_wait: HashSet<TableId>,
699 ) {
700 let timer = self.metrics.barrier_latency.start_timer();
701
702 if let Some((_, node)) = self.command_ctx_queue.last_key_value() {
703 assert_eq!(
704 command_ctx.barrier_info.prev_epoch.value(),
705 node.command_ctx.barrier_info.curr_epoch.value()
706 );
707 }
708
709 tracing::trace!(
710 prev_epoch = command_ctx.barrier_info.prev_epoch(),
711 ?creating_jobs_to_wait,
712 ?node_to_collect,
713 "enqueue command"
714 );
715 self.command_ctx_queue.insert(
716 command_ctx.barrier_info.prev_epoch(),
717 EpochNode {
718 enqueue_time: timer,
719 state: BarrierEpochState {
720 node_to_collect,
721 resps: vec![],
722 creating_jobs_to_wait,
723 finished_jobs: HashMap::new(),
724 },
725 command_ctx,
726 notifiers,
727 },
728 );
729 }
730
731 fn barrier_collected(
734 &mut self,
735 resp: BarrierCompleteResponse,
736 periodic_barriers: &mut PeriodicBarriers,
737 ) -> MetaResult<()> {
738 let worker_id = resp.worker_id;
739 let prev_epoch = resp.epoch;
740 tracing::trace!(
741 worker_id,
742 prev_epoch,
743 partial_graph_id = resp.partial_graph_id,
744 "barrier collected"
745 );
746 let creating_job_id = from_partial_graph_id(resp.partial_graph_id);
747 match creating_job_id {
748 None => {
749 if let Some(node) = self.command_ctx_queue.get_mut(&prev_epoch) {
750 assert!(
751 node.state
752 .node_to_collect
753 .remove(&(worker_id as _))
754 .is_some()
755 );
756 node.state.resps.push(resp);
757 } else {
758 panic!(
759 "collect barrier on non-existing barrier: {}, {}",
760 prev_epoch, worker_id
761 );
762 }
763 }
764 Some(creating_job_id) => {
765 let should_merge_to_upstream = self
766 .creating_streaming_job_controls
767 .get_mut(&creating_job_id)
768 .expect("should exist")
769 .collect(resp);
770 if should_merge_to_upstream {
771 periodic_barriers.force_checkpoint_in_next_barrier();
772 }
773 }
774 }
775 Ok(())
776 }
777
778 fn can_inject_barrier(&self, in_flight_barrier_nums: usize) -> bool {
780 self.command_ctx_queue
781 .values()
782 .filter(|x| x.state.is_inflight())
783 .count()
784 < in_flight_barrier_nums
785 }
786
787 pub(crate) fn is_valid_after_worker_err(&mut self, worker_id: WorkerId) -> bool {
789 for epoch_node in self.command_ctx_queue.values_mut() {
790 if !is_valid_after_worker_err(&mut epoch_node.state.node_to_collect, worker_id) {
791 return false;
792 }
793 }
794 true
796 }
797}
798
799impl DatabaseCheckpointControl {
800 fn collect_backfill_pinned_upstream_log_epoch(
802 &self,
803 ) -> HashMap<TableId, (u64, HashSet<TableId>)> {
804 self.creating_streaming_job_controls
805 .iter()
806 .filter_map(|(table_id, creating_job)| {
807 creating_job
808 .pinned_upstream_log_epoch()
809 .map(|progress_epoch| {
810 (
811 *table_id,
812 (
813 progress_epoch,
814 creating_job.snapshot_backfill_upstream_tables.clone(),
815 ),
816 )
817 })
818 })
819 .collect()
820 }
821
822 fn next_complete_barrier_task(
823 &mut self,
824 task: &mut Option<CompleteBarrierTask>,
825 mut context: Option<(&mut PeriodicBarriers, &mut ControlStreamManager)>,
826 hummock_version_stats: &HummockVersionStats,
827 ) {
828 let mut creating_jobs_task = vec![];
830 {
831 let mut finished_jobs = Vec::new();
833 let min_upstream_inflight_barrier = self
834 .command_ctx_queue
835 .first_key_value()
836 .map(|(epoch, _)| *epoch);
837 for (table_id, job) in &mut self.creating_streaming_job_controls {
838 if let Some((epoch, resps, status)) =
839 job.start_completing(min_upstream_inflight_barrier)
840 {
841 let is_first_time = match status {
842 CompleteJobType::First => true,
843 CompleteJobType::Normal => false,
844 CompleteJobType::Finished => {
845 finished_jobs.push((*table_id, epoch, resps));
846 continue;
847 }
848 };
849 creating_jobs_task.push((*table_id, epoch, resps, is_first_time));
850 }
851 }
852 if !finished_jobs.is_empty()
853 && let Some((_, control_stream_manager)) = &mut context
854 {
855 control_stream_manager.remove_partial_graph(
856 self.database_id,
857 finished_jobs
858 .iter()
859 .map(|(table_id, _, _)| *table_id)
860 .collect(),
861 );
862 }
863 for (table_id, epoch, resps) in finished_jobs {
864 let epoch_state = &mut self
865 .command_ctx_queue
866 .get_mut(&epoch)
867 .expect("should exist")
868 .state;
869 assert!(epoch_state.creating_jobs_to_wait.remove(&table_id));
870 debug!(epoch, ?table_id, "finish creating job");
871 let creating_streaming_job = self
874 .creating_streaming_job_controls
875 .remove(&table_id)
876 .expect("should exist");
877 assert!(creating_streaming_job.is_finished());
878 assert!(epoch_state.finished_jobs.insert(table_id, resps).is_none());
879 }
880 }
881 assert!(self.completing_barrier.is_none());
882 while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value()
883 && !state.is_inflight()
884 {
885 {
886 let (_, mut node) = self.command_ctx_queue.pop_first().expect("non-empty");
887 assert!(node.state.creating_jobs_to_wait.is_empty());
888 assert!(node.state.node_to_collect.is_empty());
889 let mut finished_jobs = self.create_mview_tracker.apply_collected_command(
890 node.command_ctx.command.as_ref(),
891 &node.command_ctx.barrier_info,
892 &node.state.resps,
893 hummock_version_stats,
894 );
895 if !node.command_ctx.barrier_info.kind.is_checkpoint() {
896 assert!(finished_jobs.is_empty());
897 node.notifiers.into_iter().for_each(|notifier| {
898 notifier.notify_collected();
899 });
900 if let Some((periodic_barriers, _)) = &mut context
901 && self.create_mview_tracker.has_pending_finished_jobs()
902 && self
903 .command_ctx_queue
904 .values()
905 .all(|node| !node.command_ctx.barrier_info.kind.is_checkpoint())
906 {
907 periodic_barriers.force_checkpoint_in_next_barrier();
908 }
909 continue;
910 }
911 node.state
912 .finished_jobs
913 .drain()
914 .for_each(|(job_id, resps)| {
915 node.state.resps.extend(resps);
916 finished_jobs.push(TrackingJob::New(TrackingCommand {
917 job_id,
918 replace_stream_job: None,
919 }));
920 });
921 let task = task.get_or_insert_default();
922 node.command_ctx.collect_commit_epoch_info(
923 &mut task.commit_info,
924 take(&mut node.state.resps),
925 self.collect_backfill_pinned_upstream_log_epoch(),
926 );
927 self.completing_barrier = Some(node.command_ctx.barrier_info.prev_epoch());
928 task.finished_jobs.extend(finished_jobs);
929 task.notifiers.extend(node.notifiers);
930 task.epoch_infos
931 .try_insert(
932 self.database_id,
933 (Some((node.command_ctx, node.enqueue_time)), vec![]),
934 )
935 .expect("non duplicate");
936 break;
937 }
938 }
939 if !creating_jobs_task.is_empty() {
940 let task = task.get_or_insert_default();
941 for (table_id, epoch, resps, is_first_time) in creating_jobs_task {
942 collect_creating_job_commit_epoch_info(
943 &mut task.commit_info,
944 epoch,
945 resps,
946 self.creating_streaming_job_controls[&table_id].state_table_ids(),
947 is_first_time,
948 );
949 let (_, creating_job_epochs) =
950 task.epoch_infos.entry(self.database_id).or_default();
951 creating_job_epochs.push((table_id, epoch));
952 }
953 }
954 }
955
956 fn ack_completed(
957 &mut self,
958 command_prev_epoch: Option<u64>,
959 creating_job_epochs: Vec<(TableId, u64)>,
960 ) {
961 {
962 if let Some(prev_epoch) = self.completing_barrier.take() {
963 assert_eq!(command_prev_epoch, Some(prev_epoch));
964 self.committed_epoch = Some(prev_epoch);
965 } else {
966 assert_eq!(command_prev_epoch, None);
967 };
968 for (table_id, epoch) in creating_job_epochs {
969 self.creating_streaming_job_controls
970 .get_mut(&table_id)
971 .expect("should exist")
972 .ack_completed(epoch)
973 }
974 }
975 }
976}
977
978struct EpochNode {
980 enqueue_time: HistogramTimer,
982
983 state: BarrierEpochState,
985
986 command_ctx: CommandContext,
988 notifiers: Vec<Notifier>,
990}
991
992#[derive(Debug)]
993struct BarrierEpochState {
995 node_to_collect: NodeToCollect,
996
997 resps: Vec<BarrierCompleteResponse>,
998
999 creating_jobs_to_wait: HashSet<TableId>,
1000
1001 finished_jobs: HashMap<TableId, Vec<BarrierCompleteResponse>>,
1002}
1003
1004impl BarrierEpochState {
1005 fn is_inflight(&self) -> bool {
1006 !self.node_to_collect.is_empty() || !self.creating_jobs_to_wait.is_empty()
1007 }
1008}
1009
1010impl DatabaseCheckpointControl {
1011 fn handle_new_barrier(
1013 &mut self,
1014 command: Option<(Command, Vec<Notifier>)>,
1015 checkpoint: bool,
1016 span: tracing::Span,
1017 control_stream_manager: &mut ControlStreamManager,
1018 hummock_version_stats: &HummockVersionStats,
1019 curr_epoch: TracedEpoch,
1020 ) -> MetaResult<()> {
1021 let (mut command, mut notifiers) = if let Some((command, notifiers)) = command {
1022 (Some(command), notifiers)
1023 } else {
1024 (None, vec![])
1025 };
1026
1027 for table_to_cancel in command
1028 .as_ref()
1029 .map(Command::tables_to_drop)
1030 .into_iter()
1031 .flatten()
1032 {
1033 if self
1034 .creating_streaming_job_controls
1035 .contains_key(&table_to_cancel)
1036 {
1037 warn!(
1038 table_id = table_to_cancel.table_id,
1039 "ignore cancel command on creating streaming job"
1040 );
1041 for notifier in notifiers {
1042 notifier
1043 .notify_start_failed(anyhow!("cannot cancel creating streaming job, the job will continue creating until created or recovery").into());
1044 }
1045 return Ok(());
1046 }
1047 }
1048
1049 if let Some(Command::RescheduleFragment { .. }) = &command {
1050 if !self.creating_streaming_job_controls.is_empty() {
1051 warn!("ignore reschedule when creating streaming job with snapshot backfill");
1052 for notifier in notifiers {
1053 notifier.notify_start_failed(
1054 anyhow!(
1055 "cannot reschedule when creating streaming job with snapshot backfill",
1056 )
1057 .into(),
1058 );
1059 }
1060 return Ok(());
1061 }
1062 }
1063
1064 let Some(barrier_info) =
1065 self.state
1066 .next_barrier_info(command.as_ref(), checkpoint, curr_epoch)
1067 else {
1068 for mut notifier in notifiers {
1070 notifier.notify_started();
1071 notifier.notify_collected();
1072 }
1073 return Ok(());
1074 };
1075
1076 let mut edges = self
1077 .state
1078 .inflight_graph_info
1079 .build_edge(command.as_ref(), &*control_stream_manager);
1080
1081 if let Some(Command::CreateStreamingJob {
1083 job_type,
1084 info,
1085 cross_db_snapshot_backfill_info,
1086 }) = &mut command
1087 {
1088 match job_type {
1089 CreateStreamingJobType::Normal | CreateStreamingJobType::SinkIntoTable(_) => {
1090 for fragment in info.stream_job_fragments.inner.fragments.values_mut() {
1091 fill_snapshot_backfill_epoch(
1092 &mut fragment.nodes,
1093 None,
1094 cross_db_snapshot_backfill_info,
1095 )?;
1096 }
1097 }
1098 CreateStreamingJobType::SnapshotBackfill(snapshot_backfill_info) => {
1099 if self.state.is_paused() {
1100 warn!("cannot create streaming job with snapshot backfill when paused");
1101 for notifier in notifiers {
1102 notifier.notify_start_failed(
1103 anyhow!("cannot create streaming job with snapshot backfill when paused",)
1104 .into(),
1105 );
1106 }
1107 return Ok(());
1108 }
1109 for snapshot_backfill_epoch in snapshot_backfill_info
1111 .upstream_mv_table_id_to_backfill_epoch
1112 .values_mut()
1113 {
1114 assert_eq!(
1115 snapshot_backfill_epoch.replace(barrier_info.prev_epoch()),
1116 None,
1117 "must not set previously"
1118 );
1119 }
1120 for fragment in info.stream_job_fragments.inner.fragments.values_mut() {
1121 if let Err(e) = fill_snapshot_backfill_epoch(
1122 &mut fragment.nodes,
1123 Some(snapshot_backfill_info),
1124 cross_db_snapshot_backfill_info,
1125 ) {
1126 warn!(e = %e.as_report(), "failed to fill snapshot backfill epoch");
1127 for notifier in notifiers {
1128 notifier.notify_start_failed(e.clone());
1129 }
1130 return Ok(());
1131 };
1132 }
1133 let job_id = info.stream_job_fragments.stream_job_id();
1134 let snapshot_backfill_upstream_tables = snapshot_backfill_info
1135 .upstream_mv_table_id_to_backfill_epoch
1136 .keys()
1137 .cloned()
1138 .collect();
1139
1140 self.creating_streaming_job_controls.insert(
1141 job_id,
1142 CreatingStreamingJobControl::new(
1143 info,
1144 snapshot_backfill_upstream_tables,
1145 barrier_info.prev_epoch(),
1146 hummock_version_stats,
1147 control_stream_manager,
1148 edges.as_mut().expect("should exist"),
1149 )?,
1150 );
1151 }
1152 }
1153 }
1154
1155 if let (BarrierKind::Checkpoint(_), None) = (&barrier_info.kind, &command)
1157 && let Some(jobs_to_merge) = self.jobs_to_merge()
1158 {
1159 command = Some(Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge));
1160 }
1161
1162 let command = command;
1163
1164 let (
1165 pre_applied_graph_info,
1166 pre_applied_subscription_info,
1167 table_ids_to_commit,
1168 jobs_to_wait,
1169 prev_paused_reason,
1170 ) = self.state.apply_command(command.as_ref());
1171
1172 barrier_info.prev_epoch.span().in_scope(|| {
1174 tracing::info!(target: "rw_tracing", epoch = barrier_info.curr_epoch.value().0, "new barrier enqueued");
1175 });
1176 span.record("epoch", barrier_info.curr_epoch.value().0);
1177
1178 for creating_job in &mut self.creating_streaming_job_controls.values_mut() {
1179 creating_job.on_new_command(control_stream_manager, command.as_ref(), &barrier_info)?;
1180 }
1181
1182 let node_to_collect = match control_stream_manager.inject_command_ctx_barrier(
1183 self.database_id,
1184 command.as_ref(),
1185 &barrier_info,
1186 prev_paused_reason,
1187 &pre_applied_graph_info,
1188 &self.state.inflight_graph_info,
1189 &mut edges,
1190 ) {
1191 Ok(node_to_collect) => node_to_collect,
1192 Err(err) => {
1193 for notifier in notifiers {
1194 notifier.notify_start_failed(err.clone());
1195 }
1196 fail_point!("inject_barrier_err_success");
1197 return Err(err);
1198 }
1199 };
1200
1201 notifiers.iter_mut().for_each(|n| n.notify_started());
1203
1204 let command_ctx = CommandContext::new(
1205 barrier_info,
1206 pre_applied_subscription_info,
1207 table_ids_to_commit.clone(),
1208 command,
1209 span,
1210 );
1211
1212 self.enqueue_command(command_ctx, notifiers, node_to_collect, jobs_to_wait);
1214
1215 Ok(())
1216 }
1217}