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 TrackingJob {
235 pub(crate) async fn finish(self, metadata_manager: &MetadataManager) -> MetaResult<()> {
237 match self {
238 TrackingJob::New(command) => {
239 metadata_manager
240 .catalog_controller
241 .finish_streaming_job(
242 command.job_id.table_id as i32,
243 command.replace_stream_job.clone(),
244 )
245 .await?;
246 Ok(())
247 }
248 TrackingJob::Recovered(recovered) => {
249 metadata_manager
250 .catalog_controller
251 .finish_streaming_job(recovered.id, None)
252 .await?;
253 Ok(())
254 }
255 }
256 }
257
258 pub(crate) fn table_to_create(&self) -> TableId {
259 match self {
260 TrackingJob::New(command) => command.job_id,
261 TrackingJob::Recovered(recovered) => (recovered.id as u32).into(),
262 }
263 }
264}
265
266impl std::fmt::Debug for TrackingJob {
267 fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
268 match self {
269 TrackingJob::New(command) => write!(f, "TrackingJob::New({:?})", command.job_id),
270 TrackingJob::Recovered(recovered) => {
271 write!(f, "TrackingJob::RecoveredV2({:?})", recovered.id)
272 }
273 }
274 }
275}
276
277pub struct RecoveredTrackingJob {
278 pub id: ObjectId,
279}
280
281pub(super) struct TrackingCommand {
283 pub job_id: TableId,
284 pub replace_stream_job: Option<ReplaceStreamJobPlan>,
285}
286
287pub(super) enum UpdateProgressResult {
288 None,
289 Finished(TrackingJob),
290 BackfillNodeFinished(Vec<FragmentId>),
291}
292
293#[derive(Default, Debug)]
299pub(super) struct CreateMviewProgressTracker {
300 progress_map: HashMap<TableId, (Progress, TrackingJob)>,
302
303 actor_map: HashMap<ActorId, TableId>,
304
305 pending_finished_jobs: Vec<TrackingJob>,
307
308 pending_backfill_nodes: Vec<FragmentId>,
310}
311
312impl CreateMviewProgressTracker {
313 pub fn recover(
321 mviews: impl IntoIterator<Item = (TableId, (String, &StreamJobFragments))>,
322 version_stats: &HummockVersionStats,
323 ) -> Self {
324 let mut actor_map = HashMap::new();
325 let mut progress_map = HashMap::new();
326 for (creating_table_id, (definition, table_fragments)) in mviews {
327 let mut states = HashMap::new();
328 let mut backfill_upstream_types = HashMap::new();
329 let actors = table_fragments.tracking_progress_actor_ids();
330 for (actor, backfill_upstream_type) in actors {
331 actor_map.insert(actor, creating_table_id);
332 states.insert(actor, BackfillState::ConsumingUpstream(Epoch(0), 0));
333 backfill_upstream_types.insert(actor, backfill_upstream_type);
334 }
335
336 let progress = Self::recover_progress(
337 states,
338 backfill_upstream_types,
339 table_fragments.upstream_table_counts(),
340 definition,
341 version_stats,
342 );
343 let tracking_job = TrackingJob::Recovered(RecoveredTrackingJob {
344 id: creating_table_id.table_id as i32,
345 });
346 progress_map.insert(creating_table_id, (progress, tracking_job));
347 }
348 Self {
349 progress_map,
350 actor_map,
351 pending_finished_jobs: Vec::new(),
352 pending_backfill_nodes: Vec::new(),
353 }
354 }
355
356 fn recover_progress(
362 states: HashMap<ActorId, BackfillState>,
363 backfill_upstream_types: HashMap<ActorId, BackfillUpstreamType>,
364 upstream_mv_count: HashMap<TableId, usize>,
365 definition: String,
366 version_stats: &HummockVersionStats,
367 ) -> Progress {
368 let upstream_mvs_total_key_count =
369 calculate_total_key_count(&upstream_mv_count, version_stats);
370 Progress {
371 states,
372 backfill_order_state: Default::default(),
373 backfill_upstream_types,
374 done_count: 0, upstream_mv_count,
376 upstream_mvs_total_key_count,
377 mv_backfill_consumed_rows: 0, source_backfill_consumed_rows: 0, definition,
380 }
381 }
382
383 pub fn gen_ddl_progress(&self) -> HashMap<u32, DdlProgress> {
384 self.progress_map
385 .iter()
386 .map(|(table_id, (x, _))| {
387 let table_id = table_id.table_id;
388 let ddl_progress = DdlProgress {
389 id: table_id as u64,
390 statement: x.definition.clone(),
391 progress: x.calculate_progress(),
392 };
393 (table_id, ddl_progress)
394 })
395 .collect()
396 }
397
398 pub(super) fn update_tracking_jobs<'a>(
399 &mut self,
400 info: Option<(
401 &CreateStreamingJobCommandInfo,
402 Option<&ReplaceStreamJobPlan>,
403 )>,
404 create_mview_progress: impl IntoIterator<Item = &'a CreateMviewProgress>,
405 version_stats: &HummockVersionStats,
406 ) {
407 {
408 {
409 let finished_commands = {
411 let mut commands = vec![];
412 if let Some((create_job_info, replace_stream_job)) = info
414 && let Some(command) =
415 self.add(create_job_info, replace_stream_job, version_stats)
416 {
417 commands.push(command);
419 }
420 for progress in create_mview_progress {
422 match self.update(progress, version_stats) {
424 UpdateProgressResult::None => {
425 tracing::trace!(?progress, "update progress");
426 }
427 UpdateProgressResult::Finished(command) => {
428 tracing::trace!(?progress, "finish progress");
429 commands.push(command);
430 }
431 UpdateProgressResult::BackfillNodeFinished(next_backfill_nodes) => {
432 tracing::trace!(
433 ?progress,
434 ?next_backfill_nodes,
435 "start next backfill node"
436 );
437 self.queue_backfill(next_backfill_nodes);
438 }
439 }
440 }
441 commands
442 };
443
444 for command in finished_commands {
445 self.stash_command_to_finish(command);
446 }
447 }
448 }
449 }
450
451 pub(super) fn apply_collected_command(
454 &mut self,
455 command: Option<&Command>,
456 barrier_info: &BarrierInfo,
457 resps: impl IntoIterator<Item = &PbBarrierCompleteResponse>,
458 version_stats: &HummockVersionStats,
459 ) -> Vec<TrackingJob> {
460 let new_tracking_job_info =
461 if let Some(Command::CreateStreamingJob { info, job_type, .. }) = command {
462 match job_type {
463 CreateStreamingJobType::Normal => Some((info, None)),
464 CreateStreamingJobType::SinkIntoTable(replace_stream_job) => {
465 Some((info, Some(replace_stream_job)))
466 }
467 CreateStreamingJobType::SnapshotBackfill(_) => {
468 None
470 }
471 }
472 } else {
473 None
474 };
475 self.update_tracking_jobs(
476 new_tracking_job_info,
477 resps
478 .into_iter()
479 .flat_map(|resp| resp.create_mview_progress.iter()),
480 version_stats,
481 );
482 for table_id in command.map(Command::tables_to_drop).into_iter().flatten() {
483 self.cancel_command(table_id);
486 }
487 if barrier_info.kind.is_checkpoint() {
488 self.take_finished_jobs()
489 } else {
490 vec![]
491 }
492 }
493
494 pub(super) fn stash_command_to_finish(&mut self, finished_job: TrackingJob) {
496 self.pending_finished_jobs.push(finished_job);
497 }
498
499 fn queue_backfill(&mut self, backfill_nodes: impl IntoIterator<Item = FragmentId>) {
500 self.pending_backfill_nodes.extend(backfill_nodes);
501 }
502
503 pub(super) fn take_finished_jobs(&mut self) -> Vec<TrackingJob> {
505 tracing::trace!(finished_jobs=?self.pending_finished_jobs, progress_map=?self.progress_map, "take_finished_jobs");
506 take(&mut self.pending_finished_jobs)
507 }
508
509 pub(super) fn take_pending_backfill_nodes(&mut self) -> Vec<FragmentId> {
510 take(&mut self.pending_backfill_nodes)
511 }
512
513 pub(super) fn has_pending_finished_jobs(&self) -> bool {
514 !self.pending_finished_jobs.is_empty()
515 }
516
517 pub(super) fn cancel_command(&mut self, id: TableId) {
518 let _ = self.progress_map.remove(&id);
519 self.pending_finished_jobs
520 .retain(|x| x.table_to_create() != id);
521 self.actor_map.retain(|_, table_id| *table_id != id);
522 }
523
524 pub fn abort_all(&mut self) {
526 self.actor_map.clear();
527 self.pending_finished_jobs.clear();
528 self.progress_map.clear();
529 }
530
531 pub fn add(
535 &mut self,
536 info: &CreateStreamingJobCommandInfo,
537 replace_stream_job: Option<&ReplaceStreamJobPlan>,
538 version_stats: &HummockVersionStats,
539 ) -> Option<TrackingJob> {
540 tracing::trace!(?info, "add job to track");
541 let (info, actors, replace_table_info) = {
542 let CreateStreamingJobCommandInfo {
543 stream_job_fragments,
544 ..
545 } = info;
546 let actors = stream_job_fragments.tracking_progress_actor_ids();
547 if actors.is_empty() {
548 return Some(TrackingJob::New(TrackingCommand {
550 job_id: info.stream_job_fragments.stream_job_id,
551 replace_stream_job: replace_stream_job.cloned(),
552 }));
553 }
554 (info.clone(), actors, replace_stream_job.cloned())
555 };
556
557 let CreateStreamingJobCommandInfo {
558 stream_job_fragments: table_fragments,
559 definition,
560 job_type,
561 create_type,
562 fragment_backfill_ordering,
563 ..
564 } = info;
565
566 let creating_mv_id = table_fragments.stream_job_id();
567 let upstream_mv_count = table_fragments.upstream_table_counts();
568 let upstream_total_key_count: u64 =
569 calculate_total_key_count(&upstream_mv_count, version_stats);
570
571 for (actor, _backfill_upstream_type) in &actors {
572 self.actor_map.insert(*actor, creating_mv_id);
573 }
574
575 let backfill_order_state =
576 BackfillOrderState::new(fragment_backfill_ordering, &table_fragments);
577 let progress = Progress::new(
578 actors,
579 upstream_mv_count,
580 upstream_total_key_count,
581 definition.clone(),
582 backfill_order_state,
583 );
584 if job_type == StreamingJobType::Sink && create_type == CreateType::Background {
585 Some(TrackingJob::New(TrackingCommand {
591 job_id: creating_mv_id,
592 replace_stream_job: replace_table_info,
593 }))
594 } else {
595 let old = self.progress_map.insert(
596 creating_mv_id,
597 (
598 progress,
599 TrackingJob::New(TrackingCommand {
600 job_id: creating_mv_id,
601 replace_stream_job: replace_table_info,
602 }),
603 ),
604 );
605 assert!(old.is_none());
606 None
607 }
608 }
609
610 pub fn update(
614 &mut self,
615 progress: &CreateMviewProgress,
616 version_stats: &HummockVersionStats,
617 ) -> UpdateProgressResult {
618 tracing::trace!(?progress, "update progress");
619 let actor = progress.backfill_actor_id;
620 let Some(table_id) = self.actor_map.get(&actor).copied() else {
621 tracing::info!(
628 "no tracked progress for actor {}, the stream job could already be finished",
629 actor
630 );
631 return UpdateProgressResult::None;
632 };
633
634 let new_state = if progress.done {
635 BackfillState::Done(progress.consumed_rows)
636 } else {
637 BackfillState::ConsumingUpstream(progress.consumed_epoch.into(), progress.consumed_rows)
638 };
639
640 match self.progress_map.entry(table_id) {
641 Entry::Occupied(mut o) => {
642 let progress = &mut o.get_mut().0;
643
644 let upstream_total_key_count: u64 =
645 calculate_total_key_count(&progress.upstream_mv_count, version_stats);
646
647 tracing::debug!(?table_id, "updating progress for table");
648 let next_backfill_nodes =
649 progress.update(actor, new_state, upstream_total_key_count);
650
651 if progress.is_done() {
652 tracing::debug!(
653 "all actors done for creating mview with table_id {}!",
654 table_id
655 );
656
657 for actor in o.get().0.actors() {
659 self.actor_map.remove(&actor);
660 }
661 assert!(next_backfill_nodes.is_empty());
662 UpdateProgressResult::Finished(o.remove().1)
663 } else if !next_backfill_nodes.is_empty() {
664 tracing::debug!("scheduling next backfill nodes: {:?}", next_backfill_nodes);
665 UpdateProgressResult::BackfillNodeFinished(next_backfill_nodes)
666 } else {
667 UpdateProgressResult::None
668 }
669 }
670 Entry::Vacant(_) => {
671 tracing::warn!(
672 "update the progress of an non-existent creating streaming job: {progress:?}, which could be cancelled"
673 );
674 UpdateProgressResult::None
675 }
676 }
677 }
678}
679
680fn calculate_total_key_count(
681 table_count: &HashMap<TableId, usize>,
682 version_stats: &HummockVersionStats,
683) -> u64 {
684 table_count
685 .iter()
686 .map(|(table_id, count)| {
687 assert_ne!(*count, 0);
688 *count as u64
689 * version_stats
690 .table_stats
691 .get(&table_id.table_id)
692 .map_or(0, |stat| stat.total_key_count as u64)
693 })
694 .sum()
695}