From 4d58a70f1bfb8981cc4c3972aed20ebdcbf98c42 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 4 Oct 2024 19:39:22 +0800 Subject: [PATCH 01/19] feat(snapshot-backfill): control log store back pressure in backfill executor --- proto/stream_service.proto | 6 + .../barrier/creating_job/barrier_control.rs | 92 +------ src/meta/src/barrier/creating_job/mod.rs | 248 +++++------------- src/meta/src/barrier/creating_job/status.rs | 160 ++++++++--- src/meta/src/barrier/mod.rs | 109 ++++---- src/meta/src/barrier/state.rs | 25 +- .../executor/backfill/snapshot_backfill.rs | 242 +++++++---------- src/stream/src/task/barrier_manager.rs | 17 +- .../src/task/barrier_manager/managed_state.rs | 25 +- .../src/task/barrier_manager/progress.rs | 56 +++- 10 files changed, 475 insertions(+), 505 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 45703554c2367..3d2faf5e4e56d 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -29,6 +29,11 @@ message BarrierCompleteResponse { uint64 consumed_epoch = 3; uint64 consumed_rows = 4; } + message CreateMviewLogStoreProgress { + uint32 backfill_actor_id = 1; + bool done = 2; + uint32 pending_barrier_num = 3; + } string request_id = 1; common.Status status = 2; repeated CreateMviewProgress create_mview_progress = 3; @@ -46,6 +51,7 @@ message BarrierCompleteResponse { uint32 partial_graph_id = 8; // prev_epoch of barrier uint64 epoch = 9; + repeated CreateMviewLogStoreProgress create_mview_log_store_progress = 10; } message WaitEpochCommitRequest { diff --git a/src/meta/src/barrier/creating_job/barrier_control.rs b/src/meta/src/barrier/creating_job/barrier_control.rs index 76a66113c254f..f0f092a7ff853 100644 --- a/src/meta/src/barrier/creating_job/barrier_control.rs +++ b/src/meta/src/barrier/creating_job/barrier_control.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::Bound::{Excluded, Unbounded}; use std::collections::{BTreeMap, HashSet, VecDeque}; use std::mem::take; use std::time::Instant; @@ -26,22 +25,13 @@ use tracing::debug; use crate::manager::WorkerId; use crate::rpc::metrics::MetaMetrics; -#[derive(Debug)] -pub(super) enum CreatingStreamingJobBarrierType { - Snapshot, - LogStore, - Upstream, -} - #[derive(Debug)] struct CreatingStreamingJobEpochState { epoch: u64, node_to_collect: HashSet, resps: Vec, - upstream_epoch_to_notify: Option, is_checkpoint: bool, enqueue_time: Instant, - barrier_type: CreatingStreamingJobBarrierType, } #[derive(Debug)] @@ -49,9 +39,9 @@ pub(super) struct CreatingStreamingJobBarrierControl { table_id: TableId, // key is prev_epoch of barrier inflight_barrier_queue: BTreeMap, + backfill_epoch: u64, initial_epoch: Option, max_collected_epoch: Option, - max_attached_epoch: Option, // newer epoch at the front. should all be checkpoint barrier pending_barriers_to_complete: VecDeque, completing_barrier: Option<(CreatingStreamingJobEpochState, HistogramTimer)>, @@ -59,21 +49,20 @@ pub(super) struct CreatingStreamingJobBarrierControl { // metrics consuming_snapshot_barrier_latency: LabelGuardedHistogram<2>, consuming_log_store_barrier_latency: LabelGuardedHistogram<2>, - consuming_upstream_barrier_latency: LabelGuardedHistogram<2>, wait_commit_latency: LabelGuardedHistogram<1>, inflight_barrier_num: LabelGuardedIntGauge<1>, } impl CreatingStreamingJobBarrierControl { - pub(super) fn new(table_id: TableId, metrics: &MetaMetrics) -> Self { + pub(super) fn new(table_id: TableId, backfill_epoch: u64, metrics: &MetaMetrics) -> Self { let table_id_str = format!("{}", table_id.table_id); Self { table_id, inflight_barrier_queue: Default::default(), + backfill_epoch, initial_epoch: None, max_collected_epoch: None, - max_attached_epoch: None, pending_barriers_to_complete: Default::default(), completing_barrier: None, @@ -83,9 +72,6 @@ impl CreatingStreamingJobBarrierControl { consuming_log_store_barrier_latency: metrics .snapshot_backfill_barrier_latency .with_guarded_label_values(&[&table_id_str, "consuming_log_store"]), - consuming_upstream_barrier_latency: metrics - .snapshot_backfill_barrier_latency - .with_guarded_label_values(&[&table_id_str, "consuming_upstream"]), wait_commit_latency: metrics .snapshot_backfill_wait_commit_latency .with_guarded_label_values(&[&table_id_str]), @@ -127,7 +113,6 @@ impl CreatingStreamingJobBarrierControl { epoch: u64, node_to_collect: HashSet, is_checkpoint: bool, - barrier_type: CreatingStreamingJobBarrierType, ) { debug!( epoch, @@ -142,17 +127,12 @@ impl CreatingStreamingJobBarrierControl { if let Some(latest_epoch) = self.latest_epoch() { assert!(epoch > latest_epoch, "{} {}", epoch, latest_epoch); } - if let Some(max_attached_epoch) = self.max_attached_epoch { - assert!(epoch > max_attached_epoch); - } let epoch_state = CreatingStreamingJobEpochState { epoch, node_to_collect, resps: vec![], - upstream_epoch_to_notify: None, is_checkpoint, enqueue_time: Instant::now(), - barrier_type, }; if epoch_state.node_to_collect.is_empty() && self.inflight_barrier_queue.is_empty() { self.add_collected(epoch_state); @@ -163,41 +143,6 @@ impl CreatingStreamingJobBarrierControl { .set(self.inflight_barrier_queue.len() as _); } - pub(super) fn unattached_epochs(&self) -> impl Iterator + '_ { - let range_start = if let Some(max_attached_epoch) = self.max_attached_epoch { - Excluded(max_attached_epoch) - } else { - Unbounded - }; - self.inflight_barrier_queue - .range((range_start, Unbounded)) - .map(|(epoch, state)| (*epoch, state.is_checkpoint)) - } - - /// Attach an `upstream_epoch` to the `epoch` of the creating job. - /// - /// The `upstream_epoch` won't be completed until the `epoch` of the creating job is completed so that - /// the `upstream_epoch` should wait for the progress of creating job, and we can ensure that the downstream - /// creating job can eventually catch up with the upstream. - pub(super) fn attach_upstream_epoch(&mut self, epoch: u64, upstream_epoch: u64) { - debug!( - epoch, - upstream_epoch, - table_id = ?self.table_id.table_id, - "attach epoch" - ); - if let Some(max_attached_epoch) = self.max_attached_epoch { - assert!(epoch > max_attached_epoch); - } - self.max_attached_epoch = Some(epoch); - let epoch_state = self - .inflight_barrier_queue - .get_mut(&epoch) - .expect("should exist"); - assert!(epoch_state.upstream_epoch_to_notify.is_none()); - epoch_state.upstream_epoch_to_notify = Some(upstream_epoch); - } - pub(super) fn collect( &mut self, epoch: u64, @@ -228,46 +173,35 @@ impl CreatingStreamingJobBarrierControl { .set(self.inflight_barrier_queue.len() as _); } - #[expect(clippy::type_complexity)] - /// Return (`upstream_epochs_to_notify`, Some((epoch, resps, `is_first_commit`))) - /// - /// `upstream_epochs_to_notify` is the upstream epochs of non-checkpoint barriers to be notified about barrier completing. - /// These non-checkpoint barriers does not need to call `commit_epoch` and therefore can be completed as long as collected. - pub(super) fn start_completing( - &mut self, - ) -> (Vec, Option<(u64, Vec, bool)>) { + /// Return Some((epoch, resps, `is_first_commit`)) + pub(super) fn start_completing(&mut self) -> Option<(u64, Vec, bool)> { if self.completing_barrier.is_some() { - return (vec![], None); + return None; } - let mut upstream_epochs_to_notify = vec![]; while let Some(mut epoch_state) = self.pending_barriers_to_complete.pop_back() { let epoch = epoch_state.epoch; let is_first = self.initial_epoch.expect("should have set") == epoch; if is_first { assert!(epoch_state.is_checkpoint); } else if !epoch_state.is_checkpoint { - if let Some(upstream_epoch) = epoch_state.upstream_epoch_to_notify { - upstream_epochs_to_notify.push(upstream_epoch); - } continue; } let resps = take(&mut epoch_state.resps); self.completing_barrier = Some((epoch_state, self.wait_commit_latency.start_timer())); - return (upstream_epochs_to_notify, Some((epoch, resps, is_first))); + return Some((epoch, resps, is_first)); } - (upstream_epochs_to_notify, None) + None } /// Ack on completing a checkpoint barrier. /// /// Return the upstream epoch to be notified when there is any. - pub(super) fn ack_completed(&mut self, completed_epoch: u64) -> Option { + pub(super) fn ack_completed(&mut self, completed_epoch: u64) { let (epoch_state, wait_commit_timer) = self.completing_barrier.take().expect("should exist"); wait_commit_timer.observe_duration(); assert_eq!(epoch_state.epoch, completed_epoch); - epoch_state.upstream_epoch_to_notify } fn add_collected(&mut self, epoch_state: CreatingStreamingJobEpochState) { @@ -280,10 +214,10 @@ impl CreatingStreamingJobBarrierControl { } self.max_collected_epoch = Some(epoch_state.epoch); let barrier_latency = epoch_state.enqueue_time.elapsed().as_secs_f64(); - let barrier_latency_metrics = match &epoch_state.barrier_type { - CreatingStreamingJobBarrierType::Snapshot => &self.consuming_snapshot_barrier_latency, - CreatingStreamingJobBarrierType::LogStore => &self.consuming_log_store_barrier_latency, - CreatingStreamingJobBarrierType::Upstream => &self.consuming_upstream_barrier_latency, + let barrier_latency_metrics = if epoch_state.epoch < self.backfill_epoch { + &self.consuming_snapshot_barrier_latency + } else { + &self.consuming_log_store_barrier_latency }; barrier_latency_metrics.observe(barrier_latency); self.pending_barriers_to_complete.push_front(epoch_state); diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/creating_job/mod.rs index 0e2a948b6eb4b..760b14571de77 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/creating_job/mod.rs @@ -17,24 +17,19 @@ mod status; use std::cmp::max; use std::collections::HashMap; -use std::mem::take; use std::sync::Arc; -use std::time::Duration; use prometheus::HistogramTimer; use risingwave_common::metrics::{LabelGuardedHistogram, LabelGuardedIntGauge}; -use risingwave_common::util::epoch::Epoch; use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_service::BarrierCompleteResponse; -use tracing::{debug, info}; +use tracing::info; use crate::barrier::command::CommandContext; -use crate::barrier::creating_job::barrier_control::{ - CreatingStreamingJobBarrierControl, CreatingStreamingJobBarrierType, -}; +use crate::barrier::creating_job::barrier_control::CreatingStreamingJobBarrierControl; use crate::barrier::creating_job::status::{ CreatingJobInjectBarrierInfo, CreatingStreamingJobStatus, }; @@ -73,6 +68,7 @@ impl CreatingStreamingJobControl { definition = info.definition, "new creating job" ); + let snapshot_backfill_actors = info.table_fragments.snapshot_backfill_actor_ids(); let mut create_mview_tracker = CreateMviewProgressTracker::default(); create_mview_tracker.update_tracking_jobs(Some((&info, None)), [], version_stat); let fragment_info: HashMap<_, _> = info.new_fragment_info().collect(); @@ -85,13 +81,18 @@ impl CreatingStreamingJobControl { Self { info, snapshot_backfill_info, - barrier_control: CreatingStreamingJobBarrierControl::new(table_id, metrics), + barrier_control: CreatingStreamingJobBarrierControl::new( + table_id, + backfill_epoch, + metrics, + ), backfill_epoch, status: CreatingStreamingJobStatus::ConsumingSnapshot { prev_epoch_fake_physical_time: 0, pending_commands: vec![], version_stats: version_stat.clone(), create_mview_tracker, + snapshot_backfill_actors, graph_info: InflightGraphInfo::new(fragment_info), backfill_epoch, pending_non_checkpoint_barriers: vec![], @@ -106,6 +107,25 @@ impl CreatingStreamingJobControl { } } + /// Return whether the job is finished or not + pub(super) fn attach_upstream_wait_finish_epoch(&mut self, upstream_epoch: u64) -> bool { + match &mut self.status { + CreatingStreamingJobStatus::Finishing(upstream_epoch_to_notify) => { + assert_eq!(*upstream_epoch_to_notify, None); + if self.barrier_control.is_empty() { + true + } else { + *upstream_epoch_to_notify = Some(upstream_epoch); + false + } + } + CreatingStreamingJobStatus::ConsumingSnapshot { .. } + | CreatingStreamingJobStatus::ConsumingLogStore { .. } => { + unreachable!("should not attach upstream wait finish epoch at non-finishing status") + } + } + } + pub(super) fn start_wait_progress_timer(&self) -> HistogramTimer { self.upstream_wait_progress_latency.start_timer() } @@ -142,32 +162,15 @@ impl CreatingStreamingJobControl { } } CreatingStreamingJobStatus::ConsumingLogStore { - start_consume_log_store_epoch, + log_store_progress_tracker, .. } => { - let max_collected_epoch = max( - self.barrier_control.max_collected_epoch().unwrap_or(0), - self.backfill_epoch, - ); - let lag = Duration::from_millis( - Epoch(*start_consume_log_store_epoch) - .physical_time() - .saturating_sub(Epoch(max_collected_epoch).physical_time()), - ); - format!( - "LogStore [remain lag: {:?}, epoch cnt: {}]", - lag, - self.barrier_control.inflight_barrier_count() - ) - } - CreatingStreamingJobStatus::ConsumingUpstream { .. } => { format!( - "Upstream [unattached: {}, epoch cnt: {}]", - self.barrier_control.unattached_epochs().count(), - self.barrier_control.inflight_barrier_count(), + "LogStore [{}]", + log_store_progress_tracker.gen_ddl_progress() ) } - CreatingStreamingJobStatus::Finishing { .. } => { + CreatingStreamingJobStatus::Finishing(_) => { format!( "Finishing [epoch count: {}]", self.barrier_control.inflight_barrier_count() @@ -182,52 +185,22 @@ impl CreatingStreamingJobControl { } pub(super) fn pinned_upstream_log_epoch(&self) -> Option { - let stop_consume_log_store_epoch = match &self.status { - CreatingStreamingJobStatus::ConsumingSnapshot { .. } - | CreatingStreamingJobStatus::ConsumingLogStore { .. } => None, - CreatingStreamingJobStatus::ConsumingUpstream { - start_consume_upstream_epoch, - .. - } - | CreatingStreamingJobStatus::Finishing { - start_consume_upstream_epoch, - .. - } => Some(*start_consume_upstream_epoch), - }; - if let Some(max_collected_epoch) = self.barrier_control.max_collected_epoch() { - if max_collected_epoch < self.backfill_epoch { - Some(self.backfill_epoch) - } else if let Some(stop_consume_log_store_epoch) = stop_consume_log_store_epoch - && max_collected_epoch >= stop_consume_log_store_epoch - { - None - } else { - Some(max_collected_epoch) - } - } else { - Some(self.backfill_epoch) + if matches!(&self.status, CreatingStreamingJobStatus::Finishing(_)) { + return None; } + // TODO: when supporting recoverable snapshot backfill, we should use the max epoch that has committed + Some(max( + self.barrier_control.max_collected_epoch().unwrap_or(0), + self.backfill_epoch, + )) } pub(super) fn may_inject_fake_barrier( &mut self, control_stream_manager: &mut ControlStreamManager, - upstream_prev_epoch: u64, is_checkpoint: bool, ) -> MetaResult<()> { - if let Some((barriers_to_inject, graph_info)) = - self.status.may_inject_fake_barrier(is_checkpoint) - { - if let Some(graph_info) = graph_info { - info!( - table_id = self.info.table_fragments.table_id().table_id, - upstream_prev_epoch, "start consuming log store" - ); - self.status = CreatingStreamingJobStatus::ConsumingLogStore { - graph_info, - start_consume_log_store_epoch: upstream_prev_epoch, - }; - } + if let Some(barriers_to_inject) = self.status.may_inject_fake_barrier(is_checkpoint) { let graph_info = self .status .active_graph_info() @@ -256,7 +229,6 @@ impl CreatingStreamingJobControl { prev_epoch.value().0, node_to_collect, kind.is_checkpoint(), - CreatingStreamingJobBarrierType::Snapshot, ); } } @@ -267,7 +239,7 @@ impl CreatingStreamingJobControl { &mut self, control_stream_manager: &mut ControlStreamManager, command_ctx: &Arc, - ) -> MetaResult>> { + ) -> MetaResult<()> { let table_id = self.info.table_fragments.table_id(); let start_consume_upstream = if let Command::MergeSnapshotBackfillStreamingJobs( jobs_to_merge, @@ -290,7 +262,7 @@ impl CreatingStreamingJobControl { .0 .saturating_sub(progress_epoch) as _, ); - let graph_to_finish = match &mut self.status { + match &mut self.status { CreatingStreamingJobStatus::ConsumingSnapshot { pending_commands, .. } => { @@ -299,21 +271,19 @@ impl CreatingStreamingJobControl { "should not start consuming upstream for a job that are consuming snapshot" ); pending_commands.push(command_ctx.clone()); - None } CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } => { let node_to_collect = control_stream_manager.inject_barrier( Some(table_id), - if start_consume_upstream { - // erase the mutation on upstream except the last command - command_ctx.to_mutation() - } else { - None - }, + None, (&command_ctx.curr_epoch, &command_ctx.prev_epoch), &command_ctx.kind, graph_info, - Some(graph_info), + if start_consume_upstream { + None + } else { + Some(graph_info) + }, None, vec![], vec![], @@ -322,93 +292,24 @@ impl CreatingStreamingJobControl { command_ctx.prev_epoch.value().0, node_to_collect, command_ctx.kind.is_checkpoint(), - CreatingStreamingJobBarrierType::LogStore, ); let prev_epoch = command_ctx.prev_epoch.value().0; if start_consume_upstream { - let graph_info = take(graph_info); info!( table_id = self.info.table_fragments.table_id().table_id, prev_epoch, "start consuming upstream" ); - self.status = CreatingStreamingJobStatus::ConsumingUpstream { - start_consume_upstream_epoch: prev_epoch, - graph_info, - }; + self.status = CreatingStreamingJobStatus::Finishing(None); } - None - } - CreatingStreamingJobStatus::ConsumingUpstream { - start_consume_upstream_epoch, - graph_info, - } => { - assert!( - !start_consume_upstream, - "should not start consuming upstream for a job again" - ); - - let should_finish = command_ctx.kind.is_checkpoint() - && self.barrier_control.unattached_epochs().next().is_none(); - let node_to_collect = control_stream_manager.inject_barrier( - Some(table_id), - // do not send the upstream barrier mutation because in `ConsumingUpstream` stage, - // barriers are still injected and collected independently on the creating jobs. - None, - (&command_ctx.curr_epoch, &command_ctx.prev_epoch), - &command_ctx.kind, - graph_info, - if should_finish { - None - } else { - Some(graph_info) - }, - None, - vec![], - vec![], - )?; - let prev_epoch = command_ctx.prev_epoch.value().0; - self.barrier_control.enqueue_epoch( - prev_epoch, - node_to_collect, - command_ctx.kind.is_checkpoint(), - CreatingStreamingJobBarrierType::Upstream, - ); - let graph_info = if should_finish { - info!(prev_epoch, table_id = ?self.info.table_fragments.table_id(), "mark as finishing"); - self.barrier_control - .attach_upstream_epoch(prev_epoch, prev_epoch); - let graph_info = take(graph_info); - self.status = CreatingStreamingJobStatus::Finishing { - start_consume_upstream_epoch: *start_consume_upstream_epoch, - }; - Some(Some(graph_info)) - } else { - let mut unattached_epochs_iter = self.barrier_control.unattached_epochs(); - let mut epoch_to_attach = unattached_epochs_iter.next().expect("non-empty").0; - let mut remain_count = 5; - while remain_count > 0 - && let Some((epoch, _)) = unattached_epochs_iter.next() - { - remain_count -= 1; - epoch_to_attach = epoch; - } - drop(unattached_epochs_iter); - self.barrier_control - .attach_upstream_epoch(epoch_to_attach, prev_epoch); - Some(None) - }; - - graph_info } CreatingStreamingJobStatus::Finishing { .. } => { assert!( !start_consume_upstream, "should not start consuming upstream for a job again" ); - None } }; - Ok(graph_to_finish) + Ok(()) } pub(super) fn collect( @@ -417,50 +318,39 @@ impl CreatingStreamingJobControl { worker_id: WorkerId, resp: BarrierCompleteResponse, ) { - self.status.update_progress(&resp.create_mview_progress); + self.status.update_progress( + &resp.create_mview_progress, + &resp.create_mview_log_store_progress, + ); self.barrier_control.collect(epoch, worker_id, resp); } pub(super) fn should_merge_to_upstream(&self) -> Option { - if let ( - CreatingStreamingJobStatus::ConsumingLogStore { - graph_info, - start_consume_log_store_epoch, - }, - Some(max_collected_epoch), - ) = (&self.status, self.barrier_control.max_collected_epoch()) + if let CreatingStreamingJobStatus::ConsumingLogStore { + graph_info, + ref log_store_progress_tracker, + } = &self.status + && log_store_progress_tracker.is_finished() { - if max_collected_epoch >= *start_consume_log_store_epoch { - Some(graph_info.clone()) - } else { - let lag = Duration::from_millis( - Epoch(*start_consume_log_store_epoch).physical_time() - - Epoch(max_collected_epoch).physical_time(), - ); - debug!( - ?lag, - max_collected_epoch, start_consume_log_store_epoch, "wait consuming log store" - ); - None - } + Some(graph_info.clone()) } else { None } } - #[expect(clippy::type_complexity)] - pub(super) fn start_completing( - &mut self, - ) -> (Vec, Option<(u64, Vec, bool)>) { + pub(super) fn start_completing(&mut self) -> Option<(u64, Vec, bool)> { self.barrier_control.start_completing() } - pub(super) fn ack_completed(&mut self, completed_epoch: u64) -> Option<(u64, bool)> { - let upstream_epoch_to_notify = self.barrier_control.ack_completed(completed_epoch); - if let Some(upstream_epoch_to_notify) = upstream_epoch_to_notify { - Some((upstream_epoch_to_notify, self.is_finished())) + pub(super) fn ack_completed(&mut self, completed_epoch: u64) -> Option { + self.barrier_control.ack_completed(completed_epoch); + if let CreatingStreamingJobStatus::Finishing(upstream_epoch_to_notify) = &self.status { + if self.barrier_control.is_empty() { + *upstream_epoch_to_notify + } else { + None + } } else { - assert!(!self.is_finished()); None } } diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/creating_job/status.rs index 6f205d7ced99f..511f52aa624a3 100644 --- a/src/meta/src/barrier/creating_job/status.rs +++ b/src/meta/src/barrier/creating_job/status.rs @@ -12,15 +12,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashMap; +use std::collections::hash_map::Entry; +use std::collections::{HashMap, HashSet}; use std::mem::take; use std::sync::Arc; +use itertools::Itertools; +use risingwave_common::hash::ActorId; use risingwave_common::util::epoch::Epoch; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::StreamActor; -use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; +use risingwave_pb::stream_service::barrier_complete_response::{ + CreateMviewProgress, PbCreateMviewLogStoreProgress, +}; +use tracing::warn; use crate::barrier::command::CommandContext; use crate::barrier::info::InflightGraphInfo; @@ -28,6 +34,70 @@ use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::{BarrierKind, TracedEpoch}; use crate::manager::WorkerId; +#[derive(Debug)] +pub(super) struct CreateMviewLogStoreProgressTracker { + /// `actor_id` -> `pending_barrier_count` + ongoing_actors: HashMap, + finished_actors: HashSet, +} + +impl CreateMviewLogStoreProgressTracker { + fn new(actors: impl Iterator, initial_pending_count: usize) -> Self { + Self { + ongoing_actors: HashMap::from_iter(actors.map(|actor| (actor, initial_pending_count))), + finished_actors: HashSet::new(), + } + } + + pub(super) fn gen_ddl_progress(&self) -> String { + let sum = self.ongoing_actors.values().sum::() as f64; + let count = if self.ongoing_actors.is_empty() { + 1 + } else { + self.ongoing_actors.len() + } as f64; + let avg = sum / count; + format!( + "finished: {}/{}, avg epoch count {}", + self.finished_actors.len(), + self.ongoing_actors.len() + self.finished_actors.len(), + avg + ) + } + + fn update(&mut self, progress: impl IntoIterator) { + for progress in progress { + match self.ongoing_actors.entry(progress.backfill_actor_id) { + Entry::Occupied(mut entry) => { + if progress.done { + entry.remove_entry(); + assert!( + self.finished_actors.insert(progress.backfill_actor_id), + "non-duplicate" + ); + } else { + *entry.get_mut() = progress.pending_barrier_num as _; + } + } + Entry::Vacant(_) => { + if cfg!(debug_assertions) { + panic!( + "reporting progress on non-inflight actor: {:?} {:?}", + progress, self + ); + } else { + warn!(?progress, progress_tracker = ?self, "reporting progress on non-inflight actor"); + } + } + } + } + } + + pub(super) fn is_finished(&self) -> bool { + self.ongoing_actors.is_empty() + } +} + #[derive(Debug)] pub(super) enum CreatingStreamingJobStatus { ConsumingSnapshot { @@ -36,6 +106,7 @@ pub(super) enum CreatingStreamingJobStatus { version_stats: HummockVersionStats, create_mview_tracker: CreateMviewProgressTracker, graph_info: InflightGraphInfo, + snapshot_backfill_actors: HashSet, backfill_epoch: u64, /// The `prev_epoch` of pending non checkpoint barriers pending_non_checkpoint_barriers: Vec, @@ -45,15 +116,9 @@ pub(super) enum CreatingStreamingJobStatus { }, ConsumingLogStore { graph_info: InflightGraphInfo, - start_consume_log_store_epoch: u64, - }, - ConsumingUpstream { - start_consume_upstream_epoch: u64, - graph_info: InflightGraphInfo, - }, - Finishing { - start_consume_upstream_epoch: u64, + log_store_progress_tracker: CreateMviewLogStoreProgressTracker, }, + Finishing(Option), } pub(super) struct CreatingJobInjectBarrierInfo { @@ -68,9 +133,8 @@ impl CreatingStreamingJobStatus { pub(super) fn active_graph_info(&self) -> Option<&InflightGraphInfo> { match self { CreatingStreamingJobStatus::ConsumingSnapshot { graph_info, .. } - | CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } - | CreatingStreamingJobStatus::ConsumingUpstream { graph_info, .. } => Some(graph_info), - CreatingStreamingJobStatus::Finishing { .. } => { + | CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } => Some(graph_info), + CreatingStreamingJobStatus::Finishing(_) => { // when entering `Finishing`, the graph will have been added to the upstream graph, // and therefore the separate graph info is inactive. None @@ -81,32 +145,53 @@ impl CreatingStreamingJobStatus { pub(super) fn update_progress( &mut self, create_mview_progress: impl IntoIterator, + log_store_progress: impl IntoIterator, ) { - if let Self::ConsumingSnapshot { - create_mview_tracker, - ref version_stats, - .. - } = self - { - create_mview_tracker.update_tracking_jobs(None, create_mview_progress, version_stats); + match self { + Self::ConsumingSnapshot { + create_mview_tracker, + ref version_stats, + .. + } => { + create_mview_tracker.update_tracking_jobs( + None, + create_mview_progress, + version_stats, + ); + let mut log_store_progress = log_store_progress.into_iter().peekable(); + if log_store_progress.peek().is_some() { + warn!(log_store_progress = ?log_store_progress.collect_vec(), "ignore log store progress when consuming snapshot"); + } + } + CreatingStreamingJobStatus::ConsumingLogStore { + log_store_progress_tracker, + .. + } => { + log_store_progress_tracker.update(log_store_progress); + let mut create_mview_progress = create_mview_progress.into_iter().peekable(); + if create_mview_progress.peek().is_some() { + warn!(create_mview_progress = ?create_mview_progress.collect_vec(), "ignore create_mview_progress when consuming log store"); + } + } + CreatingStreamingJobStatus::Finishing(_) => {} } } /// return /// - Some(vec[(`curr_epoch`, `prev_epoch`, `barrier_kind`)]) of barriers to newly inject - /// - Some(`graph_info`) when the status should transit to `ConsumingLogStore` pub(super) fn may_inject_fake_barrier( &mut self, is_checkpoint: bool, - ) -> Option<(Vec, Option)> { + ) -> Option> { if let CreatingStreamingJobStatus::ConsumingSnapshot { prev_epoch_fake_physical_time, pending_commands, create_mview_tracker, - graph_info, + ref graph_info, pending_non_checkpoint_barriers, ref backfill_epoch, initial_barrier_info, + ref snapshot_backfill_actors, .. } = self { @@ -115,7 +200,7 @@ impl CreatingStreamingJobStatus { pending_non_checkpoint_barriers.push(*backfill_epoch); let prev_epoch = Epoch::from_physical_time(*prev_epoch_fake_physical_time); - let barriers_to_inject = + let barriers_to_inject: Vec<_> = [CreatingJobInjectBarrierInfo { curr_epoch: TracedEpoch::new(Epoch(*backfill_epoch)), prev_epoch: TracedEpoch::new(prev_epoch), @@ -135,8 +220,14 @@ impl CreatingStreamingJobStatus { })) .collect(); - let graph_info = take(graph_info); - Some((barriers_to_inject, Some(graph_info))) + *self = CreatingStreamingJobStatus::ConsumingLogStore { + graph_info: graph_info.clone(), + log_store_progress_tracker: CreateMviewLogStoreProgressTracker::new( + snapshot_backfill_actors.iter().cloned(), + barriers_to_inject.len(), + ), + }; + Some(barriers_to_inject) } else { let prev_epoch = TracedEpoch::new(Epoch::from_physical_time(*prev_epoch_fake_physical_time)); @@ -155,16 +246,13 @@ impl CreatingStreamingJobStatus { } else { Default::default() }; - Some(( - vec![CreatingJobInjectBarrierInfo { - curr_epoch, - prev_epoch, - kind, - new_actors, - mutation, - }], - None, - )) + Some(vec![CreatingJobInjectBarrierInfo { + curr_epoch, + prev_epoch, + kind, + new_actors, + mutation, + }]) } } else { None diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 463de3f6febe4..c8eb3507b6add 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -308,22 +308,29 @@ impl CheckpointControl { ?jobs_to_wait, "enqueue command" ); + let mut finished_table_ids = HashMap::new(); let creating_jobs_to_wait = jobs_to_wait .into_iter() - .map(|table_id| { - ( - table_id, - if node_to_collect.is_empty() { - Some( - self.creating_streaming_job_controls - .get(&table_id) - .expect("should exist") - .start_wait_progress_timer(), - ) - } else { - None - }, - ) + .filter_map(|table_id| { + let Entry::Occupied(mut entry) = + self.creating_streaming_job_controls.entry(table_id) + else { + unreachable!("should exist") + }; + if entry + .get_mut() + .attach_upstream_wait_finish_epoch(command_ctx.prev_epoch.value().0) + { + let (table_id, creating_job) = entry.remove_entry(); + finished_table_ids + .try_insert(table_id, creating_job.info) + .unwrap(); + None + } else if node_to_collect.is_empty() { + Some((table_id, Some(entry.get().start_wait_progress_timer()))) + } else { + Some((table_id, None)) + } }) .collect(); self.command_ctx_queue.insert( @@ -334,7 +341,7 @@ impl CheckpointControl { node_to_collect, resps: vec![], creating_jobs_to_wait, - finished_table_ids: HashMap::new(), + finished_table_ids, table_ids_to_commit, }, command_ctx, @@ -948,7 +955,8 @@ impl GlobalBarrierManager { } } - let Some((prev_epoch, curr_epoch)) = self.state.next_epoch_pair(&command) else { + let Some((prev_epoch, curr_epoch, jobs_to_wait)) = self.state.next_epoch_pair(&command) + else { // skip the command when there is nothing to do with the barrier for mut notifier in notifiers { notifier.notify_started(); @@ -997,11 +1005,7 @@ impl GlobalBarrierManager { .creating_streaming_job_controls .values_mut() { - creating_job.may_inject_fake_barrier( - &mut self.control_stream_manager, - prev_epoch.value().0, - checkpoint, - )? + creating_job.may_inject_fake_barrier(&mut self.control_stream_manager, checkpoint)? } self.pending_non_checkpoint_barriers @@ -1048,18 +1052,12 @@ impl GlobalBarrierManager { send_latency_timer.observe_duration(); - let mut jobs_to_wait = HashSet::new(); - - for (table_id, creating_job) in &mut self.checkpoint_control.creating_streaming_job_controls + for creating_job in &mut self + .checkpoint_control + .creating_streaming_job_controls + .values_mut() { - if let Some(wait_job) = - creating_job.on_new_command(&mut self.control_stream_manager, &command_ctx)? - { - jobs_to_wait.insert(*table_id); - if let Some(graph_to_finish) = wait_job { - self.state.inflight_graph_info.extend(graph_to_finish); - } - } + creating_job.on_new_command(&mut self.control_stream_manager, &command_ctx)?; } let node_to_collect = match self.control_stream_manager.inject_command_ctx_barrier( @@ -1423,20 +1421,7 @@ impl CheckpointControl { }; } else { for (table_id, job) in &mut self.creating_streaming_job_controls { - let (upstream_epochs_to_notify, commit_info) = job.start_completing(); - for upstream_epoch in upstream_epochs_to_notify { - let wait_progress_timer = self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .creating_jobs_to_wait - .remove(table_id) - .expect("should exist"); - if let Some(timer) = wait_progress_timer { - timer.observe_duration(); - } - } + let commit_info = job.start_completing(); if let Some((epoch, resps, is_first_time)) = commit_info { let tables_to_commit = job .info @@ -1514,7 +1499,7 @@ impl CheckpointControl { CompletingCommand::None }; self.completing_command = next_completing_command_status; - if let Some((upstream_epoch, is_finished)) = self + if let Some(upstream_epoch) = self .creating_streaming_job_controls .get_mut(&table_id) .expect("should exist") @@ -1531,22 +1516,20 @@ impl CheckpointControl { if let Some(timer) = wait_progress_timer { timer.observe_duration(); } - if is_finished { - debug!(epoch, ?table_id, "finish creating job"); - let creating_streaming_job = self - .creating_streaming_job_controls - .remove(&table_id) - .expect("should exist"); - assert!(creating_streaming_job.is_finished()); - assert!(self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .finished_table_ids - .insert(table_id, creating_streaming_job.info) - .is_none()); - } + debug!(epoch, ?table_id, "finish creating job"); + let creating_streaming_job = self + .creating_streaming_job_controls + .remove(&table_id) + .expect("should exist"); + assert!(creating_streaming_job.is_finished()); + assert!(self + .command_ctx_queue + .get_mut(&upstream_epoch) + .expect("should exist") + .state + .finished_table_ids + .insert(table_id, creating_streaming_job.info) + .is_none()); } join_result.map(|_| None) } diff --git a/src/meta/src/barrier/state.rs b/src/meta/src/barrier/state.rs index db2ded5629d7a..5f9e17ca16658 100644 --- a/src/meta/src/barrier/state.rs +++ b/src/meta/src/barrier/state.rs @@ -12,6 +12,10 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::collections::HashSet; +use std::mem::take; + +use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::Epoch; use risingwave_pb::meta::PausedReason; @@ -25,6 +29,7 @@ pub struct BarrierManagerState { /// There's no need to persist this field. On recovery, we will restore this from the latest /// committed snapshot in `HummockManager`. in_flight_prev_epoch: Option, + creating_jobs_to_wait: HashSet, /// Inflight running actors info. pub(crate) inflight_graph_info: InflightGraphInfo, @@ -44,6 +49,7 @@ impl BarrierManagerState { ) -> Self { Self { in_flight_prev_epoch, + creating_jobs_to_wait: Default::default(), inflight_graph_info, inflight_subscription_info, paused_reason, @@ -66,7 +72,10 @@ impl BarrierManagerState { } /// Returns the epoch pair for the next barrier, and updates the state. - pub fn next_epoch_pair(&mut self, command: &Command) -> Option<(TracedEpoch, TracedEpoch)> { + pub fn next_epoch_pair( + &mut self, + command: &Command, + ) -> Option<(TracedEpoch, TracedEpoch, HashSet)> { if self.inflight_graph_info.is_empty() && !matches!(&command, Command::CreateStreamingJob { .. }) { @@ -78,7 +87,11 @@ impl BarrierManagerState { let prev_epoch = in_flight_prev_epoch.clone(); let next_epoch = prev_epoch.next(); *in_flight_prev_epoch = next_epoch.clone(); - Some((prev_epoch, next_epoch)) + Some(( + prev_epoch, + next_epoch, + take(&mut self.creating_jobs_to_wait), + )) } /// Returns the inflight actor infos that have included the newly added actors in the given command. The dropped actors @@ -108,6 +121,14 @@ impl BarrierManagerState { if let Some(fragment_changes) = fragment_changes { self.inflight_graph_info.post_apply(&fragment_changes); } + if let Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge) = command { + for (_, graph_info) in jobs_to_merge.values() { + self.inflight_graph_info.extend(graph_info.clone()); + } + self.creating_jobs_to_wait + .extend(jobs_to_merge.keys().cloned()); + } + self.inflight_subscription_info.post_apply(command); (info, subscription_info) diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index 89801a3cf4133..bbbd5e9bc61b2 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cmp::min; use std::collections::VecDeque; use std::future::{pending, Future}; use std::mem::replace; @@ -20,9 +21,7 @@ use std::sync::Arc; use anyhow::anyhow; use futures::future::Either; use futures::{pin_mut, Stream, TryStreamExt}; -use itertools::Itertools; use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::catalog::TableId; use risingwave_common::metrics::LabelGuardedIntCounter; use risingwave_common::row::OwnedRow; use risingwave_common::util::chunk_coalesce::DataChunkBuilder; @@ -32,7 +31,6 @@ use risingwave_storage::table::batch_table::storage_table::StorageTable; use risingwave_storage::table::ChangeLogRow; use risingwave_storage::StateStore; use tokio::select; -use tokio::sync::mpsc; use tokio::sync::mpsc::UnboundedReceiver; use crate::executor::backfill::utils::{create_builder, mapping_chunk}; @@ -40,7 +38,7 @@ use crate::executor::monitor::StreamingMetrics; use crate::executor::prelude::{try_stream, StreamExt}; use crate::executor::{ expect_first_barrier, ActorContextRef, BackfillExecutor, Barrier, BoxedMessageStream, - DispatcherBarrier, DispatcherMessage, Execute, MergeExecutorInput, Message, Mutation, + DispatcherBarrier, DispatcherMessage, Execute, MergeExecutorInput, Message, StreamExecutorError, StreamExecutorResult, }; use crate::task::CreateMviewProgressReporter; @@ -101,35 +99,16 @@ impl SnapshotBackfillExecutor { #[try_stream(ok = Message, error = StreamExecutorError)] async fn execute_inner(mut self) { debug!("snapshot backfill executor start"); - let upstream_table_id = self.upstream_table.table_id(); let first_barrier = expect_first_barrier(&mut self.upstream).await?; debug!(epoch = ?first_barrier.epoch, "get first upstream barrier"); let first_recv_barrier = receive_next_barrier(&mut self.barrier_rx).await?; debug!(epoch = ?first_recv_barrier.epoch, "get first inject barrier"); let should_backfill = first_barrier.epoch != first_recv_barrier.epoch; - let mut barrier_epoch = { - if should_backfill { - let subscriber_ids = first_recv_barrier - .added_subscriber_on_mv_table(upstream_table_id) - .collect_vec(); - let snapshot_backfill_table_fragment_id = match subscriber_ids.as_slice() { - [] => { - return Err(anyhow!( - "first recv barrier on backfill should add subscriber on upstream" - ) - .into()); - } - [snapshot_backfill_table_fragment_id] => *snapshot_backfill_table_fragment_id, - multiple => { - return Err(anyhow!( - "first recv barrier on backfill have multiple subscribers {:?} on upstream table {}", - multiple, upstream_table_id.table_id - ) - .into()); - } - }; + let barrier_manager = self.progress.barrier_manager().clone(); + let (mut barrier_epoch, mut need_report_finish) = { + if should_backfill { let table_id_str = format!("{}", self.upstream_table.table_id().table_id); let actor_id_str = format!("{}", self.actor_ctx.id); @@ -138,12 +117,8 @@ impl SnapshotBackfillExecutor { .snapshot_backfill_consume_row_count .with_guarded_label_values(&[&table_id_str, &actor_id_str, "consume_upstream"]); - let mut upstream_buffer = UpstreamBuffer::new( - &mut self.upstream, - upstream_table_id, - snapshot_backfill_table_fragment_id, - consume_upstream_row_count, - ); + let mut upstream_buffer = + UpstreamBuffer::new(&mut self.upstream, consume_upstream_row_count); let first_barrier_epoch = first_barrier.epoch; @@ -187,8 +162,8 @@ impl SnapshotBackfillExecutor { yield Message::Barrier(recv_barrier); } - let mut upstream_buffer = - upstream_buffer.start_consuming_log_store(&mut self.barrier_rx); + // TODO: may set the `max_pending_barrier_num_gap` by config + let mut upstream_buffer = upstream_buffer.start_consuming_log_store(10); let mut barrier_epoch = first_barrier_epoch; @@ -210,7 +185,9 @@ impl SnapshotBackfillExecutor { ]); // Phase 2: consume upstream log store - while let Some(barrier) = upstream_buffer.take_buffered_barrier().await? { + while let Some(upstream_barrier) = upstream_buffer.next_barrier().await? { + let barrier = receive_next_barrier(&mut self.barrier_rx).await?; + assert_eq!(upstream_barrier.epoch, barrier.epoch); assert_eq!(barrier_epoch.curr, barrier.epoch.prev); barrier_epoch = barrier.epoch; @@ -240,6 +217,12 @@ impl SnapshotBackfillExecutor { debug!(?barrier_epoch, "after consume change log"); + barrier_manager.update_create_mview_log_store_progress( + barrier.epoch, + self.actor_ctx.id, + upstream_buffer.state.barrier_count(), + ); + yield Message::Barrier(barrier); } @@ -248,7 +231,7 @@ impl SnapshotBackfillExecutor { table_id = self.upstream_table.table_id().table_id, "finish consuming log store" ); - barrier_epoch + (barrier_epoch, true) } else { info!( table_id = self.upstream_table.table_id().table_id, @@ -257,7 +240,7 @@ impl SnapshotBackfillExecutor { let first_recv_barrier = receive_next_barrier(&mut self.barrier_rx).await?; assert_eq!(first_barrier.epoch, first_recv_barrier.epoch); yield Message::Barrier(first_recv_barrier); - first_barrier.epoch + (first_barrier.epoch, false) } }; let mut upstream = self.upstream.into_executor(self.barrier_rx).execute(); @@ -266,6 +249,10 @@ impl SnapshotBackfillExecutor { if let Message::Barrier(barrier) = &msg { assert_eq!(barrier.epoch.prev, barrier_epoch.curr); barrier_epoch = barrier.epoch; + if need_report_finish { + need_report_finish = false; + barrier_manager.finish_consuming_log_store(barrier_epoch, self.actor_ctx.id); + } } yield msg; } @@ -329,8 +316,8 @@ async fn read_change_log( } trait UpstreamBufferState { - // The future must be cancellation-safe - async fn is_finished(&mut self) -> StreamExecutorResult; + /// Return whether to be able to consume upstream. This is used to control backpressure to upstream + fn can_consume_upstream(&self) -> bool; fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier); } @@ -339,9 +326,9 @@ struct StateOfConsumingSnapshot { } impl UpstreamBufferState for StateOfConsumingSnapshot { - async fn is_finished(&mut self) -> StreamExecutorResult { - // never finish when consuming snapshot - Ok(false) + fn can_consume_upstream(&self) -> bool { + // no backpressure to upstream when consuming snapshot + true } fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier) { @@ -349,51 +336,31 @@ impl UpstreamBufferState for StateOfConsumingSnapshot { } } -struct StateOfConsumingLogStore<'a> { - barrier_rx: &'a mut mpsc::UnboundedReceiver, +struct StateOfConsumingLogStore { /// Barriers received from upstream but not yet received the barrier from local barrier worker /// newer barrier at the front upstream_pending_barriers: VecDeque, - /// Barriers received from both upstream and local barrier worker - /// newer barrier at the front - barriers: VecDeque, - is_finished: bool, - current_subscriber_id: u32, - upstream_table_id: TableId, + max_pending_barrier_num: usize, + /// The amount of `max_pending_barrier_num` to decrease whenever emitting a checkpoint barrier to downstream + max_pending_barrier_num_gap: usize, } -impl<'a> StateOfConsumingLogStore<'a> { +impl StateOfConsumingLogStore { fn barrier_count(&self) -> usize { - self.upstream_pending_barriers.len() + self.barriers.len() - } - - async fn handle_one_pending_barrier(&mut self) -> StreamExecutorResult { - assert!(!self.is_finished); - let barrier = receive_next_barrier(self.barrier_rx).await?; - assert_eq!( - self.upstream_pending_barriers - .pop_back() - .expect("non-empty") - .epoch, - barrier.epoch - ); - if is_finish_barrier(&barrier, self.current_subscriber_id, self.upstream_table_id) { - self.is_finished = true; - } - Ok(barrier) + self.upstream_pending_barriers.len() } } -impl<'a> UpstreamBufferState for StateOfConsumingLogStore<'a> { - async fn is_finished(&mut self) -> StreamExecutorResult { - while !self.upstream_pending_barriers.is_empty() { - let barrier = self.handle_one_pending_barrier().await?; - self.barriers.push_front(barrier); - } - if self.is_finished { - assert!(self.upstream_pending_barriers.is_empty()); +impl UpstreamBufferState for StateOfConsumingLogStore { + fn can_consume_upstream(&self) -> bool { + if let Some(prev_barrier) = self.upstream_pending_barriers.front() + && prev_barrier.kind.is_barrier() + { + // allow consuming upstream when the barrier is non-checkpoint barrier to avoid deadlock + true + } else { + self.upstream_pending_barriers.len() < self.max_pending_barrier_num } - Ok(self.is_finished) } fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier) { @@ -404,70 +371,73 @@ impl<'a> UpstreamBufferState for StateOfConsumingLogStore<'a> { struct UpstreamBuffer<'a, S> { upstream: &'a mut MergeExecutorInput, state: S, + /// Whether we have started polling any upstream data before the next barrier. + /// When `true`, we should continue polling until the next barrier, because + /// some data in this epoch have been discarded and data in this epoch + /// must be read from log store + is_polling_epoch_data: bool, consume_upstream_row_count: LabelGuardedIntCounter<3>, - upstream_table_id: TableId, - current_subscriber_id: u32, } impl<'a> UpstreamBuffer<'a, StateOfConsumingSnapshot> { fn new( upstream: &'a mut MergeExecutorInput, - upstream_table_id: TableId, - current_subscriber_id: u32, consume_upstream_row_count: LabelGuardedIntCounter<3>, ) -> Self { Self { upstream, + is_polling_epoch_data: false, state: StateOfConsumingSnapshot { pending_barriers: vec![], }, consume_upstream_row_count, - upstream_table_id, - current_subscriber_id, } } - fn start_consuming_log_store<'s>( + fn start_consuming_log_store( self, - barrier_rx: &'s mut UnboundedReceiver, - ) -> UpstreamBuffer<'a, StateOfConsumingLogStore<'s>> { + max_pending_barrier_num_gap: usize, + ) -> UpstreamBuffer<'a, StateOfConsumingLogStore> { let StateOfConsumingSnapshot { pending_barriers } = self.state; let mut upstream_pending_barriers = VecDeque::with_capacity(pending_barriers.len()); for pending_barrier in pending_barriers { upstream_pending_barriers.push_front(pending_barrier); } + let max_pending_barrier_num = upstream_pending_barriers.len(); UpstreamBuffer { upstream: self.upstream, state: StateOfConsumingLogStore { - barrier_rx, upstream_pending_barriers, - barriers: Default::default(), - is_finished: false, - current_subscriber_id: self.current_subscriber_id, - upstream_table_id: self.upstream_table_id, + max_pending_barrier_num, + max_pending_barrier_num_gap, }, + is_polling_epoch_data: self.is_polling_epoch_data, consume_upstream_row_count: self.consume_upstream_row_count, - upstream_table_id: self.upstream_table_id, - current_subscriber_id: self.current_subscriber_id, } } } impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { async fn concurrently_consume_upstream(&mut self) -> StreamExecutorError { - if let Err(e) = try { - while !self.state.is_finished().await? { - self.consume_until_next_barrier().await?; + { + loop { + if let Err(e) = try { + if !self.state.can_consume_upstream() && !self.is_polling_epoch_data { + // pause the future to block consuming upstream + return pending().await; + } + let barrier = self.consume_until_next_barrier().await?; + self.state.on_upstream_barrier(barrier); + } { + break e; + } } - } { - return e; } - pending().await } /// Consume the upstream until seeing the next barrier. /// `pending_barriers` must be non-empty after this method returns. - async fn consume_until_next_barrier(&mut self) -> StreamExecutorResult<()> { + async fn consume_until_next_barrier(&mut self) -> StreamExecutorResult { loop { let msg: DispatcherMessage = self .upstream @@ -476,59 +446,47 @@ impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { .ok_or_else(|| anyhow!("end of upstream"))?; match msg { DispatcherMessage::Chunk(chunk) => { + self.is_polling_epoch_data = true; self.consume_upstream_row_count .inc_by(chunk.cardinality() as _); } DispatcherMessage::Barrier(barrier) => { - self.state.on_upstream_barrier(barrier); - break Ok(()); + self.is_polling_epoch_data = false; + break Ok(barrier); + } + DispatcherMessage::Watermark(_) => { + self.is_polling_epoch_data = true; } - DispatcherMessage::Watermark(_) => {} } } } } -impl<'a, 's> UpstreamBuffer<'a, StateOfConsumingLogStore<'s>> { - async fn take_buffered_barrier(&mut self) -> StreamExecutorResult> { - Ok(if let Some(barrier) = self.state.barriers.pop_back() { - Some(barrier) - } else if !self.state.upstream_pending_barriers.is_empty() { - let barrier = self.state.handle_one_pending_barrier().await?; - Some(barrier) - } else if self.state.is_finished { - None - } else { - self.consume_until_next_barrier().await?; - let barrier = self.state.handle_one_pending_barrier().await?; - Some(barrier) - }) - } -} - -fn is_finish_barrier( - barrier: &Barrier, - current_subscriber_id: u32, - upstream_table_id: TableId, -) -> bool { - if let Some(Mutation::DropSubscriptions { - subscriptions_to_drop, - }) = barrier.mutation.as_deref() - { - let is_finished = subscriptions_to_drop - .iter() - .any(|(subscriber_id, _)| *subscriber_id == current_subscriber_id); - if is_finished { - assert!(subscriptions_to_drop.iter().any( - |(subscriber_id, subscribed_upstream_table_id)| { - *subscriber_id == current_subscriber_id - && upstream_table_id == *subscribed_upstream_table_id +impl<'a> UpstreamBuffer<'a, StateOfConsumingLogStore> { + async fn next_barrier(&mut self) -> StreamExecutorResult> { + Ok( + if let Some(barrier) = self.state.upstream_pending_barriers.pop_back() { + // only update the `max_pending_barrier_num` on checkpoint barrier to avoid deadlock caused by + // downstream waiting on a checkpoint barrier to bump up. + if barrier.kind.is_checkpoint() { + self.state.max_pending_barrier_num = min( + self.state.upstream_pending_barriers.len(), + self.state + .max_pending_barrier_num + .saturating_sub(self.state.max_pending_barrier_num_gap), + ) } - )) - } - is_finished - } else { - false + Some(barrier) + } else { + self.state.max_pending_barrier_num = 0; + if self.is_polling_epoch_data { + let barrier = self.consume_until_next_barrier().await?; + Some(barrier) + } else { + None + } + }, + ) } } diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 139f58a696f42..6cfa439d59af0 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -24,7 +24,7 @@ use futures::StreamExt; use itertools::Itertools; use risingwave_common::error::tonic::extra::Score; use risingwave_pb::stream_service::barrier_complete_response::{ - PbCreateMviewProgress, PbLocalSstableInfo, + PbCreateMviewLogStoreProgress, PbCreateMviewProgress, PbLocalSstableInfo, }; use risingwave_rpc_client::error::{ToTonicStatus, TonicStatusWrapper}; use thiserror_ext::AsReport; @@ -78,6 +78,8 @@ pub struct BarrierCompleteResult { /// The updated creation progress of materialized view after this barrier. pub create_mview_progress: Vec, + + pub create_mview_log_store_progress: Vec, } pub(super) struct ControlStreamHandle { @@ -187,6 +189,12 @@ pub(super) enum LocalBarrierEvent { actor: ActorId, state: BackfillState, }, + ReportCreateMviewLogStoreProgress { + epoch: EpochPair, + actor: ActorId, + /// `None` means it has finished processing log store + pending_barrier_num: Option, + }, RegisterBarrierSender { actor_id: ActorId, barrier_sender: mpsc::UnboundedSender, @@ -407,6 +415,11 @@ impl LocalBarrierWorker { } => { self.update_create_mview_progress(epoch, actor, state); } + LocalBarrierEvent::ReportCreateMviewLogStoreProgress { + epoch, + actor, + pending_barrier_num, + } => self.update_create_mview_log_store_progress(epoch, actor, pending_barrier_num), LocalBarrierEvent::RegisterBarrierSender { actor_id, barrier_sender, @@ -467,6 +480,7 @@ impl LocalBarrierWorker { let BarrierCompleteResult { create_mview_progress, sync_result, + create_mview_log_store_progress, } = result; let (synced_sstables, table_watermarks, old_value_ssts) = sync_result @@ -488,6 +502,7 @@ impl LocalBarrierWorker { epoch, status: None, create_mview_progress, + create_mview_log_store_progress, synced_sstables: synced_sstables .into_iter() .map( diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 462f78233f25b..b49fc2ceeec08 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -94,7 +94,9 @@ mod await_epoch_completed_future { use futures::future::BoxFuture; use futures::FutureExt; use risingwave_hummock_sdk::SyncResult; - use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewProgress; + use risingwave_pb::stream_service::barrier_complete_response::{ + PbCreateMviewLogStoreProgress, PbCreateMviewProgress, + }; use crate::error::StreamResult; use crate::executor::Barrier; @@ -108,6 +110,7 @@ mod await_epoch_completed_future { barrier: Barrier, barrier_await_tree_reg: Option<&await_tree::Registry>, create_mview_progress: Vec, + create_mview_log_store_progress: Vec, ) -> AwaitEpochCompletedFuture { let prev_epoch = barrier.epoch.prev; let future = async move { @@ -124,6 +127,7 @@ mod await_epoch_completed_future { result.map(|sync_result| BarrierCompleteResult { sync_result, create_mview_progress, + create_mview_log_store_progress, }), ) }); @@ -142,6 +146,7 @@ mod await_epoch_completed_future { use await_epoch_completed_future::*; use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; +use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewLogStoreProgress; use risingwave_pb::stream_service::InjectBarrierRequest; fn sync_epoch( @@ -378,6 +383,8 @@ pub(super) struct PartialGraphManagedBarrierState { /// in [`BarrierCompleteResult`]. pub(super) create_mview_progress: HashMap>, + pub(super) create_mview_log_store_progress: HashMap>>, + pub(super) state_store: StateStoreImpl, pub(super) streaming_metrics: Arc, @@ -399,6 +406,7 @@ impl PartialGraphManagedBarrierState { epoch_barrier_state_map: Default::default(), prev_barrier_table_ids: None, create_mview_progress: Default::default(), + create_mview_log_store_progress: Default::default(), await_epoch_completed_futures: Default::default(), state_store, streaming_metrics, @@ -743,6 +751,20 @@ impl PartialGraphManagedBarrierState { .map(|(actor, state)| state.to_pb(actor)) .collect(); + let create_mview_log_store_progress = self + .create_mview_log_store_progress + .remove(&barrier_state.barrier.epoch.curr) + .unwrap_or_default() + .into_iter() + .map( + |(actor, pending_barrier_num)| PbCreateMviewLogStoreProgress { + backfill_actor_id: actor, + done: pending_barrier_num.is_none(), + pending_barrier_num: pending_barrier_num.unwrap_or_default() as _, + }, + ) + .collect(); + let complete_barrier_future = match kind { BarrierKind::Unspecified => unreachable!(), BarrierKind::Initial => { @@ -774,6 +796,7 @@ impl PartialGraphManagedBarrierState { barrier, self.barrier_await_tree_reg.as_ref(), create_mview_progress, + create_mview_log_store_progress, ) }); } diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 9b2820bb3bfed..a991a533f0bfa 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -18,7 +18,9 @@ use risingwave_common::util::epoch::EpochPair; use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewProgress; use super::LocalBarrierManager; -use crate::task::barrier_manager::LocalBarrierEvent::ReportCreateProgress; +use crate::task::barrier_manager::LocalBarrierEvent::{ + ReportCreateMviewLogStoreProgress, ReportCreateProgress, +}; use crate::task::barrier_manager::LocalBarrierWorker; use crate::task::ActorId; @@ -79,6 +81,31 @@ impl LocalBarrierWorker { warn!(?epoch, actor, ?state, "ignore create mview progress"); } } + + pub(crate) fn update_create_mview_log_store_progress( + &mut self, + epoch: EpochPair, + actor: ActorId, + pending_barrier_num: Option, + ) { + if let Some(actor_state) = self.state.actor_states.get(&actor) + && let Some(partial_graph_id) = actor_state.inflight_barriers.get(&epoch.prev) + && let Some(graph_state) = self.state.graph_states.get_mut(partial_graph_id) + { + graph_state + .create_mview_log_store_progress + .entry(epoch.curr) + .or_default() + .insert(actor, pending_barrier_num); + } else { + warn!( + ?epoch, + actor, + ?pending_barrier_num, + "ignore create mview log store progress" + ); + } + } } impl LocalBarrierManager { @@ -191,6 +218,10 @@ impl CreateMviewProgressReporter { } self.update_inner(epoch, BackfillState::Done(current_consumed_rows)); } + + pub(crate) fn barrier_manager(&self) -> &LocalBarrierManager { + &self.barrier_manager + } } impl LocalBarrierManager { @@ -201,11 +232,32 @@ impl LocalBarrierManager { /// /// When all backfill executors of the creating mview finish, the creation progress will be done at /// frontend and the mview will be exposed to the user. - pub fn register_create_mview_progress( + pub(crate) fn register_create_mview_progress( &self, backfill_actor_id: ActorId, ) -> CreateMviewProgressReporter { trace!("register create mview progress: {}", backfill_actor_id); CreateMviewProgressReporter::new(self.clone(), backfill_actor_id) } + + pub(crate) fn update_create_mview_log_store_progress( + &self, + epoch: EpochPair, + actor: ActorId, + pending_barrier_num: usize, + ) { + self.send_event(ReportCreateMviewLogStoreProgress { + epoch, + actor, + pending_barrier_num: Some(pending_barrier_num), + }) + } + + pub(crate) fn finish_consuming_log_store(&self, epoch: EpochPair, actor: ActorId) { + self.send_event(ReportCreateMviewLogStoreProgress { + epoch, + actor, + pending_barrier_num: None, + }) + } } From 648a770029d371b399527307e61baf8583ea7a59 Mon Sep 17 00:00:00 2001 From: William Wen Date: Sun, 6 Oct 2024 14:48:48 +0800 Subject: [PATCH 02/19] merge create mview progress --- proto/stream_service.proto | 7 +- src/meta/src/barrier/creating_job/mod.rs | 13 +- src/meta/src/barrier/creating_job/status.rs | 16 +- .../executor/backfill/snapshot_backfill.rs | 11 +- src/stream/src/task/barrier_manager.rs | 17 +- .../src/task/barrier_manager/managed_state.rs | 25 +-- .../src/task/barrier_manager/progress.rs | 155 ++++++++++-------- 7 files changed, 102 insertions(+), 142 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 3d2faf5e4e56d..9fc0da18d7d73 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -28,11 +28,7 @@ message BarrierCompleteResponse { bool done = 2; uint64 consumed_epoch = 3; uint64 consumed_rows = 4; - } - message CreateMviewLogStoreProgress { - uint32 backfill_actor_id = 1; - bool done = 2; - uint32 pending_barrier_num = 3; + uint32 pending_barrier_num = 5; } string request_id = 1; common.Status status = 2; @@ -51,7 +47,6 @@ message BarrierCompleteResponse { uint32 partial_graph_id = 8; // prev_epoch of barrier uint64 epoch = 9; - repeated CreateMviewLogStoreProgress create_mview_log_store_progress = 10; } message WaitEpochCommitRequest { diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/creating_job/mod.rs index 760b14571de77..089d7039aacc1 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/creating_job/mod.rs @@ -107,11 +107,15 @@ impl CreatingStreamingJobControl { } } - /// Return whether the job is finished or not + /// Attach an upstream epoch to be notified on the finish of the creating job. + /// Return whether the job is finished or not, and if finished, the upstream_epoch won't be attached. pub(super) fn attach_upstream_wait_finish_epoch(&mut self, upstream_epoch: u64) -> bool { match &mut self.status { CreatingStreamingJobStatus::Finishing(upstream_epoch_to_notify) => { - assert_eq!(*upstream_epoch_to_notify, None); + assert_eq!( + *upstream_epoch_to_notify, None, + "should not attach wait finish epoch for twice" + ); if self.barrier_control.is_empty() { true } else { @@ -318,10 +322,7 @@ impl CreatingStreamingJobControl { worker_id: WorkerId, resp: BarrierCompleteResponse, ) { - self.status.update_progress( - &resp.create_mview_progress, - &resp.create_mview_log_store_progress, - ); + self.status.update_progress(&resp.create_mview_progress); self.barrier_control.collect(epoch, worker_id, resp); } diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/creating_job/status.rs index 511f52aa624a3..337611327a814 100644 --- a/src/meta/src/barrier/creating_job/status.rs +++ b/src/meta/src/barrier/creating_job/status.rs @@ -17,14 +17,13 @@ use std::collections::{HashMap, HashSet}; use std::mem::take; use std::sync::Arc; -use itertools::Itertools; use risingwave_common::hash::ActorId; use risingwave_common::util::epoch::Epoch; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; use risingwave_pb::stream_plan::StreamActor; use risingwave_pb::stream_service::barrier_complete_response::{ - CreateMviewProgress, PbCreateMviewLogStoreProgress, + CreateMviewProgress, PbCreateMviewProgress, }; use tracing::warn; @@ -65,7 +64,7 @@ impl CreateMviewLogStoreProgressTracker { ) } - fn update(&mut self, progress: impl IntoIterator) { + fn update(&mut self, progress: impl IntoIterator) { for progress in progress { match self.ongoing_actors.entry(progress.backfill_actor_id) { Entry::Occupied(mut entry) => { @@ -145,7 +144,6 @@ impl CreatingStreamingJobStatus { pub(super) fn update_progress( &mut self, create_mview_progress: impl IntoIterator, - log_store_progress: impl IntoIterator, ) { match self { Self::ConsumingSnapshot { @@ -158,20 +156,12 @@ impl CreatingStreamingJobStatus { create_mview_progress, version_stats, ); - let mut log_store_progress = log_store_progress.into_iter().peekable(); - if log_store_progress.peek().is_some() { - warn!(log_store_progress = ?log_store_progress.collect_vec(), "ignore log store progress when consuming snapshot"); - } } CreatingStreamingJobStatus::ConsumingLogStore { log_store_progress_tracker, .. } => { - log_store_progress_tracker.update(log_store_progress); - let mut create_mview_progress = create_mview_progress.into_iter().peekable(); - if create_mview_progress.peek().is_some() { - warn!(create_mview_progress = ?create_mview_progress.collect_vec(), "ignore create_mview_progress when consuming log store"); - } + log_store_progress_tracker.update(create_mview_progress); } CreatingStreamingJobStatus::Finishing(_) => {} } diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index bbbd5e9bc61b2..6a740eafa0503 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -105,8 +105,6 @@ impl SnapshotBackfillExecutor { debug!(epoch = ?first_recv_barrier.epoch, "get first inject barrier"); let should_backfill = first_barrier.epoch != first_recv_barrier.epoch; - let barrier_manager = self.progress.barrier_manager().clone(); - let (mut barrier_epoch, mut need_report_finish) = { if should_backfill { let table_id_str = format!("{}", self.upstream_table.table_id().table_id); @@ -140,7 +138,7 @@ impl SnapshotBackfillExecutor { self.rate_limit, &mut self.barrier_rx, &self.output_indices, - self.progress, + &mut self.progress, first_recv_barrier, ); @@ -217,9 +215,8 @@ impl SnapshotBackfillExecutor { debug!(?barrier_epoch, "after consume change log"); - barrier_manager.update_create_mview_log_store_progress( + self.progress.update_create_mview_log_store_progress( barrier.epoch, - self.actor_ctx.id, upstream_buffer.state.barrier_count(), ); @@ -251,7 +248,7 @@ impl SnapshotBackfillExecutor { barrier_epoch = barrier.epoch; if need_report_finish { need_report_finish = false; - barrier_manager.finish_consuming_log_store(barrier_epoch, self.actor_ctx.id); + self.progress.finish_consuming_log_store(barrier_epoch); } } yield msg; @@ -547,7 +544,7 @@ async fn make_consume_snapshot_stream<'a, S: StateStore>( rate_limit: Option, barrier_rx: &'a mut UnboundedReceiver, output_indices: &'a [usize], - mut progress: CreateMviewProgressReporter, + progress: &'a mut CreateMviewProgressReporter, first_recv_barrier: Barrier, ) { let mut barrier_epoch = first_recv_barrier.epoch; diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 6cfa439d59af0..139f58a696f42 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -24,7 +24,7 @@ use futures::StreamExt; use itertools::Itertools; use risingwave_common::error::tonic::extra::Score; use risingwave_pb::stream_service::barrier_complete_response::{ - PbCreateMviewLogStoreProgress, PbCreateMviewProgress, PbLocalSstableInfo, + PbCreateMviewProgress, PbLocalSstableInfo, }; use risingwave_rpc_client::error::{ToTonicStatus, TonicStatusWrapper}; use thiserror_ext::AsReport; @@ -78,8 +78,6 @@ pub struct BarrierCompleteResult { /// The updated creation progress of materialized view after this barrier. pub create_mview_progress: Vec, - - pub create_mview_log_store_progress: Vec, } pub(super) struct ControlStreamHandle { @@ -189,12 +187,6 @@ pub(super) enum LocalBarrierEvent { actor: ActorId, state: BackfillState, }, - ReportCreateMviewLogStoreProgress { - epoch: EpochPair, - actor: ActorId, - /// `None` means it has finished processing log store - pending_barrier_num: Option, - }, RegisterBarrierSender { actor_id: ActorId, barrier_sender: mpsc::UnboundedSender, @@ -415,11 +407,6 @@ impl LocalBarrierWorker { } => { self.update_create_mview_progress(epoch, actor, state); } - LocalBarrierEvent::ReportCreateMviewLogStoreProgress { - epoch, - actor, - pending_barrier_num, - } => self.update_create_mview_log_store_progress(epoch, actor, pending_barrier_num), LocalBarrierEvent::RegisterBarrierSender { actor_id, barrier_sender, @@ -480,7 +467,6 @@ impl LocalBarrierWorker { let BarrierCompleteResult { create_mview_progress, sync_result, - create_mview_log_store_progress, } = result; let (synced_sstables, table_watermarks, old_value_ssts) = sync_result @@ -502,7 +488,6 @@ impl LocalBarrierWorker { epoch, status: None, create_mview_progress, - create_mview_log_store_progress, synced_sstables: synced_sstables .into_iter() .map( diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index b49fc2ceeec08..462f78233f25b 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -94,9 +94,7 @@ mod await_epoch_completed_future { use futures::future::BoxFuture; use futures::FutureExt; use risingwave_hummock_sdk::SyncResult; - use risingwave_pb::stream_service::barrier_complete_response::{ - PbCreateMviewLogStoreProgress, PbCreateMviewProgress, - }; + use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewProgress; use crate::error::StreamResult; use crate::executor::Barrier; @@ -110,7 +108,6 @@ mod await_epoch_completed_future { barrier: Barrier, barrier_await_tree_reg: Option<&await_tree::Registry>, create_mview_progress: Vec, - create_mview_log_store_progress: Vec, ) -> AwaitEpochCompletedFuture { let prev_epoch = barrier.epoch.prev; let future = async move { @@ -127,7 +124,6 @@ mod await_epoch_completed_future { result.map(|sync_result| BarrierCompleteResult { sync_result, create_mview_progress, - create_mview_log_store_progress, }), ) }); @@ -146,7 +142,6 @@ mod await_epoch_completed_future { use await_epoch_completed_future::*; use risingwave_pb::stream_plan::SubscriptionUpstreamInfo; -use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewLogStoreProgress; use risingwave_pb::stream_service::InjectBarrierRequest; fn sync_epoch( @@ -383,8 +378,6 @@ pub(super) struct PartialGraphManagedBarrierState { /// in [`BarrierCompleteResult`]. pub(super) create_mview_progress: HashMap>, - pub(super) create_mview_log_store_progress: HashMap>>, - pub(super) state_store: StateStoreImpl, pub(super) streaming_metrics: Arc, @@ -406,7 +399,6 @@ impl PartialGraphManagedBarrierState { epoch_barrier_state_map: Default::default(), prev_barrier_table_ids: None, create_mview_progress: Default::default(), - create_mview_log_store_progress: Default::default(), await_epoch_completed_futures: Default::default(), state_store, streaming_metrics, @@ -751,20 +743,6 @@ impl PartialGraphManagedBarrierState { .map(|(actor, state)| state.to_pb(actor)) .collect(); - let create_mview_log_store_progress = self - .create_mview_log_store_progress - .remove(&barrier_state.barrier.epoch.curr) - .unwrap_or_default() - .into_iter() - .map( - |(actor, pending_barrier_num)| PbCreateMviewLogStoreProgress { - backfill_actor_id: actor, - done: pending_barrier_num.is_none(), - pending_barrier_num: pending_barrier_num.unwrap_or_default() as _, - }, - ) - .collect(); - let complete_barrier_future = match kind { BarrierKind::Unspecified => unreachable!(), BarrierKind::Initial => { @@ -796,7 +774,6 @@ impl PartialGraphManagedBarrierState { barrier, self.barrier_await_tree_reg.as_ref(), create_mview_progress, - create_mview_log_store_progress, ) }); } diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index a991a533f0bfa..5a34dd7448e70 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -18,9 +18,7 @@ use risingwave_common::util::epoch::EpochPair; use risingwave_pb::stream_service::barrier_complete_response::PbCreateMviewProgress; use super::LocalBarrierManager; -use crate::task::barrier_manager::LocalBarrierEvent::{ - ReportCreateMviewLogStoreProgress, ReportCreateProgress, -}; +use crate::task::barrier_manager::LocalBarrierEvent::ReportCreateProgress; use crate::task::barrier_manager::LocalBarrierWorker; use crate::task::ActorId; @@ -29,23 +27,30 @@ type ConsumedRows = u64; #[derive(Debug, Clone, Copy)] pub(crate) enum BackfillState { - ConsumingUpstream(ConsumedEpoch, ConsumedRows), - Done(ConsumedRows), + ConsumingUpstreamTable(ConsumedEpoch, ConsumedRows), + DoneConsumingUpstreamTable(ConsumedRows), + ConsumingLogStore { pending_barrier_num: usize }, + DoneConsumingLogStore, } impl BackfillState { pub fn to_pb(self, actor_id: ActorId) -> PbCreateMviewProgress { + let (done, consumed_epoch, consumed_rows, pending_barrier_num) = match self { + BackfillState::ConsumingUpstreamTable(consumed_epoch, consumed_rows) => { + (false, consumed_epoch, consumed_rows, 0) + } + BackfillState::DoneConsumingUpstreamTable(consumed_rows) => (true, 0, consumed_rows, 0), /* unused field for done */ + BackfillState::ConsumingLogStore { + pending_barrier_num, + } => (false, 0, 0, pending_barrier_num as _), + BackfillState::DoneConsumingLogStore => (true, 0, 0, 0), + }; PbCreateMviewProgress { backfill_actor_id: actor_id, - done: matches!(self, BackfillState::Done(_)), - consumed_epoch: match self { - BackfillState::ConsumingUpstream(consumed_epoch, _) => consumed_epoch, - BackfillState::Done(_) => 0, // unused field for done - }, - consumed_rows: match self { - BackfillState::ConsumingUpstream(_, consumed_rows) => consumed_rows, - BackfillState::Done(consumed_rows) => consumed_rows, - }, + done, + consumed_epoch, + consumed_rows, + pending_barrier_num, } } } @@ -53,10 +58,27 @@ impl BackfillState { impl Display for BackfillState { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { match self { - BackfillState::ConsumingUpstream(epoch, rows) => { - write!(f, "ConsumingUpstream(epoch: {}, rows: {})", epoch, rows) + BackfillState::ConsumingUpstreamTable(epoch, rows) => { + write!( + f, + "ConsumingUpstreamTable(epoch: {}, rows: {})", + epoch, rows + ) + } + BackfillState::DoneConsumingUpstreamTable(rows) => { + write!(f, "DoneConsumingUpstreamTable(rows: {})", rows) + } + BackfillState::ConsumingLogStore { + pending_barrier_num, + } => { + write!( + f, + "ConsumingLogStore(pending_barrier_num: {pending_barrier_num})" + ) + } + BackfillState::DoneConsumingLogStore => { + write!(f, "DoneConsumingLogStore") } - BackfillState::Done(rows) => write!(f, "Done(rows: {})", rows), } } } @@ -81,31 +103,6 @@ impl LocalBarrierWorker { warn!(?epoch, actor, ?state, "ignore create mview progress"); } } - - pub(crate) fn update_create_mview_log_store_progress( - &mut self, - epoch: EpochPair, - actor: ActorId, - pending_barrier_num: Option, - ) { - if let Some(actor_state) = self.state.actor_states.get(&actor) - && let Some(partial_graph_id) = actor_state.inflight_barriers.get(&epoch.prev) - && let Some(graph_state) = self.state.graph_states.get_mut(partial_graph_id) - { - graph_state - .create_mview_log_store_progress - .entry(epoch.curr) - .or_default() - .insert(actor, pending_barrier_num); - } else { - warn!( - ?epoch, - actor, - ?pending_barrier_num, - "ignore create mview log store progress" - ); - } - } } impl LocalBarrierManager { @@ -192,7 +189,7 @@ impl CreateMviewProgressReporter { current_consumed_rows: ConsumedRows, ) { match self.state { - Some(BackfillState::ConsumingUpstream(last, last_consumed_rows)) => { + Some(BackfillState::ConsumingUpstreamTable(last, last_consumed_rows)) => { assert!( last < consumed_epoch, "last_epoch: {:#?} must be greater than consumed epoch: {:#?}", @@ -201,26 +198,65 @@ impl CreateMviewProgressReporter { ); assert!(last_consumed_rows <= current_consumed_rows); } - Some(BackfillState::Done(_)) => unreachable!(), + Some(state) => { + panic!( + "should not update consuming progress at invalid state: {:?}", + state + ) + } None => {} }; self.update_inner( epoch, - BackfillState::ConsumingUpstream(consumed_epoch, current_consumed_rows), + BackfillState::ConsumingUpstreamTable(consumed_epoch, current_consumed_rows), ); } /// Finish the progress. If the progress is already finished, then perform no-op. /// `current_epoch` should be provided to locate the barrier under concurrent checkpoint. pub fn finish(&mut self, epoch: EpochPair, current_consumed_rows: ConsumedRows) { - if let Some(BackfillState::Done(_)) = self.state { + if let Some(BackfillState::DoneConsumingUpstreamTable(_)) = self.state { return; } - self.update_inner(epoch, BackfillState::Done(current_consumed_rows)); + self.update_inner( + epoch, + BackfillState::DoneConsumingUpstreamTable(current_consumed_rows), + ); + } + + pub(crate) fn update_create_mview_log_store_progress( + &mut self, + epoch: EpochPair, + pending_barrier_num: usize, + ) { + match self.state { + Some(BackfillState::DoneConsumingUpstreamTable(_)) + | Some(BackfillState::ConsumingLogStore { .. }) => { + // valid state + } + state => { + panic!("cannot update log store progress at state {:?}", state) + } + } + self.update_inner( + epoch, + BackfillState::ConsumingLogStore { + pending_barrier_num, + }, + ); } - pub(crate) fn barrier_manager(&self) -> &LocalBarrierManager { - &self.barrier_manager + pub(crate) fn finish_consuming_log_store(&mut self, epoch: EpochPair) { + match self.state { + Some(BackfillState::DoneConsumingUpstreamTable(_)) + | Some(BackfillState::ConsumingLogStore { .. }) => { + // valid state + } + state => { + panic!("cannot finish log store progress at state {:?}", state) + } + } + self.update_inner(epoch, BackfillState::DoneConsumingLogStore); } } @@ -239,25 +275,4 @@ impl LocalBarrierManager { trace!("register create mview progress: {}", backfill_actor_id); CreateMviewProgressReporter::new(self.clone(), backfill_actor_id) } - - pub(crate) fn update_create_mview_log_store_progress( - &self, - epoch: EpochPair, - actor: ActorId, - pending_barrier_num: usize, - ) { - self.send_event(ReportCreateMviewLogStoreProgress { - epoch, - actor, - pending_barrier_num: Some(pending_barrier_num), - }) - } - - pub(crate) fn finish_consuming_log_store(&self, epoch: EpochPair, actor: ActorId) { - self.send_event(ReportCreateMviewLogStoreProgress { - epoch, - actor, - pending_barrier_num: None, - }) - } } From b6cb330b5b7e6ff677d059419c8d2007dd940a05 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 7 Oct 2024 12:51:46 +0800 Subject: [PATCH 03/19] refactor and refine --- .../barrier/creating_job/barrier_control.rs | 24 ++- src/meta/src/barrier/creating_job/mod.rs | 105 +++++---- src/meta/src/barrier/creating_job/status.rs | 5 +- src/meta/src/barrier/mod.rs | 200 ++++++++---------- src/meta/src/barrier/state.rs | 32 ++- src/meta/src/rpc/metrics.rs | 12 +- 6 files changed, 183 insertions(+), 195 deletions(-) diff --git a/src/meta/src/barrier/creating_job/barrier_control.rs b/src/meta/src/barrier/creating_job/barrier_control.rs index f0f092a7ff853..f5a0453edcf18 100644 --- a/src/meta/src/barrier/creating_job/barrier_control.rs +++ b/src/meta/src/barrier/creating_job/barrier_control.rs @@ -14,6 +14,8 @@ use std::collections::{BTreeMap, HashSet, VecDeque}; use std::mem::take; +use std::ops::Bound::Unbounded; +use std::ops::{Bound, RangeBounds}; use std::time::Instant; use prometheus::HistogramTimer; @@ -42,7 +44,7 @@ pub(super) struct CreatingStreamingJobBarrierControl { backfill_epoch: u64, initial_epoch: Option, max_collected_epoch: Option, - // newer epoch at the front. should all be checkpoint barrier + // newer epoch at the front. pending_barriers_to_complete: VecDeque, completing_barrier: Option<(CreatingStreamingJobEpochState, HistogramTimer)>, @@ -174,11 +176,21 @@ impl CreatingStreamingJobBarrierControl { } /// Return Some((epoch, resps, `is_first_commit`)) - pub(super) fn start_completing(&mut self) -> Option<(u64, Vec, bool)> { - if self.completing_barrier.is_some() { - return None; - } - while let Some(mut epoch_state) = self.pending_barriers_to_complete.pop_back() { + /// + /// Only epoch within the `epoch_end_bound` can be started. + pub(super) fn start_completing( + &mut self, + epoch_end_bound: Bound, + ) -> Option<(u64, Vec, bool)> { + assert!(self.completing_barrier.is_none()); + let epoch_range: (Bound, Bound) = (Unbounded, epoch_end_bound); + while let Some(epoch_state) = self.pending_barriers_to_complete.back() + && epoch_range.contains(&epoch_state.epoch) + { + let mut epoch_state = self + .pending_barriers_to_complete + .pop_back() + .expect("non-empty"); let epoch = epoch_state.epoch; let is_first = self.initial_epoch.expect("should have set") == epoch; if is_first { diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/creating_job/mod.rs index 089d7039aacc1..ebec98e6d4eec 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/creating_job/mod.rs @@ -17,10 +17,10 @@ mod status; use std::cmp::max; use std::collections::HashMap; +use std::ops::Bound::{Excluded, Unbounded}; use std::sync::Arc; -use prometheus::HistogramTimer; -use risingwave_common::metrics::{LabelGuardedHistogram, LabelGuardedIntGauge}; +use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::DdlProgress; use risingwave_pb::hummock::HummockVersionStats; @@ -51,7 +51,6 @@ pub(super) struct CreatingStreamingJobControl { status: CreatingStreamingJobStatus, upstream_lag: LabelGuardedIntGauge<1>, - upstream_wait_progress_latency: LabelGuardedHistogram<1>, } impl CreatingStreamingJobControl { @@ -101,39 +100,9 @@ impl CreatingStreamingJobControl { upstream_lag: metrics .snapshot_backfill_lag .with_guarded_label_values(&[&table_id_str]), - upstream_wait_progress_latency: metrics - .snapshot_backfill_upstream_wait_progress_latency - .with_guarded_label_values(&[&table_id_str]), } } - /// Attach an upstream epoch to be notified on the finish of the creating job. - /// Return whether the job is finished or not, and if finished, the upstream_epoch won't be attached. - pub(super) fn attach_upstream_wait_finish_epoch(&mut self, upstream_epoch: u64) -> bool { - match &mut self.status { - CreatingStreamingJobStatus::Finishing(upstream_epoch_to_notify) => { - assert_eq!( - *upstream_epoch_to_notify, None, - "should not attach wait finish epoch for twice" - ); - if self.barrier_control.is_empty() { - true - } else { - *upstream_epoch_to_notify = Some(upstream_epoch); - false - } - } - CreatingStreamingJobStatus::ConsumingSnapshot { .. } - | CreatingStreamingJobStatus::ConsumingLogStore { .. } => { - unreachable!("should not attach upstream wait finish epoch at non-finishing status") - } - } - } - - pub(super) fn start_wait_progress_timer(&self) -> HistogramTimer { - self.upstream_wait_progress_latency.start_timer() - } - pub(super) fn is_wait_on_worker(&self, worker_id: WorkerId) -> bool { self.barrier_control.is_wait_on_worker(worker_id) || self @@ -303,7 +272,8 @@ impl CreatingStreamingJobControl { table_id = self.info.table_fragments.table_id().table_id, prev_epoch, "start consuming upstream" ); - self.status = CreatingStreamingJobStatus::Finishing(None); + assert!(command_ctx.kind.is_checkpoint()); + self.status = CreatingStreamingJobStatus::Finishing(prev_epoch); } } CreatingStreamingJobStatus::Finishing { .. } => { @@ -338,22 +308,65 @@ impl CreatingStreamingJobControl { None } } +} + +pub(super) enum CompleteJobType { + /// The first barrier + First, + Normal, + /// The last barrier to complete + Finished, +} - pub(super) fn start_completing(&mut self) -> Option<(u64, Vec, bool)> { - self.barrier_control.start_completing() +impl CreatingStreamingJobControl { + pub(super) fn start_completing( + &mut self, + min_upstream_inflight_epoch: Option, + ) -> Option<(u64, Vec, CompleteJobType)> { + let (finished_at_epoch, epoch_end_bound) = match &self.status { + CreatingStreamingJobStatus::Finishing(finish_at_epoch) => { + let epoch_end_bound = min_upstream_inflight_epoch + .map(|upstream_epoch| { + if upstream_epoch < *finish_at_epoch { + Excluded(upstream_epoch) + } else { + Unbounded + } + }) + .unwrap_or(Unbounded); + (Some(*finish_at_epoch), epoch_end_bound) + } + CreatingStreamingJobStatus::ConsumingSnapshot { .. } + | CreatingStreamingJobStatus::ConsumingLogStore { .. } => ( + None, + min_upstream_inflight_epoch + .map(Excluded) + .unwrap_or(Unbounded), + ), + }; + self.barrier_control.start_completing(epoch_end_bound).map( + |(epoch, resps, is_first_commit)| { + let status = if let Some(finish_at_epoch) = finished_at_epoch { + assert!(!is_first_commit); + if epoch == finish_at_epoch { + self.barrier_control.ack_completed(epoch); + assert!(self.barrier_control.is_empty()); + CompleteJobType::Finished + } else { + CompleteJobType::Normal + } + } else if is_first_commit { + CompleteJobType::First + } else { + CompleteJobType::Normal + }; + (epoch, resps, status) + }, + ) } - pub(super) fn ack_completed(&mut self, completed_epoch: u64) -> Option { + pub(super) fn ack_completed(&mut self, completed_epoch: u64) { self.barrier_control.ack_completed(completed_epoch); - if let CreatingStreamingJobStatus::Finishing(upstream_epoch_to_notify) = &self.status { - if self.barrier_control.is_empty() { - *upstream_epoch_to_notify - } else { - None - } - } else { - None - } } pub(super) fn is_finished(&self) -> bool { diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/creating_job/status.rs index 337611327a814..39825243df57d 100644 --- a/src/meta/src/barrier/creating_job/status.rs +++ b/src/meta/src/barrier/creating_job/status.rs @@ -117,7 +117,10 @@ pub(super) enum CreatingStreamingJobStatus { graph_info: InflightGraphInfo, log_store_progress_tracker: CreateMviewLogStoreProgressTracker, }, - Finishing(Option), + /// All backfill actors have started consuming upstream, and the job + /// will be finished when all previously injected barriers have been collected + /// Store the `prev_epoch` that will finish at. + Finishing(u64), } pub(super) struct CreatingJobInjectBarrierInfo { diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index c8eb3507b6add..f2976f6876976 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -51,7 +51,7 @@ use tracing::{debug, error, info, warn, Instrument}; use self::command::CommandContext; use self::notifier::Notifier; -use crate::barrier::creating_job::CreatingStreamingJobControl; +use crate::barrier::creating_job::{CompleteJobType, CreatingStreamingJobControl}; use crate::barrier::info::InflightGraphInfo; use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob}; use crate::barrier::rpc::{merge_node_rpc_errors, ControlStreamManager}; @@ -291,7 +291,7 @@ impl CheckpointControl { command_ctx: Arc, notifiers: Vec, node_to_collect: HashSet, - jobs_to_wait: HashSet, + creating_jobs_to_wait: HashSet, table_ids_to_commit: HashSet, ) { let timer = self.context.metrics.barrier_latency.start_timer(); @@ -305,34 +305,9 @@ impl CheckpointControl { tracing::trace!( prev_epoch = command_ctx.prev_epoch.value().0, - ?jobs_to_wait, + ?creating_jobs_to_wait, "enqueue command" ); - let mut finished_table_ids = HashMap::new(); - let creating_jobs_to_wait = jobs_to_wait - .into_iter() - .filter_map(|table_id| { - let Entry::Occupied(mut entry) = - self.creating_streaming_job_controls.entry(table_id) - else { - unreachable!("should exist") - }; - if entry - .get_mut() - .attach_upstream_wait_finish_epoch(command_ctx.prev_epoch.value().0) - { - let (table_id, creating_job) = entry.remove_entry(); - finished_table_ids - .try_insert(table_id, creating_job.info) - .unwrap(); - None - } else if node_to_collect.is_empty() { - Some((table_id, Some(entry.get().start_wait_progress_timer()))) - } else { - Some((table_id, None)) - } - }) - .collect(); self.command_ctx_queue.insert( command_ctx.prev_epoch.value().0, EpochNode { @@ -341,7 +316,7 @@ impl CheckpointControl { node_to_collect, resps: vec![], creating_jobs_to_wait, - finished_table_ids, + finished_jobs: HashMap::new(), table_ids_to_commit, }, command_ctx, @@ -364,19 +339,6 @@ impl CheckpointControl { if resp.partial_graph_id == u32::MAX { if let Some(node) = self.command_ctx_queue.get_mut(&prev_epoch) { assert!(node.state.node_to_collect.remove(&worker_id)); - if node.state.node_to_collect.is_empty() { - node.state - .creating_jobs_to_wait - .iter_mut() - .for_each(|(table_id, timer)| { - *timer = Some( - self.creating_streaming_job_controls - .get(table_id) - .expect("should exist") - .start_wait_progress_timer(), - ); - }); - } node.state.resps.push(resp); } else { panic!( @@ -552,9 +514,9 @@ struct BarrierEpochState { resps: Vec, - creating_jobs_to_wait: HashMap>, + creating_jobs_to_wait: HashSet, - finished_table_ids: HashMap, + finished_jobs: HashMap)>, table_ids_to_commit: HashSet, } @@ -955,8 +917,7 @@ impl GlobalBarrierManager { } } - let Some((prev_epoch, curr_epoch, jobs_to_wait)) = self.state.next_epoch_pair(&command) - else { + let Some((prev_epoch, curr_epoch)) = self.state.next_epoch_pair(&command) else { // skip the command when there is nothing to do with the barrier for mut notifier in notifiers { notifier.notify_started(); @@ -1026,8 +987,14 @@ impl GlobalBarrierManager { command = Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge); } - let (pre_applied_graph_info, pre_applied_subscription_info) = - self.state.apply_command(&command); + let command = command; + + let ( + pre_applied_graph_info, + pre_applied_subscription_info, + table_ids_to_commit, + jobs_to_wait, + ) = self.state.apply_command(&command); // Tracing related stuff prev_epoch.span().in_scope(|| { @@ -1035,8 +1002,6 @@ impl GlobalBarrierManager { }); span.record("epoch", curr_epoch.value().0); - let table_ids_to_commit: HashSet<_> = pre_applied_graph_info.existing_table_ids().collect(); - let command_ctx = Arc::new(CommandContext::new( self.active_streaming_nodes.current().clone(), pre_applied_subscription_info, @@ -1199,21 +1164,22 @@ impl GlobalBarrierManagerContext { command_ctx, notifiers, enqueue_time, - state, + mut state, .. } = node; assert!(state.node_to_collect.is_empty()); assert!(state.creating_jobs_to_wait.is_empty()); let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); - if !state.finished_table_ids.is_empty() { + if !state.finished_jobs.is_empty() { assert!(command_ctx.kind.is_checkpoint()); + finished_jobs.extend(state.finished_jobs.into_values().map(|(info, resps)| { + state.resps.extend(resps); + TrackingJob::New(TrackingCommand { + info, + replace_table_info: None, + }) + })); } - finished_jobs.extend(state.finished_table_ids.into_values().map(|info| { - TrackingJob::New(TrackingCommand { - info, - replace_table_info: None, - }) - })); let result = self .update_snapshot( @@ -1387,6 +1353,65 @@ impl CheckpointControl { pub(super) async fn next_completed_barrier( &mut self, ) -> MetaResult> { + if matches!(&self.completing_command, CompletingCommand::None) { + // `Vec::new` is a const fn, and do not have memory allocation, and therefore is lightweight enough + let mut finished_jobs = Vec::new(); + let min_upstream_inflight_barrier = self + .command_ctx_queue + .first_key_value() + .map(|(epoch, _)| *epoch); + for (table_id, job) in &mut self.creating_streaming_job_controls { + if let Some((epoch, resps, status)) = + job.start_completing(min_upstream_inflight_barrier) + { + let is_first_time = match status { + CompleteJobType::First => true, + CompleteJobType::Normal => false, + CompleteJobType::Finished => { + finished_jobs.push((*table_id, epoch, resps)); + continue; + } + }; + let tables_to_commit = job + .info + .table_fragments + .all_table_ids() + .map(TableId::new) + .collect(); + let join_handle = + tokio::spawn(self.context.clone().complete_creating_job_barrier( + epoch, + resps, + tables_to_commit, + is_first_time, + )); + self.completing_command = CompletingCommand::CreatingStreamingJob { + table_id: *table_id, + epoch, + join_handle, + }; + break; + } + } + for (table_id, epoch, resps) in finished_jobs { + let epoch_state = &mut self + .command_ctx_queue + .get_mut(&epoch) + .expect("should exist") + .state; + assert!(epoch_state.creating_jobs_to_wait.remove(&table_id)); + debug!(epoch, ?table_id, "finish creating job"); + let creating_streaming_job = self + .creating_streaming_job_controls + .remove(&table_id) + .expect("should exist"); + assert!(creating_streaming_job.is_finished()); + assert!(epoch_state + .finished_jobs + .insert(table_id, (creating_streaming_job.info, resps)) + .is_none()); + } + } if matches!(&self.completing_command, CompletingCommand::None) { // If there is no completing barrier, try to start completing the earliest barrier if // it has been collected. @@ -1395,7 +1420,7 @@ impl CheckpointControl { { let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); assert!(node.state.creating_jobs_to_wait.is_empty()); - let table_ids_to_finish = node.state.finished_table_ids.keys().cloned().collect(); + let table_ids_to_finish = node.state.finished_jobs.keys().cloned().collect(); let finished_jobs = self .create_mview_tracker .apply_collected_command(&node, &self.hummock_version_stats); @@ -1419,31 +1444,6 @@ impl CheckpointControl { join_handle, table_ids_to_finish, }; - } else { - for (table_id, job) in &mut self.creating_streaming_job_controls { - let commit_info = job.start_completing(); - if let Some((epoch, resps, is_first_time)) = commit_info { - let tables_to_commit = job - .info - .table_fragments - .all_table_ids() - .map(TableId::new) - .collect(); - let join_handle = - tokio::spawn(self.context.clone().complete_creating_job_barrier( - epoch, - resps, - tables_to_commit, - is_first_time, - )); - self.completing_command = CompletingCommand::CreatingStreamingJob { - table_id: *table_id, - epoch, - join_handle, - }; - break; - } - } } } @@ -1499,38 +1499,10 @@ impl CheckpointControl { CompletingCommand::None }; self.completing_command = next_completing_command_status; - if let Some(upstream_epoch) = self - .creating_streaming_job_controls + self.creating_streaming_job_controls .get_mut(&table_id) .expect("should exist") - .ack_completed(epoch) - { - let wait_progress_timer = self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .creating_jobs_to_wait - .remove(&table_id) - .expect("should exist"); - if let Some(timer) = wait_progress_timer { - timer.observe_duration(); - } - debug!(epoch, ?table_id, "finish creating job"); - let creating_streaming_job = self - .creating_streaming_job_controls - .remove(&table_id) - .expect("should exist"); - assert!(creating_streaming_job.is_finished()); - assert!(self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .finished_table_ids - .insert(table_id, creating_streaming_job.info) - .is_none()); - } + .ack_completed(epoch); join_result.map(|_| None) } CompletingCommand::None | CompletingCommand::Err(_) => pending().await, diff --git a/src/meta/src/barrier/state.rs b/src/meta/src/barrier/state.rs index 5f9e17ca16658..1edd924a393f6 100644 --- a/src/meta/src/barrier/state.rs +++ b/src/meta/src/barrier/state.rs @@ -13,7 +13,6 @@ // limitations under the License. use std::collections::HashSet; -use std::mem::take; use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::Epoch; @@ -29,7 +28,6 @@ pub struct BarrierManagerState { /// There's no need to persist this field. On recovery, we will restore this from the latest /// committed snapshot in `HummockManager`. in_flight_prev_epoch: Option, - creating_jobs_to_wait: HashSet, /// Inflight running actors info. pub(crate) inflight_graph_info: InflightGraphInfo, @@ -49,7 +47,6 @@ impl BarrierManagerState { ) -> Self { Self { in_flight_prev_epoch, - creating_jobs_to_wait: Default::default(), inflight_graph_info, inflight_subscription_info, paused_reason, @@ -72,10 +69,7 @@ impl BarrierManagerState { } /// Returns the epoch pair for the next barrier, and updates the state. - pub fn next_epoch_pair( - &mut self, - command: &Command, - ) -> Option<(TracedEpoch, TracedEpoch, HashSet)> { + pub fn next_epoch_pair(&mut self, command: &Command) -> Option<(TracedEpoch, TracedEpoch)> { if self.inflight_graph_info.is_empty() && !matches!(&command, Command::CreateStreamingJob { .. }) { @@ -87,11 +81,7 @@ impl BarrierManagerState { let prev_epoch = in_flight_prev_epoch.clone(); let next_epoch = prev_epoch.next(); *in_flight_prev_epoch = next_epoch.clone(); - Some(( - prev_epoch, - next_epoch, - take(&mut self.creating_jobs_to_wait), - )) + Some((prev_epoch, next_epoch)) } /// Returns the inflight actor infos that have included the newly added actors in the given command. The dropped actors @@ -99,7 +89,12 @@ impl BarrierManagerState { pub fn apply_command( &mut self, command: &Command, - ) -> (InflightGraphInfo, InflightSubscriptionInfo) { + ) -> ( + InflightGraphInfo, + InflightSubscriptionInfo, + HashSet, + HashSet, + ) { // update the fragment_infos outside pre_apply let fragment_changes = if let Command::CreateStreamingJob { job_type: CreateStreamingJobType::SnapshotBackfill(_), @@ -121,16 +116,19 @@ impl BarrierManagerState { if let Some(fragment_changes) = fragment_changes { self.inflight_graph_info.post_apply(&fragment_changes); } + + let mut table_ids_to_commit: HashSet<_> = info.existing_table_ids().collect(); + let mut jobs_to_wait = HashSet::new(); if let Command::MergeSnapshotBackfillStreamingJobs(jobs_to_merge) = command { - for (_, graph_info) in jobs_to_merge.values() { + for (table_id, (_, graph_info)) in jobs_to_merge { + jobs_to_wait.insert(*table_id); + table_ids_to_commit.extend(graph_info.existing_table_ids()); self.inflight_graph_info.extend(graph_info.clone()); } - self.creating_jobs_to_wait - .extend(jobs_to_merge.keys().cloned()); } self.inflight_subscription_info.post_apply(command); - (info, subscription_info) + (info, subscription_info, table_ids_to_commit, jobs_to_wait) } } diff --git a/src/meta/src/rpc/metrics.rs b/src/meta/src/rpc/metrics.rs index 382e83afa1533..ec30b1dff0585 100644 --- a/src/meta/src/rpc/metrics.rs +++ b/src/meta/src/rpc/metrics.rs @@ -84,9 +84,6 @@ pub struct MetaMetrics { pub snapshot_backfill_barrier_latency: LabelGuardedHistogramVec<2>, // (table_id, barrier_type) /// The latency of commit epoch of `table_id` pub snapshot_backfill_wait_commit_latency: LabelGuardedHistogramVec<1>, // (table_id, ) - /// The latency that the upstream waits on the snapshot backfill progress after the upstream - /// has collected the barrier. - pub snapshot_backfill_upstream_wait_progress_latency: LabelGuardedHistogramVec<1>, /* (table_id, ) */ /// The lags between the upstream epoch and the downstream epoch. pub snapshot_backfill_lag: LabelGuardedIntGaugeVec<1>, // (table_id, ) /// The number of inflight barriers of `table_id` @@ -285,13 +282,7 @@ impl MetaMetrics { ); let snapshot_backfill_wait_commit_latency = register_guarded_histogram_vec_with_registry!(opts, &["table_id"], registry).unwrap(); - let opts = histogram_opts!( - "meta_snapshot_backfill_upstream_wait_progress_latency", - "snapshot backfill upstream_wait_progress_latency", - exponential_buckets(0.1, 1.5, 20).unwrap() // max 221s - ); - let snapshot_backfill_upstream_wait_progress_latency = - register_guarded_histogram_vec_with_registry!(opts, &["table_id"], registry).unwrap(); + let snapshot_backfill_lag = register_guarded_int_gauge_vec_with_registry!( "meta_snapshot_backfill_upstream_lag", "snapshot backfill upstream_lag", @@ -769,7 +760,6 @@ impl MetaMetrics { last_committed_barrier_time, snapshot_backfill_barrier_latency, snapshot_backfill_wait_commit_latency, - snapshot_backfill_upstream_wait_progress_latency, snapshot_backfill_lag, snapshot_backfill_inflight_barrier_num, recovery_failure_cnt, From 1b359b0cb7320988ed5d61d40fc560f9bb1897db Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 9 Oct 2024 01:59:14 +0800 Subject: [PATCH 04/19] refactor --- proto/hummock.proto | 1 - src/meta/src/barrier/mod.rs | 30 ++++--- src/meta/src/hummock/manager/commit_epoch.rs | 82 +++++++++++-------- src/meta/src/hummock/manager/context.rs | 10 +-- src/meta/src/hummock/manager/time_travel.rs | 5 +- src/meta/src/hummock/manager/transaction.rs | 24 +++--- .../src/hummock/mock_hummock_meta_client.rs | 13 ++- src/storage/hummock_sdk/src/compact_task.rs | 3 - .../hummock_test/src/hummock_storage_tests.rs | 36 +++----- .../hummock_test/src/state_store_tests.rs | 9 +- src/storage/src/hummock/validator.rs | 14 +--- 11 files changed, 110 insertions(+), 117 deletions(-) diff --git a/proto/hummock.proto b/proto/hummock.proto index 0fab515e546e6..28bc5294725c5 100644 --- a/proto/hummock.proto +++ b/proto/hummock.proto @@ -525,7 +525,6 @@ message ReportCompactionTaskResponse { message ValidationTask { repeated SstableInfo sst_infos = 1; map sst_id_to_worker_id = 2; - uint64 epoch = 3; } // Delete SSTs in object store diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 463de3f6febe4..15d6600f68033 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -1166,20 +1166,23 @@ impl GlobalBarrierManagerContext { let (sst_to_context, sstables, new_table_watermarks, old_value_sst) = collect_resp_info(resps); assert!(old_value_sst.is_empty()); - let new_table_fragment_info = if is_first_time { - NewTableFragmentInfo::NewCompactionGroup { + let new_table_fragment_infos = if is_first_time { + vec![NewTableFragmentInfo::NewCompactionGroup { table_ids: tables_to_commit.clone(), - } + }] } else { - NewTableFragmentInfo::None + vec![] }; + let tables_to_commit = tables_to_commit + .into_iter() + .map(|table_id| (table_id, epoch)) + .collect(); let info = CommitEpochInfo { sstables, new_table_watermarks, sst_to_context, - new_table_fragment_info, + new_table_fragment_infos, change_log_delta: Default::default(), - committed_epoch: epoch, tables_to_commit, }; self.hummock_manager.commit_epoch(info).await?; @@ -1712,21 +1715,21 @@ fn collect_commit_epoch_info( let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) = collect_resp_info(resps); - let new_table_fragment_info = if let Command::CreateStreamingJob { info, job_type } = + let new_table_fragment_infos = if let Command::CreateStreamingJob { info, job_type } = &command_ctx.command && !matches!(job_type, CreateStreamingJobType::SnapshotBackfill(_)) { let table_fragments = &info.table_fragments; - NewTableFragmentInfo::Normal { + vec![NewTableFragmentInfo::Normal { mv_table_id: table_fragments.mv_table_id().map(TableId::new), internal_table_ids: table_fragments .internal_table_ids() .into_iter() .map(TableId::new) .collect(), - } + }] } else { - NewTableFragmentInfo::None + vec![] }; let mut mv_log_store_truncate_epoch = HashMap::new(); @@ -1767,14 +1770,17 @@ fn collect_commit_epoch_info( ); let epoch = command_ctx.prev_epoch.value().0; + let tables_to_commit = tables_to_commit + .into_iter() + .map(|table_id| (table_id, epoch)) + .collect(); CommitEpochInfo { sstables: synced_ssts, new_table_watermarks, sst_to_context, - new_table_fragment_info, + new_table_fragment_infos, change_log_delta: table_new_change_log, - committed_epoch: epoch, tables_to_commit, } } diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index e444f2c263bb3..3da93d61ab70b 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -13,6 +13,7 @@ // limitations under the License. use std::collections::{BTreeMap, HashMap, HashSet}; +use std::sync::Arc; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::change_log::ChangeLogDelta; @@ -28,6 +29,7 @@ use risingwave_hummock_sdk::{ CompactionGroupId, HummockContextId, HummockSstableObjectId, LocalSstableInfo, }; use risingwave_pb::hummock::compact_task::{self}; +use risingwave_pb::hummock::CompactionConfig; use sea_orm::TransactionTrait; use crate::hummock::error::{Error, Result}; @@ -47,7 +49,6 @@ use crate::hummock::{ }; pub enum NewTableFragmentInfo { - None, Normal { mv_table_id: Option, internal_table_ids: Vec, @@ -61,10 +62,10 @@ pub struct CommitEpochInfo { pub sstables: Vec, pub new_table_watermarks: HashMap, pub sst_to_context: HashMap, - pub new_table_fragment_info: NewTableFragmentInfo, + pub new_table_fragment_infos: Vec, pub change_log_delta: HashMap, - pub committed_epoch: u64, - pub tables_to_commit: HashSet, + /// `table_id` -> `committed_epoch` + pub tables_to_commit: HashMap, } impl HummockManager { @@ -75,9 +76,8 @@ impl HummockManager { mut sstables, new_table_watermarks, sst_to_context, - new_table_fragment_info, + new_table_fragment_infos, change_log_delta, - committed_epoch, tables_to_commit, } = commit_info; let mut versioning_guard = self.versioning.write().await; @@ -91,7 +91,6 @@ impl HummockManager { let versioning: &mut Versioning = &mut versioning_guard; self.commit_epoch_sanity_check( - committed_epoch, &tables_to_commit, &sstables, &sst_to_context, @@ -124,15 +123,18 @@ impl HummockManager { let state_table_info = &version.latest_version().state_table_info; let mut table_compaction_group_mapping = state_table_info.build_table_compaction_group_id(); + let mut new_table_ids = HashMap::new(); + let mut new_compaction_groups = HashMap::new(); + let mut compaction_group_manager_txn = None; + let mut compaction_group_config: Option> = None; // Add new table - let (new_table_ids, new_compaction_group, compaction_group_manager_txn) = + for new_table_fragment_info in new_table_fragment_infos { match new_table_fragment_info { NewTableFragmentInfo::Normal { mv_table_id, internal_table_ids, } => { - let mut new_table_ids = HashMap::new(); on_handle_add_new_table( state_table_info, &internal_table_ids, @@ -148,24 +150,40 @@ impl HummockManager { &mut table_compaction_group_mapping, &mut new_table_ids, )?; - (new_table_ids, None, None) } NewTableFragmentInfo::NewCompactionGroup { table_ids } => { - let compaction_group_manager_guard = - self.compaction_group_manager.write().await; - let compaction_group_config = - compaction_group_manager_guard.default_compaction_config(); - let mut compaction_group_manager = - CompactionGroupManager::start_owned_compaction_groups_txn( - compaction_group_manager_guard, - ); - let mut new_table_ids = HashMap::new(); + let (compaction_group_manager, compaction_group_config) = + if let Some(compaction_group_manager) = &mut compaction_group_manager_txn { + ( + compaction_group_manager, + (*compaction_group_config + .as_ref() + .expect("must be set with compaction_group_manager_txn")) + .clone(), + ) + } else { + let compaction_group_manager_guard = + self.compaction_group_manager.write().await; + let new_compaction_group_config = + compaction_group_manager_guard.default_compaction_config(); + compaction_group_config = Some(new_compaction_group_config.clone()); + ( + compaction_group_manager_txn.insert( + CompactionGroupManager::start_owned_compaction_groups_txn( + compaction_group_manager_guard, + ), + ), + new_compaction_group_config, + ) + }; let new_compaction_group_id = next_compaction_group_id(&self.env).await?; + new_compaction_groups + .insert(new_compaction_group_id, compaction_group_config.clone()); compaction_group_manager.insert( new_compaction_group_id, CompactionGroup { group_id: new_compaction_group_id, - compaction_config: compaction_group_config.clone(), + compaction_config: compaction_group_config, }, ); @@ -176,14 +194,9 @@ impl HummockManager { &mut table_compaction_group_mapping, &mut new_table_ids, )?; - ( - new_table_ids, - Some((new_compaction_group_id, (*compaction_group_config).clone())), - Some(compaction_group_manager), - ) } - NewTableFragmentInfo::None => (HashMap::new(), None, None), - }; + } + } let commit_sstables = self .correct_commit_ssts(sstables, &table_compaction_group_mapping) @@ -192,9 +205,8 @@ impl HummockManager { let modified_compaction_groups: Vec<_> = commit_sstables.keys().cloned().collect(); let time_travel_delta = version.pre_commit_epoch( - committed_epoch, &tables_to_commit, - new_compaction_group, + new_compaction_groups, commit_sstables, &new_table_ids, new_table_watermarks, @@ -253,9 +265,14 @@ impl HummockManager { .values() .map(|g| (g.group_id, g.parent_group_id)) .collect(); - let time_travel_tables_to_commit = table_compaction_group_mapping - .iter() - .filter(|(table_id, _)| tables_to_commit.contains(table_id)); + let time_travel_tables_to_commit = + table_compaction_group_mapping + .iter() + .filter_map(|(table_id, cg_id)| { + tables_to_commit + .get(table_id) + .map(|committed_epoch| (table_id, cg_id, *committed_epoch)) + }); let mut txn = sql_store.conn.begin().await?; let version_snapshot_sst_ids = self .write_time_travel_metadata( @@ -265,7 +282,6 @@ impl HummockManager { &group_parents, &versioning.last_time_travel_snapshot_sst_ids, time_travel_tables_to_commit, - committed_epoch, ) .await?; commit_multi_var_with_provided_txn!( diff --git a/src/meta/src/hummock/manager/context.rs b/src/meta/src/hummock/manager/context.rs index b42dd9e54d0ed..257613c9926be 100644 --- a/src/meta/src/hummock/manager/context.rs +++ b/src/meta/src/hummock/manager/context.rs @@ -19,7 +19,7 @@ use itertools::Itertools; use risingwave_common::catalog::TableId; use risingwave_hummock_sdk::version::HummockVersion; use risingwave_hummock_sdk::{ - HummockContextId, HummockEpoch, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, + HummockContextId, HummockSstableObjectId, HummockVersionId, LocalSstableInfo, INVALID_VERSION_ID, }; use risingwave_pb::hummock::{HummockPinnedVersion, ValidationTask}; @@ -188,8 +188,7 @@ impl HummockManager { pub async fn commit_epoch_sanity_check( &self, - committed_epoch: HummockEpoch, - tables_to_commit: &HashSet, + tables_to_commit: &HashMap, sstables: &[LocalSstableInfo], sst_to_context: &HashMap, current_version: &HummockVersion, @@ -215,9 +214,9 @@ impl HummockManager { } // sanity check on monotonically increasing table committed epoch - for table_id in tables_to_commit { + for (table_id, committed_epoch) in tables_to_commit { if let Some(info) = current_version.state_table_info.info().get(table_id) { - if committed_epoch <= info.committed_epoch { + if *committed_epoch <= info.committed_epoch { return Err(anyhow::anyhow!( "table {} Epoch {} <= committed_epoch {}", table_id, @@ -264,7 +263,6 @@ impl HummockManager { .send_event(ResponseEvent::ValidationTask(ValidationTask { sst_infos: sst_infos.into_iter().map(|sst| sst.into()).collect_vec(), sst_id_to_worker_id: sst_to_context.clone(), - epoch: committed_epoch, })) .is_err() { diff --git a/src/meta/src/hummock/manager/time_travel.rs b/src/meta/src/hummock/manager/time_travel.rs index 044d8503fc94e..42037e65f7423 100644 --- a/src/meta/src/hummock/manager/time_travel.rs +++ b/src/meta/src/hummock/manager/time_travel.rs @@ -377,8 +377,7 @@ impl HummockManager { delta: HummockVersionDelta, group_parents: &HashMap, skip_sst_ids: &HashSet, - tables_to_commit: impl Iterator, - committed_epoch: u64, + tables_to_commit: impl Iterator, ) -> Result>> { let select_groups = group_parents .iter() @@ -415,7 +414,7 @@ impl HummockManager { Ok(count) } - for (table_id, cg_id) in tables_to_commit { + for (table_id, cg_id, committed_epoch) in tables_to_commit { if !select_groups.contains(cg_id) { continue; } diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index b8e9335a161b6..87ec960bca1d1 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeMap, HashMap, HashSet}; +use std::collections::{BTreeMap, HashMap}; use std::ops::{Deref, DerefMut}; +use std::sync::Arc; use risingwave_common::catalog::TableId; +use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::change_log::ChangeLogDelta; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -23,9 +25,7 @@ use risingwave_hummock_sdk::table_watermark::TableWatermarks; use risingwave_hummock_sdk::version::{ GroupDelta, HummockVersion, HummockVersionDelta, IntraLevelDelta, }; -use risingwave_hummock_sdk::{ - CompactionGroupId, FrontendHummockVersionDelta, HummockEpoch, HummockVersionId, -}; +use risingwave_hummock_sdk::{CompactionGroupId, FrontendHummockVersionDelta, HummockVersionId}; use risingwave_pb::hummock::{ CompactionConfig, CompatibilityVersion, GroupConstruct, HummockVersionDeltas, HummockVersionStats, StateTableInfoDelta, @@ -113,9 +113,8 @@ impl<'a> HummockVersionTransaction<'a> { /// Returns a duplicate delta, used by time travel. pub(super) fn pre_commit_epoch( &mut self, - committed_epoch: HummockEpoch, - tables_to_commit: &HashSet, - new_compaction_group: Option<(CompactionGroupId, CompactionConfig)>, + tables_to_commit: &HashMap, + new_compaction_groups: HashMap>, commit_sstables: BTreeMap>, new_table_ids: &HashMap, new_table_watermarks: HashMap, @@ -125,7 +124,7 @@ impl<'a> HummockVersionTransaction<'a> { new_version_delta.new_table_watermarks = new_table_watermarks; new_version_delta.change_log_delta = change_log_delta; - if let Some((compaction_group_id, compaction_group_config)) = new_compaction_group { + for (compaction_group_id, compaction_group_config) in new_compaction_groups { { let group_deltas = &mut new_version_delta .group_deltas @@ -135,7 +134,7 @@ impl<'a> HummockVersionTransaction<'a> { #[expect(deprecated)] group_deltas.push(GroupDelta::GroupConstruct(GroupConstruct { - group_config: Some(compaction_group_config.clone()), + group_config: Some((*compaction_group_config).clone()), group_id: compaction_group_id, parent_group_id: StaticCompactionGroupId::NewCompactionGroup as CompactionGroupId, @@ -160,7 +159,7 @@ impl<'a> HummockVersionTransaction<'a> { .last() .map(|level| level.sub_level_id + 1) }) - .unwrap_or(committed_epoch); + .unwrap_or(Epoch::now().0); let group_deltas = &mut new_version_delta .group_deltas .entry(compaction_group_id) @@ -185,6 +184,7 @@ impl<'a> HummockVersionTransaction<'a> { "newly added table exists previously: {:?}", table_id ); + let committed_epoch = *tables_to_commit.get(table_id).expect("newly added table must exist in tables_to_commit"); delta.state_table_info_delta.insert( *table_id, StateTableInfoDelta { @@ -194,7 +194,7 @@ impl<'a> HummockVersionTransaction<'a> { ); } - for table_id in tables_to_commit { + for (table_id, committed_epoch) in tables_to_commit { if new_table_ids.contains_key(table_id) { continue; } @@ -206,7 +206,7 @@ impl<'a> HummockVersionTransaction<'a> { .insert( *table_id, StateTableInfoDelta { - committed_epoch, + committed_epoch: *committed_epoch, compaction_group_id: info.compaction_group_id, } ) diff --git a/src/meta/src/hummock/mock_hummock_meta_client.rs b/src/meta/src/hummock/mock_hummock_meta_client.rs index ba54bc64969ad..c045a46dd1d95 100644 --- a/src/meta/src/hummock/mock_hummock_meta_client.rs +++ b/src/meta/src/hummock/mock_hummock_meta_client.rs @@ -171,20 +171,20 @@ impl HummockMetaClient for MockHummockMetaClient { .chain(table_ids.iter().cloned()) .collect::>(); - let new_table_fragment_info = if commit_table_ids + let new_table_fragment_infos = if commit_table_ids .iter() .all(|table_id| table_ids.contains(table_id)) { - NewTableFragmentInfo::None + vec![] } else { - NewTableFragmentInfo::Normal { + vec![NewTableFragmentInfo::Normal { mv_table_id: None, internal_table_ids: commit_table_ids .iter() .cloned() .map(TableId::from) .collect_vec(), - } + }] }; let sst_to_context = sync_result @@ -215,13 +215,12 @@ impl HummockMetaClient for MockHummockMetaClient { sstables: sync_result.uncommitted_ssts, new_table_watermarks: new_table_watermark, sst_to_context, - new_table_fragment_info, + new_table_fragment_infos, change_log_delta: table_change_log, - committed_epoch: epoch, tables_to_commit: commit_table_ids .iter() .cloned() - .map(TableId::from) + .map(|table_id| (TableId::new(table_id), epoch)) .collect(), }) .await diff --git a/src/storage/hummock_sdk/src/compact_task.rs b/src/storage/hummock_sdk/src/compact_task.rs index ff76e3a70dde8..126aced6e9afe 100644 --- a/src/storage/hummock_sdk/src/compact_task.rs +++ b/src/storage/hummock_sdk/src/compact_task.rs @@ -325,7 +325,6 @@ impl From<&CompactTask> for PbCompactTask { pub struct ValidationTask { pub sst_infos: Vec, pub sst_id_to_worker_id: HashMap, - pub epoch: u64, } impl From for ValidationTask { @@ -337,7 +336,6 @@ impl From for ValidationTask { .map(SstableInfo::from) .collect_vec(), sst_id_to_worker_id: pb_validation_task.sst_id_to_worker_id.clone(), - epoch: pb_validation_task.epoch, } } } @@ -351,7 +349,6 @@ impl From for PbValidationTask { .map(|sst| sst.into()) .collect_vec(), sst_id_to_worker_id: validation_task.sst_id_to_worker_id.clone(), - epoch: validation_task.epoch, } } } diff --git a/src/storage/hummock_test/src/hummock_storage_tests.rs b/src/storage/hummock_test/src/hummock_storage_tests.rs index 7c70721f04d82..4e6ab26a539c6 100644 --- a/src/storage/hummock_test/src/hummock_storage_tests.rs +++ b/src/storage/hummock_test/src/hummock_storage_tests.rs @@ -2585,9 +2585,12 @@ async fn test_commit_multi_epoch() { let initial_epoch = INVALID_EPOCH; let commit_epoch = - |epoch, sst: SstableInfo, new_table_fragment_info, tables_to_commit: &[TableId]| { + |epoch, sst: SstableInfo, new_table_fragment_infos, tables_to_commit: &[TableId]| { let manager = &test_env.manager; - let tables_to_commit = tables_to_commit.iter().cloned().collect(); + let tables_to_commit = tables_to_commit + .iter() + .map(|table_id| (*table_id, epoch)) + .collect(); async move { manager .commit_epoch(CommitEpochInfo { @@ -2610,9 +2613,8 @@ async fn test_commit_multi_epoch() { sst_info: sst, created_at: u64::MAX, }], - new_table_fragment_info, + new_table_fragment_infos, change_log_delta: Default::default(), - committed_epoch: epoch, tables_to_commit, }) .await @@ -2633,10 +2635,10 @@ async fn test_commit_multi_epoch() { commit_epoch( epoch1, sst1_epoch1.clone(), - NewTableFragmentInfo::Normal { + vec![NewTableFragmentInfo::Normal { mv_table_id: None, internal_table_ids: vec![existing_table_id], - }, + }], &[existing_table_id], ) .await; @@ -2678,13 +2680,7 @@ async fn test_commit_multi_epoch() { let epoch2 = epoch1.next_epoch(); - commit_epoch( - epoch2, - sst1_epoch2.clone(), - NewTableFragmentInfo::None, - &[existing_table_id], - ) - .await; + commit_epoch(epoch2, sst1_epoch2.clone(), vec![], &[existing_table_id]).await; { let version = test_env.manager.get_current_version().await; @@ -2727,9 +2723,9 @@ async fn test_commit_multi_epoch() { commit_epoch( epoch1, sst2_epoch1.clone(), - NewTableFragmentInfo::NewCompactionGroup { + vec![NewTableFragmentInfo::NewCompactionGroup { table_ids: HashSet::from_iter([new_table_id]), - }, + }], &[new_table_id], ) .await; @@ -2764,13 +2760,7 @@ async fn test_commit_multi_epoch() { ..Default::default() }; - commit_epoch( - epoch2, - sst2_epoch2.clone(), - NewTableFragmentInfo::None, - &[new_table_id], - ) - .await; + commit_epoch(epoch2, sst2_epoch2.clone(), vec![], &[new_table_id]).await; { let version = test_env.manager.get_current_version().await; @@ -2804,7 +2794,7 @@ async fn test_commit_multi_epoch() { commit_epoch( epoch3, sst_epoch3.clone(), - NewTableFragmentInfo::None, + vec![], &[existing_table_id, new_table_id], ) .await; diff --git a/src/storage/hummock_test/src/state_store_tests.rs b/src/storage/hummock_test/src/state_store_tests.rs index 7b2c5b5e60649..0c511dbfb2068 100644 --- a/src/storage/hummock_test/src/state_store_tests.rs +++ b/src/storage/hummock_test/src/state_store_tests.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::HashSet; +use std::collections::{HashMap, HashSet}; use std::ops::Bound; use std::ops::Bound::{Excluded, Included, Unbounded}; use std::sync::Arc; @@ -31,7 +31,7 @@ use risingwave_hummock_sdk::{ HummockReadEpoch, HummockSstableObjectId, LocalSstableInfo, SyncResult, }; use risingwave_meta::hummock::test_utils::setup_compute_env; -use risingwave_meta::hummock::{CommitEpochInfo, NewTableFragmentInfo}; +use risingwave_meta::hummock::CommitEpochInfo; use risingwave_rpc_client::HummockMetaClient; use risingwave_storage::hummock::iterator::change_log::test_utils::{ apply_test_log_data, gen_test_data, @@ -1407,10 +1407,9 @@ async fn test_replicated_local_hummock_storage() { sstables: vec![], new_table_watermarks: Default::default(), sst_to_context: Default::default(), - new_table_fragment_info: NewTableFragmentInfo::None, + new_table_fragment_infos: vec![], change_log_delta: Default::default(), - committed_epoch: epoch0, - tables_to_commit: HashSet::from_iter([TEST_TABLE_ID]), + tables_to_commit: HashMap::from_iter([(TEST_TABLE_ID, epoch0)]), }) .await .unwrap(); diff --git a/src/storage/src/hummock/validator.rs b/src/storage/src/hummock/validator.rs index cc95b7089b664..2c0efbb3ca934 100644 --- a/src/storage/src/hummock/validator.rs +++ b/src/storage/src/hummock/validator.rs @@ -38,12 +38,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) .sst_id_to_worker_id .get(&sst.object_id) .expect("valid worker_id"); - tracing::debug!( - "Validating SST {} from worker {}, epoch {}", - sst.object_id, - worker_id, - task.epoch - ); + tracing::debug!("Validating SST {} from worker {}", sst.object_id, worker_id,); let holder = match sstable_store.sstable(&sst, unused.borrow_mut()).await { Ok(holder) => holder, Err(_err) => { @@ -100,12 +95,7 @@ pub async fn validate_ssts(task: ValidationTask, sstable_store: SstableStoreRef) break; } } - tracing::debug!( - "Validated {} keys for SST {}, epoch {}", - key_counts, - sst.object_id, - task.epoch - ); + tracing::debug!("Validated {} keys for SST {}", key_counts, sst.object_id,); iter.collect_local_statistic(&mut unused); unused.ignore(); } From 07a33c3bec959b9cd9beea752a2050a8ef45b28f Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 9 Oct 2024 10:58:20 +0800 Subject: [PATCH 05/19] refine --- src/meta/src/barrier/mod.rs | 104 ++++++++++++++---------------------- 1 file changed, 39 insertions(+), 65 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 15d6600f68033..2bbc4083299d4 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -484,11 +484,12 @@ impl CheckpointControl { let finished_jobs = self .create_mview_tracker .apply_collected_command(&node, &self.hummock_version_stats); - if let Err(e) = self - .context - .clone() - .complete_barrier(node, finished_jobs, HashMap::new()) - .await + if node.command_ctx.kind.is_checkpoint() + && let Err(e) = self + .context + .clone() + .complete_barrier(node, finished_jobs, HashMap::new()) + .await { error!( prev_epoch, @@ -568,7 +569,7 @@ enum CompletingCommand { // The join handle of a spawned task that completes the barrier. // The return value indicate whether there is some create streaming job command // that has finished but not checkpointed. If there is any, we will force checkpoint on the next barrier - join_handle: JoinHandle>>, + join_handle: JoinHandle>, }, CreatingStreamingJob { table_id: TableId, @@ -1194,7 +1195,7 @@ impl GlobalBarrierManagerContext { node: EpochNode, mut finished_jobs: Vec, backfill_pinned_log_epoch: HashMap)>, - ) -> MetaResult> { + ) -> MetaResult { tracing::trace!( prev_epoch = node.command_ctx.prev_epoch.value().0, kind = ?node.command_ctx.kind, @@ -1207,12 +1208,10 @@ impl GlobalBarrierManagerContext { state, .. } = node; + let epochs = must_match!(&command_ctx.kind, BarrierKind::Checkpoint(epochs) => epochs); assert!(state.node_to_collect.is_empty()); assert!(state.creating_jobs_to_wait.is_empty()); let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); - if !state.finished_table_ids.is_empty() { - assert!(command_ctx.kind.is_checkpoint()); - } finished_jobs.extend(state.finished_table_ids.into_values().map(|info| { TrackingJob::New(TrackingCommand { info, @@ -1220,14 +1219,18 @@ impl GlobalBarrierManagerContext { }) })); - let result = self - .update_snapshot( - &command_ctx, - state.table_ids_to_commit, + let result: MetaResult = try { + let commit_info = collect_commit_epoch_info( state.resps, + &command_ctx, + epochs, backfill_pinned_log_epoch, - ) - .await; + state.table_ids_to_commit, + ); + self.hummock_manager.commit_epoch(commit_info).await?; + command_ctx.post_collect().await?; + self.hummock_manager.get_version_stats().await + }; let version_stats = match result { Ok(version_stats) => version_stats, @@ -1255,45 +1258,6 @@ impl GlobalBarrierManagerContext { Ok(version_stats) } - async fn update_snapshot( - &self, - command_ctx: &CommandContext, - tables_to_commit: HashSet, - resps: Vec, - backfill_pinned_log_epoch: HashMap)>, - ) -> MetaResult> { - { - { - match &command_ctx.kind { - BarrierKind::Initial => {} - BarrierKind::Checkpoint(epochs) => { - let commit_info = collect_commit_epoch_info( - resps, - command_ctx, - epochs, - backfill_pinned_log_epoch, - tables_to_commit, - ); - self.hummock_manager.commit_epoch(commit_info).await?; - } - BarrierKind::Barrier => { - // if we collect a barrier(checkpoint = false), - // we need to ensure that command is Plain and the notifier's checkpoint is - // false - assert!(!command_ctx.command.need_checkpoint()); - } - } - - command_ctx.post_collect().await?; - Ok(if command_ctx.kind.is_checkpoint() { - Some(self.hummock_manager.get_version_stats().await) - } else { - None - }) - } - } - } - pub fn hummock_manager(&self) -> &HummockManagerRef { &self.hummock_manager } @@ -1392,18 +1356,26 @@ impl CheckpointControl { pub(super) async fn next_completed_barrier( &mut self, ) -> MetaResult> { - if matches!(&self.completing_command, CompletingCommand::None) { - // If there is no completing barrier, try to start completing the earliest barrier if - // it has been collected. - if let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() + // If there is no completing barrier, try to start completing the earliest barrier if + // it has been collected. + while let CompletingCommand::None = &self.completing_command + && let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() + && !state.is_inflight() + { { let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); assert!(node.state.creating_jobs_to_wait.is_empty()); - let table_ids_to_finish = node.state.finished_table_ids.keys().cloned().collect(); let finished_jobs = self .create_mview_tracker .apply_collected_command(&node, &self.hummock_version_stats); + if !node.command_ctx.kind.is_checkpoint() { + assert!(finished_jobs.is_empty()); + node.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + continue; + } + let table_ids_to_finish = node.state.finished_table_ids.keys().cloned().collect(); let command_ctx = node.command_ctx.clone(); let join_handle = tokio::spawn(self.context.clone().complete_barrier( node, @@ -1424,7 +1396,11 @@ impl CheckpointControl { join_handle, table_ids_to_finish, }; - } else { + } + } + + if matches!(&self.completing_command, CompletingCommand::None) { + { for (table_id, job) in &mut self.creating_streaming_job_controls { let (upstream_epochs_to_notify, commit_info) = job.start_completing(); for upstream_epoch in upstream_epochs_to_notify { @@ -1482,9 +1458,7 @@ impl CheckpointControl { let completed_command = replace(&mut self.completing_command, next_completing_command_status); join_result.map(move | version_stats| { - if let Some(new_version_stats) = version_stats { - self.hummock_version_stats = new_version_stats; - } + self.hummock_version_stats = version_stats; must_match!( completed_command, CompletingCommand::GlobalStreamingGraph { command_ctx, table_ids_to_finish, require_next_checkpoint, .. } => { From f50cb2bc54bef9852f41dbbc11a1e6fe47cd1bd9 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 9 Oct 2024 13:08:04 +0800 Subject: [PATCH 06/19] fix timeout --- src/meta/src/hummock/manager/transaction.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 87ec960bca1d1..cef638908e59d 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -159,7 +159,7 @@ impl<'a> HummockVersionTransaction<'a> { .last() .map(|level| level.sub_level_id + 1) }) - .unwrap_or(Epoch::now().0); + .unwrap_or_else(|| Epoch::now().0); let group_deltas = &mut new_version_delta .group_deltas .entry(compaction_group_id) From c2789e89206f0733b762501a4d105a191a2fbaba Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 9 Oct 2024 13:54:54 +0800 Subject: [PATCH 07/19] avoid calling now --- src/meta/src/barrier/mod.rs | 9 +++++---- src/meta/src/hummock/manager/transaction.rs | 9 +++++++-- 2 files changed, 12 insertions(+), 6 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 2bbc4083299d4..cfabf840e14f2 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -1244,10 +1244,11 @@ impl GlobalBarrierManagerContext { notifiers.into_iter().for_each(|notifier| { notifier.notify_collected(); }); - try_join_all(finished_jobs.into_iter().map(|finished_job| { - let metadata_manager = &self.metadata_manager; - async move { finished_job.finish(metadata_manager).await } - })) + try_join_all( + finished_jobs + .into_iter() + .map(|finished_job| finished_job.finish(&self.metadata_manager)), + ) .await?; let duration_sec = enqueue_time.stop_and_record(); self.report_complete_event(duration_sec, &command_ctx); diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index cef638908e59d..56b60a6535722 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -17,7 +17,6 @@ use std::ops::{Deref, DerefMut}; use std::sync::Arc; use risingwave_common::catalog::TableId; -use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::change_log::ChangeLogDelta; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -159,7 +158,13 @@ impl<'a> HummockVersionTransaction<'a> { .last() .map(|level| level.sub_level_id + 1) }) - .unwrap_or_else(|| Epoch::now().0); + .unwrap_or_else(|| { + tables_to_commit + .values() + .cloned() + .max() + .expect("non empty tables_to_commit") + }); let group_deltas = &mut new_version_delta .group_deltas .entry(compaction_group_id) From f86c5c63d223abfacc76ec966398d6b1055ad129 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 9 Oct 2024 18:17:25 +0800 Subject: [PATCH 08/19] refactor --- src/meta/src/barrier/mod.rs | 415 +++++++++++++++---------------- src/meta/src/barrier/schedule.rs | 4 + 2 files changed, 199 insertions(+), 220 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index cfabf840e14f2..7b75cd0c964f3 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -12,10 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; -use std::future::pending; +use std::future::{pending, Future}; use std::mem::{replace, take}; use std::sync::Arc; use std::time::Duration; @@ -55,6 +54,7 @@ use crate::barrier::creating_job::CreatingStreamingJobControl; use crate::barrier::info::InflightGraphInfo; use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob}; use crate::barrier::rpc::{merge_node_rpc_errors, ControlStreamManager}; +use crate::barrier::schedule::ScheduledBarriers; use crate::barrier::state::BarrierManagerState; use crate::error::MetaErrorInner; use crate::hummock::{CommitEpochInfo, HummockManagerRef, NewTableFragmentInfo}; @@ -292,7 +292,6 @@ impl CheckpointControl { notifiers: Vec, node_to_collect: HashSet, jobs_to_wait: HashSet, - table_ids_to_commit: HashSet, ) { let timer = self.context.metrics.barrier_latency.start_timer(); @@ -334,8 +333,7 @@ impl CheckpointControl { node_to_collect, resps: vec![], creating_jobs_to_wait, - finished_table_ids: HashMap::new(), - table_ids_to_commit, + finished_jobs: HashMap::new(), }, command_ctx, notifiers, @@ -404,7 +402,7 @@ impl CheckpointControl { CompletingCommand::None | CompletingCommand::Err(_) | CompletingCommand::CreatingStreamingJob { .. } => None, - CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => Some(command_ctx), + CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => command_ctx.as_ref(), }) .map(|command_ctx| command_ctx.command.should_pause_inject_barrier()) .unwrap_or(false); @@ -418,7 +416,7 @@ impl CheckpointControl { | CompletingCommand::Err(_) | CompletingCommand::CreatingStreamingJob { .. } => None, CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => - Some(command_ctx), + command_ctx.as_ref(), } .into_iter() ) @@ -434,16 +432,8 @@ impl CheckpointControl { // join spawned completing command to finish no matter it succeeds or not. let is_err = match replace(&mut self.completing_command, CompletingCommand::None) { CompletingCommand::None => false, - CompletingCommand::GlobalStreamingGraph { - command_ctx, - join_handle, - .. - } => { - info!( - prev_epoch = ?command_ctx.prev_epoch, - curr_epoch = ?command_ctx.curr_epoch, - "waiting for completing command to finish in recovery" - ); + CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { + info!("waiting for completing command to finish in recovery"); match join_handle.await { Err(e) => { warn!(err = ?e.as_report(), "failed to join completing task"); @@ -473,36 +463,15 @@ impl CheckpointControl { }; if !is_err { // continue to finish the pending collected barrier. - while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() - { - let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); - let (prev_epoch, curr_epoch) = ( - node.command_ctx.prev_epoch.value().0, - node.command_ctx.curr_epoch.value().0, - ); - let finished_jobs = self - .create_mview_tracker - .apply_collected_command(&node, &self.hummock_version_stats); - if node.command_ctx.kind.is_checkpoint() - && let Err(e) = self - .context - .clone() - .complete_barrier(node, finished_jobs, HashMap::new()) - .await - { + while let Some((task, _)) = self.next_collected_checkpoint_barrier(None) { + if let Err(e) = self.context.clone().complete_barrier(task).await { error!( - prev_epoch, - curr_epoch, err = ?e.as_report(), "failed to complete barrier during recovery" ); break; } else { - info!( - prev_epoch, - curr_epoch, "succeed to complete barrier during recovery" - ) + info!("succeed to complete barrier during recovery") } } } @@ -548,9 +517,7 @@ struct BarrierEpochState { creating_jobs_to_wait: HashMap>, - finished_table_ids: HashMap, - - table_ids_to_commit: HashSet, + finished_jobs: HashMap, } impl BarrierEpochState { @@ -562,9 +529,8 @@ impl BarrierEpochState { enum CompletingCommand { None, GlobalStreamingGraph { - command_ctx: Arc, + command_ctx: Option>, table_ids_to_finish: HashSet, - require_next_checkpoint: bool, // The join handle of a spawned task that completes the barrier. // The return value indicate whether there is some create streaming job command @@ -871,15 +837,9 @@ impl GlobalBarrierManager { } } } - complete_result = self.checkpoint_control.next_completed_barrier() => { + complete_result = self.checkpoint_control.next_completed_barrier(&mut self.scheduled_barriers) => { match complete_result { Ok(Some(output)) => { - // If there are remaining commands (that requires checkpoint to finish), we force - // the next barrier to be a checkpoint. - if output.require_next_checkpoint { - assert_matches!(output.command_ctx.kind, BarrierKind::Barrier); - self.scheduled_barriers.force_checkpoint_in_next_barrier(); - } self.control_stream_manager.remove_partial_graph( output.table_ids_to_finish.iter().map(|table_id| table_id.table_id).collect() ); @@ -1039,7 +999,7 @@ impl GlobalBarrierManager { pre_applied_subscription_info, prev_epoch.clone(), curr_epoch.clone(), - table_ids_to_commit.clone(), + table_ids_to_commit, self.state.paused_reason(), command, kind, @@ -1091,7 +1051,6 @@ impl GlobalBarrierManager { notifiers, node_to_collect, jobs_to_wait, - table_ids_to_commit, ); Ok(()) @@ -1156,6 +1115,14 @@ impl GlobalBarrierManager { } } +struct CompleteBarrierTask { + commit_info: CommitEpochInfo, + finished_jobs: Vec, + notifiers: Vec, + /// Some((`command_ctx`, `enqueue_time`)) + command_context: Option<(Arc, HistogramTimer)>, +} + impl GlobalBarrierManagerContext { async fn complete_creating_job_barrier( self, @@ -1190,73 +1157,43 @@ impl GlobalBarrierManagerContext { Ok(()) } - async fn complete_barrier( - self, - node: EpochNode, - mut finished_jobs: Vec, - backfill_pinned_log_epoch: HashMap)>, - ) -> MetaResult { - tracing::trace!( - prev_epoch = node.command_ctx.prev_epoch.value().0, - kind = ?node.command_ctx.kind, - "complete barrier" - ); - let EpochNode { - command_ctx, - notifiers, - enqueue_time, - state, - .. - } = node; - let epochs = must_match!(&command_ctx.kind, BarrierKind::Checkpoint(epochs) => epochs); - assert!(state.node_to_collect.is_empty()); - assert!(state.creating_jobs_to_wait.is_empty()); - let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); - finished_jobs.extend(state.finished_table_ids.into_values().map(|info| { - TrackingJob::New(TrackingCommand { - info, - replace_table_info: None, - }) - })); - - let result: MetaResult = try { - let commit_info = collect_commit_epoch_info( - state.resps, - &command_ctx, - epochs, - backfill_pinned_log_epoch, - state.table_ids_to_commit, - ); - self.hummock_manager.commit_epoch(commit_info).await?; - command_ctx.post_collect().await?; - self.hummock_manager.get_version_stats().await + async fn complete_barrier(self, task: CompleteBarrierTask) -> MetaResult { + let result: MetaResult<()> = try { + let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); + self.hummock_manager.commit_epoch(task.commit_info).await?; + if let Some((command_ctx, _)) = &task.command_context { + command_ctx.post_collect().await?; + } + + wait_commit_timer.observe_duration(); }; - let version_stats = match result { - Ok(version_stats) => version_stats, - Err(e) => { - for notifier in notifiers { + { + if let Err(e) = result { + for notifier in task.notifiers { notifier.notify_collection_failed(e.clone()); } return Err(e); } - }; - notifiers.into_iter().for_each(|notifier| { - notifier.notify_collected(); - }); - try_join_all( - finished_jobs - .into_iter() - .map(|finished_job| finished_job.finish(&self.metadata_manager)), - ) - .await?; - let duration_sec = enqueue_time.stop_and_record(); - self.report_complete_event(duration_sec, &command_ctx); - wait_commit_timer.observe_duration(); - self.metrics - .last_committed_barrier_time - .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); - Ok(version_stats) + task.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + try_join_all( + task.finished_jobs + .into_iter() + .map(|finished_job| finished_job.finish(&self.metadata_manager)), + ) + .await?; + if let Some((command_ctx, enqueue_time)) = task.command_context { + let duration_sec = enqueue_time.stop_and_record(); + self.report_complete_event(duration_sec, &command_ctx); + self.metrics + .last_committed_barrier_time + .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); + } + } + + Ok(self.hummock_manager.get_version_stats().await) } pub fn hummock_manager(&self) -> &HummockManagerRef { @@ -1323,8 +1260,6 @@ impl GlobalBarrierManagerContext { } struct BarrierCompleteOutput { - command_ctx: Arc, - require_next_checkpoint: bool, table_ids_to_finish: HashSet, } @@ -1354,19 +1289,18 @@ impl CheckpointControl { .collect() } - pub(super) async fn next_completed_barrier( + fn next_collected_checkpoint_barrier( &mut self, - ) -> MetaResult> { - // If there is no completing barrier, try to start completing the earliest barrier if - // it has been collected. - while let CompletingCommand::None = &self.completing_command - && let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() + mut scheduled_barriers: Option<&mut ScheduledBarriers>, + ) -> Option<(CompleteBarrierTask, HashSet)> { + while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() && !state.is_inflight() { { - let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); + let (_, mut node) = self.command_ctx_queue.pop_first().expect("non-empty"); assert!(node.state.creating_jobs_to_wait.is_empty()); - let finished_jobs = self + assert!(node.state.node_to_collect.is_empty()); + let mut finished_jobs = self .create_mview_tracker .apply_collected_command(&node, &self.hummock_version_stats); if !node.command_ctx.kind.is_checkpoint() { @@ -1374,29 +1308,71 @@ impl CheckpointControl { node.notifiers.into_iter().for_each(|notifier| { notifier.notify_collected(); }); + if let Some(scheduled_barriers) = &mut scheduled_barriers + && !scheduled_barriers.is_forced_next_checkpoint() + && self.create_mview_tracker.has_pending_finished_jobs() + && self + .command_ctx_queue + .values() + .all(|node| !node.command_ctx.kind.is_checkpoint()) + { + scheduled_barriers.force_checkpoint_in_next_barrier(); + } continue; } - let table_ids_to_finish = node.state.finished_table_ids.keys().cloned().collect(); - let command_ctx = node.command_ctx.clone(); - let join_handle = tokio::spawn(self.context.clone().complete_barrier( - node, - finished_jobs, + let commit_info = collect_commit_epoch_info( + take(&mut node.state.resps), + &node.command_ctx, self.collect_backfill_pinned_upstream_log_epoch(), + ); + let table_ids_to_finish = node + .state + .finished_jobs + .drain() + .map(|(table_id, info)| { + finished_jobs.push(TrackingJob::New(TrackingCommand { + info, + replace_table_info: None, + })); + table_id + }) + .collect(); + return Some(( + CompleteBarrierTask { + commit_info, + finished_jobs, + notifiers: node.notifiers, + command_context: Some((node.command_ctx, node.enqueue_time)), + }, + table_ids_to_finish, )); - let require_next_checkpoint = - if self.create_mview_tracker.has_pending_finished_jobs() { - self.command_ctx_queue - .values() - .all(|node| !node.command_ctx.kind.is_checkpoint()) - } else { - false + } + } + None + } + + pub(super) fn next_completed_barrier<'a>( + &'a mut self, + scheduled_barriers: &mut ScheduledBarriers, + ) -> impl Future>> + 'a { + // If there is no completing barrier, try to start completing the earliest barrier if + // it has been collected. + if let CompletingCommand::None = &self.completing_command { + if let Some((task, table_ids_to_finish)) = + self.next_collected_checkpoint_barrier(Some(scheduled_barriers)) + { + { + let command_ctx = task + .command_context + .as_ref() + .map(|(command_ctx, _)| command_ctx.clone()); + let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); + self.completing_command = CompletingCommand::GlobalStreamingGraph { + command_ctx, + join_handle, + table_ids_to_finish, }; - self.completing_command = CompletingCommand::GlobalStreamingGraph { - command_ctx, - require_next_checkpoint, - join_handle, - table_ids_to_finish, - }; + } } } @@ -1442,93 +1418,93 @@ impl CheckpointControl { } } - match &mut self.completing_command { - CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { - let join_result: MetaResult<_> = try { - join_handle - .await - .context("failed to join completing command")?? - }; - // It's important to reset the completing_command after await no matter the result is err - // or not, and otherwise the join handle will be polled again after ready. - let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) - } else { - CompletingCommand::None - }; - let completed_command = - replace(&mut self.completing_command, next_completing_command_status); - join_result.map(move | version_stats| { + async move { + match &mut self.completing_command { + CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { + let join_result: MetaResult<_> = try { + join_handle + .await + .context("failed to join completing command")?? + }; + // It's important to reset the completing_command after await no matter the result is err + // or not, and otherwise the join handle will be polled again after ready. + let next_completing_command_status = if let Err(e) = &join_result { + CompletingCommand::Err(e.clone()) + } else { + CompletingCommand::None + }; + let completed_command = + replace(&mut self.completing_command, next_completing_command_status); + join_result.map(move |version_stats| { self.hummock_version_stats = version_stats; must_match!( completed_command, - CompletingCommand::GlobalStreamingGraph { command_ctx, table_ids_to_finish, require_next_checkpoint, .. } => { + CompletingCommand::GlobalStreamingGraph { table_ids_to_finish, .. } => { Some(BarrierCompleteOutput { - command_ctx, - require_next_checkpoint, table_ids_to_finish, }) } ) }) - } - CompletingCommand::CreatingStreamingJob { - table_id, - epoch, - join_handle, - } => { - let table_id = *table_id; - let epoch = *epoch; - let join_result: MetaResult<_> = try { - join_handle - .await - .context("failed to join completing command")?? - }; - // It's important to reset the completing_command after await no matter the result is err - // or not, and otherwise the join handle will be polled again after ready. - let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) - } else { - CompletingCommand::None - }; - self.completing_command = next_completing_command_status; - if let Some((upstream_epoch, is_finished)) = self - .creating_streaming_job_controls - .get_mut(&table_id) - .expect("should exist") - .ack_completed(epoch) - { - let wait_progress_timer = self - .command_ctx_queue - .get_mut(&upstream_epoch) + } + CompletingCommand::CreatingStreamingJob { + table_id, + epoch, + join_handle, + } => { + let table_id = *table_id; + let epoch = *epoch; + let join_result: MetaResult<_> = try { + join_handle + .await + .context("failed to join completing command")?? + }; + // It's important to reset the completing_command after await no matter the result is err + // or not, and otherwise the join handle will be polled again after ready. + let next_completing_command_status = if let Err(e) = &join_result { + CompletingCommand::Err(e.clone()) + } else { + CompletingCommand::None + }; + self.completing_command = next_completing_command_status; + if let Some((upstream_epoch, is_finished)) = self + .creating_streaming_job_controls + .get_mut(&table_id) .expect("should exist") - .state - .creating_jobs_to_wait - .remove(&table_id) - .expect("should exist"); - if let Some(timer) = wait_progress_timer { - timer.observe_duration(); - } - if is_finished { - debug!(epoch, ?table_id, "finish creating job"); - let creating_streaming_job = self - .creating_streaming_job_controls - .remove(&table_id) - .expect("should exist"); - assert!(creating_streaming_job.is_finished()); - assert!(self + .ack_completed(epoch) + { + let wait_progress_timer = self .command_ctx_queue .get_mut(&upstream_epoch) .expect("should exist") .state - .finished_table_ids - .insert(table_id, creating_streaming_job.info) - .is_none()); + .creating_jobs_to_wait + .remove(&table_id) + .expect("should exist"); + if let Some(timer) = wait_progress_timer { + timer.observe_duration(); + } + if is_finished { + debug!(epoch, ?table_id, "finish creating job"); + let creating_streaming_job = self + .creating_streaming_job_controls + .remove(&table_id) + .expect("should exist"); + assert!(creating_streaming_job.is_finished()); + assert!(self + .command_ctx_queue + .get_mut(&upstream_epoch) + .expect("should exist") + .state + .finished_jobs + .insert(table_id, creating_streaming_job.info) + .is_none()); + } } + join_result.map(|_| None) } - join_result.map(|_| None) + CompletingCommand::None | CompletingCommand::Err(_) => pending().await, } - CompletingCommand::None | CompletingCommand::Err(_) => pending().await, } } } @@ -1683,9 +1659,7 @@ fn collect_resp_info( fn collect_commit_epoch_info( resps: Vec, command_ctx: &CommandContext, - epochs: &Vec, backfill_pinned_log_epoch: HashMap)>, - tables_to_commit: HashSet, ) -> CommitEpochInfo { let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) = collect_resp_info(resps); @@ -1740,14 +1714,15 @@ fn collect_commit_epoch_info( let table_new_change_log = build_table_change_log_delta( old_value_ssts.into_iter(), synced_ssts.iter().map(|sst| &sst.sst_info), - epochs, + must_match!(&command_ctx.kind, BarrierKind::Checkpoint(epochs) => epochs), mv_log_store_truncate_epoch.into_iter(), ); let epoch = command_ctx.prev_epoch.value().0; - let tables_to_commit = tables_to_commit - .into_iter() - .map(|table_id| (table_id, epoch)) + let tables_to_commit = command_ctx + .table_ids_to_commit + .iter() + .map(|table_id| (*table_id, epoch)) .collect(); CommitEpochInfo { diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index c39fdb56e4fb7..59eb11d4342aa 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -451,6 +451,10 @@ impl ScheduledBarriers { self.force_checkpoint = true; } + pub fn is_forced_next_checkpoint(&self) -> bool { + self.force_checkpoint + } + /// Update the `checkpoint_frequency` pub fn set_checkpoint_frequency(&mut self, frequency: usize) { self.checkpoint_frequency = frequency; From f8df42d575402116e5fc313048154644b3381831 Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 9 Oct 2024 19:35:29 +0800 Subject: [PATCH 09/19] refactor --- src/meta/src/barrier/mod.rs | 360 +++++++++---------- src/meta/src/barrier/schedule.rs | 4 - src/meta/src/hummock/manager/commit_epoch.rs | 1 + src/meta/src/lib.rs | 1 + 4 files changed, 162 insertions(+), 204 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 7b75cd0c964f3..401e983dabbcb 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -14,7 +14,7 @@ use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; -use std::future::{pending, Future}; +use std::future::pending; use std::mem::{replace, take}; use std::sync::Arc; use std::time::Duration; @@ -54,7 +54,6 @@ use crate::barrier::creating_job::CreatingStreamingJobControl; use crate::barrier::info::InflightGraphInfo; use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob}; use crate::barrier::rpc::{merge_node_rpc_errors, ControlStreamManager}; -use crate::barrier::schedule::ScheduledBarriers; use crate::barrier::state::BarrierManagerState; use crate::error::MetaErrorInner; use crate::hummock::{CommitEpochInfo, HummockManagerRef, NewTableFragmentInfo}; @@ -399,9 +398,7 @@ impl CheckpointControl { .last_key_value() .map(|(_, x)| &x.command_ctx) .or(match &self.completing_command { - CompletingCommand::None - | CompletingCommand::Err(_) - | CompletingCommand::CreatingStreamingJob { .. } => None, + CompletingCommand::None | CompletingCommand::Err(_) => None, CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => command_ctx.as_ref(), }) .map(|command_ctx| command_ctx.command.should_pause_inject_barrier()) @@ -412,9 +409,7 @@ impl CheckpointControl { .map(|node| &node.command_ctx) .chain( match &self.completing_command { - CompletingCommand::None - | CompletingCommand::Err(_) - | CompletingCommand::CreatingStreamingJob { .. } => None, + CompletingCommand::None | CompletingCommand::Err(_) => None, CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => command_ctx.as_ref(), } @@ -447,23 +442,10 @@ impl CheckpointControl { } } CompletingCommand::Err(_) => true, - CompletingCommand::CreatingStreamingJob { join_handle, .. } => { - match join_handle.await { - Err(e) => { - warn!(err = ?e.as_report(), "failed to join completing task"); - true - } - Ok(Err(e)) => { - warn!(err = ?e.as_report(), "failed to complete barrier during clear"); - true - } - Ok(Ok(_)) => false, - } - } }; if !is_err { // continue to finish the pending collected barrier. - while let Some((task, _)) = self.next_collected_checkpoint_barrier(None) { + while let Some(task) = self.next_complete_barrier_task() { if let Err(e) = self.context.clone().complete_barrier(task).await { error!( err = ?e.as_report(), @@ -531,17 +513,14 @@ enum CompletingCommand { GlobalStreamingGraph { command_ctx: Option>, table_ids_to_finish: HashSet, + creating_job_epochs: Vec<(TableId, u64)>, + require_next_checkpoint: bool, // The join handle of a spawned task that completes the barrier. // The return value indicate whether there is some create streaming job command // that has finished but not checkpointed. If there is any, we will force checkpoint on the next barrier join_handle: JoinHandle>, }, - CreatingStreamingJob { - table_id: TableId, - epoch: u64, - join_handle: JoinHandle>, - }, #[expect(dead_code)] Err(MetaError), } @@ -837,14 +816,18 @@ impl GlobalBarrierManager { } } } - complete_result = self.checkpoint_control.next_completed_barrier(&mut self.scheduled_barriers) => { + complete_result = self.checkpoint_control.next_completed_barrier() => { match complete_result { - Ok(Some(output)) => { + Ok(output) => { + // If there are remaining commands (that requires checkpoint to finish), we force + // the next barrier to be a checkpoint. + if output.require_next_checkpoint { + self.scheduled_barriers.force_checkpoint_in_next_barrier(); + } self.control_stream_manager.remove_partial_graph( output.table_ids_to_finish.iter().map(|table_id| table_id.table_id).collect() ); } - Ok(None) => {} Err(e) => { self.failure_recovery(e).await; } @@ -1115,46 +1098,48 @@ impl GlobalBarrierManager { } } +#[derive(Default)] struct CompleteBarrierTask { commit_info: CommitEpochInfo, finished_jobs: Vec, notifiers: Vec, /// Some((`command_ctx`, `enqueue_time`)) command_context: Option<(Arc, HistogramTimer)>, + table_ids_to_finish: HashSet, + creating_job_epochs: Vec<(TableId, u64)>, + require_next_checkpoint: bool, } impl GlobalBarrierManagerContext { - async fn complete_creating_job_barrier( - self, + fn collect_creating_job_commit_epoch_info( + commit_info: &mut CommitEpochInfo, epoch: u64, resps: Vec, - tables_to_commit: HashSet, + tables_to_commit: impl Iterator, is_first_time: bool, - ) -> MetaResult<()> { + ) { let (sst_to_context, sstables, new_table_watermarks, old_value_sst) = collect_resp_info(resps); assert!(old_value_sst.is_empty()); - let new_table_fragment_infos = if is_first_time { - vec![NewTableFragmentInfo::NewCompactionGroup { - table_ids: tables_to_commit.clone(), - }] - } else { - vec![] - }; - let tables_to_commit = tables_to_commit - .into_iter() - .map(|table_id| (table_id, epoch)) - .collect(); - let info = CommitEpochInfo { - sstables, - new_table_watermarks, - sst_to_context, - new_table_fragment_infos, - change_log_delta: Default::default(), - tables_to_commit, + commit_info.sst_to_context.extend(sst_to_context); + commit_info.sstables.extend(sstables); + commit_info + .new_table_watermarks + .extend(new_table_watermarks); + let tables_to_commit: HashSet<_> = tables_to_commit.collect(); + tables_to_commit.iter().for_each(|table_id| { + commit_info + .tables_to_commit + .try_insert(*table_id, epoch) + .expect("non duplicate"); + }); + if is_first_time { + commit_info + .new_table_fragment_infos + .push(NewTableFragmentInfo::NewCompactionGroup { + table_ids: tables_to_commit, + }); }; - self.hummock_manager.commit_epoch(info).await?; - Ok(()) } async fn complete_barrier(self, task: CompleteBarrierTask) -> MetaResult { @@ -1260,6 +1245,7 @@ impl GlobalBarrierManagerContext { } struct BarrierCompleteOutput { + require_next_checkpoint: bool, table_ids_to_finish: HashSet, } @@ -1289,10 +1275,8 @@ impl CheckpointControl { .collect() } - fn next_collected_checkpoint_barrier( - &mut self, - mut scheduled_barriers: Option<&mut ScheduledBarriers>, - ) -> Option<(CompleteBarrierTask, HashSet)> { + fn next_complete_barrier_task(&mut self) -> Option { + let mut task = None; while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() && !state.is_inflight() { @@ -1308,16 +1292,6 @@ impl CheckpointControl { node.notifiers.into_iter().for_each(|notifier| { notifier.notify_collected(); }); - if let Some(scheduled_barriers) = &mut scheduled_barriers - && !scheduled_barriers.is_forced_next_checkpoint() - && self.create_mview_tracker.has_pending_finished_jobs() - && self - .command_ctx_queue - .values() - .all(|node| !node.command_ctx.kind.is_checkpoint()) - { - scheduled_barriers.force_checkpoint_in_next_barrier(); - } continue; } let commit_info = collect_commit_epoch_info( @@ -1337,46 +1311,27 @@ impl CheckpointControl { table_id }) .collect(); - return Some(( - CompleteBarrierTask { - commit_info, - finished_jobs, - notifiers: node.notifiers, - command_context: Some((node.command_ctx, node.enqueue_time)), - }, - table_ids_to_finish, - )); - } - } - None - } - - pub(super) fn next_completed_barrier<'a>( - &'a mut self, - scheduled_barriers: &mut ScheduledBarriers, - ) -> impl Future>> + 'a { - // If there is no completing barrier, try to start completing the earliest barrier if - // it has been collected. - if let CompletingCommand::None = &self.completing_command { - if let Some((task, table_ids_to_finish)) = - self.next_collected_checkpoint_barrier(Some(scheduled_barriers)) - { - { - let command_ctx = task - .command_context - .as_ref() - .map(|(command_ctx, _)| command_ctx.clone()); - let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); - self.completing_command = CompletingCommand::GlobalStreamingGraph { - command_ctx, - join_handle, - table_ids_to_finish, + let require_next_checkpoint = + if self.create_mview_tracker.has_pending_finished_jobs() { + self.command_ctx_queue + .values() + .all(|node| !node.command_ctx.kind.is_checkpoint()) + } else { + false }; - } + task = Some(CompleteBarrierTask { + commit_info, + finished_jobs, + notifiers: node.notifiers, + command_context: Some((node.command_ctx, node.enqueue_time)), + table_ids_to_finish, + creating_job_epochs: vec![], + require_next_checkpoint, + }); + break; } } - - if matches!(&self.completing_command, CompletingCommand::None) { + { { for (table_id, job) in &mut self.creating_streaming_job_controls { let (upstream_epochs_to_notify, commit_info) = job.start_completing(); @@ -1394,117 +1349,122 @@ impl CheckpointControl { } } if let Some((epoch, resps, is_first_time)) = commit_info { - let tables_to_commit = job - .info - .table_fragments - .all_table_ids() - .map(TableId::new) - .collect(); - let join_handle = - tokio::spawn(self.context.clone().complete_creating_job_barrier( - epoch, - resps, - tables_to_commit, - is_first_time, - )); - self.completing_command = CompletingCommand::CreatingStreamingJob { - table_id: *table_id, + let task = task.get_or_insert_default(); + GlobalBarrierManagerContext::collect_creating_job_commit_epoch_info( + &mut task.commit_info, epoch, - join_handle, - }; - break; + resps, + job.info.table_fragments.all_table_ids().map(TableId::new), + is_first_time, + ); + task.creating_job_epochs.push((*table_id, epoch)); } } } } + task + } - async move { - match &mut self.completing_command { - CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { - let join_result: MetaResult<_> = try { - join_handle - .await - .context("failed to join completing command")?? - }; - // It's important to reset the completing_command after await no matter the result is err - // or not, and otherwise the join handle will be polled again after ready. - let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) - } else { - CompletingCommand::None + pub(super) async fn next_completed_barrier(&mut self) -> MetaResult { + // If there is no completing barrier, try to start completing the earliest barrier if + // it has been collected. + if let CompletingCommand::None = &self.completing_command { + if let Some(task) = self.next_complete_barrier_task() { + { + let command_ctx = task + .command_context + .as_ref() + .map(|(command_ctx, _)| command_ctx.clone()); + let table_ids_to_finish = task.table_ids_to_finish.clone(); + let creating_job_epochs = task.creating_job_epochs.clone(); + let require_next_checkpoint = task.require_next_checkpoint; + let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); + self.completing_command = CompletingCommand::GlobalStreamingGraph { + command_ctx, + join_handle, + table_ids_to_finish, + creating_job_epochs, + require_next_checkpoint, }; - let completed_command = - replace(&mut self.completing_command, next_completing_command_status); - join_result.map(move |version_stats| { - self.hummock_version_stats = version_stats; - must_match!( - completed_command, - CompletingCommand::GlobalStreamingGraph { table_ids_to_finish, .. } => { - Some(BarrierCompleteOutput { - table_ids_to_finish, - }) - } - ) - }) } - CompletingCommand::CreatingStreamingJob { - table_id, - epoch, - join_handle, - } => { - let table_id = *table_id; - let epoch = *epoch; - let join_result: MetaResult<_> = try { - join_handle - .await - .context("failed to join completing command")?? - }; - // It's important to reset the completing_command after await no matter the result is err - // or not, and otherwise the join handle will be polled again after ready. - let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) - } else { - CompletingCommand::None - }; - self.completing_command = next_completing_command_status; - if let Some((upstream_epoch, is_finished)) = self - .creating_streaming_job_controls - .get_mut(&table_id) + } + } + + let CompletingCommand::GlobalStreamingGraph { join_handle, .. } = + &mut self.completing_command + else { + return pending().await; + }; + + let (table_ids_to_finish, creating_job_epochs, require_next_checkpoint) = { + { + let join_result: MetaResult<_> = try { + join_handle + .await + .context("failed to join completing command")?? + }; + // It's important to reset the completing_command after await no matter the result is err + // or not, and otherwise the join handle will be polled again after ready. + let next_completing_command_status = if let Err(e) = &join_result { + CompletingCommand::Err(e.clone()) + } else { + CompletingCommand::None + }; + let completed_command = + replace(&mut self.completing_command, next_completing_command_status); + self.hummock_version_stats = join_result?; + + must_match!(completed_command, CompletingCommand::GlobalStreamingGraph { + table_ids_to_finish, + creating_job_epochs, + require_next_checkpoint, + .. + } => (table_ids_to_finish, creating_job_epochs, require_next_checkpoint)) + } + }; + + { + for (table_id, epoch) in creating_job_epochs { + if let Some((upstream_epoch, is_finished)) = self + .creating_streaming_job_controls + .get_mut(&table_id) + .expect("should exist") + .ack_completed(epoch) + { + let wait_progress_timer = self + .command_ctx_queue + .get_mut(&upstream_epoch) .expect("should exist") - .ack_completed(epoch) - { - let wait_progress_timer = self + .state + .creating_jobs_to_wait + .remove(&table_id) + .expect("should exist"); + if let Some(timer) = wait_progress_timer { + timer.observe_duration(); + } + if is_finished { + debug!(epoch, ?table_id, "finish creating job"); + let creating_streaming_job = self + .creating_streaming_job_controls + .remove(&table_id) + .expect("should exist"); + assert!(creating_streaming_job.is_finished()); + assert!(self .command_ctx_queue .get_mut(&upstream_epoch) .expect("should exist") .state - .creating_jobs_to_wait - .remove(&table_id) - .expect("should exist"); - if let Some(timer) = wait_progress_timer { - timer.observe_duration(); - } - if is_finished { - debug!(epoch, ?table_id, "finish creating job"); - let creating_streaming_job = self - .creating_streaming_job_controls - .remove(&table_id) - .expect("should exist"); - assert!(creating_streaming_job.is_finished()); - assert!(self - .command_ctx_queue - .get_mut(&upstream_epoch) - .expect("should exist") - .state - .finished_jobs - .insert(table_id, creating_streaming_job.info) - .is_none()); - } + .finished_jobs + .insert(table_id, creating_streaming_job.info) + .is_none()); } - join_result.map(|_| None) } - CompletingCommand::None | CompletingCommand::Err(_) => pending().await, } + + Ok(BarrierCompleteOutput { + require_next_checkpoint, + table_ids_to_finish, + }) } } } diff --git a/src/meta/src/barrier/schedule.rs b/src/meta/src/barrier/schedule.rs index 59eb11d4342aa..c39fdb56e4fb7 100644 --- a/src/meta/src/barrier/schedule.rs +++ b/src/meta/src/barrier/schedule.rs @@ -451,10 +451,6 @@ impl ScheduledBarriers { self.force_checkpoint = true; } - pub fn is_forced_next_checkpoint(&self) -> bool { - self.force_checkpoint - } - /// Update the `checkpoint_frequency` pub fn set_checkpoint_frequency(&mut self, frequency: usize) { self.checkpoint_frequency = frequency; diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 3da93d61ab70b..358e8907e5d5d 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -58,6 +58,7 @@ pub enum NewTableFragmentInfo { }, } +#[derive(Default)] pub struct CommitEpochInfo { pub sstables: Vec, pub new_table_watermarks: HashMap, diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index eab9dd1287ebf..ced439bac63ab 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -31,6 +31,7 @@ #![feature(const_option)] #![feature(anonymous_lifetime_in_impl_trait)] #![feature(duration_millis_float)] +#![feature(option_get_or_insert_default)] pub mod backup_restore; pub mod barrier; From d85167ed26d059e7b13ef7890ee92345c630e1df Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 10 Oct 2024 13:12:11 +0800 Subject: [PATCH 10/19] temp revert to 1b359b0cb7 for test timeout --- src/meta/src/barrier/mod.rs | 470 +++++++++++-------- src/meta/src/hummock/manager/commit_epoch.rs | 1 - src/meta/src/hummock/manager/transaction.rs | 9 +- src/meta/src/lib.rs | 1 - 4 files changed, 282 insertions(+), 199 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 401e983dabbcb..15d6600f68033 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::assert_matches::assert_matches; use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; use std::future::pending; @@ -291,6 +292,7 @@ impl CheckpointControl { notifiers: Vec, node_to_collect: HashSet, jobs_to_wait: HashSet, + table_ids_to_commit: HashSet, ) { let timer = self.context.metrics.barrier_latency.start_timer(); @@ -332,7 +334,8 @@ impl CheckpointControl { node_to_collect, resps: vec![], creating_jobs_to_wait, - finished_jobs: HashMap::new(), + finished_table_ids: HashMap::new(), + table_ids_to_commit, }, command_ctx, notifiers, @@ -398,8 +401,10 @@ impl CheckpointControl { .last_key_value() .map(|(_, x)| &x.command_ctx) .or(match &self.completing_command { - CompletingCommand::None | CompletingCommand::Err(_) => None, - CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => command_ctx.as_ref(), + CompletingCommand::None + | CompletingCommand::Err(_) + | CompletingCommand::CreatingStreamingJob { .. } => None, + CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => Some(command_ctx), }) .map(|command_ctx| command_ctx.command.should_pause_inject_barrier()) .unwrap_or(false); @@ -409,9 +414,11 @@ impl CheckpointControl { .map(|node| &node.command_ctx) .chain( match &self.completing_command { - CompletingCommand::None | CompletingCommand::Err(_) => None, + CompletingCommand::None + | CompletingCommand::Err(_) + | CompletingCommand::CreatingStreamingJob { .. } => None, CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => - command_ctx.as_ref(), + Some(command_ctx), } .into_iter() ) @@ -427,8 +434,16 @@ impl CheckpointControl { // join spawned completing command to finish no matter it succeeds or not. let is_err = match replace(&mut self.completing_command, CompletingCommand::None) { CompletingCommand::None => false, - CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { - info!("waiting for completing command to finish in recovery"); + CompletingCommand::GlobalStreamingGraph { + command_ctx, + join_handle, + .. + } => { + info!( + prev_epoch = ?command_ctx.prev_epoch, + curr_epoch = ?command_ctx.curr_epoch, + "waiting for completing command to finish in recovery" + ); match join_handle.await { Err(e) => { warn!(err = ?e.as_report(), "failed to join completing task"); @@ -442,18 +457,51 @@ impl CheckpointControl { } } CompletingCommand::Err(_) => true, + CompletingCommand::CreatingStreamingJob { join_handle, .. } => { + match join_handle.await { + Err(e) => { + warn!(err = ?e.as_report(), "failed to join completing task"); + true + } + Ok(Err(e)) => { + warn!(err = ?e.as_report(), "failed to complete barrier during clear"); + true + } + Ok(Ok(_)) => false, + } + } }; if !is_err { // continue to finish the pending collected barrier. - while let Some(task) = self.next_complete_barrier_task() { - if let Err(e) = self.context.clone().complete_barrier(task).await { + while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() + && !state.is_inflight() + { + let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); + let (prev_epoch, curr_epoch) = ( + node.command_ctx.prev_epoch.value().0, + node.command_ctx.curr_epoch.value().0, + ); + let finished_jobs = self + .create_mview_tracker + .apply_collected_command(&node, &self.hummock_version_stats); + if let Err(e) = self + .context + .clone() + .complete_barrier(node, finished_jobs, HashMap::new()) + .await + { error!( + prev_epoch, + curr_epoch, err = ?e.as_report(), "failed to complete barrier during recovery" ); break; } else { - info!("succeed to complete barrier during recovery") + info!( + prev_epoch, + curr_epoch, "succeed to complete barrier during recovery" + ) } } } @@ -499,7 +547,9 @@ struct BarrierEpochState { creating_jobs_to_wait: HashMap>, - finished_jobs: HashMap, + finished_table_ids: HashMap, + + table_ids_to_commit: HashSet, } impl BarrierEpochState { @@ -511,15 +561,19 @@ impl BarrierEpochState { enum CompletingCommand { None, GlobalStreamingGraph { - command_ctx: Option>, + command_ctx: Arc, table_ids_to_finish: HashSet, - creating_job_epochs: Vec<(TableId, u64)>, require_next_checkpoint: bool, // The join handle of a spawned task that completes the barrier. // The return value indicate whether there is some create streaming job command // that has finished but not checkpointed. If there is any, we will force checkpoint on the next barrier - join_handle: JoinHandle>, + join_handle: JoinHandle>>, + }, + CreatingStreamingJob { + table_id: TableId, + epoch: u64, + join_handle: JoinHandle>, }, #[expect(dead_code)] Err(MetaError), @@ -818,16 +872,18 @@ impl GlobalBarrierManager { } complete_result = self.checkpoint_control.next_completed_barrier() => { match complete_result { - Ok(output) => { + Ok(Some(output)) => { // If there are remaining commands (that requires checkpoint to finish), we force // the next barrier to be a checkpoint. if output.require_next_checkpoint { + assert_matches!(output.command_ctx.kind, BarrierKind::Barrier); self.scheduled_barriers.force_checkpoint_in_next_barrier(); } self.control_stream_manager.remove_partial_graph( output.table_ids_to_finish.iter().map(|table_id| table_id.table_id).collect() ); } + Ok(None) => {} Err(e) => { self.failure_recovery(e).await; } @@ -982,7 +1038,7 @@ impl GlobalBarrierManager { pre_applied_subscription_info, prev_epoch.clone(), curr_epoch.clone(), - table_ids_to_commit, + table_ids_to_commit.clone(), self.state.paused_reason(), command, kind, @@ -1034,6 +1090,7 @@ impl GlobalBarrierManager { notifiers, node_to_collect, jobs_to_wait, + table_ids_to_commit, ); Ok(()) @@ -1098,87 +1155,143 @@ impl GlobalBarrierManager { } } -#[derive(Default)] -struct CompleteBarrierTask { - commit_info: CommitEpochInfo, - finished_jobs: Vec, - notifiers: Vec, - /// Some((`command_ctx`, `enqueue_time`)) - command_context: Option<(Arc, HistogramTimer)>, - table_ids_to_finish: HashSet, - creating_job_epochs: Vec<(TableId, u64)>, - require_next_checkpoint: bool, -} - impl GlobalBarrierManagerContext { - fn collect_creating_job_commit_epoch_info( - commit_info: &mut CommitEpochInfo, + async fn complete_creating_job_barrier( + self, epoch: u64, resps: Vec, - tables_to_commit: impl Iterator, + tables_to_commit: HashSet, is_first_time: bool, - ) { + ) -> MetaResult<()> { let (sst_to_context, sstables, new_table_watermarks, old_value_sst) = collect_resp_info(resps); assert!(old_value_sst.is_empty()); - commit_info.sst_to_context.extend(sst_to_context); - commit_info.sstables.extend(sstables); - commit_info - .new_table_watermarks - .extend(new_table_watermarks); - let tables_to_commit: HashSet<_> = tables_to_commit.collect(); - tables_to_commit.iter().for_each(|table_id| { - commit_info - .tables_to_commit - .try_insert(*table_id, epoch) - .expect("non duplicate"); - }); - if is_first_time { - commit_info - .new_table_fragment_infos - .push(NewTableFragmentInfo::NewCompactionGroup { - table_ids: tables_to_commit, - }); + let new_table_fragment_infos = if is_first_time { + vec![NewTableFragmentInfo::NewCompactionGroup { + table_ids: tables_to_commit.clone(), + }] + } else { + vec![] + }; + let tables_to_commit = tables_to_commit + .into_iter() + .map(|table_id| (table_id, epoch)) + .collect(); + let info = CommitEpochInfo { + sstables, + new_table_watermarks, + sst_to_context, + new_table_fragment_infos, + change_log_delta: Default::default(), + tables_to_commit, }; + self.hummock_manager.commit_epoch(info).await?; + Ok(()) } - async fn complete_barrier(self, task: CompleteBarrierTask) -> MetaResult { - let result: MetaResult<()> = try { - let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); - self.hummock_manager.commit_epoch(task.commit_info).await?; - if let Some((command_ctx, _)) = &task.command_context { - command_ctx.post_collect().await?; - } - - wait_commit_timer.observe_duration(); - }; + async fn complete_barrier( + self, + node: EpochNode, + mut finished_jobs: Vec, + backfill_pinned_log_epoch: HashMap)>, + ) -> MetaResult> { + tracing::trace!( + prev_epoch = node.command_ctx.prev_epoch.value().0, + kind = ?node.command_ctx.kind, + "complete barrier" + ); + let EpochNode { + command_ctx, + notifiers, + enqueue_time, + state, + .. + } = node; + assert!(state.node_to_collect.is_empty()); + assert!(state.creating_jobs_to_wait.is_empty()); + let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); + if !state.finished_table_ids.is_empty() { + assert!(command_ctx.kind.is_checkpoint()); + } + finished_jobs.extend(state.finished_table_ids.into_values().map(|info| { + TrackingJob::New(TrackingCommand { + info, + replace_table_info: None, + }) + })); + + let result = self + .update_snapshot( + &command_ctx, + state.table_ids_to_commit, + state.resps, + backfill_pinned_log_epoch, + ) + .await; - { - if let Err(e) = result { - for notifier in task.notifiers { + let version_stats = match result { + Ok(version_stats) => version_stats, + Err(e) => { + for notifier in notifiers { notifier.notify_collection_failed(e.clone()); } return Err(e); } - task.notifiers.into_iter().for_each(|notifier| { - notifier.notify_collected(); - }); - try_join_all( - task.finished_jobs - .into_iter() - .map(|finished_job| finished_job.finish(&self.metadata_manager)), - ) - .await?; - if let Some((command_ctx, enqueue_time)) = task.command_context { - let duration_sec = enqueue_time.stop_and_record(); - self.report_complete_event(duration_sec, &command_ctx); - self.metrics - .last_committed_barrier_time - .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); + }; + notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + try_join_all(finished_jobs.into_iter().map(|finished_job| { + let metadata_manager = &self.metadata_manager; + async move { finished_job.finish(metadata_manager).await } + })) + .await?; + let duration_sec = enqueue_time.stop_and_record(); + self.report_complete_event(duration_sec, &command_ctx); + wait_commit_timer.observe_duration(); + self.metrics + .last_committed_barrier_time + .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); + Ok(version_stats) + } + + async fn update_snapshot( + &self, + command_ctx: &CommandContext, + tables_to_commit: HashSet, + resps: Vec, + backfill_pinned_log_epoch: HashMap)>, + ) -> MetaResult> { + { + { + match &command_ctx.kind { + BarrierKind::Initial => {} + BarrierKind::Checkpoint(epochs) => { + let commit_info = collect_commit_epoch_info( + resps, + command_ctx, + epochs, + backfill_pinned_log_epoch, + tables_to_commit, + ); + self.hummock_manager.commit_epoch(commit_info).await?; + } + BarrierKind::Barrier => { + // if we collect a barrier(checkpoint = false), + // we need to ensure that command is Plain and the notifier's checkpoint is + // false + assert!(!command_ctx.command.need_checkpoint()); + } + } + + command_ctx.post_collect().await?; + Ok(if command_ctx.kind.is_checkpoint() { + Some(self.hummock_manager.get_version_stats().await) + } else { + None + }) } } - - Ok(self.hummock_manager.get_version_stats().await) } pub fn hummock_manager(&self) -> &HummockManagerRef { @@ -1245,6 +1358,7 @@ impl GlobalBarrierManagerContext { } struct BarrierCompleteOutput { + command_ctx: Arc, require_next_checkpoint: bool, table_ids_to_finish: HashSet, } @@ -1275,42 +1389,27 @@ impl CheckpointControl { .collect() } - fn next_complete_barrier_task(&mut self) -> Option { - let mut task = None; - while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() - { + pub(super) async fn next_completed_barrier( + &mut self, + ) -> MetaResult> { + if matches!(&self.completing_command, CompletingCommand::None) { + // If there is no completing barrier, try to start completing the earliest barrier if + // it has been collected. + if let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() + && !state.is_inflight() { - let (_, mut node) = self.command_ctx_queue.pop_first().expect("non-empty"); + let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); assert!(node.state.creating_jobs_to_wait.is_empty()); - assert!(node.state.node_to_collect.is_empty()); - let mut finished_jobs = self + let table_ids_to_finish = node.state.finished_table_ids.keys().cloned().collect(); + let finished_jobs = self .create_mview_tracker .apply_collected_command(&node, &self.hummock_version_stats); - if !node.command_ctx.kind.is_checkpoint() { - assert!(finished_jobs.is_empty()); - node.notifiers.into_iter().for_each(|notifier| { - notifier.notify_collected(); - }); - continue; - } - let commit_info = collect_commit_epoch_info( - take(&mut node.state.resps), - &node.command_ctx, + let command_ctx = node.command_ctx.clone(); + let join_handle = tokio::spawn(self.context.clone().complete_barrier( + node, + finished_jobs, self.collect_backfill_pinned_upstream_log_epoch(), - ); - let table_ids_to_finish = node - .state - .finished_jobs - .drain() - .map(|(table_id, info)| { - finished_jobs.push(TrackingJob::New(TrackingCommand { - info, - replace_table_info: None, - })); - table_id - }) - .collect(); + )); let require_next_checkpoint = if self.create_mview_tracker.has_pending_finished_jobs() { self.command_ctx_queue @@ -1319,20 +1418,13 @@ impl CheckpointControl { } else { false }; - task = Some(CompleteBarrierTask { - commit_info, - finished_jobs, - notifiers: node.notifiers, - command_context: Some((node.command_ctx, node.enqueue_time)), - table_ids_to_finish, - creating_job_epochs: vec![], + self.completing_command = CompletingCommand::GlobalStreamingGraph { + command_ctx, require_next_checkpoint, - }); - break; - } - } - { - { + join_handle, + table_ids_to_finish, + }; + } else { for (table_id, job) in &mut self.creating_streaming_job_controls { let (upstream_epochs_to_notify, commit_info) = job.start_completing(); for upstream_epoch in upstream_epochs_to_notify { @@ -1349,55 +1441,32 @@ impl CheckpointControl { } } if let Some((epoch, resps, is_first_time)) = commit_info { - let task = task.get_or_insert_default(); - GlobalBarrierManagerContext::collect_creating_job_commit_epoch_info( - &mut task.commit_info, + let tables_to_commit = job + .info + .table_fragments + .all_table_ids() + .map(TableId::new) + .collect(); + let join_handle = + tokio::spawn(self.context.clone().complete_creating_job_barrier( + epoch, + resps, + tables_to_commit, + is_first_time, + )); + self.completing_command = CompletingCommand::CreatingStreamingJob { + table_id: *table_id, epoch, - resps, - job.info.table_fragments.all_table_ids().map(TableId::new), - is_first_time, - ); - task.creating_job_epochs.push((*table_id, epoch)); + join_handle, + }; + break; } } } } - task - } - pub(super) async fn next_completed_barrier(&mut self) -> MetaResult { - // If there is no completing barrier, try to start completing the earliest barrier if - // it has been collected. - if let CompletingCommand::None = &self.completing_command { - if let Some(task) = self.next_complete_barrier_task() { - { - let command_ctx = task - .command_context - .as_ref() - .map(|(command_ctx, _)| command_ctx.clone()); - let table_ids_to_finish = task.table_ids_to_finish.clone(); - let creating_job_epochs = task.creating_job_epochs.clone(); - let require_next_checkpoint = task.require_next_checkpoint; - let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); - self.completing_command = CompletingCommand::GlobalStreamingGraph { - command_ctx, - join_handle, - table_ids_to_finish, - creating_job_epochs, - require_next_checkpoint, - }; - } - } - } - - let CompletingCommand::GlobalStreamingGraph { join_handle, .. } = - &mut self.completing_command - else { - return pending().await; - }; - - let (table_ids_to_finish, creating_job_epochs, require_next_checkpoint) = { - { + match &mut self.completing_command { + CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { let join_result: MetaResult<_> = try { join_handle .await @@ -1412,19 +1481,42 @@ impl CheckpointControl { }; let completed_command = replace(&mut self.completing_command, next_completing_command_status); - self.hummock_version_stats = join_result?; - - must_match!(completed_command, CompletingCommand::GlobalStreamingGraph { - table_ids_to_finish, - creating_job_epochs, - require_next_checkpoint, - .. - } => (table_ids_to_finish, creating_job_epochs, require_next_checkpoint)) + join_result.map(move | version_stats| { + if let Some(new_version_stats) = version_stats { + self.hummock_version_stats = new_version_stats; + } + must_match!( + completed_command, + CompletingCommand::GlobalStreamingGraph { command_ctx, table_ids_to_finish, require_next_checkpoint, .. } => { + Some(BarrierCompleteOutput { + command_ctx, + require_next_checkpoint, + table_ids_to_finish, + }) + } + ) + }) } - }; - - { - for (table_id, epoch) in creating_job_epochs { + CompletingCommand::CreatingStreamingJob { + table_id, + epoch, + join_handle, + } => { + let table_id = *table_id; + let epoch = *epoch; + let join_result: MetaResult<_> = try { + join_handle + .await + .context("failed to join completing command")?? + }; + // It's important to reset the completing_command after await no matter the result is err + // or not, and otherwise the join handle will be polled again after ready. + let next_completing_command_status = if let Err(e) = &join_result { + CompletingCommand::Err(e.clone()) + } else { + CompletingCommand::None + }; + self.completing_command = next_completing_command_status; if let Some((upstream_epoch, is_finished)) = self .creating_streaming_job_controls .get_mut(&table_id) @@ -1454,17 +1546,14 @@ impl CheckpointControl { .get_mut(&upstream_epoch) .expect("should exist") .state - .finished_jobs + .finished_table_ids .insert(table_id, creating_streaming_job.info) .is_none()); } } + join_result.map(|_| None) } - - Ok(BarrierCompleteOutput { - require_next_checkpoint, - table_ids_to_finish, - }) + CompletingCommand::None | CompletingCommand::Err(_) => pending().await, } } } @@ -1619,7 +1708,9 @@ fn collect_resp_info( fn collect_commit_epoch_info( resps: Vec, command_ctx: &CommandContext, + epochs: &Vec, backfill_pinned_log_epoch: HashMap)>, + tables_to_commit: HashSet, ) -> CommitEpochInfo { let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) = collect_resp_info(resps); @@ -1674,15 +1765,14 @@ fn collect_commit_epoch_info( let table_new_change_log = build_table_change_log_delta( old_value_ssts.into_iter(), synced_ssts.iter().map(|sst| &sst.sst_info), - must_match!(&command_ctx.kind, BarrierKind::Checkpoint(epochs) => epochs), + epochs, mv_log_store_truncate_epoch.into_iter(), ); let epoch = command_ctx.prev_epoch.value().0; - let tables_to_commit = command_ctx - .table_ids_to_commit - .iter() - .map(|table_id| (*table_id, epoch)) + let tables_to_commit = tables_to_commit + .into_iter() + .map(|table_id| (table_id, epoch)) .collect(); CommitEpochInfo { diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 358e8907e5d5d..3da93d61ab70b 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -58,7 +58,6 @@ pub enum NewTableFragmentInfo { }, } -#[derive(Default)] pub struct CommitEpochInfo { pub sstables: Vec, pub new_table_watermarks: HashMap, diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 56b60a6535722..87ec960bca1d1 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -17,6 +17,7 @@ use std::ops::{Deref, DerefMut}; use std::sync::Arc; use risingwave_common::catalog::TableId; +use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::change_log::ChangeLogDelta; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -158,13 +159,7 @@ impl<'a> HummockVersionTransaction<'a> { .last() .map(|level| level.sub_level_id + 1) }) - .unwrap_or_else(|| { - tables_to_commit - .values() - .cloned() - .max() - .expect("non empty tables_to_commit") - }); + .unwrap_or(Epoch::now().0); let group_deltas = &mut new_version_delta .group_deltas .entry(compaction_group_id) diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index ced439bac63ab..eab9dd1287ebf 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -31,7 +31,6 @@ #![feature(const_option)] #![feature(anonymous_lifetime_in_impl_trait)] #![feature(duration_millis_float)] -#![feature(option_get_or_insert_default)] pub mod backup_restore; pub mod barrier; From a734be6d22306e7bd2d18a7cc0ea76447e5df649 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 10 Oct 2024 15:26:25 +0800 Subject: [PATCH 11/19] Revert "temp revert to 1b359b0cb7 for test timeout" This reverts commit d85167ed26d059e7b13ef7890ee92345c630e1df. --- src/meta/src/barrier/mod.rs | 470 ++++++++----------- src/meta/src/hummock/manager/commit_epoch.rs | 1 + src/meta/src/hummock/manager/transaction.rs | 9 +- src/meta/src/lib.rs | 1 + 4 files changed, 199 insertions(+), 282 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 15d6600f68033..401e983dabbcb 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::assert_matches::assert_matches; use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; use std::future::pending; @@ -292,7 +291,6 @@ impl CheckpointControl { notifiers: Vec, node_to_collect: HashSet, jobs_to_wait: HashSet, - table_ids_to_commit: HashSet, ) { let timer = self.context.metrics.barrier_latency.start_timer(); @@ -334,8 +332,7 @@ impl CheckpointControl { node_to_collect, resps: vec![], creating_jobs_to_wait, - finished_table_ids: HashMap::new(), - table_ids_to_commit, + finished_jobs: HashMap::new(), }, command_ctx, notifiers, @@ -401,10 +398,8 @@ impl CheckpointControl { .last_key_value() .map(|(_, x)| &x.command_ctx) .or(match &self.completing_command { - CompletingCommand::None - | CompletingCommand::Err(_) - | CompletingCommand::CreatingStreamingJob { .. } => None, - CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => Some(command_ctx), + CompletingCommand::None | CompletingCommand::Err(_) => None, + CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => command_ctx.as_ref(), }) .map(|command_ctx| command_ctx.command.should_pause_inject_barrier()) .unwrap_or(false); @@ -414,11 +409,9 @@ impl CheckpointControl { .map(|node| &node.command_ctx) .chain( match &self.completing_command { - CompletingCommand::None - | CompletingCommand::Err(_) - | CompletingCommand::CreatingStreamingJob { .. } => None, + CompletingCommand::None | CompletingCommand::Err(_) => None, CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => - Some(command_ctx), + command_ctx.as_ref(), } .into_iter() ) @@ -434,16 +427,8 @@ impl CheckpointControl { // join spawned completing command to finish no matter it succeeds or not. let is_err = match replace(&mut self.completing_command, CompletingCommand::None) { CompletingCommand::None => false, - CompletingCommand::GlobalStreamingGraph { - command_ctx, - join_handle, - .. - } => { - info!( - prev_epoch = ?command_ctx.prev_epoch, - curr_epoch = ?command_ctx.curr_epoch, - "waiting for completing command to finish in recovery" - ); + CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { + info!("waiting for completing command to finish in recovery"); match join_handle.await { Err(e) => { warn!(err = ?e.as_report(), "failed to join completing task"); @@ -457,51 +442,18 @@ impl CheckpointControl { } } CompletingCommand::Err(_) => true, - CompletingCommand::CreatingStreamingJob { join_handle, .. } => { - match join_handle.await { - Err(e) => { - warn!(err = ?e.as_report(), "failed to join completing task"); - true - } - Ok(Err(e)) => { - warn!(err = ?e.as_report(), "failed to complete barrier during clear"); - true - } - Ok(Ok(_)) => false, - } - } }; if !is_err { // continue to finish the pending collected barrier. - while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() - { - let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); - let (prev_epoch, curr_epoch) = ( - node.command_ctx.prev_epoch.value().0, - node.command_ctx.curr_epoch.value().0, - ); - let finished_jobs = self - .create_mview_tracker - .apply_collected_command(&node, &self.hummock_version_stats); - if let Err(e) = self - .context - .clone() - .complete_barrier(node, finished_jobs, HashMap::new()) - .await - { + while let Some(task) = self.next_complete_barrier_task() { + if let Err(e) = self.context.clone().complete_barrier(task).await { error!( - prev_epoch, - curr_epoch, err = ?e.as_report(), "failed to complete barrier during recovery" ); break; } else { - info!( - prev_epoch, - curr_epoch, "succeed to complete barrier during recovery" - ) + info!("succeed to complete barrier during recovery") } } } @@ -547,9 +499,7 @@ struct BarrierEpochState { creating_jobs_to_wait: HashMap>, - finished_table_ids: HashMap, - - table_ids_to_commit: HashSet, + finished_jobs: HashMap, } impl BarrierEpochState { @@ -561,19 +511,15 @@ impl BarrierEpochState { enum CompletingCommand { None, GlobalStreamingGraph { - command_ctx: Arc, + command_ctx: Option>, table_ids_to_finish: HashSet, + creating_job_epochs: Vec<(TableId, u64)>, require_next_checkpoint: bool, // The join handle of a spawned task that completes the barrier. // The return value indicate whether there is some create streaming job command // that has finished but not checkpointed. If there is any, we will force checkpoint on the next barrier - join_handle: JoinHandle>>, - }, - CreatingStreamingJob { - table_id: TableId, - epoch: u64, - join_handle: JoinHandle>, + join_handle: JoinHandle>, }, #[expect(dead_code)] Err(MetaError), @@ -872,18 +818,16 @@ impl GlobalBarrierManager { } complete_result = self.checkpoint_control.next_completed_barrier() => { match complete_result { - Ok(Some(output)) => { + Ok(output) => { // If there are remaining commands (that requires checkpoint to finish), we force // the next barrier to be a checkpoint. if output.require_next_checkpoint { - assert_matches!(output.command_ctx.kind, BarrierKind::Barrier); self.scheduled_barriers.force_checkpoint_in_next_barrier(); } self.control_stream_manager.remove_partial_graph( output.table_ids_to_finish.iter().map(|table_id| table_id.table_id).collect() ); } - Ok(None) => {} Err(e) => { self.failure_recovery(e).await; } @@ -1038,7 +982,7 @@ impl GlobalBarrierManager { pre_applied_subscription_info, prev_epoch.clone(), curr_epoch.clone(), - table_ids_to_commit.clone(), + table_ids_to_commit, self.state.paused_reason(), command, kind, @@ -1090,7 +1034,6 @@ impl GlobalBarrierManager { notifiers, node_to_collect, jobs_to_wait, - table_ids_to_commit, ); Ok(()) @@ -1155,143 +1098,87 @@ impl GlobalBarrierManager { } } +#[derive(Default)] +struct CompleteBarrierTask { + commit_info: CommitEpochInfo, + finished_jobs: Vec, + notifiers: Vec, + /// Some((`command_ctx`, `enqueue_time`)) + command_context: Option<(Arc, HistogramTimer)>, + table_ids_to_finish: HashSet, + creating_job_epochs: Vec<(TableId, u64)>, + require_next_checkpoint: bool, +} + impl GlobalBarrierManagerContext { - async fn complete_creating_job_barrier( - self, + fn collect_creating_job_commit_epoch_info( + commit_info: &mut CommitEpochInfo, epoch: u64, resps: Vec, - tables_to_commit: HashSet, + tables_to_commit: impl Iterator, is_first_time: bool, - ) -> MetaResult<()> { + ) { let (sst_to_context, sstables, new_table_watermarks, old_value_sst) = collect_resp_info(resps); assert!(old_value_sst.is_empty()); - let new_table_fragment_infos = if is_first_time { - vec![NewTableFragmentInfo::NewCompactionGroup { - table_ids: tables_to_commit.clone(), - }] - } else { - vec![] - }; - let tables_to_commit = tables_to_commit - .into_iter() - .map(|table_id| (table_id, epoch)) - .collect(); - let info = CommitEpochInfo { - sstables, - new_table_watermarks, - sst_to_context, - new_table_fragment_infos, - change_log_delta: Default::default(), - tables_to_commit, + commit_info.sst_to_context.extend(sst_to_context); + commit_info.sstables.extend(sstables); + commit_info + .new_table_watermarks + .extend(new_table_watermarks); + let tables_to_commit: HashSet<_> = tables_to_commit.collect(); + tables_to_commit.iter().for_each(|table_id| { + commit_info + .tables_to_commit + .try_insert(*table_id, epoch) + .expect("non duplicate"); + }); + if is_first_time { + commit_info + .new_table_fragment_infos + .push(NewTableFragmentInfo::NewCompactionGroup { + table_ids: tables_to_commit, + }); }; - self.hummock_manager.commit_epoch(info).await?; - Ok(()) } - async fn complete_barrier( - self, - node: EpochNode, - mut finished_jobs: Vec, - backfill_pinned_log_epoch: HashMap)>, - ) -> MetaResult> { - tracing::trace!( - prev_epoch = node.command_ctx.prev_epoch.value().0, - kind = ?node.command_ctx.kind, - "complete barrier" - ); - let EpochNode { - command_ctx, - notifiers, - enqueue_time, - state, - .. - } = node; - assert!(state.node_to_collect.is_empty()); - assert!(state.creating_jobs_to_wait.is_empty()); - let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); - if !state.finished_table_ids.is_empty() { - assert!(command_ctx.kind.is_checkpoint()); - } - finished_jobs.extend(state.finished_table_ids.into_values().map(|info| { - TrackingJob::New(TrackingCommand { - info, - replace_table_info: None, - }) - })); - - let result = self - .update_snapshot( - &command_ctx, - state.table_ids_to_commit, - state.resps, - backfill_pinned_log_epoch, - ) - .await; - - let version_stats = match result { - Ok(version_stats) => version_stats, - Err(e) => { - for notifier in notifiers { - notifier.notify_collection_failed(e.clone()); - } - return Err(e); + async fn complete_barrier(self, task: CompleteBarrierTask) -> MetaResult { + let result: MetaResult<()> = try { + let wait_commit_timer = self.metrics.barrier_wait_commit_latency.start_timer(); + self.hummock_manager.commit_epoch(task.commit_info).await?; + if let Some((command_ctx, _)) = &task.command_context { + command_ctx.post_collect().await?; } + + wait_commit_timer.observe_duration(); }; - notifiers.into_iter().for_each(|notifier| { - notifier.notify_collected(); - }); - try_join_all(finished_jobs.into_iter().map(|finished_job| { - let metadata_manager = &self.metadata_manager; - async move { finished_job.finish(metadata_manager).await } - })) - .await?; - let duration_sec = enqueue_time.stop_and_record(); - self.report_complete_event(duration_sec, &command_ctx); - wait_commit_timer.observe_duration(); - self.metrics - .last_committed_barrier_time - .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); - Ok(version_stats) - } - async fn update_snapshot( - &self, - command_ctx: &CommandContext, - tables_to_commit: HashSet, - resps: Vec, - backfill_pinned_log_epoch: HashMap)>, - ) -> MetaResult> { { - { - match &command_ctx.kind { - BarrierKind::Initial => {} - BarrierKind::Checkpoint(epochs) => { - let commit_info = collect_commit_epoch_info( - resps, - command_ctx, - epochs, - backfill_pinned_log_epoch, - tables_to_commit, - ); - self.hummock_manager.commit_epoch(commit_info).await?; - } - BarrierKind::Barrier => { - // if we collect a barrier(checkpoint = false), - // we need to ensure that command is Plain and the notifier's checkpoint is - // false - assert!(!command_ctx.command.need_checkpoint()); - } + if let Err(e) = result { + for notifier in task.notifiers { + notifier.notify_collection_failed(e.clone()); } - - command_ctx.post_collect().await?; - Ok(if command_ctx.kind.is_checkpoint() { - Some(self.hummock_manager.get_version_stats().await) - } else { - None - }) + return Err(e); + } + task.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + try_join_all( + task.finished_jobs + .into_iter() + .map(|finished_job| finished_job.finish(&self.metadata_manager)), + ) + .await?; + if let Some((command_ctx, enqueue_time)) = task.command_context { + let duration_sec = enqueue_time.stop_and_record(); + self.report_complete_event(duration_sec, &command_ctx); + self.metrics + .last_committed_barrier_time + .set(command_ctx.curr_epoch.value().as_unix_secs() as i64); } } + + Ok(self.hummock_manager.get_version_stats().await) } pub fn hummock_manager(&self) -> &HummockManagerRef { @@ -1358,7 +1245,6 @@ impl GlobalBarrierManagerContext { } struct BarrierCompleteOutput { - command_ctx: Arc, require_next_checkpoint: bool, table_ids_to_finish: HashSet, } @@ -1389,27 +1275,42 @@ impl CheckpointControl { .collect() } - pub(super) async fn next_completed_barrier( - &mut self, - ) -> MetaResult> { - if matches!(&self.completing_command, CompletingCommand::None) { - // If there is no completing barrier, try to start completing the earliest barrier if - // it has been collected. - if let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() - && !state.is_inflight() + fn next_complete_barrier_task(&mut self) -> Option { + let mut task = None; + while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() + && !state.is_inflight() + { { - let (_, node) = self.command_ctx_queue.pop_first().expect("non-empty"); + let (_, mut node) = self.command_ctx_queue.pop_first().expect("non-empty"); assert!(node.state.creating_jobs_to_wait.is_empty()); - let table_ids_to_finish = node.state.finished_table_ids.keys().cloned().collect(); - let finished_jobs = self + assert!(node.state.node_to_collect.is_empty()); + let mut finished_jobs = self .create_mview_tracker .apply_collected_command(&node, &self.hummock_version_stats); - let command_ctx = node.command_ctx.clone(); - let join_handle = tokio::spawn(self.context.clone().complete_barrier( - node, - finished_jobs, + if !node.command_ctx.kind.is_checkpoint() { + assert!(finished_jobs.is_empty()); + node.notifiers.into_iter().for_each(|notifier| { + notifier.notify_collected(); + }); + continue; + } + let commit_info = collect_commit_epoch_info( + take(&mut node.state.resps), + &node.command_ctx, self.collect_backfill_pinned_upstream_log_epoch(), - )); + ); + let table_ids_to_finish = node + .state + .finished_jobs + .drain() + .map(|(table_id, info)| { + finished_jobs.push(TrackingJob::New(TrackingCommand { + info, + replace_table_info: None, + })); + table_id + }) + .collect(); let require_next_checkpoint = if self.create_mview_tracker.has_pending_finished_jobs() { self.command_ctx_queue @@ -1418,13 +1319,20 @@ impl CheckpointControl { } else { false }; - self.completing_command = CompletingCommand::GlobalStreamingGraph { - command_ctx, - require_next_checkpoint, - join_handle, + task = Some(CompleteBarrierTask { + commit_info, + finished_jobs, + notifiers: node.notifiers, + command_context: Some((node.command_ctx, node.enqueue_time)), table_ids_to_finish, - }; - } else { + creating_job_epochs: vec![], + require_next_checkpoint, + }); + break; + } + } + { + { for (table_id, job) in &mut self.creating_streaming_job_controls { let (upstream_epochs_to_notify, commit_info) = job.start_completing(); for upstream_epoch in upstream_epochs_to_notify { @@ -1441,32 +1349,55 @@ impl CheckpointControl { } } if let Some((epoch, resps, is_first_time)) = commit_info { - let tables_to_commit = job - .info - .table_fragments - .all_table_ids() - .map(TableId::new) - .collect(); - let join_handle = - tokio::spawn(self.context.clone().complete_creating_job_barrier( - epoch, - resps, - tables_to_commit, - is_first_time, - )); - self.completing_command = CompletingCommand::CreatingStreamingJob { - table_id: *table_id, + let task = task.get_or_insert_default(); + GlobalBarrierManagerContext::collect_creating_job_commit_epoch_info( + &mut task.commit_info, epoch, - join_handle, - }; - break; + resps, + job.info.table_fragments.all_table_ids().map(TableId::new), + is_first_time, + ); + task.creating_job_epochs.push((*table_id, epoch)); } } } } + task + } - match &mut self.completing_command { - CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { + pub(super) async fn next_completed_barrier(&mut self) -> MetaResult { + // If there is no completing barrier, try to start completing the earliest barrier if + // it has been collected. + if let CompletingCommand::None = &self.completing_command { + if let Some(task) = self.next_complete_barrier_task() { + { + let command_ctx = task + .command_context + .as_ref() + .map(|(command_ctx, _)| command_ctx.clone()); + let table_ids_to_finish = task.table_ids_to_finish.clone(); + let creating_job_epochs = task.creating_job_epochs.clone(); + let require_next_checkpoint = task.require_next_checkpoint; + let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); + self.completing_command = CompletingCommand::GlobalStreamingGraph { + command_ctx, + join_handle, + table_ids_to_finish, + creating_job_epochs, + require_next_checkpoint, + }; + } + } + } + + let CompletingCommand::GlobalStreamingGraph { join_handle, .. } = + &mut self.completing_command + else { + return pending().await; + }; + + let (table_ids_to_finish, creating_job_epochs, require_next_checkpoint) = { + { let join_result: MetaResult<_> = try { join_handle .await @@ -1481,42 +1412,19 @@ impl CheckpointControl { }; let completed_command = replace(&mut self.completing_command, next_completing_command_status); - join_result.map(move | version_stats| { - if let Some(new_version_stats) = version_stats { - self.hummock_version_stats = new_version_stats; - } - must_match!( - completed_command, - CompletingCommand::GlobalStreamingGraph { command_ctx, table_ids_to_finish, require_next_checkpoint, .. } => { - Some(BarrierCompleteOutput { - command_ctx, - require_next_checkpoint, - table_ids_to_finish, - }) - } - ) - }) + self.hummock_version_stats = join_result?; + + must_match!(completed_command, CompletingCommand::GlobalStreamingGraph { + table_ids_to_finish, + creating_job_epochs, + require_next_checkpoint, + .. + } => (table_ids_to_finish, creating_job_epochs, require_next_checkpoint)) } - CompletingCommand::CreatingStreamingJob { - table_id, - epoch, - join_handle, - } => { - let table_id = *table_id; - let epoch = *epoch; - let join_result: MetaResult<_> = try { - join_handle - .await - .context("failed to join completing command")?? - }; - // It's important to reset the completing_command after await no matter the result is err - // or not, and otherwise the join handle will be polled again after ready. - let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) - } else { - CompletingCommand::None - }; - self.completing_command = next_completing_command_status; + }; + + { + for (table_id, epoch) in creating_job_epochs { if let Some((upstream_epoch, is_finished)) = self .creating_streaming_job_controls .get_mut(&table_id) @@ -1546,14 +1454,17 @@ impl CheckpointControl { .get_mut(&upstream_epoch) .expect("should exist") .state - .finished_table_ids + .finished_jobs .insert(table_id, creating_streaming_job.info) .is_none()); } } - join_result.map(|_| None) } - CompletingCommand::None | CompletingCommand::Err(_) => pending().await, + + Ok(BarrierCompleteOutput { + require_next_checkpoint, + table_ids_to_finish, + }) } } } @@ -1708,9 +1619,7 @@ fn collect_resp_info( fn collect_commit_epoch_info( resps: Vec, command_ctx: &CommandContext, - epochs: &Vec, backfill_pinned_log_epoch: HashMap)>, - tables_to_commit: HashSet, ) -> CommitEpochInfo { let (sst_to_context, synced_ssts, new_table_watermarks, old_value_ssts) = collect_resp_info(resps); @@ -1765,14 +1674,15 @@ fn collect_commit_epoch_info( let table_new_change_log = build_table_change_log_delta( old_value_ssts.into_iter(), synced_ssts.iter().map(|sst| &sst.sst_info), - epochs, + must_match!(&command_ctx.kind, BarrierKind::Checkpoint(epochs) => epochs), mv_log_store_truncate_epoch.into_iter(), ); let epoch = command_ctx.prev_epoch.value().0; - let tables_to_commit = tables_to_commit - .into_iter() - .map(|table_id| (table_id, epoch)) + let tables_to_commit = command_ctx + .table_ids_to_commit + .iter() + .map(|table_id| (*table_id, epoch)) .collect(); CommitEpochInfo { diff --git a/src/meta/src/hummock/manager/commit_epoch.rs b/src/meta/src/hummock/manager/commit_epoch.rs index 3da93d61ab70b..358e8907e5d5d 100644 --- a/src/meta/src/hummock/manager/commit_epoch.rs +++ b/src/meta/src/hummock/manager/commit_epoch.rs @@ -58,6 +58,7 @@ pub enum NewTableFragmentInfo { }, } +#[derive(Default)] pub struct CommitEpochInfo { pub sstables: Vec, pub new_table_watermarks: HashMap, diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 87ec960bca1d1..56b60a6535722 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -17,7 +17,6 @@ use std::ops::{Deref, DerefMut}; use std::sync::Arc; use risingwave_common::catalog::TableId; -use risingwave_common::util::epoch::Epoch; use risingwave_hummock_sdk::change_log::ChangeLogDelta; use risingwave_hummock_sdk::compaction_group::StaticCompactionGroupId; use risingwave_hummock_sdk::sstable_info::SstableInfo; @@ -159,7 +158,13 @@ impl<'a> HummockVersionTransaction<'a> { .last() .map(|level| level.sub_level_id + 1) }) - .unwrap_or(Epoch::now().0); + .unwrap_or_else(|| { + tables_to_commit + .values() + .cloned() + .max() + .expect("non empty tables_to_commit") + }); let group_deltas = &mut new_version_delta .group_deltas .entry(compaction_group_id) diff --git a/src/meta/src/lib.rs b/src/meta/src/lib.rs index eab9dd1287ebf..ced439bac63ab 100644 --- a/src/meta/src/lib.rs +++ b/src/meta/src/lib.rs @@ -31,6 +31,7 @@ #![feature(const_option)] #![feature(anonymous_lifetime_in_impl_trait)] #![feature(duration_millis_float)] +#![feature(option_get_or_insert_default)] pub mod backup_restore; pub mod barrier; From 9b7a63ab737c8d534c054ec39b07495ad0418f0b Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 10 Oct 2024 15:51:16 +0800 Subject: [PATCH 12/19] force checkpoint in normal barrier --- src/meta/src/barrier/mod.rs | 57 ++++++++++----------- src/meta/src/hummock/manager/transaction.rs | 17 +++--- 2 files changed, 38 insertions(+), 36 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 401e983dabbcb..4d803fd7af367 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -14,7 +14,7 @@ use std::collections::hash_map::Entry; use std::collections::{BTreeMap, HashMap, HashSet}; -use std::future::pending; +use std::future::{pending, Future}; use std::mem::{replace, take}; use std::sync::Arc; use std::time::Duration; @@ -54,6 +54,7 @@ use crate::barrier::creating_job::CreatingStreamingJobControl; use crate::barrier::info::InflightGraphInfo; use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand, TrackingJob}; use crate::barrier::rpc::{merge_node_rpc_errors, ControlStreamManager}; +use crate::barrier::schedule::ScheduledBarriers; use crate::barrier::state::BarrierManagerState; use crate::error::MetaErrorInner; use crate::hummock::{CommitEpochInfo, HummockManagerRef, NewTableFragmentInfo}; @@ -445,7 +446,7 @@ impl CheckpointControl { }; if !is_err { // continue to finish the pending collected barrier. - while let Some(task) = self.next_complete_barrier_task() { + while let Some(task) = self.next_complete_barrier_task(None) { if let Err(e) = self.context.clone().complete_barrier(task).await { error!( err = ?e.as_report(), @@ -514,7 +515,6 @@ enum CompletingCommand { command_ctx: Option>, table_ids_to_finish: HashSet, creating_job_epochs: Vec<(TableId, u64)>, - require_next_checkpoint: bool, // The join handle of a spawned task that completes the barrier. // The return value indicate whether there is some create streaming job command @@ -816,14 +816,9 @@ impl GlobalBarrierManager { } } } - complete_result = self.checkpoint_control.next_completed_barrier() => { + complete_result = self.checkpoint_control.next_completed_barrier(&mut self.scheduled_barriers) => { match complete_result { Ok(output) => { - // If there are remaining commands (that requires checkpoint to finish), we force - // the next barrier to be a checkpoint. - if output.require_next_checkpoint { - self.scheduled_barriers.force_checkpoint_in_next_barrier(); - } self.control_stream_manager.remove_partial_graph( output.table_ids_to_finish.iter().map(|table_id| table_id.table_id).collect() ); @@ -1107,7 +1102,6 @@ struct CompleteBarrierTask { command_context: Option<(Arc, HistogramTimer)>, table_ids_to_finish: HashSet, creating_job_epochs: Vec<(TableId, u64)>, - require_next_checkpoint: bool, } impl GlobalBarrierManagerContext { @@ -1245,7 +1239,6 @@ impl GlobalBarrierManagerContext { } struct BarrierCompleteOutput { - require_next_checkpoint: bool, table_ids_to_finish: HashSet, } @@ -1275,7 +1268,10 @@ impl CheckpointControl { .collect() } - fn next_complete_barrier_task(&mut self) -> Option { + fn next_complete_barrier_task( + &mut self, + mut scheduled_barriers: Option<&mut ScheduledBarriers>, + ) -> Option { let mut task = None; while let Some((_, EpochNode { state, .. })) = self.command_ctx_queue.first_key_value() && !state.is_inflight() @@ -1292,6 +1288,15 @@ impl CheckpointControl { node.notifiers.into_iter().for_each(|notifier| { notifier.notify_collected(); }); + if let Some(scheduled_barriers) = &mut scheduled_barriers + && self.create_mview_tracker.has_pending_finished_jobs() + && self + .command_ctx_queue + .values() + .all(|node| !node.command_ctx.kind.is_checkpoint()) + { + scheduled_barriers.force_checkpoint_in_next_barrier(); + } continue; } let commit_info = collect_commit_epoch_info( @@ -1311,14 +1316,6 @@ impl CheckpointControl { table_id }) .collect(); - let require_next_checkpoint = - if self.create_mview_tracker.has_pending_finished_jobs() { - self.command_ctx_queue - .values() - .all(|node| !node.command_ctx.kind.is_checkpoint()) - } else { - false - }; task = Some(CompleteBarrierTask { commit_info, finished_jobs, @@ -1326,7 +1323,6 @@ impl CheckpointControl { command_context: Some((node.command_ctx, node.enqueue_time)), table_ids_to_finish, creating_job_epochs: vec![], - require_next_checkpoint, }); break; } @@ -1365,11 +1361,14 @@ impl CheckpointControl { task } - pub(super) async fn next_completed_barrier(&mut self) -> MetaResult { + pub(super) fn next_completed_barrier<'a>( + &'a mut self, + scheduled_barriers: &mut ScheduledBarriers, + ) -> impl Future> + 'a { // If there is no completing barrier, try to start completing the earliest barrier if // it has been collected. if let CompletingCommand::None = &self.completing_command { - if let Some(task) = self.next_complete_barrier_task() { + if let Some(task) = self.next_complete_barrier_task(Some(scheduled_barriers)) { { let command_ctx = task .command_context @@ -1377,26 +1376,28 @@ impl CheckpointControl { .map(|(command_ctx, _)| command_ctx.clone()); let table_ids_to_finish = task.table_ids_to_finish.clone(); let creating_job_epochs = task.creating_job_epochs.clone(); - let require_next_checkpoint = task.require_next_checkpoint; let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); self.completing_command = CompletingCommand::GlobalStreamingGraph { command_ctx, join_handle, table_ids_to_finish, creating_job_epochs, - require_next_checkpoint, }; } } } + self.next_completed_barrier_inner() + } + + async fn next_completed_barrier_inner(&mut self) -> MetaResult { let CompletingCommand::GlobalStreamingGraph { join_handle, .. } = &mut self.completing_command else { return pending().await; }; - let (table_ids_to_finish, creating_job_epochs, require_next_checkpoint) = { + let (table_ids_to_finish, creating_job_epochs) = { { let join_result: MetaResult<_> = try { join_handle @@ -1417,9 +1418,8 @@ impl CheckpointControl { must_match!(completed_command, CompletingCommand::GlobalStreamingGraph { table_ids_to_finish, creating_job_epochs, - require_next_checkpoint, .. - } => (table_ids_to_finish, creating_job_epochs, require_next_checkpoint)) + } => (table_ids_to_finish, creating_job_epochs)) } }; @@ -1462,7 +1462,6 @@ impl CheckpointControl { } Ok(BarrierCompleteOutput { - require_next_checkpoint, table_ids_to_finish, }) } diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 56b60a6535722..5e4bf772b6454 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::cell::LazyCell; use std::collections::{BTreeMap, HashMap}; use std::ops::{Deref, DerefMut}; use std::sync::Arc; @@ -145,6 +146,14 @@ impl<'a> HummockVersionTransaction<'a> { } } + let max_epoch_to_commit = LazyCell::new(|| { + tables_to_commit + .values() + .cloned() + .max() + .expect("non empty tables_to_commit") + }); + // Append SSTs to a new version. for (compaction_group_id, inserted_table_infos) in commit_sstables { let l0_sub_level_id = new_version_delta @@ -158,13 +167,7 @@ impl<'a> HummockVersionTransaction<'a> { .last() .map(|level| level.sub_level_id + 1) }) - .unwrap_or_else(|| { - tables_to_commit - .values() - .cloned() - .max() - .expect("non empty tables_to_commit") - }); + .unwrap_or_else(|| *max_epoch_to_commit); let group_deltas = &mut new_version_delta .group_deltas .entry(compaction_group_id) From 1e82a19697f3f0e9397fade546fd935de70db013 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 10 Oct 2024 17:50:17 +0800 Subject: [PATCH 13/19] rename --- src/meta/src/barrier/mod.rs | 49 +++++++++++++++++-------------------- 1 file changed, 23 insertions(+), 26 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 327c1f99e2f5f..038aef4a550a9 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -218,7 +218,7 @@ struct CheckpointControl { /// Command that has been collected but is still completing. /// The join handle of the completing future is stored. - completing_command: CompletingCommand, + completing_task: CompletingTask, hummock_version_stats: HummockVersionStats, @@ -235,7 +235,7 @@ impl CheckpointControl { Self { command_ctx_queue: Default::default(), creating_streaming_job_controls: Default::default(), - completing_command: CompletingCommand::None, + completing_task: CompletingTask::None, hummock_version_stats: context.hummock_manager.get_version_stats().await, create_mview_tracker, context, @@ -244,8 +244,8 @@ impl CheckpointControl { fn total_command_num(&self) -> usize { self.command_ctx_queue.len() - + match &self.completing_command { - CompletingCommand::GlobalStreamingGraph { .. } => 1, + + match &self.completing_task { + CompletingTask::Completing { .. } => 1, _ => 0, } } @@ -398,9 +398,9 @@ impl CheckpointControl { .command_ctx_queue .last_key_value() .map(|(_, x)| &x.command_ctx) - .or(match &self.completing_command { - CompletingCommand::None | CompletingCommand::Err(_) => None, - CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => command_ctx.as_ref(), + .or(match &self.completing_task { + CompletingTask::None | CompletingTask::Err(_) => None, + CompletingTask::Completing { command_ctx, .. } => command_ctx.as_ref(), }) .map(|command_ctx| command_ctx.command.should_pause_inject_barrier()) .unwrap_or(false); @@ -409,10 +409,9 @@ impl CheckpointControl { .values() .map(|node| &node.command_ctx) .chain( - match &self.completing_command { - CompletingCommand::None | CompletingCommand::Err(_) => None, - CompletingCommand::GlobalStreamingGraph { command_ctx, .. } => - command_ctx.as_ref(), + match &self.completing_task { + CompletingTask::None | CompletingTask::Err(_) => None, + CompletingTask::Completing { command_ctx, .. } => command_ctx.as_ref(), } .into_iter() ) @@ -426,9 +425,9 @@ impl CheckpointControl { /// We need to make sure there are no changes when doing recovery pub async fn clear_on_err(&mut self, err: &MetaError) { // join spawned completing command to finish no matter it succeeds or not. - let is_err = match replace(&mut self.completing_command, CompletingCommand::None) { - CompletingCommand::None => false, - CompletingCommand::GlobalStreamingGraph { join_handle, .. } => { + let is_err = match replace(&mut self.completing_task, CompletingTask::None) { + CompletingTask::None => false, + CompletingTask::Completing { join_handle, .. } => { info!("waiting for completing command to finish in recovery"); match join_handle.await { Err(e) => { @@ -442,7 +441,7 @@ impl CheckpointControl { Ok(Ok(_)) => false, } } - CompletingCommand::Err(_) => true, + CompletingTask::Err(_) => true, }; if !is_err { // continue to finish the pending collected barrier. @@ -509,9 +508,9 @@ impl BarrierEpochState { } } -enum CompletingCommand { +enum CompletingTask { None, - GlobalStreamingGraph { + Completing { command_ctx: Option>, table_ids_to_finish: HashSet, creating_job_epochs: Vec<(TableId, u64)>, @@ -1369,7 +1368,7 @@ impl CheckpointControl { ) -> impl Future> + 'a { // If there is no completing barrier, try to start completing the earliest barrier if // it has been collected. - if let CompletingCommand::None = &self.completing_command { + if let CompletingTask::None = &self.completing_task { if let Some(task) = self.next_complete_barrier_task(Some(scheduled_barriers)) { { let command_ctx = task @@ -1379,7 +1378,7 @@ impl CheckpointControl { let table_ids_to_finish = task.table_ids_to_finish.clone(); let creating_job_epochs = task.creating_job_epochs.clone(); let join_handle = tokio::spawn(self.context.clone().complete_barrier(task)); - self.completing_command = CompletingCommand::GlobalStreamingGraph { + self.completing_task = CompletingTask::Completing { command_ctx, join_handle, table_ids_to_finish, @@ -1393,9 +1392,7 @@ impl CheckpointControl { } async fn next_completed_barrier_inner(&mut self) -> MetaResult { - let CompletingCommand::GlobalStreamingGraph { join_handle, .. } = - &mut self.completing_command - else { + let CompletingTask::Completing { join_handle, .. } = &mut self.completing_task else { return pending().await; }; @@ -1409,15 +1406,15 @@ impl CheckpointControl { // It's important to reset the completing_command after await no matter the result is err // or not, and otherwise the join handle will be polled again after ready. let next_completing_command_status = if let Err(e) = &join_result { - CompletingCommand::Err(e.clone()) + CompletingTask::Err(e.clone()) } else { - CompletingCommand::None + CompletingTask::None }; let completed_command = - replace(&mut self.completing_command, next_completing_command_status); + replace(&mut self.completing_task, next_completing_command_status); self.hummock_version_stats = join_result?; - must_match!(completed_command, CompletingCommand::GlobalStreamingGraph { + must_match!(completed_command, CompletingTask::Completing { table_ids_to_finish, creating_job_epochs, .. From 62106c4b7ef66733b3b4488d73103906de10c689 Mon Sep 17 00:00:00 2001 From: William Wen Date: Sat, 12 Oct 2024 14:03:10 +0800 Subject: [PATCH 14/19] transit to log store in collect --- .../barrier/creating_job/barrier_control.rs | 2 + src/meta/src/barrier/creating_job/mod.rs | 124 +++++++++++------- src/meta/src/barrier/creating_job/status.rs | 106 ++++++++------- src/meta/src/barrier/mod.rs | 26 ++-- src/meta/src/barrier/state.rs | 2 + .../src/task/barrier_manager/progress.rs | 29 ++-- 6 files changed, 154 insertions(+), 135 deletions(-) diff --git a/src/meta/src/barrier/creating_job/barrier_control.rs b/src/meta/src/barrier/creating_job/barrier_control.rs index f5a0453edcf18..c63981cea1092 100644 --- a/src/meta/src/barrier/creating_job/barrier_control.rs +++ b/src/meta/src/barrier/creating_job/barrier_control.rs @@ -178,6 +178,8 @@ impl CreatingStreamingJobBarrierControl { /// Return Some((epoch, resps, `is_first_commit`)) /// /// Only epoch within the `epoch_end_bound` can be started. + /// Usually `epoch_end_bound` is the upstream committed epoch. This is to ensure that + /// the creating job won't have higher committed epoch than the upstream. pub(super) fn start_completing( &mut self, epoch_end_bound: Bound, diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/creating_job/mod.rs index ebec98e6d4eec..51700ac739986 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/creating_job/mod.rs @@ -20,6 +20,7 @@ use std::collections::HashMap; use std::ops::Bound::{Excluded, Unbounded}; use std::sync::Arc; +use risingwave_common::catalog::TableId; use risingwave_common::metrics::LabelGuardedIntGauge; use risingwave_pb::common::WorkerNode; use risingwave_pb::ddl_service::DdlProgress; @@ -168,43 +169,32 @@ impl CreatingStreamingJobControl { )) } - pub(super) fn may_inject_fake_barrier( - &mut self, + fn inject_barrier( + table_id: TableId, control_stream_manager: &mut ControlStreamManager, - is_checkpoint: bool, + barrier_control: &mut CreatingStreamingJobBarrierControl, + pre_applied_graph_info: &InflightGraphInfo, + applied_graph_info: Option<&InflightGraphInfo>, + CreatingJobInjectBarrierInfo { + curr_epoch, + prev_epoch, + kind, + new_actors, + mutation, + }: CreatingJobInjectBarrierInfo, ) -> MetaResult<()> { - if let Some(barriers_to_inject) = self.status.may_inject_fake_barrier(is_checkpoint) { - let graph_info = self - .status - .active_graph_info() - .expect("must exist when having barriers to inject"); - let table_id = self.info.table_fragments.table_id(); - for CreatingJobInjectBarrierInfo { - curr_epoch, - prev_epoch, - kind, - new_actors, - mutation, - } in barriers_to_inject - { - let node_to_collect = control_stream_manager.inject_barrier( - Some(table_id), - mutation, - (&curr_epoch, &prev_epoch), - &kind, - graph_info, - Some(graph_info), - new_actors, - vec![], - vec![], - )?; - self.barrier_control.enqueue_epoch( - prev_epoch.value().0, - node_to_collect, - kind.is_checkpoint(), - ); - } - } + let node_to_collect = control_stream_manager.inject_barrier( + Some(table_id), + mutation, + (&curr_epoch, &prev_epoch), + &kind, + pre_applied_graph_info, + applied_graph_info, + new_actors, + vec![], + vec![], + )?; + barrier_control.enqueue_epoch(prev_epoch.value().0, node_to_collect, kind.is_checkpoint()); Ok(()) } @@ -237,35 +227,52 @@ impl CreatingStreamingJobControl { ); match &mut self.status { CreatingStreamingJobStatus::ConsumingSnapshot { - pending_commands, .. + pending_commands, + prev_epoch_fake_physical_time, + pending_non_checkpoint_barriers, + initial_barrier_info, + ref graph_info, + .. } => { assert!( !start_consume_upstream, "should not start consuming upstream for a job that are consuming snapshot" ); + let new_barrier = CreatingStreamingJobStatus::new_fake_barrier( + prev_epoch_fake_physical_time, + pending_non_checkpoint_barriers, + initial_barrier_info, + command_ctx.kind.is_checkpoint(), + ); pending_commands.push(command_ctx.clone()); + Self::inject_barrier( + self.info.table_fragments.table_id(), + control_stream_manager, + &mut self.barrier_control, + graph_info, + Some(graph_info), + new_barrier, + )?; } CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } => { - let node_to_collect = control_stream_manager.inject_barrier( - Some(table_id), - None, - (&command_ctx.curr_epoch, &command_ctx.prev_epoch), - &command_ctx.kind, + Self::inject_barrier( + self.info.table_fragments.table_id(), + control_stream_manager, + &mut self.barrier_control, graph_info, if start_consume_upstream { None } else { Some(graph_info) }, - None, - vec![], - vec![], + CreatingJobInjectBarrierInfo { + curr_epoch: command_ctx.curr_epoch.clone(), + prev_epoch: command_ctx.prev_epoch.clone(), + kind: command_ctx.kind.clone(), + new_actors: None, + mutation: None, + }, )?; - self.barrier_control.enqueue_epoch( - command_ctx.prev_epoch.value().0, - node_to_collect, - command_ctx.kind.is_checkpoint(), - ); let prev_epoch = command_ctx.prev_epoch.value().0; if start_consume_upstream { info!( @@ -291,9 +298,24 @@ impl CreatingStreamingJobControl { epoch: u64, worker_id: WorkerId, resp: BarrierCompleteResponse, - ) { - self.status.update_progress(&resp.create_mview_progress); + control_stream_manager: &mut ControlStreamManager, + ) -> MetaResult<()> { + let prev_barriers_to_inject = self.status.update_progress(&resp.create_mview_progress); self.barrier_control.collect(epoch, worker_id, resp); + if let Some((prev_barriers_to_inject, graph_info)) = prev_barriers_to_inject { + let table_id = self.info.table_fragments.table_id(); + for info in prev_barriers_to_inject { + Self::inject_barrier( + table_id, + control_stream_manager, + &mut self.barrier_control, + graph_info, + Some(graph_info), + info, + )?; + } + } + Ok(()) } pub(super) fn should_merge_to_upstream(&self) -> Option { diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/creating_job/status.rs index 39825243df57d..1e4c7ce5120d8 100644 --- a/src/meta/src/barrier/creating_job/status.rs +++ b/src/meta/src/barrier/creating_job/status.rs @@ -18,6 +18,7 @@ use std::mem::take; use std::sync::Arc; use risingwave_common::hash::ActorId; +use risingwave_common::must_match; use risingwave_common::util::epoch::Epoch; use risingwave_pb::hummock::HummockVersionStats; use risingwave_pb::stream_plan::barrier_mutation::Mutation; @@ -147,11 +148,18 @@ impl CreatingStreamingJobStatus { pub(super) fn update_progress( &mut self, create_mview_progress: impl IntoIterator, - ) { + ) -> Option<(Vec, &InflightGraphInfo)> { match self { Self::ConsumingSnapshot { create_mview_tracker, ref version_stats, + prev_epoch_fake_physical_time, + pending_commands, + ref graph_info, + pending_non_checkpoint_barriers, + ref backfill_epoch, + initial_barrier_info, + ref snapshot_backfill_actors, .. } => { create_mview_tracker.update_tracking_jobs( @@ -159,47 +167,21 @@ impl CreatingStreamingJobStatus { create_mview_progress, version_stats, ); - } - CreatingStreamingJobStatus::ConsumingLogStore { - log_store_progress_tracker, - .. - } => { - log_store_progress_tracker.update(create_mview_progress); - } - CreatingStreamingJobStatus::Finishing(_) => {} - } - } - - /// return - /// - Some(vec[(`curr_epoch`, `prev_epoch`, `barrier_kind`)]) of barriers to newly inject - pub(super) fn may_inject_fake_barrier( - &mut self, - is_checkpoint: bool, - ) -> Option> { - if let CreatingStreamingJobStatus::ConsumingSnapshot { - prev_epoch_fake_physical_time, - pending_commands, - create_mview_tracker, - ref graph_info, - pending_non_checkpoint_barriers, - ref backfill_epoch, - initial_barrier_info, - ref snapshot_backfill_actors, - .. - } = self - { - if create_mview_tracker.has_pending_finished_jobs() { - assert!(initial_barrier_info.is_none()); - pending_non_checkpoint_barriers.push(*backfill_epoch); + if create_mview_tracker.has_pending_finished_jobs() { + let (new_actors, mutation) = match initial_barrier_info.take() { + Some((new_actors, mutation)) => (Some(new_actors), Some(mutation)), + None => (None, None), + }; + assert!(initial_barrier_info.is_none()); + pending_non_checkpoint_barriers.push(*backfill_epoch); - let prev_epoch = Epoch::from_physical_time(*prev_epoch_fake_physical_time); - let barriers_to_inject: Vec<_> = - [CreatingJobInjectBarrierInfo { + let prev_epoch = Epoch::from_physical_time(*prev_epoch_fake_physical_time); + let barriers_to_inject: Vec<_> = [CreatingJobInjectBarrierInfo { curr_epoch: TracedEpoch::new(Epoch(*backfill_epoch)), prev_epoch: TracedEpoch::new(prev_epoch), kind: BarrierKind::Checkpoint(take(pending_non_checkpoint_barriers)), - new_actors: None, - mutation: None, + new_actors, + mutation, }] .into_iter() .chain(pending_commands.drain(..).map(|command_ctx| { @@ -213,15 +195,39 @@ impl CreatingStreamingJobStatus { })) .collect(); - *self = CreatingStreamingJobStatus::ConsumingLogStore { - graph_info: graph_info.clone(), - log_store_progress_tracker: CreateMviewLogStoreProgressTracker::new( - snapshot_backfill_actors.iter().cloned(), - barriers_to_inject.len(), - ), - }; - Some(barriers_to_inject) - } else { + *self = CreatingStreamingJobStatus::ConsumingLogStore { + graph_info: graph_info.clone(), + log_store_progress_tracker: CreateMviewLogStoreProgressTracker::new( + snapshot_backfill_actors.iter().cloned(), + barriers_to_inject.len(), + ), + }; + let graph_info = must_match!(self, + CreatingStreamingJobStatus::ConsumingLogStore {graph_info, ..} => graph_info); + Some((barriers_to_inject, graph_info)) + } else { + None + } + } + CreatingStreamingJobStatus::ConsumingLogStore { + log_store_progress_tracker, + .. + } => { + log_store_progress_tracker.update(create_mview_progress); + None + } + CreatingStreamingJobStatus::Finishing(_) => None, + } + } + + pub(super) fn new_fake_barrier( + prev_epoch_fake_physical_time: &mut u64, + pending_non_checkpoint_barriers: &mut Vec, + initial_barrier_info: &mut Option<(HashMap>, Mutation)>, + is_checkpoint: bool, + ) -> CreatingJobInjectBarrierInfo { + { + { let prev_epoch = TracedEpoch::new(Epoch::from_physical_time(*prev_epoch_fake_physical_time)); *prev_epoch_fake_physical_time += 1; @@ -239,16 +245,14 @@ impl CreatingStreamingJobStatus { } else { Default::default() }; - Some(vec![CreatingJobInjectBarrierInfo { + CreatingJobInjectBarrierInfo { curr_epoch, prev_epoch, kind, new_actors, mutation, - }]) + } } - } else { - None } } } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index beefc3dee4ee4..e18dbf72442f3 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -327,7 +327,11 @@ impl CheckpointControl { /// Change the state of this `prev_epoch` to `Completed`. Return continuous nodes /// with `Completed` starting from first node [`Completed`..`InFlight`) and remove them. - fn barrier_collected(&mut self, resp: BarrierCompleteResponse) { + fn barrier_collected( + &mut self, + resp: BarrierCompleteResponse, + control_stream_manager: &mut ControlStreamManager, + ) -> MetaResult<()> { let worker_id = resp.worker_id; let prev_epoch = resp.epoch; tracing::trace!( @@ -351,8 +355,9 @@ impl CheckpointControl { self.creating_streaming_job_controls .get_mut(&creating_table_id) .expect("should exist") - .collect(prev_epoch, worker_id, resp); + .collect(prev_epoch, worker_id, resp, control_stream_manager)?; } + Ok(()) } /// Pause inject barrier until True. @@ -817,12 +822,8 @@ impl GlobalBarrierManager { } } (worker_id, resp_result) = self.control_stream_manager.next_complete_barrier_response() => { - match resp_result { - Ok(resp) => { - self.checkpoint_control.barrier_collected(resp); - - } - Err(e) => { + if let Err(e) = resp_result.and_then(|resp| self.checkpoint_control.barrier_collected(resp, &mut self.control_stream_manager)) { + { let failed_command = self.checkpoint_control.command_wait_collect_from_worker(worker_id); if failed_command.is_some() || self.state.inflight_graph_info.contains_worker(worker_id) @@ -962,15 +963,6 @@ impl GlobalBarrierManager { ); } - // may inject fake barrier - for creating_job in self - .checkpoint_control - .creating_streaming_job_controls - .values_mut() - { - creating_job.may_inject_fake_barrier(&mut self.control_stream_manager, checkpoint)? - } - self.pending_non_checkpoint_barriers .push(prev_epoch.value().0); let kind = if checkpoint { diff --git a/src/meta/src/barrier/state.rs b/src/meta/src/barrier/state.rs index 1edd924a393f6..d9fe6f13c963c 100644 --- a/src/meta/src/barrier/state.rs +++ b/src/meta/src/barrier/state.rs @@ -86,6 +86,8 @@ impl BarrierManagerState { /// Returns the inflight actor infos that have included the newly added actors in the given command. The dropped actors /// will be removed from the state after the info get resolved. + /// + /// Return (`graph_info`, `subscription_info`, `table_ids_to_commit`, `jobs_to_wait`) pub fn apply_command( &mut self, command: &Command, diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 5a34dd7448e70..a91f9b8476111 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +use std::assert_matches::assert_matches; use std::fmt::{Display, Formatter}; use risingwave_common::util::epoch::EpochPair; @@ -229,15 +230,13 @@ impl CreateMviewProgressReporter { epoch: EpochPair, pending_barrier_num: usize, ) { - match self.state { + assert_matches!( + self.state, Some(BackfillState::DoneConsumingUpstreamTable(_)) - | Some(BackfillState::ConsumingLogStore { .. }) => { - // valid state - } - state => { - panic!("cannot update log store progress at state {:?}", state) - } - } + | Some(BackfillState::ConsumingLogStore { .. }), + "cannot update log store progress at state {:?}", + self.state + ); self.update_inner( epoch, BackfillState::ConsumingLogStore { @@ -247,15 +246,13 @@ impl CreateMviewProgressReporter { } pub(crate) fn finish_consuming_log_store(&mut self, epoch: EpochPair) { - match self.state { + assert_matches!( + self.state, Some(BackfillState::DoneConsumingUpstreamTable(_)) - | Some(BackfillState::ConsumingLogStore { .. }) => { - // valid state - } - state => { - panic!("cannot finish log store progress at state {:?}", state) - } - } + | Some(BackfillState::ConsumingLogStore { .. }), + "cannot finish log store progress at state {:?}", + self.state + ); self.update_inner(epoch, BackfillState::DoneConsumingLogStore); } } From 22a5e3a30aa0777ef19802d751acdf2892bf3dce Mon Sep 17 00:00:00 2001 From: William Wen Date: Sat, 12 Oct 2024 14:13:23 +0800 Subject: [PATCH 15/19] add comment --- src/stream/src/executor/backfill/snapshot_backfill.rs | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index 6a740eafa0503..ffc010d139adc 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -463,8 +463,14 @@ impl<'a> UpstreamBuffer<'a, StateOfConsumingLogStore> { async fn next_barrier(&mut self) -> StreamExecutorResult> { Ok( if let Some(barrier) = self.state.upstream_pending_barriers.pop_back() { - // only update the `max_pending_barrier_num` on checkpoint barrier to avoid deadlock caused by - // downstream waiting on a checkpoint barrier to bump up. + // Only update the `max_pending_barrier_num` on checkpoint barrier to avoid deadlock. + // + // After updating and decreasing `max_pending_barrier_num`, we won't poll upstream until + // the downstream fetches the next barrier, and this causes back pressure to upstream. + // However, downstream will be blocked at `try_wait_epoch` on a non-checkpoint epoch on upstream, + // and `try_wait_epoch` can only be unblocked when the next checkpoint epoch is processed. But since + // the upstream is blocked by back pressure, it cannot process the next checkpoint epoch, which + // causes deadlock. if barrier.kind.is_checkpoint() { self.state.max_pending_barrier_num = min( self.state.upstream_pending_barriers.len(), From f3c28f9ffbafcd9f71b248a8636a3c3282885b37 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 14 Oct 2024 12:43:24 +0800 Subject: [PATCH 16/19] remove trait --- .../executor/backfill/snapshot_backfill.rs | 116 ++++++------------ 1 file changed, 39 insertions(+), 77 deletions(-) diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index ffc010d139adc..d3060ee69d4f4 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -160,12 +160,11 @@ impl SnapshotBackfillExecutor { yield Message::Barrier(recv_barrier); } - // TODO: may set the `max_pending_barrier_num_gap` by config - let mut upstream_buffer = upstream_buffer.start_consuming_log_store(10); + let mut upstream_buffer = upstream_buffer.start_consuming_log_store(); let mut barrier_epoch = first_barrier_epoch; - let initial_pending_barrier = upstream_buffer.state.barrier_count(); + let initial_pending_barrier = upstream_buffer.barrier_count(); info!( ?barrier_epoch, table_id = self.upstream_table.table_id().table_id, @@ -217,7 +216,7 @@ impl SnapshotBackfillExecutor { self.progress.update_create_mview_log_store_progress( barrier.epoch, - upstream_buffer.state.barrier_count(), + upstream_buffer.barrier_count(), ); yield Message::Barrier(barrier); @@ -312,45 +311,14 @@ async fn read_change_log( } } -trait UpstreamBufferState { - /// Return whether to be able to consume upstream. This is used to control backpressure to upstream - fn can_consume_upstream(&self) -> bool; - fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier); -} - -struct StateOfConsumingSnapshot { - pending_barriers: Vec, -} - -impl UpstreamBufferState for StateOfConsumingSnapshot { - fn can_consume_upstream(&self) -> bool { - // no backpressure to upstream when consuming snapshot - true - } - - fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier) { - self.pending_barriers.push(upstream_barrier) - } -} - -struct StateOfConsumingLogStore { - /// Barriers received from upstream but not yet received the barrier from local barrier worker - /// newer barrier at the front - upstream_pending_barriers: VecDeque, - max_pending_barrier_num: usize, - /// The amount of `max_pending_barrier_num` to decrease whenever emitting a checkpoint barrier to downstream - max_pending_barrier_num_gap: usize, -} - -impl StateOfConsumingLogStore { - fn barrier_count(&self) -> usize { - self.upstream_pending_barriers.len() - } -} +struct ConsumingSnapshot; +struct ConsumingLogStore; -impl UpstreamBufferState for StateOfConsumingLogStore { +impl UpstreamBuffer<'_, S> { fn can_consume_upstream(&self) -> bool { - if let Some(prev_barrier) = self.upstream_pending_barriers.front() + if self.is_polling_epoch_data { + true + } else if let Some(prev_barrier) = self.upstream_pending_barriers.front() && prev_barrier.kind.is_barrier() { // allow consuming upstream when the barrier is non-checkpoint barrier to avoid deadlock @@ -359,24 +327,24 @@ impl UpstreamBufferState for StateOfConsumingLogStore { self.upstream_pending_barriers.len() < self.max_pending_barrier_num } } - - fn on_upstream_barrier(&mut self, upstream_barrier: DispatcherBarrier) { - self.upstream_pending_barriers.push_front(upstream_barrier); - } } struct UpstreamBuffer<'a, S> { upstream: &'a mut MergeExecutorInput, - state: S, + max_pending_barrier_num: usize, + /// Barriers received from upstream but not yet received the barrier from local barrier worker + /// newer barrier at the front + upstream_pending_barriers: VecDeque, /// Whether we have started polling any upstream data before the next barrier. /// When `true`, we should continue polling until the next barrier, because /// some data in this epoch have been discarded and data in this epoch /// must be read from log store is_polling_epoch_data: bool, consume_upstream_row_count: LabelGuardedIntCounter<3>, + _phase: S, } -impl<'a> UpstreamBuffer<'a, StateOfConsumingSnapshot> { +impl<'a> UpstreamBuffer<'a, ConsumingSnapshot> { fn new( upstream: &'a mut MergeExecutorInput, consume_upstream_row_count: LabelGuardedIntCounter<3>, @@ -384,47 +352,38 @@ impl<'a> UpstreamBuffer<'a, StateOfConsumingSnapshot> { Self { upstream, is_polling_epoch_data: false, - state: StateOfConsumingSnapshot { - pending_barriers: vec![], - }, consume_upstream_row_count, + upstream_pending_barriers: Default::default(), + // no limit on the number of pending barrier in the beginning + max_pending_barrier_num: usize::MAX, + _phase: ConsumingSnapshot {}, } } - fn start_consuming_log_store( - self, - max_pending_barrier_num_gap: usize, - ) -> UpstreamBuffer<'a, StateOfConsumingLogStore> { - let StateOfConsumingSnapshot { pending_barriers } = self.state; - let mut upstream_pending_barriers = VecDeque::with_capacity(pending_barriers.len()); - for pending_barrier in pending_barriers { - upstream_pending_barriers.push_front(pending_barrier); - } - let max_pending_barrier_num = upstream_pending_barriers.len(); + fn start_consuming_log_store(self) -> UpstreamBuffer<'a, ConsumingLogStore> { + let max_pending_barrier_num = self.barrier_count(); UpstreamBuffer { upstream: self.upstream, - state: StateOfConsumingLogStore { - upstream_pending_barriers, - max_pending_barrier_num, - max_pending_barrier_num_gap, - }, + upstream_pending_barriers: self.upstream_pending_barriers, + max_pending_barrier_num, is_polling_epoch_data: self.is_polling_epoch_data, consume_upstream_row_count: self.consume_upstream_row_count, + _phase: ConsumingLogStore {}, } } } -impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { +impl<'a, S> UpstreamBuffer<'a, S> { async fn concurrently_consume_upstream(&mut self) -> StreamExecutorError { { loop { if let Err(e) = try { - if !self.state.can_consume_upstream() && !self.is_polling_epoch_data { + if !self.can_consume_upstream() { // pause the future to block consuming upstream return pending().await; } let barrier = self.consume_until_next_barrier().await?; - self.state.on_upstream_barrier(barrier); + self.upstream_pending_barriers.push_front(barrier); } { break e; } @@ -459,10 +418,10 @@ impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { } } -impl<'a> UpstreamBuffer<'a, StateOfConsumingLogStore> { +impl<'a> UpstreamBuffer<'a, ConsumingLogStore> { async fn next_barrier(&mut self) -> StreamExecutorResult> { Ok( - if let Some(barrier) = self.state.upstream_pending_barriers.pop_back() { + if let Some(barrier) = self.upstream_pending_barriers.pop_back() { // Only update the `max_pending_barrier_num` on checkpoint barrier to avoid deadlock. // // After updating and decreasing `max_pending_barrier_num`, we won't poll upstream until @@ -472,16 +431,15 @@ impl<'a> UpstreamBuffer<'a, StateOfConsumingLogStore> { // the upstream is blocked by back pressure, it cannot process the next checkpoint epoch, which // causes deadlock. if barrier.kind.is_checkpoint() { - self.state.max_pending_barrier_num = min( - self.state.upstream_pending_barriers.len(), - self.state - .max_pending_barrier_num - .saturating_sub(self.state.max_pending_barrier_num_gap), + // sub(1) to ensure that the lag is monotonically decreasing. + self.max_pending_barrier_num = min( + self.upstream_pending_barriers.len(), + self.max_pending_barrier_num.saturating_sub(1), ) } Some(barrier) } else { - self.state.max_pending_barrier_num = 0; + self.max_pending_barrier_num = 0; if self.is_polling_epoch_data { let barrier = self.consume_until_next_barrier().await?; Some(barrier) @@ -493,7 +451,7 @@ impl<'a> UpstreamBuffer<'a, StateOfConsumingLogStore> { } } -impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { +impl<'a, S> UpstreamBuffer<'a, S> { /// Run a future while concurrently polling the upstream so that the upstream /// won't be back-pressured. async fn run_future>( @@ -511,6 +469,10 @@ impl<'a, S: UpstreamBufferState> UpstreamBuffer<'a, S> { } } } + + fn barrier_count(&self) -> usize { + self.upstream_pending_barriers.len() + } } async fn receive_next_barrier( From 8f54f206e58f70b7a25edbb785e6b479b3c0f920 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 14 Oct 2024 13:40:13 +0800 Subject: [PATCH 17/19] refactor to store pending barriers --- .../executor/backfill/snapshot_backfill.rs | 165 +++++++++--------- 1 file changed, 82 insertions(+), 83 deletions(-) diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs index d3060ee69d4f4..eb1325141fdfd 100644 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -15,7 +15,7 @@ use std::cmp::min; use std::collections::VecDeque; use std::future::{pending, Future}; -use std::mem::replace; +use std::mem::{replace, take}; use std::sync::Arc; use anyhow::anyhow; @@ -182,44 +182,50 @@ impl SnapshotBackfillExecutor { ]); // Phase 2: consume upstream log store - while let Some(upstream_barrier) = upstream_buffer.next_barrier().await? { - let barrier = receive_next_barrier(&mut self.barrier_rx).await?; - assert_eq!(upstream_barrier.epoch, barrier.epoch); - assert_eq!(barrier_epoch.curr, barrier.epoch.prev); - barrier_epoch = barrier.epoch; - - debug!(?barrier_epoch, kind = ?barrier.kind, "before consume change log"); - // use `upstream_buffer.run_future` to poll upstream concurrently so that we won't have back-pressure - // on the upstream. Otherwise, in `batch_iter_log_with_pk_bounds`, we may wait upstream epoch to be committed, - // and the back-pressure may cause the upstream unable to consume the barrier and then cause deadlock. - let stream = upstream_buffer - .run_future(self.upstream_table.batch_iter_log_with_pk_bounds( - barrier_epoch.prev, - HummockReadEpoch::Committed(barrier_epoch.prev), - )) - .await?; - let data_types = self.upstream_table.schema().data_types(); - let builder = create_builder(None, self.chunk_size, data_types); - let stream = read_change_log(stream, builder); - pin_mut!(stream); - while let Some(chunk) = upstream_buffer.run_future(stream.try_next()).await? { - debug!( - ?barrier_epoch, - size = chunk.cardinality(), - "consume change log yield chunk", - ); - consuming_log_store_row_count.inc_by(chunk.cardinality() as _); - yield Message::Chunk(chunk); - } + while let Some(upstream_barriers) = + upstream_buffer.next_checkpoint_barrier().await? + { + for upstream_barrier in upstream_barriers { + let barrier = receive_next_barrier(&mut self.barrier_rx).await?; + assert_eq!(upstream_barrier.epoch, barrier.epoch); + assert_eq!(barrier_epoch.curr, barrier.epoch.prev); + barrier_epoch = barrier.epoch; + + debug!(?barrier_epoch, kind = ?barrier.kind, "before consume change log"); + // use `upstream_buffer.run_future` to poll upstream concurrently so that we won't have back-pressure + // on the upstream. Otherwise, in `batch_iter_log_with_pk_bounds`, we may wait upstream epoch to be committed, + // and the back-pressure may cause the upstream unable to consume the barrier and then cause deadlock. + let stream = upstream_buffer + .run_future(self.upstream_table.batch_iter_log_with_pk_bounds( + barrier_epoch.prev, + HummockReadEpoch::Committed(barrier_epoch.prev), + )) + .await?; + let data_types = self.upstream_table.schema().data_types(); + let builder = create_builder(None, self.chunk_size, data_types); + let stream = read_change_log(stream, builder); + pin_mut!(stream); + while let Some(chunk) = + upstream_buffer.run_future(stream.try_next()).await? + { + debug!( + ?barrier_epoch, + size = chunk.cardinality(), + "consume change log yield chunk", + ); + consuming_log_store_row_count.inc_by(chunk.cardinality() as _); + yield Message::Chunk(chunk); + } - debug!(?barrier_epoch, "after consume change log"); + debug!(?barrier_epoch, "after consume change log"); - self.progress.update_create_mview_log_store_progress( - barrier.epoch, - upstream_buffer.barrier_count(), - ); + self.progress.update_create_mview_log_store_progress( + barrier.epoch, + upstream_buffer.barrier_count(), + ); - yield Message::Barrier(barrier); + yield Message::Barrier(barrier); + } } info!( @@ -314,27 +320,16 @@ async fn read_change_log( struct ConsumingSnapshot; struct ConsumingLogStore; -impl UpstreamBuffer<'_, S> { - fn can_consume_upstream(&self) -> bool { - if self.is_polling_epoch_data { - true - } else if let Some(prev_barrier) = self.upstream_pending_barriers.front() - && prev_barrier.kind.is_barrier() - { - // allow consuming upstream when the barrier is non-checkpoint barrier to avoid deadlock - true - } else { - self.upstream_pending_barriers.len() < self.max_pending_barrier_num - } - } -} - struct UpstreamBuffer<'a, S> { upstream: &'a mut MergeExecutorInput, - max_pending_barrier_num: usize, - /// Barriers received from upstream but not yet received the barrier from local barrier worker - /// newer barrier at the front - upstream_pending_barriers: VecDeque, + max_pending_checkpoint_barrier_num: usize, + pending_non_checkpoint_barriers: Vec, + /// Barriers received from upstream but not yet received the barrier from local barrier worker. + /// + /// In the outer `VecDeque`, newer barriers at the front. + /// In the inner `Vec`, newer barrier at the back, with the last barrier as checkpoint barrier, + /// and others as non-checkpoint barrier + upstream_pending_barriers: VecDeque>, /// Whether we have started polling any upstream data before the next barrier. /// When `true`, we should continue polling until the next barrier, because /// some data in this epoch have been discarded and data in this epoch @@ -353,9 +348,10 @@ impl<'a> UpstreamBuffer<'a, ConsumingSnapshot> { upstream, is_polling_epoch_data: false, consume_upstream_row_count, + pending_non_checkpoint_barriers: vec![], upstream_pending_barriers: Default::default(), // no limit on the number of pending barrier in the beginning - max_pending_barrier_num: usize::MAX, + max_pending_checkpoint_barrier_num: usize::MAX, _phase: ConsumingSnapshot {}, } } @@ -364,8 +360,9 @@ impl<'a> UpstreamBuffer<'a, ConsumingSnapshot> { let max_pending_barrier_num = self.barrier_count(); UpstreamBuffer { upstream: self.upstream, + pending_non_checkpoint_barriers: self.pending_non_checkpoint_barriers, upstream_pending_barriers: self.upstream_pending_barriers, - max_pending_barrier_num, + max_pending_checkpoint_barrier_num: max_pending_barrier_num, is_polling_epoch_data: self.is_polling_epoch_data, consume_upstream_row_count: self.consume_upstream_row_count, _phase: ConsumingLogStore {}, @@ -378,11 +375,13 @@ impl<'a, S> UpstreamBuffer<'a, S> { { loop { if let Err(e) = try { - if !self.can_consume_upstream() { + if self.upstream_pending_barriers.len() + >= self.max_pending_checkpoint_barrier_num + { // pause the future to block consuming upstream return pending().await; } - let barrier = self.consume_until_next_barrier().await?; + let barrier = self.consume_until_next_checkpoint_barrier().await?; self.upstream_pending_barriers.push_front(barrier); } { break e; @@ -393,7 +392,9 @@ impl<'a, S> UpstreamBuffer<'a, S> { /// Consume the upstream until seeing the next barrier. /// `pending_barriers` must be non-empty after this method returns. - async fn consume_until_next_barrier(&mut self) -> StreamExecutorResult { + async fn consume_until_next_checkpoint_barrier( + &mut self, + ) -> StreamExecutorResult> { loop { let msg: DispatcherMessage = self .upstream @@ -407,8 +408,14 @@ impl<'a, S> UpstreamBuffer<'a, S> { .inc_by(chunk.cardinality() as _); } DispatcherMessage::Barrier(barrier) => { - self.is_polling_epoch_data = false; - break Ok(barrier); + let is_checkpoint = barrier.kind.is_checkpoint(); + self.pending_non_checkpoint_barriers.push(barrier); + if is_checkpoint { + self.is_polling_epoch_data = false; + break Ok(take(&mut self.pending_non_checkpoint_barriers)); + } else { + self.is_polling_epoch_data = true; + } } DispatcherMessage::Watermark(_) => { self.is_polling_epoch_data = true; @@ -419,30 +426,22 @@ impl<'a, S> UpstreamBuffer<'a, S> { } impl<'a> UpstreamBuffer<'a, ConsumingLogStore> { - async fn next_barrier(&mut self) -> StreamExecutorResult> { + async fn next_checkpoint_barrier( + &mut self, + ) -> StreamExecutorResult>> { Ok( - if let Some(barrier) = self.upstream_pending_barriers.pop_back() { - // Only update the `max_pending_barrier_num` on checkpoint barrier to avoid deadlock. - // - // After updating and decreasing `max_pending_barrier_num`, we won't poll upstream until - // the downstream fetches the next barrier, and this causes back pressure to upstream. - // However, downstream will be blocked at `try_wait_epoch` on a non-checkpoint epoch on upstream, - // and `try_wait_epoch` can only be unblocked when the next checkpoint epoch is processed. But since - // the upstream is blocked by back pressure, it cannot process the next checkpoint epoch, which - // causes deadlock. - if barrier.kind.is_checkpoint() { - // sub(1) to ensure that the lag is monotonically decreasing. - self.max_pending_barrier_num = min( - self.upstream_pending_barriers.len(), - self.max_pending_barrier_num.saturating_sub(1), - ) - } - Some(barrier) + if let Some(barriers) = self.upstream_pending_barriers.pop_back() { + // sub(1) to ensure that the lag is monotonically decreasing. + self.max_pending_checkpoint_barrier_num = min( + self.upstream_pending_barriers.len(), + self.max_pending_checkpoint_barrier_num.saturating_sub(1), + ); + Some(barriers) } else { - self.max_pending_barrier_num = 0; + self.max_pending_checkpoint_barrier_num = 0; if self.is_polling_epoch_data { - let barrier = self.consume_until_next_barrier().await?; - Some(barrier) + let barriers = self.consume_until_next_checkpoint_barrier().await?; + Some(barriers) } else { None } From af34fd8ce8761dd6f107ea3e28c919b627481d21 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 14 Oct 2024 15:04:28 +0800 Subject: [PATCH 18/19] fix --- src/meta/src/barrier/mod.rs | 5 ++++- src/meta/src/hummock/manager/transaction.rs | 9 --------- 2 files changed, 4 insertions(+), 10 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 038aef4a550a9..27b83fcd87dfe 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -245,7 +245,10 @@ impl CheckpointControl { fn total_command_num(&self) -> usize { self.command_ctx_queue.len() + match &self.completing_task { - CompletingTask::Completing { .. } => 1, + CompletingTask::Completing { + command_ctx: Some(_), + .. + } => 1, _ => 0, } } diff --git a/src/meta/src/hummock/manager/transaction.rs b/src/meta/src/hummock/manager/transaction.rs index 5c2fc529dd754..a8d3645d29037 100644 --- a/src/meta/src/hummock/manager/transaction.rs +++ b/src/meta/src/hummock/manager/transaction.rs @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::cell::LazyCell; use std::collections::{BTreeMap, HashMap}; use std::ops::{Deref, DerefMut}; use std::sync::Arc; @@ -146,14 +145,6 @@ impl<'a> HummockVersionTransaction<'a> { } } - let max_epoch_to_commit = LazyCell::new(|| { - tables_to_commit - .values() - .cloned() - .max() - .expect("non empty tables_to_commit") - }); - // Append SSTs to a new version. for (compaction_group_id, inserted_table_infos) in commit_sstables { let group_deltas = &mut new_version_delta From 80df8075fcc307001af62ec11640af8a142a7e47 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 17 Oct 2024 18:45:57 +0800 Subject: [PATCH 19/19] address comment --- src/meta/src/barrier/creating_job/mod.rs | 130 +++++++------------- src/meta/src/barrier/creating_job/status.rs | 105 +++++++++++----- 2 files changed, 116 insertions(+), 119 deletions(-) diff --git a/src/meta/src/barrier/creating_job/mod.rs b/src/meta/src/barrier/creating_job/mod.rs index 783f74275cde9..0598cd319c590 100644 --- a/src/meta/src/barrier/creating_job/mod.rs +++ b/src/meta/src/barrier/creating_job/mod.rs @@ -48,6 +48,8 @@ pub(super) struct CreatingStreamingJobControl { pub(super) snapshot_backfill_info: SnapshotBackfillInfo, backfill_epoch: u64, + graph_info: InflightGraphInfo, + barrier_control: CreatingStreamingJobBarrierControl, status: CreatingStreamingJobStatus, @@ -87,13 +89,13 @@ impl CreatingStreamingJobControl { metrics, ), backfill_epoch, + graph_info: InflightGraphInfo::new(fragment_info), status: CreatingStreamingJobStatus::ConsumingSnapshot { prev_epoch_fake_physical_time: 0, - pending_commands: vec![], + pending_upstream_barriers: vec![], version_stats: version_stat.clone(), create_mview_tracker, snapshot_backfill_actors, - graph_info: InflightGraphInfo::new(fragment_info), backfill_epoch, pending_non_checkpoint_barriers: vec![], initial_barrier_info: Some((actors_to_create, initial_mutation)), @@ -106,17 +108,11 @@ impl CreatingStreamingJobControl { pub(super) fn is_wait_on_worker(&self, worker_id: WorkerId) -> bool { self.barrier_control.is_wait_on_worker(worker_id) - || self - .status - .active_graph_info() - .map(|info| info.contains_worker(worker_id)) - .unwrap_or(false) + || (self.status.is_finishing() && self.graph_info.contains_worker(worker_id)) } pub(super) fn on_new_worker_node_map(&self, node_map: &HashMap) { - if let Some(info) = self.status.active_graph_info() { - info.on_new_worker_node_map(node_map) - } + self.graph_info.on_new_worker_node_map(node_map) } pub(super) fn gen_ddl_progress(&self) -> DdlProgress { @@ -159,14 +155,15 @@ impl CreatingStreamingJobControl { } pub(super) fn pinned_upstream_log_epoch(&self) -> Option { - if matches!(&self.status, CreatingStreamingJobStatus::Finishing(_)) { - return None; + if self.status.is_finishing() { + None + } else { + // TODO: when supporting recoverable snapshot backfill, we should use the max epoch that has committed + Some(max( + self.barrier_control.max_collected_epoch().unwrap_or(0), + self.backfill_epoch, + )) } - // TODO: when supporting recoverable snapshot backfill, we should use the max epoch that has committed - Some(max( - self.barrier_control.max_collected_epoch().unwrap_or(0), - self.backfill_epoch, - )) } fn inject_barrier( @@ -212,6 +209,13 @@ impl CreatingStreamingJobControl { } else { false }; + if start_consume_upstream { + info!( + table_id = self.info.table_fragments.table_id().table_id, + prev_epoch = command_ctx.prev_epoch.value().0, + "start consuming upstream" + ); + } let progress_epoch = if let Some(max_collected_epoch) = self.barrier_control.max_collected_epoch() { max(max_collected_epoch, self.backfill_epoch) @@ -225,71 +229,23 @@ impl CreatingStreamingJobControl { .0 .saturating_sub(progress_epoch) as _, ); - match &mut self.status { - CreatingStreamingJobStatus::ConsumingSnapshot { - pending_commands, - prev_epoch_fake_physical_time, - pending_non_checkpoint_barriers, - initial_barrier_info, - ref graph_info, - .. - } => { - assert!( - !start_consume_upstream, - "should not start consuming upstream for a job that are consuming snapshot" - ); - let new_barrier = CreatingStreamingJobStatus::new_fake_barrier( - prev_epoch_fake_physical_time, - pending_non_checkpoint_barriers, - initial_barrier_info, - command_ctx.kind.is_checkpoint(), - ); - pending_commands.push(command_ctx.clone()); - Self::inject_barrier( - self.info.table_fragments.table_id(), - control_stream_manager, - &mut self.barrier_control, - graph_info, - Some(graph_info), - new_barrier, - )?; - } - CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } => { - Self::inject_barrier( - self.info.table_fragments.table_id(), - control_stream_manager, - &mut self.barrier_control, - graph_info, - if start_consume_upstream { - None - } else { - Some(graph_info) - }, - CreatingJobInjectBarrierInfo { - curr_epoch: command_ctx.curr_epoch.clone(), - prev_epoch: command_ctx.prev_epoch.clone(), - kind: command_ctx.kind.clone(), - new_actors: None, - mutation: None, - }, - )?; - let prev_epoch = command_ctx.prev_epoch.value().0; + if let Some(barrier_to_inject) = self + .status + .on_new_upstream_epoch(command_ctx, start_consume_upstream) + { + Self::inject_barrier( + self.info.table_fragments.table_id(), + control_stream_manager, + &mut self.barrier_control, + &self.graph_info, if start_consume_upstream { - info!( - table_id = self.info.table_fragments.table_id().table_id, - prev_epoch, "start consuming upstream" - ); - assert!(command_ctx.kind.is_checkpoint()); - self.status = CreatingStreamingJobStatus::Finishing(prev_epoch); - } - } - CreatingStreamingJobStatus::Finishing { .. } => { - assert!( - !start_consume_upstream, - "should not start consuming upstream for a job again" - ); - } - }; + None + } else { + Some(&self.graph_info) + }, + barrier_to_inject, + )?; + } Ok(()) } @@ -302,15 +258,15 @@ impl CreatingStreamingJobControl { ) -> MetaResult<()> { let prev_barriers_to_inject = self.status.update_progress(&resp.create_mview_progress); self.barrier_control.collect(epoch, worker_id, resp); - if let Some((prev_barriers_to_inject, graph_info)) = prev_barriers_to_inject { + if let Some(prev_barriers_to_inject) = prev_barriers_to_inject { let table_id = self.info.table_fragments.table_id(); for info in prev_barriers_to_inject { Self::inject_barrier( table_id, control_stream_manager, &mut self.barrier_control, - graph_info, - Some(graph_info), + &self.graph_info, + Some(&self.graph_info), info, )?; } @@ -320,12 +276,11 @@ impl CreatingStreamingJobControl { pub(super) fn should_merge_to_upstream(&self) -> Option { if let CreatingStreamingJobStatus::ConsumingLogStore { - graph_info, ref log_store_progress_tracker, } = &self.status && log_store_progress_tracker.is_finished() { - Some(graph_info.clone()) + Some(self.graph_info.clone()) } else { None } @@ -392,7 +347,6 @@ impl CreatingStreamingJobControl { } pub(super) fn is_finished(&self) -> bool { - self.barrier_control.is_empty() - && matches!(&self.status, CreatingStreamingJobStatus::Finishing { .. }) + self.barrier_control.is_empty() && self.status.is_finishing() } } diff --git a/src/meta/src/barrier/creating_job/status.rs b/src/meta/src/barrier/creating_job/status.rs index 5a4967a9192d9..093747249f1df 100644 --- a/src/meta/src/barrier/creating_job/status.rs +++ b/src/meta/src/barrier/creating_job/status.rs @@ -18,7 +18,6 @@ use std::mem::take; use std::sync::Arc; use risingwave_common::hash::ActorId; -use risingwave_common::must_match; use risingwave_common::util::epoch::Epoch; use risingwave_meta_model_v2::WorkerId; use risingwave_pb::hummock::HummockVersionStats; @@ -30,7 +29,6 @@ use risingwave_pb::stream_service::barrier_complete_response::{ use tracing::warn; use crate::barrier::command::CommandContext; -use crate::barrier::info::InflightGraphInfo; use crate::barrier::progress::CreateMviewProgressTracker; use crate::barrier::{BarrierKind, TracedEpoch}; @@ -100,12 +98,14 @@ impl CreateMviewLogStoreProgressTracker { #[derive(Debug)] pub(super) enum CreatingStreamingJobStatus { + /// The creating job is consuming upstream snapshot. + /// Will transit to `ConsumingLogStore` on `update_progress` when + /// the snapshot has been fully consumed after `update_progress`. ConsumingSnapshot { prev_epoch_fake_physical_time: u64, - pending_commands: Vec>, + pending_upstream_barriers: Vec<(TracedEpoch, TracedEpoch, BarrierKind)>, version_stats: HummockVersionStats, create_mview_tracker: CreateMviewProgressTracker, - graph_info: InflightGraphInfo, snapshot_backfill_actors: HashSet, backfill_epoch: u64, /// The `prev_epoch` of pending non checkpoint barriers @@ -114,8 +114,10 @@ pub(super) enum CreatingStreamingJobStatus { /// Take the mutation out when injecting the first barrier initial_barrier_info: Option<(HashMap>, Mutation)>, }, + /// The creating job is consuming log store. + /// + /// Will transit to `Finishing` on `on_new_upstream_epoch` when `start_consume_upstream` is `true`. ConsumingLogStore { - graph_info: InflightGraphInfo, log_store_progress_tracker: CreateMviewLogStoreProgressTracker, }, /// All backfill actors have started consuming upstream, and the job @@ -133,29 +135,16 @@ pub(super) struct CreatingJobInjectBarrierInfo { } impl CreatingStreamingJobStatus { - pub(super) fn active_graph_info(&self) -> Option<&InflightGraphInfo> { - match self { - CreatingStreamingJobStatus::ConsumingSnapshot { graph_info, .. } - | CreatingStreamingJobStatus::ConsumingLogStore { graph_info, .. } => Some(graph_info), - CreatingStreamingJobStatus::Finishing(_) => { - // when entering `Finishing`, the graph will have been added to the upstream graph, - // and therefore the separate graph info is inactive. - None - } - } - } - pub(super) fn update_progress( &mut self, create_mview_progress: impl IntoIterator, - ) -> Option<(Vec, &InflightGraphInfo)> { + ) -> Option> { match self { Self::ConsumingSnapshot { create_mview_tracker, ref version_stats, prev_epoch_fake_physical_time, - pending_commands, - ref graph_info, + pending_upstream_barriers, pending_non_checkpoint_barriers, ref backfill_epoch, initial_barrier_info, @@ -184,27 +173,24 @@ impl CreatingStreamingJobStatus { mutation, }] .into_iter() - .chain(pending_commands.drain(..).map(|command_ctx| { - CreatingJobInjectBarrierInfo { - curr_epoch: command_ctx.curr_epoch.clone(), - prev_epoch: command_ctx.prev_epoch.clone(), - kind: command_ctx.kind.clone(), + .chain(pending_upstream_barriers.drain(..).map( + |(prev_epoch, curr_epoch, kind)| CreatingJobInjectBarrierInfo { + curr_epoch, + prev_epoch, + kind, new_actors: None, mutation: None, - } - })) + }, + )) .collect(); *self = CreatingStreamingJobStatus::ConsumingLogStore { - graph_info: graph_info.clone(), log_store_progress_tracker: CreateMviewLogStoreProgressTracker::new( snapshot_backfill_actors.iter().cloned(), barriers_to_inject.len(), ), }; - let graph_info = must_match!(self, - CreatingStreamingJobStatus::ConsumingLogStore {graph_info, ..} => graph_info); - Some((barriers_to_inject, graph_info)) + Some(barriers_to_inject) } else { None } @@ -220,6 +206,59 @@ impl CreatingStreamingJobStatus { } } + pub(super) fn on_new_upstream_epoch( + &mut self, + command_ctx: &Arc, + start_consume_upstream: bool, + ) -> Option { + match self { + CreatingStreamingJobStatus::ConsumingSnapshot { + pending_upstream_barriers, + prev_epoch_fake_physical_time, + pending_non_checkpoint_barriers, + initial_barrier_info, + .. + } => { + assert!( + !start_consume_upstream, + "should not start consuming upstream for a job that are consuming snapshot" + ); + pending_upstream_barriers.push(( + command_ctx.prev_epoch.clone(), + command_ctx.curr_epoch.clone(), + command_ctx.kind.clone(), + )); + Some(CreatingStreamingJobStatus::new_fake_barrier( + prev_epoch_fake_physical_time, + pending_non_checkpoint_barriers, + initial_barrier_info, + command_ctx.kind.is_checkpoint(), + )) + } + CreatingStreamingJobStatus::ConsumingLogStore { .. } => { + let prev_epoch = command_ctx.prev_epoch.value().0; + if start_consume_upstream { + assert!(command_ctx.kind.is_checkpoint()); + *self = CreatingStreamingJobStatus::Finishing(prev_epoch); + } + Some(CreatingJobInjectBarrierInfo { + curr_epoch: command_ctx.curr_epoch.clone(), + prev_epoch: command_ctx.prev_epoch.clone(), + kind: command_ctx.kind.clone(), + new_actors: None, + mutation: None, + }) + } + CreatingStreamingJobStatus::Finishing { .. } => { + assert!( + !start_consume_upstream, + "should not start consuming upstream for a job again" + ); + None + } + } + } + pub(super) fn new_fake_barrier( prev_epoch_fake_physical_time: &mut u64, pending_non_checkpoint_barriers: &mut Vec, @@ -255,4 +294,8 @@ impl CreatingStreamingJobStatus { } } } + + pub(super) fn is_finishing(&self) -> bool { + matches!(self, Self::Finishing(_)) + } }