1use std::collections::HashMap;
16use std::collections::hash_map::Entry;
17use std::mem::take;
18
19use risingwave_common::catalog::TableId;
20use risingwave_common::util::epoch::Epoch;
21use risingwave_meta_model::ObjectId;
22use risingwave_pb::catalog::CreateType;
23use risingwave_pb::ddl_service::DdlProgress;
24use risingwave_pb::hummock::HummockVersionStats;
25use risingwave_pb::stream_service::PbBarrierCompleteResponse;
26use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress;
27
28use crate::MetaResult;
29use crate::barrier::backfill_order_control::BackfillOrderState;
30use crate::barrier::info::BarrierInfo;
31use crate::barrier::{
32 Command, CreateStreamingJobCommandInfo, CreateStreamingJobType, ReplaceStreamJobPlan,
33};
34use crate::manager::{MetadataManager, StreamingJobType};
35use crate::model::{ActorId, BackfillUpstreamType, FragmentId, StreamJobFragments};
36
37type ConsumedRows = u64;
38
39#[derive(Clone, Copy, Debug)]
40enum BackfillState {
41 Init,
42 ConsumingUpstream(#[expect(dead_code)] Epoch, ConsumedRows),
43 Done(ConsumedRows),
44}
45
46#[derive(Debug)]
48pub(super) struct Progress {
49 states: HashMap<ActorId, BackfillState>,
51 backfill_order_state: BackfillOrderState,
52 done_count: usize,
53
54 backfill_upstream_types: HashMap<ActorId, BackfillUpstreamType>,
56
57 upstream_mv_count: HashMap<TableId, usize>,
62 upstream_mvs_total_key_count: u64,
64 mv_backfill_consumed_rows: u64,
65 source_backfill_consumed_rows: u64,
66
67 definition: String,
69}
70
71impl Progress {
72 fn new(
74 actors: impl IntoIterator<Item = (ActorId, BackfillUpstreamType)>,
75 upstream_mv_count: HashMap<TableId, usize>,
76 upstream_total_key_count: u64,
77 definition: String,
78 backfill_order_state: BackfillOrderState,
79 ) -> Self {
80 let mut states = HashMap::new();
81 let mut backfill_upstream_types = HashMap::new();
82 for (actor, backfill_upstream_type) in actors {
83 states.insert(actor, BackfillState::Init);
84 backfill_upstream_types.insert(actor, backfill_upstream_type);
85 }
86 assert!(!states.is_empty());
87
88 Self {
89 states,
90 backfill_upstream_types,
91 done_count: 0,
92 upstream_mv_count,
93 upstream_mvs_total_key_count: upstream_total_key_count,
94 mv_backfill_consumed_rows: 0,
95 source_backfill_consumed_rows: 0,
96 definition,
97 backfill_order_state,
98 }
99 }
100
101 fn update(
103 &mut self,
104 actor: ActorId,
105 new_state: BackfillState,
106 upstream_total_key_count: u64,
107 ) -> Vec<FragmentId> {
108 let mut next_backfill_nodes = vec![];
109 self.upstream_mvs_total_key_count = upstream_total_key_count;
110 let total_actors = self.states.len();
111 let backfill_upstream_type = self.backfill_upstream_types.get(&actor).unwrap();
112 tracing::debug!(?actor, states = ?self.states, "update progress for actor");
113
114 let mut old = 0;
115 let mut new = 0;
116 match self.states.remove(&actor).unwrap() {
117 BackfillState::Init => {}
118 BackfillState::ConsumingUpstream(_, old_consumed_rows) => {
119 old = old_consumed_rows;
120 }
121 BackfillState::Done(_) => panic!("should not report done multiple times"),
122 };
123 match &new_state {
124 BackfillState::Init => {}
125 BackfillState::ConsumingUpstream(_, new_consumed_rows) => {
126 new = *new_consumed_rows;
127 }
128 BackfillState::Done(new_consumed_rows) => {
129 tracing::debug!("actor {} done", actor);
130 new = *new_consumed_rows;
131 self.done_count += 1;
132 next_backfill_nodes = self.backfill_order_state.finish_actor(actor);
133 tracing::debug!(
134 "{} actors out of {} complete",
135 self.done_count,
136 total_actors,
137 );
138 }
139 };
140 debug_assert!(new >= old, "backfill progress should not go backward");
141 match backfill_upstream_type {
142 BackfillUpstreamType::MView => {
143 self.mv_backfill_consumed_rows += new - old;
144 }
145 BackfillUpstreamType::Source => {
146 self.source_backfill_consumed_rows += new - old;
147 }
148 BackfillUpstreamType::Values => {
149 }
151 }
152 self.states.insert(actor, new_state);
153 next_backfill_nodes
154 }
155
156 fn is_done(&self) -> bool {
158 tracing::trace!(
159 "Progress::is_done? {}, {}, {:?}",
160 self.done_count,
161 self.states.len(),
162 self.states
163 );
164 self.done_count == self.states.len()
165 }
166
167 fn actors(&self) -> impl Iterator<Item = ActorId> + '_ {
170 self.states.keys().cloned()
171 }
172
173 fn calculate_progress(&self) -> String {
175 if self.is_done() || self.states.is_empty() {
176 return "100%".to_owned();
177 }
178 let mut mv_count = 0;
179 let mut source_count = 0;
180 for backfill_upstream_type in self.backfill_upstream_types.values() {
181 match backfill_upstream_type {
182 BackfillUpstreamType::MView => mv_count += 1,
183 BackfillUpstreamType::Source => source_count += 1,
184 BackfillUpstreamType::Values => (),
185 }
186 }
187
188 let mv_progress = (mv_count > 0).then_some({
189 if self.upstream_mvs_total_key_count == 0 {
190 "99.99%".to_owned()
191 } else {
192 let mut progress = self.mv_backfill_consumed_rows as f64
193 / (self.upstream_mvs_total_key_count as f64);
194 if progress > 1.0 {
195 progress = 0.9999;
196 }
197 format!(
198 "{:.2}% ({}/{})",
199 progress * 100.0,
200 self.mv_backfill_consumed_rows,
201 self.upstream_mvs_total_key_count
202 )
203 }
204 });
205 let source_progress = (source_count > 0).then_some(format!(
206 "{} rows consumed",
207 self.source_backfill_consumed_rows
208 ));
209 match (mv_progress, source_progress) {
210 (Some(mv_progress), Some(source_progress)) => {
211 format!(
212 "MView Backfill: {}, Source Backfill: {}",
213 mv_progress, source_progress
214 )
215 }
216 (Some(mv_progress), None) => mv_progress,
217 (None, Some(source_progress)) => source_progress,
218 (None, None) => "Unknown".to_owned(),
219 }
220 }
221}
222
223pub enum TrackingJob {
230 New(TrackingCommand),
231 Recovered(RecoveredTrackingJob),
232}
233
234impl std::fmt::Display for TrackingJob {
235 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
236 match self {
237 TrackingJob::New(command) => write!(f, "{}", command.job_id),
238 TrackingJob::Recovered(recovered) => write!(f, "{}<recovered>", recovered.id),
239 }
240 }
241}
242
243impl TrackingJob {
244 pub(crate) async fn finish(self, metadata_manager: &MetadataManager) -> MetaResult<()> {
246 match self {
247 TrackingJob::New(command) => {
248 metadata_manager
249 .catalog_controller
250 .finish_streaming_job(
251 command.job_id.table_id as i32,
252 command.replace_stream_job.clone(),
253 )
254 .await?;
255 Ok(())
256 }
257 TrackingJob::Recovered(recovered) => {
258 metadata_manager
259 .catalog_controller
260 .finish_streaming_job(recovered.id, None)
261 .await?;
262 Ok(())
263 }
264 }
265 }
266
267 pub(crate) fn table_to_create(&self) -> TableId {
268 match self {
269 TrackingJob::New(command) => command.job_id,
270 TrackingJob::Recovered(recovered) => (recovered.id as u32).into(),
271 }
272 }
273}
274
275impl std::fmt::Debug for TrackingJob {
276 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
277 match self {
278 TrackingJob::New(command) => write!(f, "TrackingJob::New({:?})", command.job_id),
279 TrackingJob::Recovered(recovered) => {
280 write!(f, "TrackingJob::RecoveredV2({:?})", recovered.id)
281 }
282 }
283 }
284}
285
286pub struct RecoveredTrackingJob {
287 pub id: ObjectId,
288}
289
290pub(super) struct TrackingCommand {
292 pub job_id: TableId,
293 pub replace_stream_job: Option<ReplaceStreamJobPlan>,
294}
295
296pub(super) enum UpdateProgressResult {
297 None,
298 Finished(TrackingJob),
299 BackfillNodeFinished(Vec<FragmentId>),
300}
301
302#[derive(Default, Debug)]
308pub(super) struct CreateMviewProgressTracker {
309 progress_map: HashMap<TableId, (Progress, TrackingJob)>,
311
312 actor_map: HashMap<ActorId, TableId>,
313
314 pending_finished_jobs: Vec<TrackingJob>,
316
317 pending_backfill_nodes: Vec<FragmentId>,
319}
320
321impl CreateMviewProgressTracker {
322 pub fn recover(
330 mviews: impl IntoIterator<Item = (TableId, (String, &StreamJobFragments))>,
331 version_stats: &HummockVersionStats,
332 ) -> Self {
333 let mut actor_map = HashMap::new();
334 let mut progress_map = HashMap::new();
335 for (creating_table_id, (definition, table_fragments)) in mviews {
336 let mut states = HashMap::new();
337 let mut backfill_upstream_types = HashMap::new();
338 let actors = table_fragments.tracking_progress_actor_ids();
339 for (actor, backfill_upstream_type) in actors {
340 actor_map.insert(actor, creating_table_id);
341 states.insert(actor, BackfillState::ConsumingUpstream(Epoch(0), 0));
342 backfill_upstream_types.insert(actor, backfill_upstream_type);
343 }
344
345 let progress = Self::recover_progress(
346 states,
347 backfill_upstream_types,
348 table_fragments.upstream_table_counts(),
349 definition,
350 version_stats,
351 );
352 let tracking_job = TrackingJob::Recovered(RecoveredTrackingJob {
353 id: creating_table_id.table_id as i32,
354 });
355 progress_map.insert(creating_table_id, (progress, tracking_job));
356 }
357 Self {
358 progress_map,
359 actor_map,
360 pending_finished_jobs: Vec::new(),
361 pending_backfill_nodes: Vec::new(),
362 }
363 }
364
365 fn recover_progress(
371 states: HashMap<ActorId, BackfillState>,
372 backfill_upstream_types: HashMap<ActorId, BackfillUpstreamType>,
373 upstream_mv_count: HashMap<TableId, usize>,
374 definition: String,
375 version_stats: &HummockVersionStats,
376 ) -> Progress {
377 let upstream_mvs_total_key_count =
378 calculate_total_key_count(&upstream_mv_count, version_stats);
379 Progress {
380 states,
381 backfill_order_state: Default::default(),
382 backfill_upstream_types,
383 done_count: 0, upstream_mv_count,
385 upstream_mvs_total_key_count,
386 mv_backfill_consumed_rows: 0, source_backfill_consumed_rows: 0, definition,
389 }
390 }
391
392 pub fn gen_ddl_progress(&self) -> HashMap<u32, DdlProgress> {
393 self.progress_map
394 .iter()
395 .map(|(table_id, (x, _))| {
396 let table_id = table_id.table_id;
397 let ddl_progress = DdlProgress {
398 id: table_id as u64,
399 statement: x.definition.clone(),
400 progress: x.calculate_progress(),
401 };
402 (table_id, ddl_progress)
403 })
404 .collect()
405 }
406
407 pub(super) fn update_tracking_jobs<'a>(
408 &mut self,
409 info: Option<(
410 &CreateStreamingJobCommandInfo,
411 Option<&ReplaceStreamJobPlan>,
412 )>,
413 create_mview_progress: impl IntoIterator<Item = &'a CreateMviewProgress>,
414 version_stats: &HummockVersionStats,
415 ) {
416 {
417 {
418 let finished_commands = {
420 let mut commands = vec![];
421 if let Some((create_job_info, replace_stream_job)) = info
423 && let Some(command) =
424 self.add(create_job_info, replace_stream_job, version_stats)
425 {
426 commands.push(command);
428 }
429 for progress in create_mview_progress {
431 match self.update(progress, version_stats) {
433 UpdateProgressResult::None => {
434 tracing::trace!(?progress, "update progress");
435 }
436 UpdateProgressResult::Finished(command) => {
437 tracing::trace!(?progress, "finish progress");
438 commands.push(command);
439 }
440 UpdateProgressResult::BackfillNodeFinished(next_backfill_nodes) => {
441 tracing::trace!(
442 ?progress,
443 ?next_backfill_nodes,
444 "start next backfill node"
445 );
446 self.queue_backfill(next_backfill_nodes);
447 }
448 }
449 }
450 commands
451 };
452
453 for command in finished_commands {
454 self.stash_command_to_finish(command);
455 }
456 }
457 }
458 }
459
460 pub(super) fn apply_collected_command(
463 &mut self,
464 command: Option<&Command>,
465 barrier_info: &BarrierInfo,
466 resps: impl IntoIterator<Item = &PbBarrierCompleteResponse>,
467 version_stats: &HummockVersionStats,
468 ) -> Vec<TrackingJob> {
469 let new_tracking_job_info =
470 if let Some(Command::CreateStreamingJob { info, job_type, .. }) = command {
471 match job_type {
472 CreateStreamingJobType::Normal => Some((info, None)),
473 CreateStreamingJobType::SinkIntoTable(replace_stream_job) => {
474 Some((info, Some(replace_stream_job)))
475 }
476 CreateStreamingJobType::SnapshotBackfill(_) => {
477 None
479 }
480 }
481 } else {
482 None
483 };
484 self.update_tracking_jobs(
485 new_tracking_job_info,
486 resps
487 .into_iter()
488 .flat_map(|resp| resp.create_mview_progress.iter()),
489 version_stats,
490 );
491 for table_id in command.map(Command::tables_to_drop).into_iter().flatten() {
492 self.cancel_command(table_id);
495 }
496 if barrier_info.kind.is_checkpoint() {
497 self.take_finished_jobs()
498 } else {
499 vec![]
500 }
501 }
502
503 pub(super) fn stash_command_to_finish(&mut self, finished_job: TrackingJob) {
505 self.pending_finished_jobs.push(finished_job);
506 }
507
508 fn queue_backfill(&mut self, backfill_nodes: impl IntoIterator<Item = FragmentId>) {
509 self.pending_backfill_nodes.extend(backfill_nodes);
510 }
511
512 pub(super) fn take_finished_jobs(&mut self) -> Vec<TrackingJob> {
514 tracing::trace!(finished_jobs=?self.pending_finished_jobs, progress_map=?self.progress_map, "take_finished_jobs");
515 take(&mut self.pending_finished_jobs)
516 }
517
518 pub(super) fn take_pending_backfill_nodes(&mut self) -> Vec<FragmentId> {
519 take(&mut self.pending_backfill_nodes)
520 }
521
522 pub(super) fn has_pending_finished_jobs(&self) -> bool {
523 !self.pending_finished_jobs.is_empty()
524 }
525
526 pub(super) fn cancel_command(&mut self, id: TableId) {
527 let _ = self.progress_map.remove(&id);
528 self.pending_finished_jobs
529 .retain(|x| x.table_to_create() != id);
530 self.actor_map.retain(|_, table_id| *table_id != id);
531 }
532
533 pub fn abort_all(&mut self) {
535 self.actor_map.clear();
536 self.pending_finished_jobs.clear();
537 self.progress_map.clear();
538 }
539
540 pub fn add(
544 &mut self,
545 info: &CreateStreamingJobCommandInfo,
546 replace_stream_job: Option<&ReplaceStreamJobPlan>,
547 version_stats: &HummockVersionStats,
548 ) -> Option<TrackingJob> {
549 tracing::trace!(?info, "add job to track");
550 let (info, actors, replace_table_info) = {
551 let CreateStreamingJobCommandInfo {
552 stream_job_fragments,
553 ..
554 } = info;
555 let actors = stream_job_fragments.tracking_progress_actor_ids();
556 if actors.is_empty() {
557 return Some(TrackingJob::New(TrackingCommand {
559 job_id: info.stream_job_fragments.stream_job_id,
560 replace_stream_job: replace_stream_job.cloned(),
561 }));
562 }
563 (info.clone(), actors, replace_stream_job.cloned())
564 };
565
566 let CreateStreamingJobCommandInfo {
567 stream_job_fragments: table_fragments,
568 definition,
569 job_type,
570 create_type,
571 fragment_backfill_ordering,
572 ..
573 } = info;
574
575 let creating_mv_id = table_fragments.stream_job_id();
576 let upstream_mv_count = table_fragments.upstream_table_counts();
577 let upstream_total_key_count: u64 =
578 calculate_total_key_count(&upstream_mv_count, version_stats);
579
580 for (actor, _backfill_upstream_type) in &actors {
581 self.actor_map.insert(*actor, creating_mv_id);
582 }
583
584 let backfill_order_state =
585 BackfillOrderState::new(fragment_backfill_ordering, &table_fragments);
586 let progress = Progress::new(
587 actors,
588 upstream_mv_count,
589 upstream_total_key_count,
590 definition.clone(),
591 backfill_order_state,
592 );
593 if job_type == StreamingJobType::Sink && create_type == CreateType::Background {
594 Some(TrackingJob::New(TrackingCommand {
600 job_id: creating_mv_id,
601 replace_stream_job: replace_table_info,
602 }))
603 } else {
604 let old = self.progress_map.insert(
605 creating_mv_id,
606 (
607 progress,
608 TrackingJob::New(TrackingCommand {
609 job_id: creating_mv_id,
610 replace_stream_job: replace_table_info,
611 }),
612 ),
613 );
614 assert!(old.is_none());
615 None
616 }
617 }
618
619 pub fn update(
623 &mut self,
624 progress: &CreateMviewProgress,
625 version_stats: &HummockVersionStats,
626 ) -> UpdateProgressResult {
627 tracing::trace!(?progress, "update progress");
628 let actor = progress.backfill_actor_id;
629 let Some(table_id) = self.actor_map.get(&actor).copied() else {
630 tracing::info!(
637 "no tracked progress for actor {}, the stream job could already be finished",
638 actor
639 );
640 return UpdateProgressResult::None;
641 };
642
643 let new_state = if progress.done {
644 BackfillState::Done(progress.consumed_rows)
645 } else {
646 BackfillState::ConsumingUpstream(progress.consumed_epoch.into(), progress.consumed_rows)
647 };
648
649 match self.progress_map.entry(table_id) {
650 Entry::Occupied(mut o) => {
651 let progress = &mut o.get_mut().0;
652
653 let upstream_total_key_count: u64 =
654 calculate_total_key_count(&progress.upstream_mv_count, version_stats);
655
656 tracing::debug!(?table_id, "updating progress for table");
657 let next_backfill_nodes =
658 progress.update(actor, new_state, upstream_total_key_count);
659
660 if progress.is_done() {
661 tracing::debug!(
662 "all actors done for creating mview with table_id {}!",
663 table_id
664 );
665
666 for actor in o.get().0.actors() {
668 self.actor_map.remove(&actor);
669 }
670 assert!(next_backfill_nodes.is_empty());
671 UpdateProgressResult::Finished(o.remove().1)
672 } else if !next_backfill_nodes.is_empty() {
673 tracing::debug!("scheduling next backfill nodes: {:?}", next_backfill_nodes);
674 UpdateProgressResult::BackfillNodeFinished(next_backfill_nodes)
675 } else {
676 UpdateProgressResult::None
677 }
678 }
679 Entry::Vacant(_) => {
680 tracing::warn!(
681 "update the progress of an non-existent creating streaming job: {progress:?}, which could be cancelled"
682 );
683 UpdateProgressResult::None
684 }
685 }
686 }
687}
688
689fn calculate_total_key_count(
690 table_count: &HashMap<TableId, usize>,
691 version_stats: &HummockVersionStats,
692) -> u64 {
693 table_count
694 .iter()
695 .map(|(table_id, count)| {
696 assert_ne!(*count, 0);
697 *count as u64
698 * version_stats
699 .table_stats
700 .get(&table_id.table_id)
701 .map_or(0, |stat| stat.total_key_count as u64)
702 })
703 .sum()
704}