From f983579f4d34ff3e3d5cc2094e5fabf993a2c247 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 19 Jul 2024 14:51:56 +0800 Subject: [PATCH 01/19] feat: support inject and barrier from partial graph --- proto/stream_service.proto | 13 +- src/meta/src/barrier/mod.rs | 40 ++- src/meta/src/barrier/recovery.rs | 11 +- src/meta/src/barrier/rpc.rs | 78 ++--- .../executor/backfill/arrangement_backfill.rs | 4 +- .../src/executor/backfill/cdc/cdc_backfill.rs | 2 +- .../executor/backfill/no_shuffle_backfill.rs | 4 +- .../executor/backfill/snapshot_backfill.rs | 130 +++++++ src/stream/src/executor/chain.rs | 4 +- src/stream/src/executor/rearranged_chain.rs | 6 +- src/stream/src/executor/values.rs | 2 +- src/stream/src/task/barrier_manager.rs | 89 ++--- .../src/task/barrier_manager/managed_state.rs | 327 +++++++++++------- .../src/task/barrier_manager/progress.rs | 45 ++- src/stream/src/task/mod.rs | 15 + 15 files changed, 511 insertions(+), 259 deletions(-) create mode 100644 src/stream/src/executor/backfill/snapshot_backfill.rs diff --git a/proto/stream_service.proto b/proto/stream_service.proto index fd97bde85348..0b58744eec3c 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -52,14 +52,18 @@ message DropActorsResponse { common.Status status = 2; } -message InjectBarrierRequest { - string request_id = 1; - stream_plan.Barrier barrier = 2; +message PartialGraphInfo { repeated uint32 actor_ids_to_send = 3; repeated uint32 actor_ids_to_collect = 4; repeated uint32 table_ids_to_sync = 5; } +message InjectBarrierRequest { + string request_id = 1; + stream_plan.Barrier barrier = 2; + map graph_info = 3; +} + message BarrierCompleteResponse { message CreateMviewProgress { uint32 backfill_actor_id = 1; @@ -80,6 +84,9 @@ message BarrierCompleteResponse { uint32 worker_id = 5; map table_watermarks = 6; repeated hummock.SstableInfo old_value_sstables = 7; + uint32 partial_graph_id = 8; + // prev_epoch of barrier + uint64 epoch = 9; } // Before starting streaming, the leader node broadcast the actor-host table to needed workers. diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index df1a34544e6c..b7613f4a1040 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -50,10 +50,9 @@ use tracing::{error, info, warn, Instrument}; use self::command::CommandContext; use self::notifier::Notifier; -use self::progress::TrackingCommand; use crate::barrier::info::InflightActorInfo; use crate::barrier::notifier::BarrierInfo; -use crate::barrier::progress::CreateMviewProgressTracker; +use crate::barrier::progress::{CreateMviewProgressTracker, TrackingCommand}; use crate::barrier::rpc::ControlStreamManager; use crate::barrier::state::BarrierManagerState; use crate::error::MetaErrorInner; @@ -275,12 +274,10 @@ 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, - worker_id: WorkerId, - prev_epoch: u64, - resp: BarrierCompleteResponse, - ) { + fn barrier_collected(&mut self, resp: BarrierCompleteResponse) { + let worker_id = resp.worker_id; + let prev_epoch = resp.epoch; + assert_eq!(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)); node.state.resps.push(resp); @@ -389,10 +386,20 @@ impl CheckpointControl { node.enqueue_time.observe_duration(); } } + + fn report_collect_failure(&self, worker_id: WorkerId, e: &MetaError) { + for epoch_node in self.command_ctx_queue.values() { + if epoch_node.state.node_to_collect.contains(&worker_id) { + self.context + .report_collect_failure(&epoch_node.command_ctx, e); + break; + } + } + } } /// The state and message of this barrier, a node for concurrent checkpoint. -pub struct EpochNode { +struct EpochNode { /// Timer for recording barrier latency, taken after `complete_barriers`. enqueue_time: HistogramTimer, @@ -674,13 +681,14 @@ impl GlobalBarrierManager { _ => {} } } - resp_result = self.control_stream_manager.next_complete_barrier_response() => { + (worker_id, resp_result) = self.control_stream_manager.next_complete_barrier_response() => { match resp_result { - Ok((worker_id, prev_epoch, resp)) => { - self.checkpoint_control.barrier_collected(worker_id, prev_epoch, resp); + Ok(resp) => { + self.checkpoint_control.barrier_collected(resp); } Err(e) => { + self.checkpoint_control.report_collect_failure(worker_id, &e); self.failure_recovery(e).await; } } @@ -754,10 +762,10 @@ impl GlobalBarrierManager { send_latency_timer.observe_duration(); - let node_to_collect = match self.control_stream_manager.inject_barrier( - command_ctx.clone(), - self.state.inflight_actor_infos.existing_table_ids(), - ) { + let node_to_collect = match self + .control_stream_manager + .inject_barrier(command_ctx.clone(), &self.state.inflight_actor_infos) + { Ok(node_to_collect) => node_to_collect, Err(err) => { for notifier in notifiers { diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index 4bb9d2f669c0..946f1cc78562 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -372,13 +372,14 @@ impl GlobalBarrierManager { tracing::Span::current(), // recovery span )); - let mut node_to_collect = control_stream_manager - .inject_barrier(command_ctx.clone(), info.existing_table_ids())?; + let mut node_to_collect = + control_stream_manager.inject_barrier(command_ctx.clone(), &info)?; while !node_to_collect.is_empty() { - let (worker_id, prev_epoch, _) = control_stream_manager + let (worker_id, result) = control_stream_manager .next_complete_barrier_response() - .await?; - assert_eq!(prev_epoch, command_ctx.prev_epoch.value().0); + .await; + let resp = result?; + assert_eq!(resp.epoch, command_ctx.prev_epoch.value().0); assert!(node_to_collect.remove(&worker_id)); } diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index c1a337bde046..3901e7d9bb18 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{HashMap, HashSet, VecDeque}; +use std::collections::{HashMap, HashSet}; use std::error::Error; use std::future::Future; use std::sync::Arc; @@ -24,7 +24,6 @@ use futures::future::try_join_all; use futures::stream::{BoxStream, FuturesUnordered}; use futures::{pin_mut, FutureExt, StreamExt}; use itertools::Itertools; -use risingwave_common::catalog::TableId; use risingwave_common::hash::ActorId; use risingwave_common::util::tracing::TracingContext; use risingwave_pb::common::{ActorInfo, WorkerNode}; @@ -32,8 +31,8 @@ use risingwave_pb::stream_plan::{Barrier, BarrierMutation}; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, BarrierCompleteResponse, BroadcastActorInfoTableRequest, BuildActorInfo, BuildActorsRequest, DropActorsRequest, - InjectBarrierRequest, StreamingControlStreamRequest, StreamingControlStreamResponse, - UpdateActorsRequest, + InjectBarrierRequest, PbPartialGraphInfo, StreamingControlStreamRequest, + StreamingControlStreamResponse, UpdateActorsRequest, }; use risingwave_rpc_client::error::RpcError; use risingwave_rpc_client::StreamClient; @@ -47,6 +46,7 @@ use uuid::Uuid; use super::command::CommandContext; use super::GlobalBarrierManagerContext; +use crate::barrier::info::InflightActorInfo; use crate::manager::{MetaSrvEnv, WorkerId}; use crate::{MetaError, MetaResult}; @@ -55,8 +55,6 @@ const COLLECT_ERROR_TIMEOUT: Duration = Duration::from_secs(3); struct ControlStreamNode { worker: WorkerNode, sender: UnboundedSender, - // earlier epoch at the front - inflight_barriers: VecDeque>, } fn into_future( @@ -180,43 +178,32 @@ impl ControlStreamManager { pub(super) async fn next_complete_barrier_response( &mut self, - ) -> MetaResult<(WorkerId, u64, BarrierCompleteResponse)> { + ) -> (WorkerId, MetaResult) { loop { let (worker_id, result) = pending_on_none(self.next_response()).await; match result { Ok(resp) => match resp.response { Some(streaming_control_stream_response::Response::CompleteBarrier(resp)) => { - let node = self - .nodes - .get_mut(&worker_id) - .expect("should exist when get collect resp"); - let command = node - .inflight_barriers - .pop_front() - .expect("should exist when get collect resp"); - break Ok((worker_id, command.prev_epoch.value().0, resp)); + assert_eq!(worker_id, resp.worker_id); + return (worker_id, Ok(resp)); } resp => { - break Err(anyhow!("get unexpected resp: {:?}", resp).into()); + break ( + worker_id, + Err(anyhow!("get unexpected resp: {:?}", resp).into()), + ); } }, Err(err) => { - let mut node = self + let node = self .nodes .remove(&worker_id) .expect("should exist when get collect resp"); // Note: No need to use `?` as the backtrace is from meta and not useful. warn!(node = ?node.worker, err = %err.as_report(), "get error from response stream"); - if let Some(command) = node.inflight_barriers.pop_front() { - let errors = self.collect_errors(node.worker.id, err).await; - let err = merge_node_rpc_errors("get error from control stream", errors); - self.context.report_collect_failure(&command, &err); - break Err(err); - } else { - // for node with no inflight barrier, simply ignore the error - info!(node = ?node.worker, "no inflight barrier no node. Ignore error"); - continue; - } + let errors = self.collect_errors(node.worker.id, err).await; + let err = merge_node_rpc_errors("get error from control stream", errors); + break (worker_id, Err(err)); } } } @@ -248,7 +235,7 @@ impl ControlStreamManager { pub(super) fn inject_barrier( &mut self, command_context: Arc, - table_ids_to_sync: HashSet, + inflight_actor_info_after_apply_command: &InflightActorInfo, ) -> MetaResult> { fail_point!("inject_barrier_err", |_| risingwave_common::bail!( "inject_barrier_err" @@ -266,9 +253,25 @@ impl ControlStreamManager { // No need to send or collect barrier for this node. assert!(actor_ids_to_send.is_empty()); } + let graph_info = HashMap::from_iter([( + u32::MAX, + PbPartialGraphInfo { + actor_ids_to_send, + actor_ids_to_collect, + table_ids_to_sync: inflight_actor_info_after_apply_command + .existing_table_ids() + .iter() + .map(|table_id| table_id.table_id) + .collect(), + }, + )]); + { let Some(node) = self.nodes.get_mut(node_id) else { - if actor_ids_to_collect.is_empty() { + if graph_info + .values() + .all(|info| info.actor_ids_to_collect.is_empty()) + { // Worker node get disconnected but has no actor to collect. Simply skip it. return Ok(()); } @@ -298,12 +301,7 @@ impl ControlStreamManager { InjectBarrierRequest { request_id: StreamRpcManager::new_request_id(), barrier: Some(barrier), - actor_ids_to_send, - actor_ids_to_collect, - table_ids_to_sync: table_ids_to_sync - .iter() - .map(|table_id| table_id.table_id) - .collect(), + graph_info, }, ), ), @@ -316,7 +314,6 @@ impl ControlStreamManager { )) })?; - node.inflight_barriers.push_back(command_context.clone()); node_need_collect.insert(*node_id); Result::<_, MetaError>::Ok(()) } @@ -359,14 +356,17 @@ impl GlobalBarrierManagerContext { ControlStreamNode { worker: node.clone(), sender: handle.request_sender, - inflight_barriers: VecDeque::new(), }, handle.response_stream, )) } /// Send barrier-complete-rpc and wait for responses from all CNs - fn report_collect_failure(&self, command_context: &CommandContext, error: &MetaError) { + pub(super) fn report_collect_failure( + &self, + command_context: &CommandContext, + error: &MetaError, + ) { // Record failure in event log. use risingwave_pb::meta::event_log; let event = event_log::EventCollectBarrierFail { diff --git a/src/stream/src/executor/backfill/arrangement_backfill.rs b/src/stream/src/executor/backfill/arrangement_backfill.rs index 7920e8dceee8..e3979496731b 100644 --- a/src/stream/src/executor/backfill/arrangement_backfill.rs +++ b/src/stream/src/executor/backfill/arrangement_backfill.rs @@ -462,7 +462,7 @@ where // May need to revisit it. // Need to check it after scale-in / scale-out. self.progress.update( - barrier.epoch.curr, + barrier.epoch, snapshot_read_epoch, total_snapshot_processed_rows, ); @@ -577,7 +577,7 @@ where } self.progress - .finish(barrier.epoch.curr, total_snapshot_processed_rows); + .finish(barrier.epoch, total_snapshot_processed_rows); yield msg; break; } diff --git a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs index 59686f4bb8fd..943059355f05 100644 --- a/src/stream/src/executor/backfill/cdc/cdc_backfill.rs +++ b/src/stream/src/executor/backfill/cdc/cdc_backfill.rs @@ -655,7 +655,7 @@ impl CdcBackfillExecutor { // mark progress as finished if let Some(progress) = self.progress.as_mut() { - progress.finish(barrier.epoch.curr, total_snapshot_row_count); + progress.finish(barrier.epoch, total_snapshot_row_count); } yield msg; // break after the state have been saved diff --git a/src/stream/src/executor/backfill/no_shuffle_backfill.rs b/src/stream/src/executor/backfill/no_shuffle_backfill.rs index e368086a9773..a65c5f1c4487 100644 --- a/src/stream/src/executor/backfill/no_shuffle_backfill.rs +++ b/src/stream/src/executor/backfill/no_shuffle_backfill.rs @@ -417,7 +417,7 @@ where snapshot_read_epoch = barrier.epoch.prev; self.progress.update( - barrier.epoch.curr, + barrier.epoch, snapshot_read_epoch, total_snapshot_processed_rows, ); @@ -540,7 +540,7 @@ where // and backfill which just finished, we need to update mview tracker, // it does not persist this information. self.progress - .finish(barrier.epoch.curr, total_snapshot_processed_rows); + .finish(barrier.epoch, total_snapshot_processed_rows); tracing::trace!( epoch = ?barrier.epoch, "Updated CreateMaterializedTracker" diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs new file mode 100644 index 000000000000..85b55a5f42b4 --- /dev/null +++ b/src/stream/src/executor/backfill/snapshot_backfill.rs @@ -0,0 +1,130 @@ +// Copyright 2024 RisingWave Labs +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +use std::pin::pin; + +use futures::TryStreamExt; +use risingwave_common::array::{Op, StreamChunk}; +use risingwave_common::util::chunk_coalesce::DataChunkBuilder; +use risingwave_hummock_sdk::HummockReadEpoch; +use risingwave_storage::table::batch_table::storage_table::StorageTable; +use risingwave_storage::StateStore; + +use crate::executor::backfill::utils::mapping_chunk; +use crate::executor::prelude::{try_stream, StreamExt}; +use crate::executor::{ + expect_first_barrier, ActorContextRef, BackfillExecutor, BoxedMessageStream, Execute, Executor, + Message, StreamExecutorError, +}; +use crate::task::CreateMviewProgress; + +/// Similar to [`super::no_shuffle_backfill::BackfillExecutor`]. +/// Main differences: +/// - [`crate::executor::ArrangementBackfillExecutor`] can reside on a different CN, so it can be scaled +/// independently. +/// - To synchronize upstream shared buffer, it is initialized with a [`ReplicatedStateTable`]. +pub struct SnapshotBackfillExecutor { + /// Upstream table + upstream_table: StorageTable, + + /// Upstream with the same schema with the upstream table. + upstream: Executor, + + /// The column indices need to be forwarded to the downstream from the upstream and table scan. + output_indices: Vec, + + progress: CreateMviewProgress, + + chunk_size: usize, +} + +impl SnapshotBackfillExecutor { + pub fn new( + upstream_table: StorageTable, + upstream: Executor, + output_indices: Vec, + _actor_ctx: ActorContextRef, + progress: CreateMviewProgress, + chunk_size: usize, + ) -> Self { + Self { + upstream_table, + upstream, + output_indices, + progress, + chunk_size, + } + } + + #[try_stream(ok = Message, error = StreamExecutorError)] + async fn execute_inner(mut self) { + let upstream = self.upstream.execute(); + let mut upstream = pin!(upstream); + let first_barrier = expect_first_barrier(&mut upstream).await?; + let prev_epoch = first_barrier.epoch.prev; + yield Message::Barrier(first_barrier); + let snapshot_stream = BackfillExecutor::make_snapshot_stream( + &self.upstream_table, + HummockReadEpoch::Committed(prev_epoch), + None, + false, + &None, + ); + let mut count = 0; + let mut snapshot_stream = pin!(snapshot_stream); + let data_types = self.upstream_table.schema().data_types(); + let mut builder = DataChunkBuilder::new(data_types, self.chunk_size); + while let Some(row) = snapshot_stream.try_next().await? { + count += 1; + if let Some(data_chunk) = builder.append_one_row(row) { + let ops = vec![Op::Insert; data_chunk.capacity()]; + yield Message::Chunk(mapping_chunk( + StreamChunk::from_parts(ops, data_chunk), + &self.output_indices, + )); + } + } + if let Some(data_chunk) = builder.consume_all() { + let ops = vec![Op::Insert; data_chunk.capacity()]; + yield Message::Chunk(mapping_chunk( + StreamChunk::from_parts(ops, data_chunk), + &self.output_indices, + )); + } + let barrier = loop { + match upstream.try_next().await? { + None => { + return Ok(()); + } + Some(Message::Barrier(barrier)) => { + break barrier; + } + Some(msg) => { + yield msg; + } + } + }; + self.progress.finish(barrier.epoch, count); + yield Message::Barrier(barrier); + while let Some(msg) = upstream.try_next().await? { + yield msg; + } + } +} + +impl Execute for SnapshotBackfillExecutor { + fn execute(self: Box) -> BoxedMessageStream { + self.execute_inner().boxed() + } +} diff --git a/src/stream/src/executor/chain.rs b/src/stream/src/executor/chain.rs index 96b9422a97b2..8a4131bb2635 100644 --- a/src/stream/src/executor/chain.rs +++ b/src/stream/src/executor/chain.rs @@ -64,7 +64,7 @@ impl ChainExecutor { // If the barrier is a conf change of creating this mview, and the snapshot is not to be // consumed, we can finish the progress immediately. if barrier.is_newly_added(self.actor_id) && self.upstream_only { - self.progress.finish(barrier.epoch.curr, 0); + self.progress.finish(barrier.epoch, 0); } // The first barrier message should be propagated. @@ -88,7 +88,7 @@ impl ChainExecutor { for msg in upstream { let msg = msg?; if to_consume_snapshot && let Message::Barrier(barrier) = &msg { - self.progress.finish(barrier.epoch.curr, 0); + self.progress.finish(barrier.epoch, 0); } yield msg; } diff --git a/src/stream/src/executor/rearranged_chain.rs b/src/stream/src/executor/rearranged_chain.rs index 19ebfeabc298..37717d270d90 100644 --- a/src/stream/src/executor/rearranged_chain.rs +++ b/src/stream/src/executor/rearranged_chain.rs @@ -155,7 +155,7 @@ impl RearrangedChainExecutor { // Update the progress since we've consumed all chunks before this // phantom. self.progress.update( - last_rearranged_epoch.curr, + last_rearranged_epoch, barrier.epoch.curr, processed_rows, ); @@ -201,7 +201,7 @@ impl RearrangedChainExecutor { continue; }; if let Some(barrier) = msg.as_barrier() { - self.progress.finish(barrier.epoch.curr, processed_rows); + self.progress.finish(barrier.epoch, processed_rows); } yield msg; } @@ -214,7 +214,7 @@ impl RearrangedChainExecutor { for msg in upstream { let msg: Message = msg?; if let Some(barrier) = msg.as_barrier() { - self.progress.finish(barrier.epoch.curr, processed_rows); + self.progress.finish(barrier.epoch, processed_rows); } yield msg; } diff --git a/src/stream/src/executor/values.rs b/src/stream/src/executor/values.rs index dfa5579d66b4..f2e737701cd2 100644 --- a/src/stream/src/executor/values.rs +++ b/src/stream/src/executor/values.rs @@ -121,7 +121,7 @@ impl ValuesExecutor { while let Some(barrier) = barrier_receiver.recv().await { if emit { - progress.finish(barrier.epoch.curr, 0); + progress.finish(barrier.epoch, 0); } yield Message::Barrier(barrier); } diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index b0ce6ad30540..17a49763fd82 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeSet, HashMap, HashSet}; +use std::collections::{BTreeSet, HashMap}; use std::fmt::Display; use std::future::pending; use std::sync::Arc; @@ -38,7 +38,8 @@ use tonic::{Code, Status}; use self::managed_state::ManagedBarrierState; use crate::error::{IntoUnexpectedExit, StreamError, StreamResult}; use crate::task::{ - ActorHandle, ActorId, AtomicU64Ref, SharedContext, StreamEnvironment, UpDownActorIds, + ActorHandle, ActorId, AtomicU64Ref, PartialGraphId, SharedContext, StreamEnvironment, + UpDownActorIds, }; mod managed_state; @@ -47,7 +48,6 @@ mod progress; mod tests; pub use progress::CreateMviewProgress; -use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; @@ -57,7 +57,7 @@ use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::streaming_control_stream_request::{InitRequest, Request}; use risingwave_pb::stream_service::streaming_control_stream_response::InitResponse; use risingwave_pb::stream_service::{ - streaming_control_stream_response, BarrierCompleteResponse, BuildActorInfo, + streaming_control_stream_response, BarrierCompleteResponse, BuildActorInfo, PartialGraphInfo, StreamingControlStreamRequest, StreamingControlStreamResponse, }; @@ -190,7 +190,7 @@ pub(super) enum LocalBarrierEvent { barrier: Barrier, }, ReportCreateProgress { - current_epoch: u64, + epoch: EpochPair, actor: ActorId, state: BackfillState, }, @@ -424,8 +424,8 @@ impl LocalBarrierWorker { (sender, create_actors_result) = self.actor_manager_state.next_created_actors() => { self.handle_actor_created(sender, create_actors_result); } - completed_epoch = self.state.next_completed_epoch() => { - let result = self.on_epoch_completed(completed_epoch); + (partial_graph_id, completed_epoch) = self.state.next_completed_epoch() => { + let result = self.on_epoch_completed(partial_graph_id, completed_epoch); if let Err(err) = result { self.notify_other_failure(err, "failed to complete epoch").await; } @@ -489,15 +489,7 @@ impl LocalBarrierWorker { match request.request.expect("should not be empty") { Request::InjectBarrier(req) => { let barrier = Barrier::from_protobuf(req.get_barrier().unwrap())?; - self.send_barrier( - &barrier, - req.actor_ids_to_send.into_iter().collect(), - req.actor_ids_to_collect.into_iter().collect(), - req.table_ids_to_sync - .into_iter() - .map(TableId::new) - .collect(), - )?; + self.send_barrier(barrier, req.graph_info)?; Ok(()) } Request::Init(_) => { @@ -512,11 +504,11 @@ impl LocalBarrierWorker { self.collect(actor_id, &barrier) } LocalBarrierEvent::ReportCreateProgress { - current_epoch, + epoch, actor, state, } => { - self.update_create_mview_progress(current_epoch, actor, state); + self.update_create_mview_progress(epoch, actor, state); } LocalBarrierEvent::SubscribeBarrierMutation { actor_id, @@ -586,9 +578,17 @@ impl LocalBarrierWorker { // event handler impl LocalBarrierWorker { - fn on_epoch_completed(&mut self, epoch: u64) -> StreamResult<()> { - let result = self + fn on_epoch_completed( + &mut self, + partial_graph_id: PartialGraphId, + epoch: u64, + ) -> StreamResult<()> { + let state = self .state + .graph_states + .get_mut(&partial_graph_id) + .expect("should exist"); + let result = state .pop_completed_epoch(epoch) .expect("should exist") .expect("should have completed")?; @@ -613,6 +613,8 @@ impl LocalBarrierWorker { streaming_control_stream_response::Response::CompleteBarrier( BarrierCompleteResponse { request_id: "todo".to_string(), + partial_graph_id: partial_graph_id.into(), + epoch, status: None, create_mview_progress, synced_sstables: synced_sstables @@ -666,19 +668,22 @@ impl LocalBarrierWorker { /// to find the root cause. fn send_barrier( &mut self, - barrier: &Barrier, - to_send: HashSet, - to_collect: HashSet, - table_ids: HashSet, + barrier: Barrier, + graph_infos: HashMap, ) -> StreamResult<()> { + let all_to_collect = || { + graph_infos + .values() + .flat_map(|info| info.actor_ids_to_collect.iter()) + .cloned() + }; if !cfg!(test) { // The barrier might be outdated and been injected after recovery in some certain extreme // scenarios. So some newly creating actors in the barrier are possibly not rebuilt during // recovery. Check it here and return an error here if some actors are not found to // avoid collection hang. We need some refine in meta side to remove this workaround since // it will cause another round of unnecessary recovery. - let missing_actor_ids = to_collect - .iter() + let missing_actor_ids = all_to_collect() .filter(|id| !self.actor_manager_state.handles.contains_key(id)) .collect_vec(); if !missing_actor_ids.is_empty() { @@ -686,7 +691,7 @@ impl LocalBarrierWorker { "to collect actors not found, they should be cleaned when recovering: {:?}", missing_actor_ids ); - return Err(anyhow!("to collect actors not found: {:?}", to_collect).into()); + return Err(anyhow!("to collect actors not found: {:?}", missing_actor_ids).into()); } } @@ -697,29 +702,31 @@ impl LocalBarrierWorker { } debug!( target: "events::stream::barrier::manager::send", - "send barrier {:?}, senders = {:?}, actor_ids_to_collect = {:?}", + "send barrier {:?}, graph_info: {:?}", barrier, - to_send, - to_collect + graph_infos, ); - for actor_id in &to_collect { - if self.failure_actors.contains_key(actor_id) { + for actor_id in all_to_collect() { + if self.failure_actors.contains_key(&actor_id) { // The failure actors could exit before the barrier is issued, while their // up-downstream actors could be stuck somehow. Return error directly to trigger the // recovery. return Err(StreamError::barrier_send( barrier.clone(), - *actor_id, + actor_id, "actor has already failed", )); } } - self.state - .transform_to_issued(barrier, to_collect, table_ids); + self.state.transform_to_issued(&barrier, &graph_infos); - for actor_id in to_send { + for actor_id in graph_infos + .values() + .flat_map(|infos| infos.actor_ids_to_send.iter()) + .cloned() + { match self.barrier_senders.get(&actor_id) { Some(senders) => { for sender in senders { @@ -774,14 +781,12 @@ impl LocalBarrierWorker { self.add_failure(actor_id, err.clone()); let root_err = self.try_find_root_failure().await.unwrap(); // always `Some` because we just added one - let failed_epochs = self.state.epochs_await_on_actor(actor_id).collect_vec(); - if !failed_epochs.is_empty() { + if let Some(actor_state) = self.state.actor_states.get(&actor_id) + && !actor_state.inflight_barriers.is_empty() + { self.control_stream_handle.reset_stream_with_err( anyhow!(root_err) - .context(format!( - "failed to collect barrier for epoch {:?}", - failed_epochs - )) + .context("failed to collect barrier") .to_status_unnamed(Code::Internal), ); } diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index f4a3fb31c03c..f67fb2d45298 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -15,9 +15,10 @@ use std::assert_matches::assert_matches; use std::collections::{BTreeMap, BTreeSet, HashMap, HashSet}; use std::fmt::{Debug, Display, Formatter}; -use std::future::Future; +use std::future::{poll_fn, Future}; use std::mem::replace; use std::sync::Arc; +use std::task::{ready, Context, Poll}; use anyhow::anyhow; use await_tree::InstrumentAwait; @@ -31,8 +32,8 @@ use risingwave_common::util::epoch::EpochPair; use risingwave_hummock_sdk::SyncResult; use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; +use risingwave_pb::stream_service::PartialGraphInfo; use risingwave_storage::{dispatch_state_store, StateStore, StateStoreImpl}; -use rw_futures_util::pending_on_none; use thiserror_ext::AsReport; use tokio::sync::mpsc; @@ -41,7 +42,7 @@ use super::{BarrierCompleteResult, SubscribeMutationItem}; use crate::error::StreamResult; use crate::executor::monitor::StreamingMetrics; use crate::executor::{Barrier, Mutation}; -use crate::task::{await_tree_key, ActorId}; +use crate::task::{await_tree_key, ActorId, PartialGraphId}; struct IssuedState { pub mutation: Option>, @@ -114,17 +115,24 @@ fn sync_epoch( .boxed() } -#[derive(Debug)] pub(super) struct ManagedBarrierStateDebugInfo<'a> { - epoch_barrier_state_map: &'a BTreeMap, - - create_mview_progress: &'a HashMap>, + graph_states: &'a HashMap, } impl Display for ManagedBarrierStateDebugInfo<'_> { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + for (partial_graph_id, graph_states) in self.graph_states { + writeln!(f, "--- Partial Group {}", partial_graph_id.0)?; + write!(f, "{}", graph_states)?; + } + Ok(()) + } +} + +impl Display for &'_ PartialGraphManagedBarrierState { fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { let mut prev_epoch = 0u64; - for (epoch, barrier_state) in self.epoch_barrier_state_map { + for (epoch, barrier_state) in &self.epoch_barrier_state_map { write!(f, "> Epoch {}: ", epoch)?; match &barrier_state.inner { ManagedBarrierStateInner::Issued(state) => { @@ -172,7 +180,7 @@ impl Display for ManagedBarrierStateDebugInfo<'_> { if !self.create_mview_progress.is_empty() { writeln!(f, "Create MView Progress:")?; - for (epoch, progress) in self.create_mview_progress { + for (epoch, progress) in &self.create_mview_progress { write!(f, "> Epoch {}:", epoch)?; for (actor_id, state) in progress { write!(f, ">> Actor {}: {}, ", actor_id, state)?; @@ -185,36 +193,67 @@ impl Display for ManagedBarrierStateDebugInfo<'_> { } #[derive(Default)] -struct ActorMutationSubscribers { +pub(super) struct InflightActorState { pending_subscribers: BTreeMap>>, started_subscribers: Vec>, + // epoch -> partial graph id + pub(super) inflight_barriers: BTreeMap, + is_stopped: bool, } -impl ActorMutationSubscribers { - fn is_empty(&self) -> bool { - self.pending_subscribers.is_empty() && self.started_subscribers.is_empty() - } -} - -pub(super) struct ManagedBarrierState { +pub(super) struct PartialGraphManagedBarrierState { + /// This is a temporary workaround for the need to still calling `seal_epoch` for storage. + /// Can be removed after `seal_epoch` is deprecated in storage. + need_seal_epoch: bool, /// Record barrier state for each epoch of concurrent checkpoints. /// /// The key is `prev_epoch`, and the first value is `curr_epoch` epoch_barrier_state_map: BTreeMap, - mutation_subscribers: HashMap, - prev_barrier_table_ids: Option<(EpochPair, HashSet)>, /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. pub(super) create_mview_progress: HashMap>, + /// Futures will be finished in the order of epoch in ascending order. + await_epoch_completed_futures: FuturesOrdered, + pub(super) state_store: StateStoreImpl, pub(super) streaming_metrics: Arc, - /// Futures will be finished in the order of epoch in ascending order. - await_epoch_completed_futures: FuturesOrdered, + /// Manages the await-trees of all barriers. + barrier_await_tree_reg: Option, +} + +impl PartialGraphManagedBarrierState { + fn new( + need_seal_epoch: bool, + state_store: StateStoreImpl, + streaming_metrics: Arc, + barrier_await_tree_reg: Option, + ) -> Self { + Self { + need_seal_epoch, + epoch_barrier_state_map: Default::default(), + prev_barrier_table_ids: None, + create_mview_progress: Default::default(), + await_epoch_completed_futures: Default::default(), + state_store, + streaming_metrics, + barrier_await_tree_reg, + } + } +} + +pub(super) struct ManagedBarrierState { + pub(super) actor_states: HashMap, + + pub(super) graph_states: HashMap, + + pub(super) state_store: StateStoreImpl, + + pub(super) streaming_metrics: Arc, /// Manages the await-trees of all barriers. barrier_await_tree_reg: Option, @@ -237,21 +276,17 @@ impl ManagedBarrierState { barrier_await_tree_reg: Option, ) -> Self { Self { - epoch_barrier_state_map: BTreeMap::default(), - mutation_subscribers: Default::default(), - prev_barrier_table_ids: None, - create_mview_progress: Default::default(), + actor_states: Default::default(), + graph_states: Default::default(), state_store, streaming_metrics, - await_epoch_completed_futures: FuturesOrdered::new(), barrier_await_tree_reg, } } pub(super) fn to_debug_info(&self) -> ManagedBarrierStateDebugInfo<'_> { ManagedBarrierStateDebugInfo { - epoch_barrier_state_map: &self.epoch_barrier_state_map, - create_mview_progress: &self.create_mview_progress, + graph_states: &self.graph_states, } } @@ -261,15 +296,29 @@ impl ManagedBarrierState { start_prev_epoch: u64, tx: mpsc::UnboundedSender, ) { - let subscribers = self.mutation_subscribers.entry(actor_id).or_default(); - if let Some(state) = self.epoch_barrier_state_map.get(&start_prev_epoch) { - match &state.inner { + let actor_state = self.actor_states.entry(actor_id).or_default(); + if let Some(start_partial_graph_id) = actor_state.inflight_barriers.get(&start_prev_epoch) { + let start_graph_state = self + .graph_states + .get(start_partial_graph_id) + .expect("should exist") + .epoch_barrier_state_map + .get(&start_prev_epoch) + .expect("should exist"); + match &start_graph_state.inner { ManagedBarrierStateInner::Issued(issued_state) => { assert!(issued_state.remaining_actors.contains(&actor_id)); - for (prev_epoch, state) in - self.epoch_barrier_state_map.range(start_prev_epoch..) + for (prev_epoch, partial_graph_id) in + actor_state.inflight_barriers.range(start_prev_epoch..) { - match &state.inner { + let graph_state = self + .graph_states + .get(partial_graph_id) + .expect("should exist") + .epoch_barrier_state_map + .get(prev_epoch) + .expect("should exist"); + match &graph_state.inner { ManagedBarrierStateInner::Issued(issued_state) => { if issued_state.remaining_actors.contains(&actor_id) { if tx @@ -293,7 +342,7 @@ impl ManagedBarrierState { } } } - subscribers.started_subscribers.push(tx); + actor_state.started_subscribers.push(tx); } state @ ManagedBarrierStateInner::AllCollected | state @ ManagedBarrierStateInner::Completed(_) => { @@ -305,7 +354,7 @@ impl ManagedBarrierState { } } else { // Barrier has not issued yet. Store the pending tx - if let Some((last_epoch, _)) = self.epoch_barrier_state_map.last_key_value() { + if let Some((last_epoch, _)) = actor_state.inflight_barriers.last_key_value() { assert!( *last_epoch < start_prev_epoch, "later barrier {} has been issued, but skip the start epoch {:?}", @@ -313,7 +362,7 @@ impl ManagedBarrierState { start_prev_epoch ); } - subscribers + actor_state .pending_subscribers .entry(start_prev_epoch) .or_default() @@ -321,6 +370,105 @@ impl ManagedBarrierState { } } + pub(super) fn transform_to_issued( + &mut self, + barrier: &Barrier, + graph_infos: &HashMap, + ) { + let actor_to_stop = barrier.all_stop_actors(); + for (partial_graph_id, graph_info) in graph_infos { + let partial_graph_id = PartialGraphId::new(*partial_graph_id); + for actor_id in &graph_info.actor_ids_to_collect { + let actor_state = self.actor_states.entry(*actor_id).or_default(); + if let Some((first_epoch, _)) = actor_state.pending_subscribers.first_key_value() { + assert!( + *first_epoch >= barrier.epoch.prev, + "barrier epoch {:?} skip subscribed epoch {}", + barrier.epoch, + first_epoch + ); + if *first_epoch == barrier.epoch.prev { + actor_state.started_subscribers.extend( + actor_state + .pending_subscribers + .pop_first() + .expect("should exist") + .1, + ); + } + } + actor_state.started_subscribers.retain(|tx| { + tx.send((barrier.epoch.prev, barrier.mutation.clone())) + .is_ok() + }); + if let Some((prev_epoch, _)) = actor_state.inflight_barriers.first_key_value() { + assert!(*prev_epoch < barrier.epoch.prev); + } + actor_state + .inflight_barriers + .insert(barrier.epoch.prev, partial_graph_id); + if let Some(actor_to_stop) = actor_to_stop + && actor_to_stop.contains(actor_id) + { + actor_state.is_stopped = true; + } + } + + let graph_state = self + .graph_states + .entry(partial_graph_id) + .or_insert_with(|| { + PartialGraphManagedBarrierState::new( + partial_graph_id.0 == u32::MAX, + self.state_store.clone(), + self.streaming_metrics.clone(), + self.barrier_await_tree_reg.clone(), + ) + }); + graph_state.transform_to_issued( + barrier, + graph_info.actor_ids_to_collect.iter().cloned().collect(), + graph_info + .table_ids_to_sync + .iter() + .map(|table_id| TableId::new(*table_id)) + .collect(), + ); + } + } + + pub(super) fn next_completed_epoch( + &mut self, + ) -> impl Future + '_ { + poll_fn(|cx| { + for (partial_graph_id, graph_state) in &mut self.graph_states { + let poll = graph_state.poll_next_completed_epoch(cx); + if poll.is_ready() { + return poll.map(|epoch| (*partial_graph_id, epoch)); + } + } + Poll::Pending + }) + } + + pub(super) fn collect(&mut self, actor_id: ActorId, barrier: &Barrier) { + let actor_states = self.actor_states.get_mut(&actor_id).expect("should exist"); + let (prev_epoch, partial_graph_id) = actor_states + .inflight_barriers + .pop_first() + .expect("should not be empty"); + assert_eq!(prev_epoch, barrier.epoch.prev); + if actor_states.is_stopped && actor_states.inflight_barriers.is_empty() { + self.actor_states.remove(&actor_id); + } + self.graph_states + .get_mut(&partial_graph_id) + .expect("should exist") + .collect(actor_id, barrier); + } +} + +impl PartialGraphManagedBarrierState { /// This method is called when barrier state is modified in either `Issued` or `Stashed` /// to transform the state to `AllCollected` and start state store `sync` when the barrier /// has been collected from all actors for an `Issued` barrier. @@ -396,14 +544,18 @@ impl ManagedBarrierState { None } BarrierKind::Barrier => { - dispatch_state_store!(&self.state_store, state_store, { - state_store.seal_epoch(prev_epoch, kind.is_checkpoint()); - }); + if self.need_seal_epoch { + dispatch_state_store!(&self.state_store, state_store, { + state_store.seal_epoch(prev_epoch, kind.is_checkpoint()); + }); + } None } BarrierKind::Checkpoint => { dispatch_state_store!(&self.state_store, state_store, { - state_store.seal_epoch(prev_epoch, kind.is_checkpoint()); + if self.need_seal_epoch { + state_store.seal_epoch(prev_epoch, kind.is_checkpoint()); + } Some(sync_epoch( state_store, &self.streaming_metrics, @@ -446,35 +598,6 @@ impl ManagedBarrierState { } } - /// Returns an iterator on epochs that is awaiting on `actor_id`. - /// This is used on notifying actor failure. On actor failure, the - /// barrier manager can call this method to iterate on epochs that - /// waits on the failed actor and then notify failure on the result - /// sender of the epoch. - pub(crate) fn epochs_await_on_actor( - &self, - actor_id: ActorId, - ) -> impl Iterator + '_ { - self.epoch_barrier_state_map - .iter() - .filter_map(move |(prev_epoch, barrier_state)| { - #[allow(clippy::single_match)] - match barrier_state.inner { - ManagedBarrierStateInner::Issued(IssuedState { - ref remaining_actors, - .. - }) => { - if remaining_actors.contains(&actor_id) { - Some(*prev_epoch) - } else { - None - } - } - _ => None, - } - }) - } - /// Collect a `barrier` from the actor with `actor_id`. pub(super) fn collect(&mut self, actor_id: ActorId, barrier: &Barrier) { tracing::debug!( @@ -587,45 +710,6 @@ impl ManagedBarrierState { } }; - for (actor_id, subscribers) in &mut self.mutation_subscribers { - if actor_ids_to_collect.contains(actor_id) { - if let Some((first_epoch, _)) = subscribers.pending_subscribers.first_key_value() { - assert!( - *first_epoch >= barrier.epoch.prev, - "barrier epoch {:?} skip subscribed epoch {}", - barrier.epoch, - first_epoch - ); - if *first_epoch == barrier.epoch.prev { - subscribers.started_subscribers.extend( - subscribers - .pending_subscribers - .pop_first() - .expect("should exist") - .1, - ); - } - } - subscribers.started_subscribers.retain(|tx| { - tx.send((barrier.epoch.prev, barrier.mutation.clone())) - .is_ok() - }); - } else { - subscribers.started_subscribers.clear(); - if let Some((first_epoch, _)) = subscribers.pending_subscribers.first_key_value() { - assert!( - *first_epoch > barrier.epoch.prev, - "barrier epoch {:?} skip subscribed epoch {}", - barrier.epoch, - first_epoch - ); - } - } - } - - self.mutation_subscribers - .retain(|_, subscribers| !subscribers.is_empty()); - self.epoch_barrier_state_map.insert( barrier.epoch.prev, BarrierState { @@ -643,17 +727,20 @@ impl ManagedBarrierState { } /// Return a future that yields the next completed epoch. The future is cancellation safe. - pub(crate) fn next_completed_epoch(&mut self) -> impl Future + '_ { - pending_on_none(self.await_epoch_completed_futures.next()).map(|(prev_epoch, result)| { - let state = self - .epoch_barrier_state_map - .get_mut(&prev_epoch) - .expect("should exist"); - // sanity check on barrier state - assert_matches!(&state.inner, ManagedBarrierStateInner::AllCollected); - state.inner = ManagedBarrierStateInner::Completed(result); - prev_epoch - }) + pub(crate) fn poll_next_completed_epoch(&mut self, cx: &mut Context<'_>) -> Poll { + ready!(self.await_epoch_completed_futures.next().poll_unpin(cx)) + .map(|(prev_epoch, result)| { + let state = self + .epoch_barrier_state_map + .get_mut(&prev_epoch) + .expect("should exist"); + // sanity check on barrier state + assert_matches!(&state.inner, ManagedBarrierStateInner::AllCollected); + state.inner = ManagedBarrierStateInner::Completed(result); + prev_epoch + }) + .map(Poll::Ready) + .unwrap_or(Poll::Pending) } /// Pop the completion result of an completed epoch. diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 69c603a4b1ab..4fe6e98d4cf9 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -14,6 +14,8 @@ use std::fmt::{Display, Formatter}; +use risingwave_common::util::epoch::EpochPair; + use super::LocalBarrierManager; use crate::task::barrier_manager::LocalBarrierEvent::ReportCreateProgress; use crate::task::barrier_manager::LocalBarrierWorker; @@ -42,27 +44,27 @@ impl Display for BackfillState { impl LocalBarrierWorker { pub(crate) fn update_create_mview_progress( &mut self, - current_epoch: u64, + epoch: EpochPair, actor: ActorId, state: BackfillState, ) { - self.state - .create_mview_progress - .entry(current_epoch) - .or_default() - .insert(actor, state); + 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_progress + .entry(epoch.curr) + .or_default() + .insert(actor, state); + } } } impl LocalBarrierManager { - fn update_create_mview_progress( - &self, - current_epoch: u64, - actor: ActorId, - state: BackfillState, - ) { + fn update_create_mview_progress(&self, epoch: EpochPair, actor: ActorId, state: BackfillState) { self.send_event(ReportCreateProgress { - current_epoch, + epoch, actor, state, }) @@ -126,13 +128,10 @@ impl CreateMviewProgress { self.backfill_actor_id } - fn update_inner(&mut self, current_epoch: u64, state: BackfillState) { + fn update_inner(&mut self, epoch: EpochPair, state: BackfillState) { self.state = Some(state); - self.barrier_manager.update_create_mview_progress( - current_epoch, - self.backfill_actor_id, - state, - ); + self.barrier_manager + .update_create_mview_progress(epoch, self.backfill_actor_id, state); } /// Update the progress to `ConsumingUpstream(consumed_epoch, consumed_rows)`. The epoch must be @@ -141,7 +140,7 @@ impl CreateMviewProgress { /// `current_consumed_rows` is an accumulated value. pub fn update( &mut self, - current_epoch: u64, + epoch: EpochPair, consumed_epoch: ConsumedEpoch, current_consumed_rows: ConsumedRows, ) { @@ -159,18 +158,18 @@ impl CreateMviewProgress { None => {} }; self.update_inner( - current_epoch, + epoch, BackfillState::ConsumingUpstream(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, current_epoch: u64, current_consumed_rows: ConsumedRows) { + pub fn finish(&mut self, epoch: EpochPair, current_consumed_rows: ConsumedRows) { if let Some(BackfillState::Done(_)) = self.state { return; } - self.update_inner(current_epoch, BackfillState::Done(current_consumed_rows)); + self.update_inner(epoch, BackfillState::Done(current_consumed_rows)); } } diff --git a/src/stream/src/task/mod.rs b/src/stream/src/task/mod.rs index 77f21b52406f..007bdd40207d 100644 --- a/src/stream/src/task/mod.rs +++ b/src/stream/src/task/mod.rs @@ -39,6 +39,21 @@ pub type DispatcherId = u64; pub type UpDownActorIds = (ActorId, ActorId); pub type UpDownFragmentIds = (FragmentId, FragmentId); +#[derive(Hash, Eq, PartialEq, Copy, Clone, Debug)] +struct PartialGraphId(u32); + +impl PartialGraphId { + fn new(id: u32) -> Self { + Self(id) + } +} + +impl Into for PartialGraphId { + fn into(self) -> u32 { + self.0 + } +} + /// Stores the information which may be modified from the data plane. /// /// The data structure is created in `LocalBarrierWorker` and is shared by actors created From f013ba31f722e5d22caad9bee71a3299b20acf5f Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 19 Jul 2024 15:29:40 +0800 Subject: [PATCH 02/19] fix test compile --- src/meta/src/barrier/rpc.rs | 16 ++++----- src/stream/src/task/barrier_manager.rs | 14 +++++--- .../src/task/barrier_manager/managed_state.rs | 33 ++++++++++--------- src/stream/src/task/mod.rs | 6 ++-- 4 files changed, 37 insertions(+), 32 deletions(-) diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 3901e7d9bb18..34d70a441941 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -179,20 +179,18 @@ impl ControlStreamManager { pub(super) async fn next_complete_barrier_response( &mut self, ) -> (WorkerId, MetaResult) { - loop { + { let (worker_id, result) = pending_on_none(self.next_response()).await; match result { Ok(resp) => match resp.response { Some(streaming_control_stream_response::Response::CompleteBarrier(resp)) => { assert_eq!(worker_id, resp.worker_id); - return (worker_id, Ok(resp)); - } - resp => { - break ( - worker_id, - Err(anyhow!("get unexpected resp: {:?}", resp).into()), - ); + (worker_id, Ok(resp)) } + resp => ( + worker_id, + Err(anyhow!("get unexpected resp: {:?}", resp).into()), + ), }, Err(err) => { let node = self @@ -203,7 +201,7 @@ impl ControlStreamManager { warn!(node = ?node.worker, err = %err.as_report(), "get error from response stream"); let errors = self.collect_errors(node.worker.id, err).await; let err = merge_node_rpc_errors("get error from control stream", errors); - break (worker_id, Err(err)); + (worker_id, Err(err)) } } } diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 421c48883f71..29093dc60740 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -1039,6 +1039,7 @@ impl LocalBarrierManager { #[cfg(test)] pub(crate) mod barrier_test_utils { + use std::collections::HashMap; use std::sync::Arc; use assert_matches::assert_matches; @@ -1046,7 +1047,7 @@ pub(crate) mod barrier_test_utils { use risingwave_pb::stream_service::streaming_control_stream_request::InitRequest; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, InjectBarrierRequest, - StreamingControlStreamRequest, StreamingControlStreamResponse, + PbPartialGraphInfo, StreamingControlStreamRequest, StreamingControlStreamResponse, }; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; use tokio_stream::wrappers::UnboundedReceiverStream; @@ -1108,9 +1109,14 @@ pub(crate) mod barrier_test_utils { InjectBarrierRequest { request_id: "".to_string(), barrier: Some(barrier.to_protobuf()), - actor_ids_to_send: actor_to_send.into_iter().collect(), - actor_ids_to_collect: actor_to_collect.into_iter().collect(), - table_ids_to_sync: vec![], + graph_info: HashMap::from_iter([( + u32::MAX, + PbPartialGraphInfo { + actor_ids_to_send: actor_to_send.into_iter().collect(), + actor_ids_to_collect: actor_to_collect.into_iter().collect(), + table_ids_to_sync: vec![], + }, + )]), }, )), })) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 93aeb1e26ac0..9ff59bbca5d7 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -244,6 +244,16 @@ impl PartialGraphManagedBarrierState { barrier_await_tree_reg, } } + + #[cfg(test)] + pub(crate) fn for_test() -> Self { + Self::new( + true, + StateStoreImpl::for_test(), + Arc::new(StreamingMetrics::unused()), + None, + ) + } } pub(super) struct ManagedBarrierState { @@ -260,15 +270,6 @@ pub(super) struct ManagedBarrierState { } impl ManagedBarrierState { - #[cfg(test)] - pub(crate) fn for_test() -> Self { - Self::new( - StateStoreImpl::for_test(), - Arc::new(StreamingMetrics::unused()), - None, - ) - } - /// Create a barrier manager state. This will be called only once. pub(super) fn new( state_store: StateStoreImpl, @@ -451,20 +452,20 @@ impl ManagedBarrierState { }) } - pub(super) fn collect(&mut self, actor_id: ActorId, barrier: &Barrier) { + pub(super) fn collect(&mut self, actor_id: ActorId, epoch: EpochPair) { let actor_states = self.actor_states.get_mut(&actor_id).expect("should exist"); let (prev_epoch, partial_graph_id) = actor_states .inflight_barriers .pop_first() .expect("should not be empty"); - assert_eq!(prev_epoch, barrier.epoch.prev); + assert_eq!(prev_epoch, epoch.prev); if actor_states.is_stopped && actor_states.inflight_barriers.is_empty() { self.actor_states.remove(&actor_id); } self.graph_states .get_mut(&partial_graph_id) .expect("should exist") - .collect(actor_id, barrier); + .collect(actor_id, epoch); } } @@ -783,7 +784,7 @@ impl PartialGraphManagedBarrierState { #[cfg(test)] async fn pop_next_completed_epoch(&mut self) -> u64 { - let epoch = self.next_completed_epoch().await; + let epoch = poll_fn(|cx| self.poll_next_completed_epoch(cx)).await; let _ = self.pop_completed_epoch(epoch).unwrap().unwrap(); epoch } @@ -796,11 +797,11 @@ mod tests { use risingwave_common::util::epoch::test_epoch; use crate::executor::Barrier; - use crate::task::barrier_manager::managed_state::ManagedBarrierState; + use crate::task::barrier_manager::managed_state::PartialGraphManagedBarrierState; #[tokio::test] async fn test_managed_state_add_actor() { - let mut managed_barrier_state = ManagedBarrierState::for_test(); + let mut managed_barrier_state = PartialGraphManagedBarrierState::for_test(); let barrier1 = Barrier::new_test_barrier(test_epoch(1)); let barrier2 = Barrier::new_test_barrier(test_epoch(2)); let barrier3 = Barrier::new_test_barrier(test_epoch(3)); @@ -850,7 +851,7 @@ mod tests { #[tokio::test] async fn test_managed_state_stop_actor() { - let mut managed_barrier_state = ManagedBarrierState::for_test(); + let mut managed_barrier_state = PartialGraphManagedBarrierState::for_test(); let barrier1 = Barrier::new_test_barrier(test_epoch(1)); let barrier2 = Barrier::new_test_barrier(test_epoch(2)); let barrier3 = Barrier::new_test_barrier(test_epoch(3)); diff --git a/src/stream/src/task/mod.rs b/src/stream/src/task/mod.rs index 1eabe00ce725..a889582299e4 100644 --- a/src/stream/src/task/mod.rs +++ b/src/stream/src/task/mod.rs @@ -49,9 +49,9 @@ impl PartialGraphId { } } -impl Into for PartialGraphId { - fn into(self) -> u32 { - self.0 +impl From for u32 { + fn from(val: PartialGraphId) -> u32 { + val.0 } } From cec9ba05447f2b74bed73fee6262ee4507b81c56 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 19 Jul 2024 17:11:45 +0800 Subject: [PATCH 03/19] fix test --- proto/stream_service.proto | 4 ++- src/meta/src/stream/stream_manager.rs | 48 ++++++++++++++++----------- 2 files changed, 32 insertions(+), 20 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 0b58744eec3c..802cde35dfca 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -61,7 +61,9 @@ message PartialGraphInfo { message InjectBarrierRequest { string request_id = 1; stream_plan.Barrier barrier = 2; - map graph_info = 3; + reserved 3, 4, 5; + reserved "actor_ids_to_send", "actor_ids_to_collect", "table_ids_to_sync"; + map graph_info = 6; } message BarrierCompleteResponse { diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 2756d71a8a6c..dc6e6661a031 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -834,7 +834,6 @@ mod tests { #[cfg(feature = "failpoints")] use tokio::sync::Notify; use tokio::task::JoinHandle; - use tokio::time::sleep; use tokio_stream::wrappers::UnboundedReceiverStream; use tonic::{Request, Response, Status, Streaming}; @@ -844,7 +843,7 @@ mod tests { use crate::manager::sink_coordination::SinkCoordinatorManager; use crate::manager::{ CatalogManager, CatalogManagerRef, ClusterManager, FragmentManager, FragmentManagerRef, - RelationIdEnum, StreamingClusterInfo, + RelationIdEnum, StreamingClusterInfo, WorkerId, }; use crate::rpc::ddl_controller::DropMode; use crate::rpc::metrics::MetaMetrics; @@ -858,6 +857,7 @@ mod tests { } struct FakeStreamService { + worker_id: WorkerId, inner: Arc, } @@ -925,6 +925,7 @@ mod tests { let (tx, rx) = unbounded_channel(); let mut request_stream = request.into_inner(); let inner = self.inner.clone(); + let worker_id = self.worker_id; let _join_handle = spawn(async move { while let Ok(Some(request)) = request_stream.try_next().await { match request.request.unwrap() { @@ -938,11 +939,21 @@ mod tests { )), })); } - streaming_control_stream_request::Request::InjectBarrier(_) => { + streaming_control_stream_request::Request::InjectBarrier(req) => { + assert_eq!(req.graph_info.len(), 1); let _ = tx.send(Ok(StreamingControlStreamResponse { response: Some( streaming_control_stream_response::Response::CompleteBarrier( - BarrierCompleteResponse::default(), + BarrierCompleteResponse { + epoch: req.barrier.unwrap().epoch.unwrap().prev, + worker_id, + partial_graph_id: *req + .graph_info + .keys() + .next() + .unwrap(), + ..BarrierCompleteResponse::default() + }, ), ), })); @@ -978,21 +989,7 @@ mod tests { actor_infos: Mutex::new(HashMap::new()), }); - let fake_service = FakeStreamService { - inner: state.clone(), - }; - let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel::<()>(); - let stream_srv = StreamServiceServer::new(fake_service); - let join_handle = tokio::spawn(async move { - tonic::transport::Server::builder() - .add_service(stream_srv) - .serve_with_shutdown(addr, async move { shutdown_rx.await.unwrap() }) - .await - .unwrap(); - }); - - sleep(Duration::from_secs(1)).await; let env = MetaSrvEnv::for_test_opts(MetaOpts::test(enable_recovery)).await; let system_params = env.system_params_reader().await; @@ -1004,7 +1001,7 @@ mod tests { port: port as i32, }; let fake_parallelism = 4; - cluster_manager + let worker_node = cluster_manager .add_worker_node( WorkerType::ComputeNode, host.clone(), @@ -1019,6 +1016,19 @@ mod tests { .await?; cluster_manager.activate_worker_node(host).await?; + let fake_service = FakeStreamService { + worker_id: worker_node.id, + inner: state.clone(), + }; + let stream_srv = StreamServiceServer::new(fake_service); + let join_handle = tokio::spawn(async move { + tonic::transport::Server::builder() + .add_service(stream_srv) + .serve_with_shutdown(addr, async move { shutdown_rx.await.unwrap() }) + .await + .unwrap(); + }); + let catalog_manager = Arc::new(CatalogManager::new(env.clone()).await?); let fragment_manager = Arc::new(FragmentManager::new(env.clone()).await?); From 9e5f205eeeb861293396012b49b8edffe667ee06 Mon Sep 17 00:00:00 2001 From: William Wen Date: Sun, 21 Jul 2024 17:55:02 +0800 Subject: [PATCH 04/19] allow skip node with empty actor to collect --- src/meta/src/barrier/rpc.rs | 8 +------- .../src/task/barrier_manager/managed_state.rs | 13 +++++++------ 2 files changed, 8 insertions(+), 13 deletions(-) diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 34d70a441941..6a1cf4b49315 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -250,6 +250,7 @@ impl ControlStreamManager { if actor_ids_to_collect.is_empty() { // No need to send or collect barrier for this node. assert!(actor_ids_to_send.is_empty()); + return Ok(()); } let graph_info = HashMap::from_iter([( u32::MAX, @@ -266,13 +267,6 @@ impl ControlStreamManager { { let Some(node) = self.nodes.get_mut(node_id) else { - if graph_info - .values() - .all(|info| info.actor_ids_to_collect.is_empty()) - { - // Worker node get disconnected but has no actor to collect. Simply skip it. - return Ok(()); - } return Err( anyhow!("unconnected worker node: {:?}", worker_node.host).into() ); diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 9ff59bbca5d7..8e9734ccbf4a 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -671,17 +671,18 @@ impl PartialGraphManagedBarrierState { "non empty table_ids at initial barrier: {:?}", self.prev_barrier_table_ids ); - info!(epoch = ?barrier.epoch, "initialize at Initial barrier"); + debug!(epoch = ?barrier.epoch, "initialize at Initial barrier"); self.prev_barrier_table_ids = Some((barrier.epoch, table_ids)); None } BarrierKind::Barrier => { - if let Some((prev_epoch, prev_table_ids)) = self.prev_barrier_table_ids.as_mut() { - assert_eq!(prev_epoch.curr, barrier.epoch.prev); + if let Some((prev_epoch, prev_table_ids)) = self.prev_barrier_table_ids.as_mut() + && prev_epoch.curr == barrier.epoch.prev + { assert_eq!(prev_table_ids, &table_ids); *prev_epoch = barrier.epoch; } else { - info!(epoch = ?barrier.epoch, "initialize at non-checkpoint barrier"); + debug!(epoch = ?barrier.epoch, "reinitialize at non-checkpoint barrier"); self.prev_barrier_table_ids = Some((barrier.epoch, table_ids)); } None @@ -690,11 +691,11 @@ impl PartialGraphManagedBarrierState { if let Some((prev_epoch, prev_table_ids)) = self .prev_barrier_table_ids .replace((barrier.epoch, table_ids)) + && prev_epoch.curr == barrier.epoch.prev { - assert_eq!(prev_epoch.curr, barrier.epoch.prev); prev_table_ids } else { - info!(epoch = ?barrier.epoch, "initialize at Checkpoint barrier"); + debug!(epoch = ?barrier.epoch, "reinitialize at Checkpoint barrier"); HashSet::new() }, ), From ea42c2919d5fd857ed896da08232c2db619fc489 Mon Sep 17 00:00:00 2001 From: William Wen Date: Sun, 21 Jul 2024 22:37:20 +0800 Subject: [PATCH 05/19] refactor --- src/stream/src/task/barrier_manager.rs | 3 +- .../src/task/barrier_manager/managed_state.rs | 459 +++++++++++++----- .../src/task/barrier_manager/progress.rs | 3 +- 3 files changed, 334 insertions(+), 131 deletions(-) diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 29093dc60740..d1468325169b 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -785,7 +785,8 @@ impl LocalBarrierWorker { let root_err = self.try_find_root_failure().await.unwrap(); // always `Some` because we just added one if let Some(actor_state) = self.state.actor_states.get(&actor_id) - && !actor_state.inflight_barriers.is_empty() + && let Some(inflight_barriers) = actor_state.inflight_barriers() + && !inflight_barriers.is_empty() { self.control_stream_handle.reset_stream_with_err( anyhow!(root_err) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 8e9734ccbf4a..a04cf3378366 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -42,6 +42,7 @@ use super::{BarrierCompleteResult, SubscribeMutationItem}; use crate::error::StreamResult; use crate::executor::monitor::StreamingMetrics; use crate::executor::{Barrier, Mutation}; +use crate::task::barrier_manager::managed_state::actor_status::InflightActorState; use crate::task::{await_tree_key, ActorId, PartialGraphId}; struct IssuedState { @@ -192,13 +193,239 @@ impl Display for &'_ PartialGraphManagedBarrierState { } } -#[derive(Default)] -pub(super) struct InflightActorState { - pending_subscribers: BTreeMap>>, - started_subscribers: Vec>, - // epoch -> partial graph id - pub(super) inflight_barriers: BTreeMap, - is_stopped: bool, +mod actor_status { + use std::collections::BTreeMap; + use std::sync::Arc; + + use risingwave_common::must_match; + use risingwave_common::util::epoch::EpochPair; + use tokio::sync::mpsc; + + use crate::executor::{Barrier, Mutation}; + use crate::task::{PartialGraphId, SubscribeMutationItem}; + + enum InflightActorStatus { + /// The actor has not been issued any barrier yet + NotStarted, + /// The actor has been issued some barriers, but has collected all the barrier. + /// Waiting for new barrier to issue. + Pending { + /// The latest `partial_graph_id` before entering `Pending` status. + /// The actor should be in the `inflight_actors` of the graph. + prev_partial_graph_id: PartialGraphId, + /// The `prev_epoch` of the previous barrier + prev_epoch: u64, + }, + /// The actor has been issued with some barriers, and waiting for collecting some barriers. + Running { + /// `prev_epoch` -> partial graph id + /// Store the barriers that has been issued but not collected. + /// Must be non-empty when in this variant, or transit to `Pending`, or the states gets removed when stopped. + /// + /// The actor should be in the `inflight_actors` of graph whose `partial_graph_id` of the first graph id. + inflight_barriers: BTreeMap>)>, + /// Whether the actor has been issued a stop barrier + is_stopping: bool, + }, + } + + pub(crate) struct InflightActorState { + pending_subscribers: BTreeMap>>, + started_subscribers: Vec>, + status: InflightActorStatus, + } + + impl InflightActorState { + pub(super) fn not_started() -> Self { + Self { + pending_subscribers: Default::default(), + started_subscribers: vec![], + status: InflightActorStatus::NotStarted, + } + } + + #[expect(clippy::type_complexity)] + pub(crate) fn inflight_barriers( + &self, + ) -> Option<&BTreeMap>)>> { + if let InflightActorStatus::Running { + inflight_barriers, .. + } = &self.status + { + Some(inflight_barriers) + } else { + None + } + } + + pub(super) fn subscribe_actor_mutation( + &mut self, + start_prev_epoch: u64, + tx: mpsc::UnboundedSender, + ) { + match &self.status { + InflightActorStatus::NotStarted => { + self.pending_subscribers + .entry(start_prev_epoch) + .or_default() + .push(tx); + } + InflightActorStatus::Pending { prev_epoch, .. } => { + assert!(*prev_epoch < start_prev_epoch); + self.pending_subscribers + .entry(start_prev_epoch) + .or_default() + .push(tx); + } + InflightActorStatus::Running { + inflight_barriers, + is_stopping, + .. + } => { + if inflight_barriers.contains_key(&start_prev_epoch) { + for (prev_epoch, (_, mutation)) in + inflight_barriers.range(start_prev_epoch..) + { + if tx.send((*prev_epoch, mutation.clone())).is_err() { + // No more subscribe on the mutation. Simply return. + return; + } + } + if !*is_stopping { + self.started_subscribers.push(tx); + } + } else { + // Barrier has not issued yet. Store the pending tx + if let Some((last_epoch, _)) = inflight_barriers.last_key_value() { + assert!( + *last_epoch < start_prev_epoch, + "later barrier {} has been issued, but skip the start epoch {:?}", + last_epoch, + start_prev_epoch + ); + } + self.pending_subscribers + .entry(start_prev_epoch) + .or_default() + .push(tx); + } + } + } + } + + #[must_use] + pub(super) fn issue_barrier( + &mut self, + partial_graph_id: PartialGraphId, + barrier: &Barrier, + is_stop: bool, + ) -> ( + // Some(prev_partial_graph_id) when the actor was in status + // InflightActorStatus::Pending { .. } with a different graph id + Option, + // whether the actor is new to the `partial_graph_id` + bool, + ) { + let (prev_partial_graph_id, is_new_in_graph) = match &self.status { + InflightActorStatus::NotStarted => { + self.status = InflightActorStatus::Running { + inflight_barriers: Default::default(), + is_stopping: false, + }; + (None, true) + } + InflightActorStatus::Pending { + prev_partial_graph_id, + prev_epoch, + } => { + assert!(*prev_epoch < barrier.epoch.prev); + let prev_partial_graph_id = *prev_partial_graph_id; + self.status = InflightActorStatus::Running { + inflight_barriers: Default::default(), + is_stopping: false, + }; + if prev_partial_graph_id != partial_graph_id { + (Some(prev_partial_graph_id), true) + } else { + (None, false) + } + } + InflightActorStatus::Running { + inflight_barriers, .. + } => { + let (prev_epoch, (prev_partial_graph_id, _)) = + inflight_barriers.last_key_value().expect("non-empty"); + assert!(*prev_epoch < barrier.epoch.prev); + (None, *prev_partial_graph_id != partial_graph_id) + } + }; + + if let Some((first_epoch, _)) = self.pending_subscribers.first_key_value() { + assert!( + *first_epoch >= barrier.epoch.prev, + "barrier epoch {:?} skip subscribed epoch {}", + barrier.epoch, + first_epoch + ); + if *first_epoch == barrier.epoch.prev { + self.started_subscribers.extend( + self.pending_subscribers + .pop_first() + .expect("should exist") + .1, + ); + } + } + self.started_subscribers.retain(|tx| { + tx.send((barrier.epoch.prev, barrier.mutation.clone())) + .is_ok() + }); + + must_match!(&mut self.status, InflightActorStatus::Running { + inflight_barriers, is_stopping, + } => { + inflight_barriers.insert(barrier.epoch.prev, (partial_graph_id, barrier.mutation.clone())); + *is_stopping = is_stop; + }); + + (prev_partial_graph_id, is_new_in_graph) + } + + #[must_use] + pub(super) fn collect( + &mut self, + epoch: EpochPair, + ) -> ( + // The `partial_graph_id` of actor on the collected epoch + PartialGraphId, + // None => the partial graph id of this actor is not changed + // Some(None) => the actor has stopped, and should be removed from the return `partial_graph_id` + // Some(Some(new_partial_graph_id)) => the actor will move to the `new_partial_graph_id` + Option>, + ) { + must_match!(&mut self.status, InflightActorStatus::Running { + inflight_barriers, is_stopping + } => { + let (prev_epoch, (prev_partial_graph_id, _)) = inflight_barriers.pop_first().expect("should exist"); + assert_eq!(prev_epoch, epoch.prev); + let move_to_graph_id = if let Some((epoch, (graph_id, _))) = inflight_barriers.first_key_value() { + if *graph_id != prev_partial_graph_id { + Some(Some((prev_partial_graph_id, *epoch))) + } else { + None + } + } else if *is_stopping { + Some(None) + } else { + self.status = InflightActorStatus::Pending {prev_epoch, prev_partial_graph_id}; + // No need to move to any partial graph when transit to `Pending`. When issuing the next barrier and + // the next graph id gets different, the actor will then move to the next graph id + None + }; + (prev_partial_graph_id, move_to_graph_id) + }) + } + } } pub(super) struct PartialGraphManagedBarrierState { @@ -210,6 +437,8 @@ pub(super) struct PartialGraphManagedBarrierState { /// The key is `prev_epoch`, and the first value is `curr_epoch` epoch_barrier_state_map: BTreeMap, + inflight_actors: HashSet, + prev_barrier_table_ids: Option<(EpochPair, HashSet)>, /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. @@ -236,6 +465,7 @@ impl PartialGraphManagedBarrierState { Self { need_seal_epoch, epoch_barrier_state_map: Default::default(), + inflight_actors: Default::default(), prev_barrier_table_ids: None, create_mview_progress: Default::default(), await_epoch_completed_futures: Default::default(), @@ -297,78 +527,10 @@ impl ManagedBarrierState { start_prev_epoch: u64, tx: mpsc::UnboundedSender, ) { - let actor_state = self.actor_states.entry(actor_id).or_default(); - if let Some(start_partial_graph_id) = actor_state.inflight_barriers.get(&start_prev_epoch) { - let start_graph_state = self - .graph_states - .get(start_partial_graph_id) - .expect("should exist") - .epoch_barrier_state_map - .get(&start_prev_epoch) - .expect("should exist"); - match &start_graph_state.inner { - ManagedBarrierStateInner::Issued(issued_state) => { - assert!(issued_state.remaining_actors.contains(&actor_id)); - for (prev_epoch, partial_graph_id) in - actor_state.inflight_barriers.range(start_prev_epoch..) - { - let graph_state = self - .graph_states - .get(partial_graph_id) - .expect("should exist") - .epoch_barrier_state_map - .get(prev_epoch) - .expect("should exist"); - match &graph_state.inner { - ManagedBarrierStateInner::Issued(issued_state) => { - if issued_state.remaining_actors.contains(&actor_id) { - if tx - .send((*prev_epoch, issued_state.mutation.clone())) - .is_err() - { - // No more subscribe on the mutation. Simply return. - return; - } - } else { - // The barrier no more collect from such actor. End subscribe on mutation. - return; - } - } - state @ ManagedBarrierStateInner::AllCollected - | state @ ManagedBarrierStateInner::Completed(_) => { - unreachable!( - "should be Issued when having new subscriber, but current state: {:?}", - state - ) - } - } - } - actor_state.started_subscribers.push(tx); - } - state @ ManagedBarrierStateInner::AllCollected - | state @ ManagedBarrierStateInner::Completed(_) => { - unreachable!( - "should be Issued when having new subscriber, but current state: {:?}", - state - ) - } - } - } else { - // Barrier has not issued yet. Store the pending tx - if let Some((last_epoch, _)) = actor_state.inflight_barriers.last_key_value() { - assert!( - *last_epoch < start_prev_epoch, - "later barrier {} has been issued, but skip the start epoch {:?}", - last_epoch, - start_prev_epoch - ); - } - actor_state - .pending_subscribers - .entry(start_prev_epoch) - .or_default() - .push(tx); - } + self.actor_states + .entry(actor_id) + .or_insert_with(InflightActorState::not_started) + .subscribe_actor_mutation(start_prev_epoch, tx); } pub(super) fn transform_to_issued( @@ -377,43 +539,9 @@ impl ManagedBarrierState { graph_infos: &HashMap, ) { let actor_to_stop = barrier.all_stop_actors(); + let mut graph_actors_to_clean: HashMap> = HashMap::new(); for (partial_graph_id, graph_info) in graph_infos { let partial_graph_id = PartialGraphId::new(*partial_graph_id); - for actor_id in &graph_info.actor_ids_to_collect { - let actor_state = self.actor_states.entry(*actor_id).or_default(); - if let Some((first_epoch, _)) = actor_state.pending_subscribers.first_key_value() { - assert!( - *first_epoch >= barrier.epoch.prev, - "barrier epoch {:?} skip subscribed epoch {}", - barrier.epoch, - first_epoch - ); - if *first_epoch == barrier.epoch.prev { - actor_state.started_subscribers.extend( - actor_state - .pending_subscribers - .pop_first() - .expect("should exist") - .1, - ); - } - } - actor_state.started_subscribers.retain(|tx| { - tx.send((barrier.epoch.prev, barrier.mutation.clone())) - .is_ok() - }); - if let Some((prev_epoch, _)) = actor_state.inflight_barriers.first_key_value() { - assert!(*prev_epoch < barrier.epoch.prev); - } - actor_state - .inflight_barriers - .insert(barrier.epoch.prev, partial_graph_id); - if let Some(actor_to_stop) = actor_to_stop - && actor_to_stop.contains(actor_id) - { - actor_state.is_stopped = true; - } - } let graph_state = self .graph_states @@ -426,6 +554,7 @@ impl ManagedBarrierState { self.barrier_await_tree_reg.clone(), ) }); + graph_state.transform_to_issued( barrier, graph_info.actor_ids_to_collect.iter().cloned().collect(), @@ -435,6 +564,36 @@ impl ManagedBarrierState { .map(|table_id| TableId::new(*table_id)) .collect(), ); + + for actor_id in &graph_info.actor_ids_to_collect { + let (prev_partial_graph_id, is_new_in_graph) = self + .actor_states + .entry(*actor_id) + .or_insert_with(InflightActorState::not_started) + .issue_barrier( + partial_graph_id, + barrier, + actor_to_stop + .map(|actors| actors.contains(actor_id)) + .unwrap_or(false), + ); + if is_new_in_graph { + graph_state.add_inflight_actor(*actor_id, barrier.epoch.prev); + } + if let Some(prev_partial_graph_id) = prev_partial_graph_id { + graph_actors_to_clean + .entry(prev_partial_graph_id) + .or_default() + .push(*actor_id); + } + } + } + + for (graph_id, actors_to_clean) in graph_actors_to_clean { + let graph_state = self.graph_states.get_mut(&graph_id).expect("should exist"); + if graph_state.remove_inflight_actors(actors_to_clean) { + self.graph_states.remove(&graph_id); + } } } @@ -443,9 +602,12 @@ impl ManagedBarrierState { ) -> impl Future + '_ { poll_fn(|cx| { for (partial_graph_id, graph_state) in &mut self.graph_states { - let poll = graph_state.poll_next_completed_epoch(cx); - if poll.is_ready() { - return poll.map(|epoch| (*partial_graph_id, epoch)); + if let Poll::Ready(epoch) = graph_state.poll_next_completed_epoch(cx) { + let partial_graph_id = *partial_graph_id; + if graph_state.is_empty() { + self.graph_states.remove(&partial_graph_id); + } + return Poll::Ready((partial_graph_id, epoch)); } } Poll::Pending @@ -453,23 +615,62 @@ impl ManagedBarrierState { } pub(super) fn collect(&mut self, actor_id: ActorId, epoch: EpochPair) { - let actor_states = self.actor_states.get_mut(&actor_id).expect("should exist"); - let (prev_epoch, partial_graph_id) = actor_states - .inflight_barriers - .pop_first() - .expect("should not be empty"); - assert_eq!(prev_epoch, epoch.prev); - if actor_states.is_stopped && actor_states.inflight_barriers.is_empty() { - self.actor_states.remove(&actor_id); - } - self.graph_states - .get_mut(&partial_graph_id) + let (prev_partial_graph_id, move_to_partial_graph_id) = self + .actor_states + .get_mut(&actor_id) .expect("should exist") - .collect(actor_id, epoch); + .collect(epoch); + let prev_graph_state = self + .graph_states + .get_mut(&prev_partial_graph_id) + .expect("should exist"); + prev_graph_state.collect(actor_id, epoch); + if let Some(move_to_partial_graph_id) = move_to_partial_graph_id { + if prev_graph_state.remove_inflight_actors([actor_id]) { + self.graph_states.remove(&prev_partial_graph_id); + } + if let Some((move_to_partial_graph_id, start_epoch)) = move_to_partial_graph_id { + self.graph_states + .get_mut(&move_to_partial_graph_id) + .expect("should exist") + .add_inflight_actor(actor_id, start_epoch); + } else { + self.actor_states.remove(&actor_id); + } + } } } impl PartialGraphManagedBarrierState { + fn is_empty(&self) -> bool { + self.inflight_actors.is_empty() + && self.epoch_barrier_state_map.is_empty() + && self.await_epoch_completed_futures.is_empty() + } + + fn add_inflight_actor(&mut self, actor_id: ActorId, start_epoch: u64) { + assert!(self.inflight_actors.insert(actor_id)); + must_match!(&self.epoch_barrier_state_map.get(&start_epoch).expect("should exist").inner, ManagedBarrierStateInner::Issued(state) => { + state.remaining_actors.contains(&actor_id); + }); + if cfg!(debug_assertions) { + for (_, state) in self.epoch_barrier_state_map.range(..start_epoch) { + if let ManagedBarrierStateInner::Issued(state) = &state.inner { + // ensure that start_epoch is the first epoch to collect the barrier + assert!(!state.remaining_actors.contains(&actor_id)); + } + } + } + } + + #[must_use] + fn remove_inflight_actors(&mut self, actor_ids: impl IntoIterator) -> bool { + for actor_id in actor_ids { + assert!(self.inflight_actors.remove(&actor_id)); + } + self.is_empty() + } + /// This method is called when barrier state is modified in either `Issued` or `Stashed` /// to transform the state to `AllCollected` and start state store `sync` when the barrier /// has been collected from all actors for an `Issued` barrier. diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 4fe6e98d4cf9..7a51f1de0d72 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -49,7 +49,8 @@ impl LocalBarrierWorker { state: BackfillState, ) { 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(inflight_barriers) = actor_state.inflight_barriers() + && let Some((partial_graph_id, _)) = inflight_barriers.get(&epoch.prev) && let Some(graph_state) = self.state.graph_states.get_mut(partial_graph_id) { graph_state From 762c48a58be8b24473157aefe69e301ae03dea32 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 22 Jul 2024 13:43:04 +0800 Subject: [PATCH 06/19] disallow skip empty node --- src/meta/src/barrier/rpc.rs | 8 +++++++- src/stream/src/task/barrier_manager/managed_state.rs | 9 ++++----- 2 files changed, 11 insertions(+), 6 deletions(-) diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index 6a1cf4b49315..34d70a441941 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -250,7 +250,6 @@ impl ControlStreamManager { if actor_ids_to_collect.is_empty() { // No need to send or collect barrier for this node. assert!(actor_ids_to_send.is_empty()); - return Ok(()); } let graph_info = HashMap::from_iter([( u32::MAX, @@ -267,6 +266,13 @@ impl ControlStreamManager { { let Some(node) = self.nodes.get_mut(node_id) else { + if graph_info + .values() + .all(|info| info.actor_ids_to_collect.is_empty()) + { + // Worker node get disconnected but has no actor to collect. Simply skip it. + return Ok(()); + } return Err( anyhow!("unconnected worker node: {:?}", worker_node.host).into() ); diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index a04cf3378366..3502b2677982 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -872,18 +872,17 @@ impl PartialGraphManagedBarrierState { "non empty table_ids at initial barrier: {:?}", self.prev_barrier_table_ids ); - debug!(epoch = ?barrier.epoch, "initialize at Initial barrier"); + info!(epoch = ?barrier.epoch, "initialize at Initial barrier"); self.prev_barrier_table_ids = Some((barrier.epoch, table_ids)); None } BarrierKind::Barrier => { - if let Some((prev_epoch, prev_table_ids)) = self.prev_barrier_table_ids.as_mut() - && prev_epoch.curr == barrier.epoch.prev - { + if let Some((prev_epoch, prev_table_ids)) = self.prev_barrier_table_ids.as_mut() { + assert_eq!(prev_epoch.curr, barrier.epoch.prev); assert_eq!(prev_table_ids, &table_ids); *prev_epoch = barrier.epoch; } else { - debug!(epoch = ?barrier.epoch, "reinitialize at non-checkpoint barrier"); + info!(epoch = ?barrier.epoch, "initialize at non-checkpoint barrier"); self.prev_barrier_table_ids = Some((barrier.epoch, table_ids)); } None From c2bd4e6b2647a86a6cd12cc33a162e2fc2b7e99e Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 22 Jul 2024 13:50:03 +0800 Subject: [PATCH 07/19] remove unrelated code --- .../executor/backfill/snapshot_backfill.rs | 130 ------------------ 1 file changed, 130 deletions(-) delete mode 100644 src/stream/src/executor/backfill/snapshot_backfill.rs diff --git a/src/stream/src/executor/backfill/snapshot_backfill.rs b/src/stream/src/executor/backfill/snapshot_backfill.rs deleted file mode 100644 index 85b55a5f42b4..000000000000 --- a/src/stream/src/executor/backfill/snapshot_backfill.rs +++ /dev/null @@ -1,130 +0,0 @@ -// Copyright 2024 RisingWave Labs -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -use std::pin::pin; - -use futures::TryStreamExt; -use risingwave_common::array::{Op, StreamChunk}; -use risingwave_common::util::chunk_coalesce::DataChunkBuilder; -use risingwave_hummock_sdk::HummockReadEpoch; -use risingwave_storage::table::batch_table::storage_table::StorageTable; -use risingwave_storage::StateStore; - -use crate::executor::backfill::utils::mapping_chunk; -use crate::executor::prelude::{try_stream, StreamExt}; -use crate::executor::{ - expect_first_barrier, ActorContextRef, BackfillExecutor, BoxedMessageStream, Execute, Executor, - Message, StreamExecutorError, -}; -use crate::task::CreateMviewProgress; - -/// Similar to [`super::no_shuffle_backfill::BackfillExecutor`]. -/// Main differences: -/// - [`crate::executor::ArrangementBackfillExecutor`] can reside on a different CN, so it can be scaled -/// independently. -/// - To synchronize upstream shared buffer, it is initialized with a [`ReplicatedStateTable`]. -pub struct SnapshotBackfillExecutor { - /// Upstream table - upstream_table: StorageTable, - - /// Upstream with the same schema with the upstream table. - upstream: Executor, - - /// The column indices need to be forwarded to the downstream from the upstream and table scan. - output_indices: Vec, - - progress: CreateMviewProgress, - - chunk_size: usize, -} - -impl SnapshotBackfillExecutor { - pub fn new( - upstream_table: StorageTable, - upstream: Executor, - output_indices: Vec, - _actor_ctx: ActorContextRef, - progress: CreateMviewProgress, - chunk_size: usize, - ) -> Self { - Self { - upstream_table, - upstream, - output_indices, - progress, - chunk_size, - } - } - - #[try_stream(ok = Message, error = StreamExecutorError)] - async fn execute_inner(mut self) { - let upstream = self.upstream.execute(); - let mut upstream = pin!(upstream); - let first_barrier = expect_first_barrier(&mut upstream).await?; - let prev_epoch = first_barrier.epoch.prev; - yield Message::Barrier(first_barrier); - let snapshot_stream = BackfillExecutor::make_snapshot_stream( - &self.upstream_table, - HummockReadEpoch::Committed(prev_epoch), - None, - false, - &None, - ); - let mut count = 0; - let mut snapshot_stream = pin!(snapshot_stream); - let data_types = self.upstream_table.schema().data_types(); - let mut builder = DataChunkBuilder::new(data_types, self.chunk_size); - while let Some(row) = snapshot_stream.try_next().await? { - count += 1; - if let Some(data_chunk) = builder.append_one_row(row) { - let ops = vec![Op::Insert; data_chunk.capacity()]; - yield Message::Chunk(mapping_chunk( - StreamChunk::from_parts(ops, data_chunk), - &self.output_indices, - )); - } - } - if let Some(data_chunk) = builder.consume_all() { - let ops = vec![Op::Insert; data_chunk.capacity()]; - yield Message::Chunk(mapping_chunk( - StreamChunk::from_parts(ops, data_chunk), - &self.output_indices, - )); - } - let barrier = loop { - match upstream.try_next().await? { - None => { - return Ok(()); - } - Some(Message::Barrier(barrier)) => { - break barrier; - } - Some(msg) => { - yield msg; - } - } - }; - self.progress.finish(barrier.epoch, count); - yield Message::Barrier(barrier); - while let Some(msg) = upstream.try_next().await? { - yield msg; - } - } -} - -impl Execute for SnapshotBackfillExecutor { - fn execute(self: Box) -> BoxedMessageStream { - self.execute_inner().boxed() - } -} From 4ebc49ee79f8b677e5ea9dbcab3923e238b8653d Mon Sep 17 00:00:00 2001 From: William Wen Date: Wed, 24 Jul 2024 18:47:16 +0800 Subject: [PATCH 08/19] apply fix --- proto/stream_service.proto | 6 +++--- src/stream/src/task/barrier_manager/managed_state.rs | 9 ++++----- src/stream/src/task/mod.rs | 4 ++++ 3 files changed, 11 insertions(+), 8 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 802cde35dfca..100ad78dffbd 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -53,9 +53,9 @@ message DropActorsResponse { } message PartialGraphInfo { - repeated uint32 actor_ids_to_send = 3; - repeated uint32 actor_ids_to_collect = 4; - repeated uint32 table_ids_to_sync = 5; + repeated uint32 actor_ids_to_send = 1; + repeated uint32 actor_ids_to_collect = 2; + repeated uint32 table_ids_to_sync = 3; } message InjectBarrierRequest { diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 3502b2677982..91c46d8559a5 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -353,10 +353,9 @@ mod actor_status { InflightActorStatus::Running { inflight_barriers, .. } => { - let (prev_epoch, (prev_partial_graph_id, _)) = - inflight_barriers.last_key_value().expect("non-empty"); + let (prev_epoch, _) = inflight_barriers.last_key_value().expect("non-empty"); assert!(*prev_epoch < barrier.epoch.prev); - (None, *prev_partial_graph_id != partial_graph_id) + (None, false) } }; @@ -410,7 +409,7 @@ mod actor_status { assert_eq!(prev_epoch, epoch.prev); let move_to_graph_id = if let Some((epoch, (graph_id, _))) = inflight_barriers.first_key_value() { if *graph_id != prev_partial_graph_id { - Some(Some((prev_partial_graph_id, *epoch))) + Some(Some((*graph_id, *epoch))) } else { None } @@ -548,7 +547,7 @@ impl ManagedBarrierState { .entry(partial_graph_id) .or_insert_with(|| { PartialGraphManagedBarrierState::new( - partial_graph_id.0 == u32::MAX, + partial_graph_id.is_global_graph(), self.state_store.clone(), self.streaming_metrics.clone(), self.barrier_await_tree_reg.clone(), diff --git a/src/stream/src/task/mod.rs b/src/stream/src/task/mod.rs index a889582299e4..5a3c2fee50a3 100644 --- a/src/stream/src/task/mod.rs +++ b/src/stream/src/task/mod.rs @@ -47,6 +47,10 @@ impl PartialGraphId { fn new(id: u32) -> Self { Self(id) } + + fn is_global_graph(&self) -> bool { + self.0 == u32::MAX + } } impl From for u32 { From 043f77d141ec8fe4078ba519427bb1e6af0724d9 Mon Sep 17 00:00:00 2001 From: William Wen Date: Thu, 25 Jul 2024 18:35:09 +0800 Subject: [PATCH 09/19] simplify actor state --- proto/stream_service.proto | 5 + src/meta/src/stream/stream_manager.rs | 1 + src/stream/src/task/barrier_manager.rs | 26 +- .../src/task/barrier_manager/managed_state.rs | 399 +++++------------- .../src/task/barrier_manager/progress.rs | 3 +- 5 files changed, 136 insertions(+), 298 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 100ad78dffbd..c5a63f97cf1c 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -109,9 +109,14 @@ message StreamingControlStreamRequest { uint64 prev_epoch = 2; } + message RemovePartialGraphRequest { + repeated uint32 partial_graph_ids = 1; + } + oneof request { InitRequest init = 1; InjectBarrierRequest inject_barrier = 2; + RemovePartialGraphRequest remove_partial_graph = 3; } } diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index cbf61a6cc731..afd9fde5582a 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -964,6 +964,7 @@ mod tests { ), })); } + streaming_control_stream_request::Request::RemovePartialGraph(..) => {} } } }); diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index 1d430ce5968d..86d2effa06e9 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -495,6 +495,12 @@ impl LocalBarrierWorker { self.send_barrier(barrier, req.graph_info)?; Ok(()) } + Request::RemovePartialGraph(req) => { + self.remove_partial_graphs( + req.partial_graph_ids.into_iter().map(PartialGraphId::new), + ); + Ok(()) + } Request::Init(_) => { unreachable!() } @@ -767,6 +773,23 @@ impl LocalBarrierWorker { Ok(()) } + fn remove_partial_graphs(&mut self, partial_graph_ids: impl Iterator) { + for partial_graph_id in partial_graph_ids { + if let Some(graph) = self.state.graph_states.remove(&partial_graph_id) { + assert!( + graph.is_empty(), + "non empty graph to be removed: {}", + &graph + ); + } else { + warn!( + partial_graph_id = partial_graph_id.0, + "no partial graph to remove" + ); + } + } + } + /// Reset all internal states. pub(super) fn reset_state(&mut self) { *self = Self::new(self.actor_manager.clone()); @@ -785,8 +808,7 @@ impl LocalBarrierWorker { let root_err = self.try_find_root_failure().await.unwrap(); // always `Some` because we just added one if let Some(actor_state) = self.state.actor_states.get(&actor_id) - && let Some(inflight_barriers) = actor_state.inflight_barriers() - && !inflight_barriers.is_empty() + && !actor_state.inflight_barriers.is_empty() { self.control_stream_handle.reset_stream_with_err( anyhow!(root_err) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 91c46d8559a5..88dae3600761 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -42,7 +42,6 @@ use super::{BarrierCompleteResult, SubscribeMutationItem}; use crate::error::StreamResult; use crate::executor::monitor::StreamingMetrics; use crate::executor::{Barrier, Mutation}; -use crate::task::barrier_manager::managed_state::actor_status::InflightActorState; use crate::task::{await_tree_key, ActorId, PartialGraphId}; struct IssuedState { @@ -193,238 +192,73 @@ impl Display for &'_ PartialGraphManagedBarrierState { } } -mod actor_status { - use std::collections::BTreeMap; - use std::sync::Arc; - - use risingwave_common::must_match; - use risingwave_common::util::epoch::EpochPair; - use tokio::sync::mpsc; - - use crate::executor::{Barrier, Mutation}; - use crate::task::{PartialGraphId, SubscribeMutationItem}; - - enum InflightActorStatus { - /// The actor has not been issued any barrier yet - NotStarted, - /// The actor has been issued some barriers, but has collected all the barrier. - /// Waiting for new barrier to issue. - Pending { - /// The latest `partial_graph_id` before entering `Pending` status. - /// The actor should be in the `inflight_actors` of the graph. - prev_partial_graph_id: PartialGraphId, - /// The `prev_epoch` of the previous barrier - prev_epoch: u64, - }, - /// The actor has been issued with some barriers, and waiting for collecting some barriers. - Running { - /// `prev_epoch` -> partial graph id - /// Store the barriers that has been issued but not collected. - /// Must be non-empty when in this variant, or transit to `Pending`, or the states gets removed when stopped. - /// - /// The actor should be in the `inflight_actors` of graph whose `partial_graph_id` of the first graph id. - inflight_barriers: BTreeMap>)>, - /// Whether the actor has been issued a stop barrier - is_stopping: bool, - }, - } - - pub(crate) struct InflightActorState { - pending_subscribers: BTreeMap>>, - started_subscribers: Vec>, - status: InflightActorStatus, - } - - impl InflightActorState { - pub(super) fn not_started() -> Self { - Self { - pending_subscribers: Default::default(), - started_subscribers: vec![], - status: InflightActorStatus::NotStarted, - } - } +pub(crate) struct InflightActorState { + pending_subscribers: BTreeMap>>, + started_subscribers: Vec>, + /// `prev_epoch` -> partial graph id + pub(super) inflight_barriers: BTreeMap>)>, + /// Whether the actor has been issued a stop barrier + is_stopping: bool, +} - #[expect(clippy::type_complexity)] - pub(crate) fn inflight_barriers( - &self, - ) -> Option<&BTreeMap>)>> { - if let InflightActorStatus::Running { - inflight_barriers, .. - } = &self.status - { - Some(inflight_barriers) - } else { - None - } +impl InflightActorState { + pub(super) fn not_started() -> Self { + Self { + pending_subscribers: Default::default(), + started_subscribers: vec![], + inflight_barriers: BTreeMap::default(), + is_stopping: false, } + } - pub(super) fn subscribe_actor_mutation( - &mut self, - start_prev_epoch: u64, - tx: mpsc::UnboundedSender, - ) { - match &self.status { - InflightActorStatus::NotStarted => { - self.pending_subscribers - .entry(start_prev_epoch) - .or_default() - .push(tx); - } - InflightActorStatus::Pending { prev_epoch, .. } => { - assert!(*prev_epoch < start_prev_epoch); + pub(super) fn issue_barrier( + &mut self, + partial_graph_id: PartialGraphId, + barrier: &Barrier, + is_stop: bool, + ) { + assert!(!self.is_stopping, "stopped actor should not issue barrier"); + if let Some((first_epoch, _)) = self.pending_subscribers.first_key_value() { + assert!( + *first_epoch >= barrier.epoch.prev, + "barrier epoch {:?} skip subscribed epoch {}", + barrier.epoch, + first_epoch + ); + if *first_epoch == barrier.epoch.prev { + self.started_subscribers.extend( self.pending_subscribers - .entry(start_prev_epoch) - .or_default() - .push(tx); - } - InflightActorStatus::Running { - inflight_barriers, - is_stopping, - .. - } => { - if inflight_barriers.contains_key(&start_prev_epoch) { - for (prev_epoch, (_, mutation)) in - inflight_barriers.range(start_prev_epoch..) - { - if tx.send((*prev_epoch, mutation.clone())).is_err() { - // No more subscribe on the mutation. Simply return. - return; - } - } - if !*is_stopping { - self.started_subscribers.push(tx); - } - } else { - // Barrier has not issued yet. Store the pending tx - if let Some((last_epoch, _)) = inflight_barriers.last_key_value() { - assert!( - *last_epoch < start_prev_epoch, - "later barrier {} has been issued, but skip the start epoch {:?}", - last_epoch, - start_prev_epoch - ); - } - self.pending_subscribers - .entry(start_prev_epoch) - .or_default() - .push(tx); - } - } - } - } - - #[must_use] - pub(super) fn issue_barrier( - &mut self, - partial_graph_id: PartialGraphId, - barrier: &Barrier, - is_stop: bool, - ) -> ( - // Some(prev_partial_graph_id) when the actor was in status - // InflightActorStatus::Pending { .. } with a different graph id - Option, - // whether the actor is new to the `partial_graph_id` - bool, - ) { - let (prev_partial_graph_id, is_new_in_graph) = match &self.status { - InflightActorStatus::NotStarted => { - self.status = InflightActorStatus::Running { - inflight_barriers: Default::default(), - is_stopping: false, - }; - (None, true) - } - InflightActorStatus::Pending { - prev_partial_graph_id, - prev_epoch, - } => { - assert!(*prev_epoch < barrier.epoch.prev); - let prev_partial_graph_id = *prev_partial_graph_id; - self.status = InflightActorStatus::Running { - inflight_barriers: Default::default(), - is_stopping: false, - }; - if prev_partial_graph_id != partial_graph_id { - (Some(prev_partial_graph_id), true) - } else { - (None, false) - } - } - InflightActorStatus::Running { - inflight_barriers, .. - } => { - let (prev_epoch, _) = inflight_barriers.last_key_value().expect("non-empty"); - assert!(*prev_epoch < barrier.epoch.prev); - (None, false) - } - }; - - if let Some((first_epoch, _)) = self.pending_subscribers.first_key_value() { - assert!( - *first_epoch >= barrier.epoch.prev, - "barrier epoch {:?} skip subscribed epoch {}", - barrier.epoch, - first_epoch + .pop_first() + .expect("should exist") + .1, ); - if *first_epoch == barrier.epoch.prev { - self.started_subscribers.extend( - self.pending_subscribers - .pop_first() - .expect("should exist") - .1, - ); - } } - self.started_subscribers.retain(|tx| { - tx.send((barrier.epoch.prev, barrier.mutation.clone())) - .is_ok() - }); - - must_match!(&mut self.status, InflightActorStatus::Running { - inflight_barriers, is_stopping, - } => { - inflight_barriers.insert(barrier.epoch.prev, (partial_graph_id, barrier.mutation.clone())); - *is_stopping = is_stop; - }); - - (prev_partial_graph_id, is_new_in_graph) } + self.started_subscribers.retain(|tx| { + tx.send((barrier.epoch.prev, barrier.mutation.clone())) + .is_ok() + }); - #[must_use] - pub(super) fn collect( - &mut self, - epoch: EpochPair, - ) -> ( - // The `partial_graph_id` of actor on the collected epoch - PartialGraphId, - // None => the partial graph id of this actor is not changed - // Some(None) => the actor has stopped, and should be removed from the return `partial_graph_id` - // Some(Some(new_partial_graph_id)) => the actor will move to the `new_partial_graph_id` - Option>, - ) { - must_match!(&mut self.status, InflightActorStatus::Running { - inflight_barriers, is_stopping - } => { - let (prev_epoch, (prev_partial_graph_id, _)) = inflight_barriers.pop_first().expect("should exist"); - assert_eq!(prev_epoch, epoch.prev); - let move_to_graph_id = if let Some((epoch, (graph_id, _))) = inflight_barriers.first_key_value() { - if *graph_id != prev_partial_graph_id { - Some(Some((*graph_id, *epoch))) - } else { - None - } - } else if *is_stopping { - Some(None) - } else { - self.status = InflightActorStatus::Pending {prev_epoch, prev_partial_graph_id}; - // No need to move to any partial graph when transit to `Pending`. When issuing the next barrier and - // the next graph id gets different, the actor will then move to the next graph id - None - }; - (prev_partial_graph_id, move_to_graph_id) - }) + self.inflight_barriers.insert( + barrier.epoch.prev, + (partial_graph_id, barrier.mutation.clone()), + ); + self.is_stopping = is_stop; + if is_stop { + assert!(self.pending_subscribers.is_empty()); + self.started_subscribers.clear(); } } + + pub(super) fn collect(&mut self, epoch: EpochPair) -> (PartialGraphId, bool) { + let (prev_epoch, (prev_partial_graph_id, _)) = + self.inflight_barriers.pop_first().expect("should exist"); + assert_eq!(prev_epoch, epoch.prev); + ( + prev_partial_graph_id, + self.inflight_barriers.is_empty() && self.is_stopping, + ) + } } pub(super) struct PartialGraphManagedBarrierState { @@ -436,20 +270,18 @@ pub(super) struct PartialGraphManagedBarrierState { /// The key is `prev_epoch`, and the first value is `curr_epoch` epoch_barrier_state_map: BTreeMap, - inflight_actors: HashSet, - prev_barrier_table_ids: Option<(EpochPair, HashSet)>, /// Record the progress updates of creating mviews for each epoch of concurrent checkpoints. pub(super) create_mview_progress: HashMap>, - /// Futures will be finished in the order of epoch in ascending order. - await_epoch_completed_futures: FuturesOrdered, - pub(super) state_store: StateStoreImpl, pub(super) streaming_metrics: Arc, + /// Futures will be finished in the order of epoch in ascending order. + await_epoch_completed_futures: FuturesOrdered, + /// Manages the await-trees of all barriers. barrier_await_tree_reg: Option, } @@ -464,7 +296,6 @@ impl PartialGraphManagedBarrierState { Self { need_seal_epoch, epoch_barrier_state_map: Default::default(), - inflight_actors: Default::default(), prev_barrier_table_ids: None, create_mview_progress: Default::default(), await_epoch_completed_futures: Default::default(), @@ -483,6 +314,10 @@ impl PartialGraphManagedBarrierState { None, ) } + + pub(super) fn is_empty(&self) -> bool { + self.epoch_barrier_state_map.is_empty() + } } pub(super) struct ManagedBarrierState { @@ -519,7 +354,43 @@ impl ManagedBarrierState { graph_states: &self.graph_states, } } +} +impl InflightActorState { + pub(super) fn subscribe_actor_mutation( + &mut self, + start_prev_epoch: u64, + tx: mpsc::UnboundedSender, + ) { + if self.inflight_barriers.contains_key(&start_prev_epoch) { + for (prev_epoch, (_, mutation)) in self.inflight_barriers.range(start_prev_epoch..) { + if tx.send((*prev_epoch, mutation.clone())).is_err() { + // No more subscribe on the mutation. Simply return. + return; + } + } + if !self.is_stopping { + self.started_subscribers.push(tx); + } + } else { + // Barrier has not issued yet. Store the pending tx + if let Some((last_epoch, _)) = self.inflight_barriers.last_key_value() { + assert!( + *last_epoch < start_prev_epoch, + "later barrier {} has been issued, but skip the start epoch {:?}", + last_epoch, + start_prev_epoch + ); + } + self.pending_subscribers + .entry(start_prev_epoch) + .or_default() + .push(tx); + } + } +} + +impl ManagedBarrierState { pub(super) fn subscribe_actor_mutation( &mut self, actor_id: ActorId, @@ -538,7 +409,6 @@ impl ManagedBarrierState { graph_infos: &HashMap, ) { let actor_to_stop = barrier.all_stop_actors(); - let mut graph_actors_to_clean: HashMap> = HashMap::new(); for (partial_graph_id, graph_info) in graph_infos { let partial_graph_id = PartialGraphId::new(*partial_graph_id); @@ -565,8 +435,7 @@ impl ManagedBarrierState { ); for actor_id in &graph_info.actor_ids_to_collect { - let (prev_partial_graph_id, is_new_in_graph) = self - .actor_states + self.actor_states .entry(*actor_id) .or_insert_with(InflightActorState::not_started) .issue_barrier( @@ -576,22 +445,6 @@ impl ManagedBarrierState { .map(|actors| actors.contains(actor_id)) .unwrap_or(false), ); - if is_new_in_graph { - graph_state.add_inflight_actor(*actor_id, barrier.epoch.prev); - } - if let Some(prev_partial_graph_id) = prev_partial_graph_id { - graph_actors_to_clean - .entry(prev_partial_graph_id) - .or_default() - .push(*actor_id); - } - } - } - - for (graph_id, actors_to_clean) in graph_actors_to_clean { - let graph_state = self.graph_states.get_mut(&graph_id).expect("should exist"); - if graph_state.remove_inflight_actors(actors_to_clean) { - self.graph_states.remove(&graph_id); } } } @@ -603,9 +456,6 @@ impl ManagedBarrierState { for (partial_graph_id, graph_state) in &mut self.graph_states { if let Poll::Ready(epoch) = graph_state.poll_next_completed_epoch(cx) { let partial_graph_id = *partial_graph_id; - if graph_state.is_empty() { - self.graph_states.remove(&partial_graph_id); - } return Poll::Ready((partial_graph_id, epoch)); } } @@ -614,62 +464,23 @@ impl ManagedBarrierState { } pub(super) fn collect(&mut self, actor_id: ActorId, epoch: EpochPair) { - let (prev_partial_graph_id, move_to_partial_graph_id) = self + let (prev_partial_graph_id, is_finished) = self .actor_states .get_mut(&actor_id) .expect("should exist") .collect(epoch); + if is_finished { + self.actor_states.remove(&actor_id); + } let prev_graph_state = self .graph_states .get_mut(&prev_partial_graph_id) .expect("should exist"); prev_graph_state.collect(actor_id, epoch); - if let Some(move_to_partial_graph_id) = move_to_partial_graph_id { - if prev_graph_state.remove_inflight_actors([actor_id]) { - self.graph_states.remove(&prev_partial_graph_id); - } - if let Some((move_to_partial_graph_id, start_epoch)) = move_to_partial_graph_id { - self.graph_states - .get_mut(&move_to_partial_graph_id) - .expect("should exist") - .add_inflight_actor(actor_id, start_epoch); - } else { - self.actor_states.remove(&actor_id); - } - } } } impl PartialGraphManagedBarrierState { - fn is_empty(&self) -> bool { - self.inflight_actors.is_empty() - && self.epoch_barrier_state_map.is_empty() - && self.await_epoch_completed_futures.is_empty() - } - - fn add_inflight_actor(&mut self, actor_id: ActorId, start_epoch: u64) { - assert!(self.inflight_actors.insert(actor_id)); - must_match!(&self.epoch_barrier_state_map.get(&start_epoch).expect("should exist").inner, ManagedBarrierStateInner::Issued(state) => { - state.remaining_actors.contains(&actor_id); - }); - if cfg!(debug_assertions) { - for (_, state) in self.epoch_barrier_state_map.range(..start_epoch) { - if let ManagedBarrierStateInner::Issued(state) = &state.inner { - // ensure that start_epoch is the first epoch to collect the barrier - assert!(!state.remaining_actors.contains(&actor_id)); - } - } - } - } - - #[must_use] - fn remove_inflight_actors(&mut self, actor_ids: impl IntoIterator) -> bool { - for actor_id in actor_ids { - assert!(self.inflight_actors.remove(&actor_id)); - } - self.is_empty() - } - /// This method is called when barrier state is modified in either `Issued` or `Stashed` /// to transform the state to `AllCollected` and start state store `sync` when the barrier /// has been collected from all actors for an `Issued` barrier. diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 7a51f1de0d72..bf082fd8121f 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -49,8 +49,7 @@ impl LocalBarrierWorker { state: BackfillState, ) { if let Some(actor_state) = self.state.actor_states.get(&actor) - && let Some(inflight_barriers) = actor_state.inflight_barriers() - && let Some((partial_graph_id, _)) = inflight_barriers.get(&epoch.prev) + && 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 From bfbb9165deb780e3273264637b0e1e7d25ef2063 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 26 Jul 2024 14:36:40 +0800 Subject: [PATCH 10/19] ignore err from empty node --- src/meta/src/barrier/mod.rs | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index bbd2fdbe66ad..1362472d13ba 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -390,14 +390,16 @@ impl CheckpointControl { } } - fn report_collect_failure(&self, worker_id: WorkerId, e: &MetaError) { + /// Return whether the collect failure on `worker_id` should trigger a recovery + fn on_collect_failure(&self, worker_id: WorkerId, e: &MetaError) -> bool { for epoch_node in self.command_ctx_queue.values() { if epoch_node.state.node_to_collect.contains(&worker_id) { self.context .report_collect_failure(&epoch_node.command_ctx, e); - break; + return true; } } + false } } @@ -691,8 +693,12 @@ impl GlobalBarrierManager { } Err(e) => { - self.checkpoint_control.report_collect_failure(worker_id, &e); - self.failure_recovery(e).await; + if self.checkpoint_control.on_collect_failure(worker_id, &e) + || self.state.inflight_actor_infos.actor_map.contains_key(&worker_id) { + self.failure_recovery(e).await; + } else { + warn!(?e, worker_id, "no barrier to collect from worker, ignore err"); + } } } } From a1434ccff0af8e76a3c3192ade31977fb2a06549 Mon Sep 17 00:00:00 2001 From: William Wen Date: Fri, 26 Jul 2024 15:12:14 +0800 Subject: [PATCH 11/19] fix dylint --- src/meta/src/barrier/mod.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index 1362472d13ba..93105f752d34 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -697,7 +697,7 @@ impl GlobalBarrierManager { || self.state.inflight_actor_infos.actor_map.contains_key(&worker_id) { self.failure_recovery(e).await; } else { - warn!(?e, worker_id, "no barrier to collect from worker, ignore err"); + warn!(e = ?e.as_report(), worker_id, "no barrier to collect from worker, ignore err"); } } } From cfff494326c5e77c0a5f216c7fd2bb238535e070 Mon Sep 17 00:00:00 2001 From: William Wen Date: Sun, 28 Jul 2024 19:49:49 +0800 Subject: [PATCH 12/19] inject barrier one by one --- src/meta/src/barrier/command.rs | 20 ++-- src/meta/src/barrier/info.rs | 124 ++++++++--------------- src/meta/src/barrier/mod.rs | 15 ++- src/meta/src/barrier/recovery.rs | 11 +- src/meta/src/barrier/rpc.rs | 52 +++++----- src/meta/src/manager/catalog/fragment.rs | 2 +- 6 files changed, 100 insertions(+), 124 deletions(-) diff --git a/src/meta/src/barrier/command.rs b/src/meta/src/barrier/command.rs index 2c65f467b4a9..c67d018a1033 100644 --- a/src/meta/src/barrier/command.rs +++ b/src/meta/src/barrier/command.rs @@ -41,12 +41,10 @@ use risingwave_pb::stream_service::WaitEpochCommitRequest; use thiserror_ext::AsReport; use tracing::warn; -use super::info::{ - CommandActorChanges, CommandFragmentChanges, CommandNewFragmentInfo, InflightActorInfo, -}; +use super::info::{CommandActorChanges, CommandFragmentChanges, InflightActorInfo}; use super::trace::TracedEpoch; use crate::barrier::GlobalBarrierManagerContext; -use crate::manager::{DdlType, MetadataManager, StreamingJob, WorkerId}; +use crate::manager::{DdlType, InflightFragmentInfo, MetadataManager, StreamingJob, WorkerId}; use crate::model::{ActorId, DispatcherId, FragmentId, TableFragments, TableParallelism}; use crate::stream::{build_actor_connector_splits, SplitAssignment, ThrottleConfig}; use crate::MetaResult; @@ -109,8 +107,8 @@ impl ReplaceTablePlan { fn actor_changes(&self) -> CommandActorChanges { let mut fragment_changes = HashMap::new(); for fragment in self.new_table_fragments.fragments.values() { - let fragment_change = CommandFragmentChanges::NewFragment(CommandNewFragmentInfo { - new_actors: fragment + let fragment_change = CommandFragmentChanges::NewFragment(InflightFragmentInfo { + actors: fragment .actors .iter() .map(|actor| { @@ -124,7 +122,7 @@ impl ReplaceTablePlan { ) }) .collect(), - table_ids: fragment + state_table_ids: fragment .state_table_ids .iter() .map(|table_id| TableId::new(*table_id)) @@ -159,12 +157,12 @@ pub struct CreateStreamingJobCommandInfo { } impl CreateStreamingJobCommandInfo { - fn new_fragment_info(&self) -> impl Iterator + '_ { + fn new_fragment_info(&self) -> impl Iterator + '_ { self.table_fragments.fragments.values().map(|fragment| { ( fragment.fragment_id, - CommandNewFragmentInfo { - new_actors: fragment + InflightFragmentInfo { + actors: fragment .actors .iter() .map(|actor| { @@ -178,7 +176,7 @@ impl CreateStreamingJobCommandInfo { ) }) .collect(), - table_ids: fragment + state_table_ids: fragment .state_table_ids .iter() .map(|table_id| TableId::new(*table_id)) diff --git a/src/meta/src/barrier/info.rs b/src/meta/src/barrier/info.rs index 645d15e83a7e..44194c7f9eb3 100644 --- a/src/meta/src/barrier/info.rs +++ b/src/meta/src/barrier/info.rs @@ -22,16 +22,9 @@ use crate::barrier::Command; use crate::manager::{ActiveStreamingWorkerNodes, ActorInfos, InflightFragmentInfo, WorkerId}; use crate::model::{ActorId, FragmentId}; -#[derive(Debug, Clone)] -pub(crate) struct CommandNewFragmentInfo { - pub new_actors: HashMap, - pub table_ids: HashSet, - pub is_injectable: bool, -} - #[derive(Debug, Clone)] pub(crate) enum CommandFragmentChanges { - NewFragment(CommandNewFragmentInfo), + NewFragment(InflightFragmentInfo), Reschedule { new_actors: HashMap, to_remove: HashSet, @@ -65,9 +58,6 @@ pub struct InflightActorInfo { /// `node_id` => actors pub actor_map: HashMap>, - /// `node_id` => barrier inject actors - pub actor_map_to_send: HashMap>, - /// `actor_id` => `WorkerId` pub actor_location_map: HashMap, @@ -96,20 +86,6 @@ impl InflightActorInfo { map }; - let actor_map_to_send = { - let mut map: HashMap<_, HashSet<_>> = HashMap::new(); - for info in actor_infos - .fragment_infos - .values() - .filter(|info| info.is_injectable) - { - for (actor_id, worker_id) in &info.actors { - map.entry(*worker_id).or_default().insert(*actor_id); - } - } - map - }; - let actor_location_map = actor_infos .fragment_infos .values() @@ -124,7 +100,6 @@ impl InflightActorInfo { Self { node_map, actor_map, - actor_map_to_send, actor_location_map, mv_depended_subscriptions, fragment_infos: actor_infos.fragment_infos, @@ -167,28 +142,11 @@ impl InflightActorInfo { let mut to_add = HashMap::new(); for (fragment_id, change) in fragment_changes { match change { - CommandFragmentChanges::NewFragment(CommandNewFragmentInfo { - new_actors, - table_ids, - is_injectable, - .. - }) => { - for (actor_id, node_id) in &new_actors { - assert!(to_add - .insert(*actor_id, (*node_id, is_injectable)) - .is_none()); + CommandFragmentChanges::NewFragment(info) => { + for (actor_id, node_id) in &info.actors { + assert!(to_add.insert(*actor_id, *node_id).is_none()); } - assert!(self - .fragment_infos - .insert( - fragment_id, - InflightFragmentInfo { - actors: new_actors, - state_table_ids: table_ids, - is_injectable, - } - ) - .is_none()); + assert!(self.fragment_infos.insert(fragment_id, info).is_none()); } CommandFragmentChanges::Reschedule { new_actors, .. } => { let info = self @@ -197,30 +155,19 @@ impl InflightActorInfo { .expect("should exist"); let actors = &mut info.actors; for (actor_id, node_id) in new_actors { - assert!(to_add - .insert(actor_id, (node_id, info.is_injectable)) - .is_none()); + assert!(to_add.insert(actor_id, node_id).is_none()); assert!(actors.insert(actor_id, node_id).is_none()); } } CommandFragmentChanges::RemoveFragment => {} } } - for (actor_id, (node_id, is_injectable)) in to_add { + for (actor_id, node_id) in to_add { assert!(self.node_map.contains_key(&node_id)); assert!( self.actor_map.entry(node_id).or_default().insert(actor_id), "duplicate actor in command changes" ); - if is_injectable { - assert!( - self.actor_map_to_send - .entry(node_id) - .or_default() - .insert(actor_id), - "duplicate actor in command changes" - ); - } assert!( self.actor_location_map.insert(actor_id, node_id).is_none(), "duplicate actor in command changes" @@ -280,13 +227,8 @@ impl InflightActorInfo { .expect("actor not found"); let actor_ids = self.actor_map.get_mut(&node_id).expect("node not found"); assert!(actor_ids.remove(&actor_id), "actor not found"); - self.actor_map_to_send - .get_mut(&node_id) - .map(|actor_ids| actor_ids.remove(&actor_id)); } self.actor_map.retain(|_, actor_ids| !actor_ids.is_empty()); - self.actor_map_to_send - .retain(|_, actor_ids| !actor_ids.is_empty()); } if let Command::DropSubscription { subscription_id, @@ -310,27 +252,49 @@ impl InflightActorInfo { } /// Returns actor list to collect in the target worker node. - pub fn actor_ids_to_collect(&self, node_id: &WorkerId) -> impl Iterator { - self.actor_map - .get(node_id) - .cloned() - .unwrap_or_default() - .into_iter() + pub fn actor_ids_to_collect( + fragment_infos: &HashMap, + node_id: WorkerId, + ) -> impl Iterator + '_ { + fragment_infos.values().flat_map(move |info| { + info.actors + .iter() + .filter_map(move |(actor_id, actor_node_id)| { + if *actor_node_id == node_id { + Some(*actor_id) + } else { + None + } + }) + }) } /// Returns actor list to send in the target worker node. - pub fn actor_ids_to_send(&self, node_id: &WorkerId) -> impl Iterator { - self.actor_map_to_send - .get(node_id) - .cloned() - .unwrap_or_default() - .into_iter() + pub fn actor_ids_to_send( + fragment_infos: &HashMap, + node_id: WorkerId, + ) -> impl Iterator + '_ { + fragment_infos + .values() + .filter(|info| info.is_injectable) + .flat_map(move |info| { + info.actors + .iter() + .filter_map(move |(actor_id, actor_node_id)| { + if *actor_node_id == node_id { + Some(*actor_id) + } else { + None + } + }) + }) } - pub fn existing_table_ids(&self) -> HashSet { - self.fragment_infos + pub fn existing_table_ids( + fragment_infos: &HashMap, + ) -> impl Iterator + '_ { + fragment_infos .values() .flat_map(|info| info.state_table_ids.iter().cloned()) - .collect() } } diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index d6341ad25c5f..ed1a936da784 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -781,10 +781,11 @@ impl GlobalBarrierManager { send_latency_timer.observe_duration(); - let node_to_collect = match self - .control_stream_manager - .inject_barrier(command_ctx.clone(), &self.state.inflight_actor_infos) - { + let node_to_collect = match self.control_stream_manager.inject_barrier( + &command_ctx, + &command_ctx.info.fragment_infos, + Some(&self.state.inflight_actor_infos.fragment_infos), + ) { Ok(node_to_collect) => node_to_collect, Err(err) => { for notifier in notifiers { @@ -1207,6 +1208,7 @@ fn collect_commit_epoch_info( table_watermarks.push(resp.table_watermarks); old_value_ssts.extend(resp.old_value_sstables.into_iter().map(|s| s.into())); } + let new_table_fragment_info = if let Command::CreateStreamingJob { info, .. } = &command_ctx.command { let table_fragments = &info.table_fragments; @@ -1261,7 +1263,10 @@ fn collect_commit_epoch_info( sst_to_worker, new_table_fragment_info, table_new_change_log, - BTreeMap::from_iter([(epoch, command_ctx.info.existing_table_ids())]), + BTreeMap::from_iter([( + epoch, + InflightActorInfo::existing_table_ids(&command_ctx.info.fragment_infos).collect(), + )]), epoch, ) } diff --git a/src/meta/src/barrier/recovery.rs b/src/meta/src/barrier/recovery.rs index a0da75fcd4b1..3c0c3440b0a6 100644 --- a/src/meta/src/barrier/recovery.rs +++ b/src/meta/src/barrier/recovery.rs @@ -335,7 +335,9 @@ impl GlobalBarrierManager { let info = info; self.context - .purge_state_table_from_hummock(&info.existing_table_ids()) + .purge_state_table_from_hummock( + &InflightActorInfo::existing_table_ids(&info.fragment_infos).collect(), + ) .await .context("purge state table from hummock")?; @@ -385,8 +387,11 @@ impl GlobalBarrierManager { tracing::Span::current(), // recovery span )); - let mut node_to_collect = - control_stream_manager.inject_barrier(command_ctx.clone(), &info)?; + let mut node_to_collect = control_stream_manager.inject_barrier( + &command_ctx, + &info.fragment_infos, + Some(&info.fragment_infos), + )?; while !node_to_collect.is_empty() { let (worker_id, result) = control_stream_manager .next_complete_barrier_response() diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index ddf976c3a038..aee35b48a1ed 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -15,7 +15,6 @@ use std::collections::{HashMap, HashSet}; use std::error::Error; use std::future::Future; -use std::sync::Arc; use std::time::Duration; use anyhow::anyhow; @@ -31,7 +30,7 @@ use risingwave_pb::stream_plan::{Barrier, BarrierMutation}; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, BarrierCompleteResponse, BroadcastActorInfoTableRequest, BuildActorInfo, BuildActorsRequest, DropActorsRequest, - InjectBarrierRequest, PbPartialGraphInfo, StreamingControlStreamRequest, + InjectBarrierRequest, PartialGraphInfo, StreamingControlStreamRequest, StreamingControlStreamResponse, UpdateActorsRequest, }; use risingwave_rpc_client::error::RpcError; @@ -47,7 +46,8 @@ use uuid::Uuid; use super::command::CommandContext; use super::GlobalBarrierManagerContext; use crate::barrier::info::InflightActorInfo; -use crate::manager::{MetaSrvEnv, WorkerId}; +use crate::manager::{InflightFragmentInfo, MetaSrvEnv, WorkerId}; +use crate::model::FragmentId; use crate::{MetaError, MetaResult}; const COLLECT_ERROR_TIMEOUT: Duration = Duration::from_secs(3); @@ -248,8 +248,9 @@ impl ControlStreamManager { /// Send inject-barrier-rpc to stream service and wait for its response before returns. pub(super) fn inject_barrier( &mut self, - command_context: Arc, - inflight_actor_info_after_apply_command: &InflightActorInfo, + command_context: &CommandContext, + pre_applied_fragment_infos: &HashMap, + applied_fragment_infos: Option<&HashMap>, ) -> MetaResult> { fail_point!("inject_barrier_err", |_| risingwave_common::bail!( "inject_barrier_err" @@ -261,31 +262,27 @@ impl ControlStreamManager { info.node_map .iter() .map(|(node_id, worker_node)| { - let actor_ids_to_send = info.actor_ids_to_send(node_id).collect_vec(); - let actor_ids_to_collect = info.actor_ids_to_collect(node_id).collect_vec(); + let actor_ids_to_send: Vec<_> = + InflightActorInfo::actor_ids_to_send(pre_applied_fragment_infos, *node_id) + .collect(); + let actor_ids_to_collect: Vec<_> = + InflightActorInfo::actor_ids_to_collect(pre_applied_fragment_infos, *node_id) + .collect(); if actor_ids_to_collect.is_empty() { // No need to send or collect barrier for this node. assert!(actor_ids_to_send.is_empty()); } - let graph_info = HashMap::from_iter([( - u32::MAX, - PbPartialGraphInfo { - actor_ids_to_send, - actor_ids_to_collect, - table_ids_to_sync: inflight_actor_info_after_apply_command - .existing_table_ids() - .iter() - .map(|table_id| table_id.table_id) - .collect(), - }, - )]); + let table_ids_to_sync = if let Some(fragment_infos) = applied_fragment_infos { + InflightActorInfo::existing_table_ids(fragment_infos) + .map(|table_id| table_id.table_id) + .collect() + } else { + Default::default() + }; { let Some(node) = self.nodes.get_mut(node_id) else { - if graph_info - .values() - .all(|info| info.actor_ids_to_collect.is_empty()) - { + if actor_ids_to_collect.is_empty() { // Worker node get disconnected but has no actor to collect. Simply skip it. return Ok(()); } @@ -315,7 +312,14 @@ impl ControlStreamManager { InjectBarrierRequest { request_id: StreamRpcManager::new_request_id(), barrier: Some(barrier), - graph_info, + graph_info: HashMap::from_iter([( + u32::MAX, + PartialGraphInfo { + actor_ids_to_send, + actor_ids_to_collect, + table_ids_to_sync, + }, + )]), }, ), ), diff --git a/src/meta/src/manager/catalog/fragment.rs b/src/meta/src/manager/catalog/fragment.rs index 2ff0fc1ac971..eab1da29f4d7 100644 --- a/src/meta/src/manager/catalog/fragment.rs +++ b/src/meta/src/manager/catalog/fragment.rs @@ -173,7 +173,7 @@ pub struct FragmentManager { core: RwLock, } -#[derive(Clone)] +#[derive(Clone, Debug)] pub struct InflightFragmentInfo { pub actors: HashMap, pub state_table_ids: HashSet, From e76c11c7593c02660417dd65e7bc1e8b424dec9a Mon Sep 17 00:00:00 2001 From: William Wen Date: Sun, 28 Jul 2024 20:25:58 +0800 Subject: [PATCH 13/19] rpc inject one by one --- proto/stream_service.proto | 13 ++-- src/meta/src/barrier/rpc.rs | 16 ++--- src/meta/src/stream/stream_manager.rs | 7 +- src/stream/src/task/barrier_manager.rs | 69 ++++++++++--------- .../src/task/barrier_manager/managed_state.rs | 65 ++++++++--------- 5 files changed, 73 insertions(+), 97 deletions(-) diff --git a/proto/stream_service.proto b/proto/stream_service.proto index 0460956d8c07..08f0ff1e7684 100644 --- a/proto/stream_service.proto +++ b/proto/stream_service.proto @@ -52,18 +52,13 @@ message DropActorsResponse { common.Status status = 2; } -message PartialGraphInfo { - repeated uint32 actor_ids_to_send = 1; - repeated uint32 actor_ids_to_collect = 2; - repeated uint32 table_ids_to_sync = 3; -} - message InjectBarrierRequest { string request_id = 1; stream_plan.Barrier barrier = 2; - reserved 3, 4, 5; - reserved "actor_ids_to_send", "actor_ids_to_collect", "table_ids_to_sync"; - map graph_info = 6; + repeated uint32 actor_ids_to_send = 3; + repeated uint32 actor_ids_to_collect = 4; + repeated uint32 table_ids_to_sync = 5; + uint32 partial_graph_id = 6; } message BarrierCompleteResponse { diff --git a/src/meta/src/barrier/rpc.rs b/src/meta/src/barrier/rpc.rs index aee35b48a1ed..ae12c439e0a9 100644 --- a/src/meta/src/barrier/rpc.rs +++ b/src/meta/src/barrier/rpc.rs @@ -30,8 +30,8 @@ use risingwave_pb::stream_plan::{Barrier, BarrierMutation}; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, BarrierCompleteResponse, BroadcastActorInfoTableRequest, BuildActorInfo, BuildActorsRequest, DropActorsRequest, - InjectBarrierRequest, PartialGraphInfo, StreamingControlStreamRequest, - StreamingControlStreamResponse, UpdateActorsRequest, + InjectBarrierRequest, StreamingControlStreamRequest, StreamingControlStreamResponse, + UpdateActorsRequest, }; use risingwave_rpc_client::error::RpcError; use risingwave_rpc_client::StreamClient; @@ -312,14 +312,10 @@ impl ControlStreamManager { InjectBarrierRequest { request_id: StreamRpcManager::new_request_id(), barrier: Some(barrier), - graph_info: HashMap::from_iter([( - u32::MAX, - PartialGraphInfo { - actor_ids_to_send, - actor_ids_to_collect, - table_ids_to_sync, - }, - )]), + actor_ids_to_send, + actor_ids_to_collect, + table_ids_to_sync, + partial_graph_id: u32::MAX, }, ), ), diff --git a/src/meta/src/stream/stream_manager.rs b/src/meta/src/stream/stream_manager.rs index 032b1410bb63..42745a80423d 100644 --- a/src/meta/src/stream/stream_manager.rs +++ b/src/meta/src/stream/stream_manager.rs @@ -947,18 +947,13 @@ mod tests { })); } streaming_control_stream_request::Request::InjectBarrier(req) => { - assert_eq!(req.graph_info.len(), 1); let _ = tx.send(Ok(StreamingControlStreamResponse { response: Some( streaming_control_stream_response::Response::CompleteBarrier( BarrierCompleteResponse { epoch: req.barrier.unwrap().epoch.unwrap().prev, worker_id, - partial_graph_id: *req - .graph_info - .keys() - .next() - .unwrap(), + partial_graph_id: req.partial_graph_id, ..BarrierCompleteResponse::default() }, ), diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index ab0f6897055d..be6d10b90841 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -use std::collections::{BTreeSet, HashMap}; +use std::collections::{BTreeSet, HashMap, HashSet}; use std::fmt::Display; use std::future::pending; use std::sync::Arc; @@ -49,6 +49,7 @@ mod progress; mod tests; pub use progress::CreateMviewProgress; +use risingwave_common::catalog::TableId; use risingwave_common::util::epoch::EpochPair; use risingwave_common::util::runtime::BackgroundShutdownRuntime; use risingwave_hummock_sdk::table_stats::to_prost_table_stats_map; @@ -60,7 +61,7 @@ use risingwave_pb::stream_service::streaming_control_stream_response::{ InitResponse, ShutdownResponse, }; use risingwave_pb::stream_service::{ - streaming_control_stream_response, BarrierCompleteResponse, BuildActorInfo, PartialGraphInfo, + streaming_control_stream_response, BarrierCompleteResponse, BuildActorInfo, StreamingControlStreamRequest, StreamingControlStreamResponse, }; @@ -536,7 +537,16 @@ impl LocalBarrierWorker { match request.request.expect("should not be empty") { Request::InjectBarrier(req) => { let barrier = Barrier::from_protobuf(req.get_barrier().unwrap())?; - self.send_barrier(barrier, req.graph_info)?; + self.send_barrier( + &barrier, + req.actor_ids_to_send.into_iter().collect(), + req.actor_ids_to_collect.into_iter().collect(), + req.table_ids_to_sync + .into_iter() + .map(TableId::new) + .collect(), + PartialGraphId::new(req.partial_graph_id), + )?; Ok(()) } Request::RemovePartialGraph(req) => { @@ -721,22 +731,20 @@ impl LocalBarrierWorker { /// to find the root cause. fn send_barrier( &mut self, - barrier: Barrier, - graph_infos: HashMap, + barrier: &Barrier, + to_send: HashSet, + to_collect: HashSet, + table_ids: HashSet, + partial_graph_id: PartialGraphId, ) -> StreamResult<()> { - let all_to_collect = || { - graph_infos - .values() - .flat_map(|info| info.actor_ids_to_collect.iter()) - .cloned() - }; if !cfg!(test) { // The barrier might be outdated and been injected after recovery in some certain extreme // scenarios. So some newly creating actors in the barrier are possibly not rebuilt during // recovery. Check it here and return an error here if some actors are not found to // avoid collection hang. We need some refine in meta side to remove this workaround since // it will cause another round of unnecessary recovery. - let missing_actor_ids = all_to_collect() + let missing_actor_ids = to_collect + .iter() .filter(|id| !self.actor_manager_state.handles.contains_key(id)) .collect_vec(); if !missing_actor_ids.is_empty() { @@ -744,7 +752,7 @@ impl LocalBarrierWorker { "to collect actors not found, they should be cleaned when recovering: {:?}", missing_actor_ids ); - return Err(anyhow!("to collect actors not found: {:?}", missing_actor_ids).into()); + return Err(anyhow!("to collect actors not found: {:?}", to_collect).into()); } } @@ -755,31 +763,29 @@ impl LocalBarrierWorker { } debug!( target: "events::stream::barrier::manager::send", - "send barrier {:?}, graph_info: {:?}", + "send barrier {:?}, senders = {:?}, actor_ids_to_collect = {:?}", barrier, - graph_infos, + to_send, + to_collect ); - for actor_id in all_to_collect() { - if self.failure_actors.contains_key(&actor_id) { + for actor_id in &to_collect { + if self.failure_actors.contains_key(actor_id) { // The failure actors could exit before the barrier is issued, while their // up-downstream actors could be stuck somehow. Return error directly to trigger the // recovery. return Err(StreamError::barrier_send( barrier.clone(), - actor_id, + *actor_id, "actor has already failed", )); } } - self.state.transform_to_issued(&barrier, &graph_infos); + self.state + .transform_to_issued(barrier, to_collect, table_ids, partial_graph_id); - for actor_id in graph_infos - .values() - .flat_map(|infos| infos.actor_ids_to_send.iter()) - .cloned() - { + for actor_id in to_send { match self.barrier_senders.get(&actor_id) { Some(senders) => { for sender in senders { @@ -1137,7 +1143,6 @@ impl LocalBarrierManager { #[cfg(test)] pub(crate) mod barrier_test_utils { - use std::collections::HashMap; use std::sync::Arc; use assert_matches::assert_matches; @@ -1145,7 +1150,7 @@ pub(crate) mod barrier_test_utils { use risingwave_pb::stream_service::streaming_control_stream_request::InitRequest; use risingwave_pb::stream_service::{ streaming_control_stream_request, streaming_control_stream_response, InjectBarrierRequest, - PbPartialGraphInfo, StreamingControlStreamRequest, StreamingControlStreamResponse, + StreamingControlStreamRequest, StreamingControlStreamResponse, }; use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver, UnboundedSender}; use tokio_stream::wrappers::UnboundedReceiverStream; @@ -1207,14 +1212,10 @@ pub(crate) mod barrier_test_utils { InjectBarrierRequest { request_id: "".to_string(), barrier: Some(barrier.to_protobuf()), - graph_info: HashMap::from_iter([( - u32::MAX, - PbPartialGraphInfo { - actor_ids_to_send: actor_to_send.into_iter().collect(), - actor_ids_to_collect: actor_to_collect.into_iter().collect(), - table_ids_to_sync: vec![], - }, - )]), + actor_ids_to_send: actor_to_send.into_iter().collect(), + actor_ids_to_collect: actor_to_collect.into_iter().collect(), + table_ids_to_sync: vec![], + partial_graph_id: u32::MAX, }, )), })) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 88dae3600761..0292f199059b 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -32,7 +32,6 @@ use risingwave_common::util::epoch::EpochPair; use risingwave_hummock_sdk::SyncResult; use risingwave_pb::stream_plan::barrier::BarrierKind; use risingwave_pb::stream_service::barrier_complete_response::CreateMviewProgress; -use risingwave_pb::stream_service::PartialGraphInfo; use risingwave_storage::{dispatch_state_store, StateStore, StateStoreImpl}; use thiserror_ext::AsReport; use tokio::sync::mpsc; @@ -406,47 +405,37 @@ impl ManagedBarrierState { pub(super) fn transform_to_issued( &mut self, barrier: &Barrier, - graph_infos: &HashMap, + actor_ids_to_collect: HashSet, + table_ids: HashSet, + partial_graph_id: PartialGraphId, ) { let actor_to_stop = barrier.all_stop_actors(); - for (partial_graph_id, graph_info) in graph_infos { - let partial_graph_id = PartialGraphId::new(*partial_graph_id); - - let graph_state = self - .graph_states - .entry(partial_graph_id) - .or_insert_with(|| { - PartialGraphManagedBarrierState::new( - partial_graph_id.is_global_graph(), - self.state_store.clone(), - self.streaming_metrics.clone(), - self.barrier_await_tree_reg.clone(), - ) - }); - - graph_state.transform_to_issued( - barrier, - graph_info.actor_ids_to_collect.iter().cloned().collect(), - graph_info - .table_ids_to_sync - .iter() - .map(|table_id| TableId::new(*table_id)) - .collect(), - ); + let graph_state = self + .graph_states + .entry(partial_graph_id) + .or_insert_with(|| { + PartialGraphManagedBarrierState::new( + partial_graph_id.is_global_graph(), + self.state_store.clone(), + self.streaming_metrics.clone(), + self.barrier_await_tree_reg.clone(), + ) + }); - for actor_id in &graph_info.actor_ids_to_collect { - self.actor_states - .entry(*actor_id) - .or_insert_with(InflightActorState::not_started) - .issue_barrier( - partial_graph_id, - barrier, - actor_to_stop - .map(|actors| actors.contains(actor_id)) - .unwrap_or(false), - ); - } + for actor_id in &actor_ids_to_collect { + self.actor_states + .entry(*actor_id) + .or_insert_with(InflightActorState::not_started) + .issue_barrier( + partial_graph_id, + barrier, + actor_to_stop + .map(|actors| actors.contains(actor_id)) + .unwrap_or(false), + ); } + + graph_state.transform_to_issued(barrier, actor_ids_to_collect, table_ids); } pub(super) fn next_completed_epoch( From 76b1244f0ef27dd04d3990971b02bc11b8f618ee Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 29 Jul 2024 13:53:33 +0800 Subject: [PATCH 14/19] address comment --- src/stream/src/task/barrier_manager.rs | 2 +- .../src/task/barrier_manager/managed_state.rs | 40 +++++++++++++++---- .../src/task/barrier_manager/progress.rs | 2 + 3 files changed, 36 insertions(+), 8 deletions(-) diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index be6d10b90841..f315fc8caff2 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -858,7 +858,7 @@ impl LocalBarrierWorker { let root_err = self.try_find_root_failure().await.unwrap(); // always `Some` because we just added one if let Some(actor_state) = self.state.actor_states.get(&actor_id) - && !actor_state.inflight_barriers.is_empty() + && (!actor_state.inflight_barriers.is_empty() || !actor_state.is_running()) { self.control_stream_handle.reset_stream_with_err( anyhow!(root_err) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 0292f199059b..208c8ea1a924 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -191,13 +191,27 @@ impl Display for &'_ PartialGraphManagedBarrierState { } } +enum InflightActorStatus { + /// The actor is just spawned and not issued any barrier yet + NotStarted, + /// The actor has been issued some barriers, and not issued any stop barrier yet + Running, + /// The actor has been issued a stop barrier + Stopping, +} + +impl InflightActorStatus { + pub(super) fn is_stopping(&self) -> bool { + matches!(self, InflightActorStatus::Stopping) + } +} + pub(crate) struct InflightActorState { pending_subscribers: BTreeMap>>, started_subscribers: Vec>, /// `prev_epoch` -> partial graph id pub(super) inflight_barriers: BTreeMap>)>, - /// Whether the actor has been issued a stop barrier - is_stopping: bool, + status: InflightActorStatus, } impl InflightActorState { @@ -206,7 +220,7 @@ impl InflightActorState { pending_subscribers: Default::default(), started_subscribers: vec![], inflight_barriers: BTreeMap::default(), - is_stopping: false, + status: InflightActorStatus::NotStarted, } } @@ -216,7 +230,6 @@ impl InflightActorState { barrier: &Barrier, is_stop: bool, ) { - assert!(!self.is_stopping, "stopped actor should not issue barrier"); if let Some((first_epoch, _)) = self.pending_subscribers.first_key_value() { assert!( *first_epoch >= barrier.epoch.prev, @@ -242,10 +255,16 @@ impl InflightActorState { barrier.epoch.prev, (partial_graph_id, barrier.mutation.clone()), ); - self.is_stopping = is_stop; if is_stop { assert!(self.pending_subscribers.is_empty()); self.started_subscribers.clear(); + assert!( + !self.status.is_stopping(), + "stopped actor should not issue barrier" + ); + self.status = InflightActorStatus::Stopping; + } else { + self.status = InflightActorStatus::Running; } } @@ -255,9 +274,13 @@ impl InflightActorState { assert_eq!(prev_epoch, epoch.prev); ( prev_partial_graph_id, - self.inflight_barriers.is_empty() && self.is_stopping, + self.inflight_barriers.is_empty() && self.status.is_stopping(), ) } + + pub(super) fn is_running(&self) -> bool { + matches!(&self.status, InflightActorStatus::Running) + } } pub(super) struct PartialGraphManagedBarrierState { @@ -368,7 +391,8 @@ impl InflightActorState { return; } } - if !self.is_stopping { + if !self.status.is_stopping() { + // Only add the subscribers when the actor is not stopped yet. self.started_subscribers.push(tx); } } else { @@ -380,6 +404,8 @@ impl InflightActorState { last_epoch, start_prev_epoch ); + } else { + assert!(self.status.is_stopping(), "actor has been stopped and has not inflight barrier. unlikely to get further barrier"); } self.pending_subscribers .entry(start_prev_epoch) diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index bf082fd8121f..256c1f99976b 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -57,6 +57,8 @@ impl LocalBarrierWorker { .entry(epoch.curr) .or_default() .insert(actor, state); + } else { + warn!(?epoch, actor, ?state, "ignore create mview progress"); } } } From 13952417094993cd67faeace9f891b9a82f99780 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 29 Jul 2024 14:09:00 +0800 Subject: [PATCH 15/19] fix --- src/stream/src/task/barrier_manager/managed_state.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index 208c8ea1a924..dfe747ad1a9b 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -405,7 +405,7 @@ impl InflightActorState { start_prev_epoch ); } else { - assert!(self.status.is_stopping(), "actor has been stopped and has not inflight barrier. unlikely to get further barrier"); + assert!(!self.status.is_stopping(), "actor has been stopped and has not inflight barrier. unlikely to get further barrier"); } self.pending_subscribers .entry(start_prev_epoch) From a0acc6bd1eddacffba39c4bbe8c9495f8322e1e0 Mon Sep 17 00:00:00 2001 From: William Wen Date: Mon, 29 Jul 2024 15:20:30 +0800 Subject: [PATCH 16/19] fix ci --- src/meta/src/barrier/mod.rs | 16 +++++++------- .../src/hummock/event_handler/uploader/mod.rs | 21 ++++++++++++++++++- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/src/meta/src/barrier/mod.rs b/src/meta/src/barrier/mod.rs index ed1a936da784..f58cb3d39503 100644 --- a/src/meta/src/barrier/mod.rs +++ b/src/meta/src/barrier/mod.rs @@ -390,16 +390,14 @@ impl CheckpointControl { } } - /// Return whether the collect failure on `worker_id` should trigger a recovery - fn on_collect_failure(&self, worker_id: WorkerId, e: &MetaError) -> bool { + /// Return the earliest command waiting on the `worker_id`. + fn command_wait_collect_from_worker(&self, worker_id: WorkerId) -> Option<&CommandContext> { for epoch_node in self.command_ctx_queue.values() { if epoch_node.state.node_to_collect.contains(&worker_id) { - self.context - .report_collect_failure(&epoch_node.command_ctx, e); - return true; + return Some(&epoch_node.command_ctx); } } - false + None } } @@ -701,10 +699,14 @@ impl GlobalBarrierManager { } Err(e) => { - if self.checkpoint_control.on_collect_failure(worker_id, &e) + let failed_command = self.checkpoint_control.command_wait_collect_from_worker(worker_id); + if failed_command.is_some() || self.state.inflight_actor_infos.actor_map.contains_key(&worker_id) { let errors = self.control_stream_manager.collect_errors(worker_id, e).await; let err = merge_node_rpc_errors("get error from control stream", errors); + if let Some(failed_command) = failed_command { + self.context.report_collect_failure(failed_command, &err); + } self.failure_recovery(err).await; } else { warn!(e = ?e.as_report(), worker_id, "no barrier to collect from worker, ignore err"); diff --git a/src/storage/src/hummock/event_handler/uploader/mod.rs b/src/storage/src/hummock/event_handler/uploader/mod.rs index 884b29de5edd..8210a998974c 100644 --- a/src/storage/src/hummock/event_handler/uploader/mod.rs +++ b/src/storage/src/hummock/event_handler/uploader/mod.rs @@ -850,7 +850,26 @@ impl UnsyncData { // called `start_epoch` because we have stopped writing on it. if !table_data.unsync_epochs.contains_key(&next_epoch) { if let Some(stopped_next_epoch) = table_data.stopped_next_epoch { - assert_eq!(stopped_next_epoch, next_epoch); + if stopped_next_epoch != next_epoch { + let table_id = table_data.table_id.table_id; + let unsync_epochs = table_data.unsync_epochs.keys().collect_vec(); + if cfg!(debug_assertions) { + panic!( + "table_id {} stop epoch {} different to prev stop epoch {}. unsync epochs: {:?}, syncing epochs {:?}, max_synced_epoch {:?}", + table_id, next_epoch, stopped_next_epoch, unsync_epochs, table_data.syncing_epochs, table_data.max_synced_epoch + ); + } else { + warn!( + table_id, + stopped_next_epoch, + next_epoch, + ?unsync_epochs, + syncing_epochs = ?table_data.syncing_epochs, + max_synced_epoch = ?table_data.max_synced_epoch, + "different stop epoch" + ); + } + } } else { if let Some(max_epoch) = table_data.max_epoch() { assert_gt!(next_epoch, max_epoch); From 70fab17ee383dadc45011586aa5511173795d26c Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 30 Jul 2024 00:31:58 +0800 Subject: [PATCH 17/19] separate actor barrier and mutation --- .../src/task/barrier_manager/managed_state.rs | 97 +++++++++++++------ .../src/task/barrier_manager/progress.rs | 2 +- 2 files changed, 67 insertions(+), 32 deletions(-) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index dfe747ad1a9b..eb468e1d58e6 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -195,22 +195,32 @@ enum InflightActorStatus { /// The actor is just spawned and not issued any barrier yet NotStarted, /// The actor has been issued some barriers, and not issued any stop barrier yet - Running, + Running(u64), /// The actor has been issued a stop barrier - Stopping, + Stopping(u64), } impl InflightActorStatus { pub(super) fn is_stopping(&self) -> bool { - matches!(self, InflightActorStatus::Stopping) + matches!(self, InflightActorStatus::Stopping(_)) + } + + fn max_issued_epoch(&self) -> Option { + match self { + InflightActorStatus::NotStarted => None, + InflightActorStatus::Running(epoch) | InflightActorStatus::Stopping(epoch) => { + Some(*epoch) + } + } } } pub(crate) struct InflightActorState { pending_subscribers: BTreeMap>>, - started_subscribers: Vec>, /// `prev_epoch` -> partial graph id - pub(super) inflight_barriers: BTreeMap>)>, + pub(super) inflight_barriers: BTreeMap, + /// `prev_epoch` -> (`mutation`, `curr_epoch`) + barrier_mutations: BTreeMap>, u64)>, status: InflightActorStatus, } @@ -218,8 +228,8 @@ impl InflightActorState { pub(super) fn not_started() -> Self { Self { pending_subscribers: Default::default(), - started_subscribers: vec![], inflight_barriers: BTreeMap::default(), + barrier_mutations: Default::default(), status: InflightActorStatus::NotStarted, } } @@ -230,6 +240,10 @@ impl InflightActorState { barrier: &Barrier, is_stop: bool, ) { + if let Some(max_issued_epoch) = self.status.max_issued_epoch() { + assert!(barrier.epoch.prev > max_issued_epoch); + } + if let Some((first_epoch, _)) = self.pending_subscribers.first_key_value() { assert!( *first_epoch >= barrier.epoch.prev, @@ -238,40 +252,50 @@ impl InflightActorState { first_epoch ); if *first_epoch == barrier.epoch.prev { - self.started_subscribers.extend( + let (_, mut subscribers) = self.pending_subscribers.pop_first().expect("non empty"); + subscribers.retain(|tx| { + tx.send((barrier.epoch.prev, barrier.mutation.clone())) + .is_ok() + }); + if !is_stop && !subscribers.is_empty() { self.pending_subscribers - .pop_first() - .expect("should exist") - .1, - ); + .entry(barrier.epoch.curr) + .or_default() + .extend(subscribers); + } } } - self.started_subscribers.retain(|tx| { - tx.send((barrier.epoch.prev, barrier.mutation.clone())) - .is_ok() - }); - self.inflight_barriers.insert( - barrier.epoch.prev, - (partial_graph_id, barrier.mutation.clone()), - ); + assert!(self + .inflight_barriers + .insert(barrier.epoch.prev, partial_graph_id) + .is_none()); + + assert!(self + .barrier_mutations + .insert( + barrier.epoch.prev, + (barrier.mutation.clone(), barrier.epoch.curr), + ) + .is_none()); if is_stop { assert!(self.pending_subscribers.is_empty()); - self.started_subscribers.clear(); assert!( !self.status.is_stopping(), "stopped actor should not issue barrier" ); - self.status = InflightActorStatus::Stopping; + self.status = InflightActorStatus::Stopping(barrier.epoch.prev); } else { - self.status = InflightActorStatus::Running; + self.status = InflightActorStatus::Running(barrier.epoch.prev); } } pub(super) fn collect(&mut self, epoch: EpochPair) -> (PartialGraphId, bool) { - let (prev_epoch, (prev_partial_graph_id, _)) = + let (prev_epoch, prev_partial_graph_id) = self.inflight_barriers.pop_first().expect("should exist"); assert_eq!(prev_epoch, epoch.prev); + let (min_mutation_epoch, _) = self.barrier_mutations.pop_first().expect("should exist"); + assert_eq!(min_mutation_epoch, epoch.prev); ( prev_partial_graph_id, self.inflight_barriers.is_empty() && self.status.is_stopping(), @@ -279,7 +303,7 @@ impl InflightActorState { } pub(super) fn is_running(&self) -> bool { - matches!(&self.status, InflightActorStatus::Running) + matches!(&self.status, InflightActorStatus::Running(_)) } } @@ -384,24 +408,35 @@ impl InflightActorState { start_prev_epoch: u64, tx: mpsc::UnboundedSender, ) { - if self.inflight_barriers.contains_key(&start_prev_epoch) { - for (prev_epoch, (_, mutation)) in self.inflight_barriers.range(start_prev_epoch..) { - if tx.send((*prev_epoch, mutation.clone())).is_err() { + if let Some((mutation, start_curr_epoch)) = self.barrier_mutations.get(&start_prev_epoch) { + if tx.send((start_prev_epoch, mutation.clone())).is_err() { + return; + } + let mut prev_epoch = *start_curr_epoch; + for (mutation_prev_epoch, (mutation, mutation_curr_epoch)) in + self.barrier_mutations.range(start_curr_epoch..) + { + assert_eq!(prev_epoch, *mutation_prev_epoch); + if tx.send((prev_epoch, mutation.clone())).is_err() { // No more subscribe on the mutation. Simply return. return; } + prev_epoch = *mutation_curr_epoch; } if !self.status.is_stopping() { // Only add the subscribers when the actor is not stopped yet. - self.started_subscribers.push(tx); + self.pending_subscribers + .entry(prev_epoch) + .or_default() + .push(tx); } } else { // Barrier has not issued yet. Store the pending tx - if let Some((last_epoch, _)) = self.inflight_barriers.last_key_value() { + if let Some(max_issued_epoch) = self.status.max_issued_epoch() { assert!( - *last_epoch < start_prev_epoch, + max_issued_epoch < start_prev_epoch, "later barrier {} has been issued, but skip the start epoch {:?}", - last_epoch, + max_issued_epoch, start_prev_epoch ); } else { diff --git a/src/stream/src/task/barrier_manager/progress.rs b/src/stream/src/task/barrier_manager/progress.rs index 256c1f99976b..9a243c2e975d 100644 --- a/src/stream/src/task/barrier_manager/progress.rs +++ b/src/stream/src/task/barrier_manager/progress.rs @@ -49,7 +49,7 @@ impl LocalBarrierWorker { state: BackfillState, ) { 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(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 From 8f97fd72af3ee9eb4c146d7d3344d952ca48851a Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 30 Jul 2024 14:44:19 +0800 Subject: [PATCH 18/19] issue barrier to actor after issue to graph --- src/stream/src/task/barrier_manager/managed_state.rs | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/src/stream/src/task/barrier_manager/managed_state.rs b/src/stream/src/task/barrier_manager/managed_state.rs index eb468e1d58e6..3651dcc44d5e 100644 --- a/src/stream/src/task/barrier_manager/managed_state.rs +++ b/src/stream/src/task/barrier_manager/managed_state.rs @@ -483,20 +483,22 @@ impl ManagedBarrierState { ) }); - for actor_id in &actor_ids_to_collect { + graph_state.transform_to_issued(barrier, actor_ids_to_collect.clone(), table_ids); + + // Note: it's important to issue barrier to actor after issuing to graph to ensure that + // we call `start_epoch` on the graph before the actors receive the barrier + for actor_id in actor_ids_to_collect { self.actor_states - .entry(*actor_id) + .entry(actor_id) .or_insert_with(InflightActorState::not_started) .issue_barrier( partial_graph_id, barrier, actor_to_stop - .map(|actors| actors.contains(actor_id)) + .map(|actors| actors.contains(&actor_id)) .unwrap_or(false), ); } - - graph_state.transform_to_issued(barrier, actor_ids_to_collect, table_ids); } pub(super) fn next_completed_epoch( From 20e819c687e1bb5e05bc2351c71364f9ed41b5c2 Mon Sep 17 00:00:00 2001 From: William Wen Date: Tue, 30 Jul 2024 15:52:08 +0800 Subject: [PATCH 19/19] fix to report error on running actor --- src/stream/src/task/barrier_manager.rs | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/stream/src/task/barrier_manager.rs b/src/stream/src/task/barrier_manager.rs index f315fc8caff2..94dbc0764497 100644 --- a/src/stream/src/task/barrier_manager.rs +++ b/src/stream/src/task/barrier_manager.rs @@ -858,7 +858,7 @@ impl LocalBarrierWorker { let root_err = self.try_find_root_failure().await.unwrap(); // always `Some` because we just added one if let Some(actor_state) = self.state.actor_states.get(&actor_id) - && (!actor_state.inflight_barriers.is_empty() || !actor_state.is_running()) + && (!actor_state.inflight_barriers.is_empty() || actor_state.is_running()) { self.control_stream_handle.reset_stream_with_err( anyhow!(root_err)