risingwave_meta/barrier/
manager.rs1use std::collections::hash_map::Entry;
16use std::sync::Arc;
17
18use anyhow::Context;
19use arc_swap::ArcSwap;
20use risingwave_common::bail;
21use risingwave_hummock_sdk::HummockVersionId;
22use risingwave_meta_model::DatabaseId;
23use risingwave_pb::ddl_service::DdlProgress;
24use risingwave_pb::meta::PbRecoveryStatus;
25use tokio::sync::mpsc::unbounded_channel;
26use tokio::sync::{mpsc, oneshot};
27use tokio::task::JoinHandle;
28use tracing::warn;
29
30use crate::MetaResult;
31use crate::barrier::worker::GlobalBarrierWorker;
32use crate::barrier::{BarrierManagerRequest, BarrierManagerStatus, RecoveryReason, schedule};
33use crate::hummock::HummockManagerRef;
34use crate::manager::sink_coordination::SinkCoordinatorManager;
35use crate::manager::{MetaSrvEnv, MetadataManager};
36use crate::stream::{ScaleControllerRef, SourceManagerRef};
37
38pub struct GlobalBarrierManager {
39 status: Arc<ArcSwap<BarrierManagerStatus>>,
40 hummock_manager: HummockManagerRef,
41 request_tx: mpsc::UnboundedSender<BarrierManagerRequest>,
42 metadata_manager: MetadataManager,
43}
44
45pub type BarrierManagerRef = Arc<GlobalBarrierManager>;
46
47impl GlobalBarrierManager {
48 pub async fn get_ddl_progress(&self) -> MetaResult<Vec<DdlProgress>> {
50 let mut ddl_progress = {
51 let (tx, rx) = oneshot::channel();
52 self.request_tx
53 .send(BarrierManagerRequest::GetDdlProgress(tx))
54 .context("failed to send get ddl progress request")?;
55 rx.await.context("failed to receive get ddl progress")?
56 };
57 let mviews = self
60 .metadata_manager
61 .catalog_controller
62 .list_background_creating_mviews(true)
63 .await?;
64 for mview in mviews {
65 if let Entry::Vacant(e) = ddl_progress.entry(mview.table_id as _) {
66 warn!(
67 job_id = mview.table_id,
68 "background job has no ddl progress"
69 );
70 e.insert(DdlProgress {
71 id: mview.table_id as u64,
72 statement: mview.definition,
73 progress: "0.0%".into(),
74 });
75 }
76 }
77
78 Ok(ddl_progress.into_values().collect())
79 }
80
81 pub async fn adhoc_recovery(&self) -> MetaResult<()> {
82 let (tx, rx) = oneshot::channel();
83 self.request_tx
84 .send(BarrierManagerRequest::AdhocRecovery(tx))
85 .context("failed to send adhoc recovery request")?;
86 rx.await.context("failed to wait adhoc recovery")?;
87 Ok(())
88 }
89
90 pub async fn update_database_barrier(
91 &self,
92 database_id: DatabaseId,
93 barrier_interval_ms: Option<u32>,
94 checkpoint_frequency: Option<u64>,
95 ) -> MetaResult<()> {
96 let (tx, rx) = oneshot::channel();
97 self.request_tx
98 .send(BarrierManagerRequest::UpdateDatabaseBarrier {
99 database_id: (database_id as u32).into(),
100 barrier_interval_ms,
101 checkpoint_frequency,
102 sender: tx,
103 })
104 .context("failed to send update database barrier request")?;
105 rx.await.context("failed to wait update database barrier")?;
106 Ok(())
107 }
108
109 pub async fn get_hummock_version_id(&self) -> HummockVersionId {
110 self.hummock_manager.get_version_id().await
111 }
112}
113
114impl GlobalBarrierManager {
115 pub fn check_status_running(&self) -> MetaResult<()> {
117 let status = self.status.load();
118 match &**status {
119 BarrierManagerStatus::Starting
120 | BarrierManagerStatus::Recovering(RecoveryReason::Bootstrap) => {
121 bail!("The cluster is bootstrapping")
122 }
123 BarrierManagerStatus::Recovering(RecoveryReason::Failover(e)) => {
124 Err(anyhow::anyhow!(e.clone()).context("The cluster is recovering"))?
125 }
126 BarrierManagerStatus::Recovering(RecoveryReason::Adhoc) => {
127 bail!("The cluster is recovering-adhoc")
128 }
129 BarrierManagerStatus::Running => Ok(()),
130 }
131 }
132
133 pub fn get_recovery_status(&self) -> PbRecoveryStatus {
134 (&**self.status.load()).into()
135 }
136}
137
138impl GlobalBarrierManager {
139 pub async fn start(
140 scheduled_barriers: schedule::ScheduledBarriers,
141 env: MetaSrvEnv,
142 metadata_manager: MetadataManager,
143 hummock_manager: HummockManagerRef,
144 source_manager: SourceManagerRef,
145 sink_manager: SinkCoordinatorManager,
146 scale_controller: ScaleControllerRef,
147 ) -> (Arc<Self>, JoinHandle<()>, oneshot::Sender<()>) {
148 let (request_tx, request_rx) = unbounded_channel();
149 let hummock_manager_clone = hummock_manager.clone();
150 let metadata_manager_clone = metadata_manager.clone();
151 let barrier_worker = GlobalBarrierWorker::new(
152 scheduled_barriers,
153 env,
154 metadata_manager,
155 hummock_manager,
156 source_manager,
157 sink_manager,
158 scale_controller,
159 request_rx,
160 )
161 .await;
162 let manager = Self {
163 status: barrier_worker.context.status(),
164 hummock_manager: hummock_manager_clone,
165 request_tx,
166 metadata_manager: metadata_manager_clone,
167 };
168 let (join_handle, shutdown_tx) = barrier_worker.start();
169 (Arc::new(manager), join_handle, shutdown_tx)
170 }
171}