risingwave_meta/manager/
metadata.rs

1// Copyright 2024 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use std::collections::{BTreeMap, HashMap, HashSet};
16use std::fmt::{Debug, Formatter};
17
18use anyhow::anyhow;
19use itertools::Itertools;
20use risingwave_common::catalog::{DatabaseId, TableId, TableOption};
21use risingwave_common::id::JobId;
22use risingwave_meta_model::refresh_job::{self, RefreshState};
23use risingwave_meta_model::{SinkId, SourceId, WorkerId};
24use risingwave_pb::catalog::{PbSource, PbTable};
25use risingwave_pb::common::worker_node::{PbResource, Property as AddNodeProperty, State};
26use risingwave_pb::common::{HostAddress, PbWorkerNode, PbWorkerType, WorkerNode, WorkerType};
27use risingwave_pb::meta::list_rate_limits_response::RateLimitInfo;
28use risingwave_pb::stream_plan::{PbDispatcherType, PbStreamScanType};
29use sea_orm::TransactionTrait;
30use sea_orm::prelude::DateTime;
31use tokio::sync::mpsc::{UnboundedReceiver, unbounded_channel};
32use tokio::sync::oneshot;
33use tracing::warn;
34
35use crate::MetaResult;
36use crate::controller::catalog::CatalogControllerRef;
37use crate::controller::cluster::{ClusterControllerRef, StreamingClusterInfo, WorkerExtraInfo};
38use crate::controller::scale::find_fragment_no_shuffle_dags_detailed;
39use crate::manager::{LocalNotification, NotificationVersion};
40use crate::model::{ActorId, ClusterId, Fragment, FragmentId, StreamJobFragments, SubscriptionId};
41use crate::stream::SplitAssignment;
42use crate::telemetry::MetaTelemetryJobDesc;
43
44#[derive(Clone)]
45pub struct MetadataManager {
46    pub cluster_controller: ClusterControllerRef,
47    pub catalog_controller: CatalogControllerRef,
48}
49
50#[derive(Debug)]
51pub(crate) enum ActiveStreamingWorkerChange {
52    Add(WorkerNode),
53    Remove(WorkerNode),
54    Update(WorkerNode),
55}
56
57pub struct ActiveStreamingWorkerNodes {
58    worker_nodes: HashMap<WorkerId, WorkerNode>,
59    rx: UnboundedReceiver<LocalNotification>,
60    #[cfg_attr(not(debug_assertions), expect(dead_code))]
61    meta_manager: Option<MetadataManager>,
62}
63
64impl Debug for ActiveStreamingWorkerNodes {
65    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
66        f.debug_struct("ActiveStreamingWorkerNodes")
67            .field("worker_nodes", &self.worker_nodes)
68            .finish()
69    }
70}
71
72impl ActiveStreamingWorkerNodes {
73    pub(crate) fn uninitialized() -> Self {
74        Self {
75            worker_nodes: Default::default(),
76            rx: unbounded_channel().1,
77            meta_manager: None,
78        }
79    }
80
81    #[cfg(test)]
82    pub(crate) fn for_test(worker_nodes: HashMap<WorkerId, WorkerNode>) -> Self {
83        let (tx, rx) = unbounded_channel();
84        let _join_handle = tokio::spawn(async move {
85            let _tx = tx;
86            std::future::pending::<()>().await
87        });
88        Self {
89            worker_nodes,
90            rx,
91            meta_manager: None,
92        }
93    }
94
95    /// Return an uninitialized one as a placeholder for future initialized
96    pub(crate) async fn new_snapshot(meta_manager: MetadataManager) -> MetaResult<Self> {
97        let (nodes, rx) = meta_manager
98            .subscribe_active_streaming_compute_nodes()
99            .await?;
100        Ok(Self {
101            worker_nodes: nodes
102                .into_iter()
103                .filter_map(|node| {
104                    let is_streaming = node.property.as_ref().is_some_and(|p| p.is_streaming);
105                    is_streaming.then_some((node.id, node))
106                })
107                .collect(),
108            rx,
109            meta_manager: Some(meta_manager),
110        })
111    }
112
113    pub(crate) fn current(&self) -> &HashMap<WorkerId, WorkerNode> {
114        &self.worker_nodes
115    }
116
117    pub(crate) async fn changed(&mut self) -> ActiveStreamingWorkerChange {
118        loop {
119            let notification = self
120                .rx
121                .recv()
122                .await
123                .expect("notification stopped or uninitialized");
124            fn is_target_worker_node(worker: &WorkerNode) -> bool {
125                worker.r#type == WorkerType::ComputeNode as i32
126                    && worker.property.as_ref().unwrap().is_streaming
127            }
128            match notification {
129                LocalNotification::WorkerNodeDeleted(worker) => {
130                    let is_target_worker_node = is_target_worker_node(&worker);
131                    let Some(prev_worker) = self.worker_nodes.remove(&worker.id) else {
132                        if is_target_worker_node {
133                            warn!(
134                                ?worker,
135                                "notify to delete an non-existing streaming compute worker"
136                            );
137                        }
138                        continue;
139                    };
140                    if !is_target_worker_node {
141                        warn!(
142                            ?worker,
143                            ?prev_worker,
144                            "deleted worker has a different recent type"
145                        );
146                    }
147                    if worker.state == State::Starting as i32 {
148                        warn!(
149                            id = %worker.id,
150                            host = ?worker.host,
151                            state = worker.state,
152                            "a starting streaming worker is deleted"
153                        );
154                    }
155                    break ActiveStreamingWorkerChange::Remove(prev_worker);
156                }
157                LocalNotification::WorkerNodeActivated(worker) => {
158                    if !is_target_worker_node(&worker) {
159                        if let Some(prev_worker) = self.worker_nodes.remove(&worker.id) {
160                            warn!(
161                                ?worker,
162                                ?prev_worker,
163                                "the type of a streaming worker is changed"
164                            );
165                            break ActiveStreamingWorkerChange::Remove(prev_worker);
166                        } else {
167                            continue;
168                        }
169                    }
170                    assert_eq!(
171                        worker.state,
172                        State::Running as i32,
173                        "not started worker added: {:?}",
174                        worker
175                    );
176                    if let Some(prev_worker) = self.worker_nodes.insert(worker.id, worker.clone()) {
177                        assert_eq!(prev_worker.host, worker.host);
178                        assert_eq!(prev_worker.r#type, worker.r#type);
179                        warn!(
180                            ?prev_worker,
181                            ?worker,
182                            eq = prev_worker == worker,
183                            "notify to update an existing active worker"
184                        );
185                        if prev_worker == worker {
186                            continue;
187                        } else {
188                            break ActiveStreamingWorkerChange::Update(worker);
189                        }
190                    } else {
191                        break ActiveStreamingWorkerChange::Add(worker);
192                    }
193                }
194                _ => {
195                    continue;
196                }
197            }
198        }
199    }
200
201    #[cfg(debug_assertions)]
202    pub(crate) async fn validate_change(&self) {
203        use risingwave_pb::common::WorkerNode;
204        use thiserror_ext::AsReport;
205        let Some(meta_manager) = &self.meta_manager else {
206            return;
207        };
208        match meta_manager.list_active_streaming_compute_nodes().await {
209            Ok(worker_nodes) => {
210                let ignore_irrelevant_info = |node: &WorkerNode| {
211                    (
212                        node.id,
213                        WorkerNode {
214                            id: node.id,
215                            r#type: node.r#type,
216                            host: node.host.clone(),
217                            property: node.property.clone(),
218                            resource: node.resource.clone(),
219                            ..Default::default()
220                        },
221                    )
222                };
223                let worker_nodes: HashMap<_, _> =
224                    worker_nodes.iter().map(ignore_irrelevant_info).collect();
225                let curr_worker_nodes: HashMap<_, _> = self
226                    .current()
227                    .values()
228                    .map(ignore_irrelevant_info)
229                    .collect();
230                if worker_nodes != curr_worker_nodes {
231                    warn!(
232                        ?worker_nodes,
233                        ?curr_worker_nodes,
234                        "different to global snapshot"
235                    );
236                }
237            }
238            Err(e) => {
239                warn!(e = ?e.as_report(), "fail to list_active_streaming_compute_nodes to compare with local snapshot");
240            }
241        }
242    }
243}
244
245impl MetadataManager {
246    pub fn new(
247        cluster_controller: ClusterControllerRef,
248        catalog_controller: CatalogControllerRef,
249    ) -> Self {
250        Self {
251            cluster_controller,
252            catalog_controller,
253        }
254    }
255
256    pub async fn get_worker_by_id(&self, worker_id: WorkerId) -> MetaResult<Option<PbWorkerNode>> {
257        self.cluster_controller.get_worker_by_id(worker_id).await
258    }
259
260    pub async fn count_worker_node(&self) -> MetaResult<HashMap<WorkerType, u64>> {
261        let node_map = self.cluster_controller.count_worker_by_type().await?;
262        Ok(node_map
263            .into_iter()
264            .map(|(ty, cnt)| (ty.into(), cnt as u64))
265            .collect())
266    }
267
268    pub async fn get_worker_info_by_id(&self, worker_id: WorkerId) -> Option<WorkerExtraInfo> {
269        self.cluster_controller
270            .get_worker_info_by_id(worker_id as _)
271            .await
272    }
273
274    pub async fn add_worker_node(
275        &self,
276        r#type: PbWorkerType,
277        host_address: HostAddress,
278        property: AddNodeProperty,
279        resource: PbResource,
280    ) -> MetaResult<WorkerId> {
281        self.cluster_controller
282            .add_worker(r#type, host_address, property, resource)
283            .await
284            .map(|id| id as WorkerId)
285    }
286
287    pub async fn list_worker_node(
288        &self,
289        worker_type: Option<WorkerType>,
290        worker_state: Option<State>,
291    ) -> MetaResult<Vec<PbWorkerNode>> {
292        self.cluster_controller
293            .list_workers(worker_type.map(Into::into), worker_state.map(Into::into))
294            .await
295    }
296
297    pub async fn subscribe_active_streaming_compute_nodes(
298        &self,
299    ) -> MetaResult<(Vec<WorkerNode>, UnboundedReceiver<LocalNotification>)> {
300        self.cluster_controller
301            .subscribe_active_streaming_compute_nodes()
302            .await
303    }
304
305    pub async fn list_active_streaming_compute_nodes(&self) -> MetaResult<Vec<PbWorkerNode>> {
306        self.cluster_controller
307            .list_active_streaming_workers()
308            .await
309    }
310
311    pub async fn list_active_serving_compute_nodes(&self) -> MetaResult<Vec<PbWorkerNode>> {
312        self.cluster_controller.list_active_serving_workers().await
313    }
314
315    pub async fn list_active_database_ids(&self) -> MetaResult<HashSet<DatabaseId>> {
316        Ok(self
317            .catalog_controller
318            .list_fragment_database_ids(None)
319            .await?
320            .into_iter()
321            .map(|(_, database_id)| database_id)
322            .collect())
323    }
324
325    pub async fn split_fragment_map_by_database<T: Debug>(
326        &self,
327        fragment_map: HashMap<FragmentId, T>,
328    ) -> MetaResult<HashMap<DatabaseId, HashMap<FragmentId, T>>> {
329        let fragment_to_database_map: HashMap<_, _> = self
330            .catalog_controller
331            .list_fragment_database_ids(Some(
332                fragment_map
333                    .keys()
334                    .map(|fragment_id| *fragment_id as _)
335                    .collect(),
336            ))
337            .await?
338            .into_iter()
339            .map(|(fragment_id, database_id)| (fragment_id as FragmentId, database_id))
340            .collect();
341        let mut ret: HashMap<_, HashMap<_, _>> = HashMap::new();
342        for (fragment_id, value) in fragment_map {
343            let database_id = *fragment_to_database_map
344                .get(&fragment_id)
345                .ok_or_else(|| anyhow!("cannot get database_id of fragment {fragment_id}"))?;
346            ret.entry(database_id)
347                .or_default()
348                .try_insert(fragment_id, value)
349                .expect("non duplicate");
350        }
351        Ok(ret)
352    }
353
354    pub async fn list_background_creating_jobs(&self) -> MetaResult<HashSet<JobId>> {
355        self.catalog_controller
356            .list_background_creating_jobs(false, None)
357            .await
358    }
359
360    pub async fn list_sources(&self) -> MetaResult<Vec<PbSource>> {
361        self.catalog_controller.list_sources().await
362    }
363
364    /// Get and filter the "**root**" fragments of the specified relations.
365    /// The root fragment is the bottom-most fragment of its fragment graph, and can be a `MView` or a `Source`.
366    ///
367    /// See also [`crate::controller::catalog::CatalogController::get_root_fragments`].
368    pub async fn get_upstream_root_fragments(
369        &self,
370        upstream_table_ids: &HashSet<TableId>,
371    ) -> MetaResult<HashMap<JobId, Fragment>> {
372        let upstream_root_fragments = self
373            .catalog_controller
374            .get_root_fragments(upstream_table_ids.iter().map(|id| id.as_job_id()).collect())
375            .await?;
376
377        Ok(upstream_root_fragments)
378    }
379
380    pub async fn get_streaming_cluster_info(&self) -> MetaResult<StreamingClusterInfo> {
381        self.cluster_controller.get_streaming_cluster_info().await
382    }
383
384    pub async fn get_all_table_options(&self) -> MetaResult<HashMap<TableId, TableOption>> {
385        self.catalog_controller.get_all_table_options().await
386    }
387
388    pub async fn get_table_name_type_mapping(
389        &self,
390    ) -> MetaResult<HashMap<TableId, (String, String)>> {
391        self.catalog_controller.get_table_name_type_mapping().await
392    }
393
394    pub async fn get_created_table_ids(&self) -> MetaResult<Vec<TableId>> {
395        self.catalog_controller.get_created_table_ids().await
396    }
397
398    pub async fn get_table_associated_source_id(
399        &self,
400        table_id: TableId,
401    ) -> MetaResult<Option<SourceId>> {
402        self.catalog_controller
403            .get_table_associated_source_id(table_id)
404            .await
405    }
406
407    pub async fn get_table_catalog_by_ids(&self, ids: &[TableId]) -> MetaResult<Vec<PbTable>> {
408        self.catalog_controller
409            .get_table_by_ids(ids.to_vec(), false)
410            .await
411    }
412
413    pub async fn list_refresh_jobs(&self) -> MetaResult<Vec<refresh_job::Model>> {
414        self.catalog_controller.list_refresh_jobs().await
415    }
416
417    pub async fn list_refreshable_table_ids(&self) -> MetaResult<Vec<TableId>> {
418        self.catalog_controller.list_refreshable_table_ids().await
419    }
420
421    pub async fn ensure_refresh_job(&self, table_id: TableId) -> MetaResult<()> {
422        self.catalog_controller.ensure_refresh_job(table_id).await
423    }
424
425    pub async fn update_refresh_job_status(
426        &self,
427        table_id: TableId,
428        status: RefreshState,
429        trigger_time: Option<DateTime>,
430        is_success: bool,
431    ) -> MetaResult<()> {
432        self.catalog_controller
433            .update_refresh_job_status(table_id, status, trigger_time, is_success)
434            .await
435    }
436
437    pub async fn reset_all_refresh_jobs_to_idle(&self) -> MetaResult<()> {
438        self.catalog_controller
439            .reset_all_refresh_jobs_to_idle()
440            .await
441    }
442
443    pub async fn update_refresh_job_interval(
444        &self,
445        table_id: TableId,
446        trigger_interval_secs: Option<i64>,
447    ) -> MetaResult<()> {
448        self.catalog_controller
449            .update_refresh_job_interval(table_id, trigger_interval_secs)
450            .await
451    }
452
453    pub async fn get_sink_state_table_ids(&self, sink_id: SinkId) -> MetaResult<Vec<TableId>> {
454        self.catalog_controller
455            .get_sink_state_table_ids(sink_id)
456            .await
457    }
458
459    pub async fn get_table_catalog_by_cdc_table_id(
460        &self,
461        cdc_table_id: &String,
462    ) -> MetaResult<Vec<PbTable>> {
463        self.catalog_controller
464            .get_table_by_cdc_table_id(cdc_table_id)
465            .await
466    }
467
468    pub async fn get_downstream_fragments(
469        &self,
470        job_id: JobId,
471    ) -> MetaResult<Vec<(PbDispatcherType, Fragment)>> {
472        self.catalog_controller
473            .get_downstream_fragments(job_id)
474            .await
475    }
476
477    pub async fn get_job_id_to_internal_table_ids_mapping(
478        &self,
479    ) -> Option<Vec<(JobId, Vec<TableId>)>> {
480        self.catalog_controller.get_job_internal_table_ids().await
481    }
482
483    pub async fn get_job_fragments_by_id(&self, job_id: JobId) -> MetaResult<StreamJobFragments> {
484        Ok(self
485            .catalog_controller
486            .get_job_fragments_by_id(job_id)
487            .await?
488            .0)
489    }
490
491    pub fn get_running_actors_of_fragment(&self, id: FragmentId) -> MetaResult<HashSet<ActorId>> {
492        self.catalog_controller
493            .get_running_actors_of_fragment(id as _)
494    }
495
496    // (backfill_actor_id, upstream_source_actor_id)
497    pub async fn get_running_actors_for_source_backfill(
498        &self,
499        source_backfill_fragment_id: FragmentId,
500        source_fragment_id: FragmentId,
501    ) -> MetaResult<HashSet<(ActorId, ActorId)>> {
502        let actor_ids = self
503            .catalog_controller
504            .get_running_actors_for_source_backfill(
505                source_backfill_fragment_id as _,
506                source_fragment_id as _,
507            )
508            .await?;
509        Ok(actor_ids
510            .into_iter()
511            .map(|(id, upstream)| (id as ActorId, upstream as ActorId))
512            .collect())
513    }
514
515    pub fn worker_actor_count(&self) -> MetaResult<HashMap<WorkerId, usize>> {
516        let actor_cnt = self.catalog_controller.worker_actor_count()?;
517        Ok(actor_cnt
518            .into_iter()
519            .map(|(id, cnt)| (id as WorkerId, cnt))
520            .collect())
521    }
522
523    pub async fn count_streaming_job(&self) -> MetaResult<usize> {
524        self.catalog_controller.count_streaming_jobs().await
525    }
526
527    pub async fn list_stream_job_desc(&self) -> MetaResult<Vec<MetaTelemetryJobDesc>> {
528        self.catalog_controller
529            .list_stream_job_desc_for_telemetry()
530            .await
531    }
532
533    pub async fn update_source_rate_limit_by_source_id(
534        &self,
535        source_id: SourceId,
536        rate_limit: Option<u32>,
537    ) -> MetaResult<(HashSet<JobId>, HashSet<FragmentId>)> {
538        self.catalog_controller
539            .update_source_rate_limit_by_source_id(source_id as _, rate_limit)
540            .await
541    }
542
543    pub async fn update_backfill_rate_limit_by_job_id(
544        &self,
545        job_id: JobId,
546        rate_limit: Option<u32>,
547    ) -> MetaResult<HashSet<FragmentId>> {
548        self.catalog_controller
549            .update_backfill_rate_limit_by_job_id(job_id, rate_limit)
550            .await
551    }
552
553    pub async fn update_sink_rate_limit_by_sink_id(
554        &self,
555        sink_id: SinkId,
556        rate_limit: Option<u32>,
557    ) -> MetaResult<HashSet<FragmentId>> {
558        self.catalog_controller
559            .update_sink_rate_limit_by_job_id(sink_id, rate_limit)
560            .await
561    }
562
563    pub async fn update_dml_rate_limit_by_job_id(
564        &self,
565        job_id: JobId,
566        rate_limit: Option<u32>,
567    ) -> MetaResult<HashSet<FragmentId>> {
568        self.catalog_controller
569            .update_dml_rate_limit_by_job_id(job_id, rate_limit)
570            .await
571    }
572
573    pub async fn update_sink_props_by_sink_id(
574        &self,
575        sink_id: SinkId,
576        props: BTreeMap<String, String>,
577    ) -> MetaResult<HashMap<String, String>> {
578        let new_props = self
579            .catalog_controller
580            .update_sink_props_by_sink_id(sink_id, props)
581            .await?;
582        Ok(new_props)
583    }
584
585    pub async fn update_iceberg_table_props_by_table_id(
586        &self,
587        table_id: TableId,
588        props: BTreeMap<String, String>,
589        alter_iceberg_table_props: Option<
590            risingwave_pb::meta::alter_connector_props_request::PbExtraOptions,
591        >,
592    ) -> MetaResult<(HashMap<String, String>, SinkId)> {
593        let (new_props, sink_id) = self
594            .catalog_controller
595            .update_iceberg_table_props_by_table_id(table_id, props, alter_iceberg_table_props)
596            .await?;
597        Ok((new_props, sink_id))
598    }
599
600    pub async fn update_fragment_rate_limit_by_fragment_id(
601        &self,
602        fragment_id: FragmentId,
603        rate_limit: Option<u32>,
604    ) -> MetaResult<()> {
605        self.catalog_controller
606            .update_fragment_rate_limit_by_fragment_id(fragment_id as _, rate_limit)
607            .await
608    }
609
610    #[await_tree::instrument]
611    pub async fn update_fragment_splits(
612        &self,
613        split_assignment: &SplitAssignment,
614    ) -> MetaResult<()> {
615        let fragment_splits = split_assignment
616            .iter()
617            .map(|(fragment_id, splits)| {
618                (
619                    *fragment_id as _,
620                    splits.values().flatten().cloned().collect_vec(),
621                )
622            })
623            .collect();
624
625        let inner = self.catalog_controller.inner.write().await;
626
627        self.catalog_controller
628            .update_fragment_splits(&inner.db, &fragment_splits)
629            .await
630    }
631
632    pub async fn get_mv_depended_subscriptions(
633        &self,
634        database_id: Option<DatabaseId>,
635    ) -> MetaResult<HashMap<TableId, HashMap<SubscriptionId, u64>>> {
636        Ok(self
637            .catalog_controller
638            .get_mv_depended_subscriptions(database_id)
639            .await?
640            .into_iter()
641            .map(|(table_id, subscriptions)| {
642                (
643                    table_id,
644                    subscriptions
645                        .into_iter()
646                        .map(|(subscription_id, retention_time)| {
647                            (subscription_id as SubscriptionId, retention_time)
648                        })
649                        .collect(),
650                )
651            })
652            .collect())
653    }
654
655    pub async fn get_job_max_parallelism(&self, job_id: JobId) -> MetaResult<usize> {
656        self.catalog_controller
657            .get_max_parallelism_by_id(job_id)
658            .await
659    }
660
661    pub async fn get_existing_job_resource_group(
662        &self,
663        streaming_job_id: JobId,
664    ) -> MetaResult<String> {
665        self.catalog_controller
666            .get_existing_job_resource_group(streaming_job_id)
667            .await
668    }
669
670    pub async fn get_database_resource_group(&self, database_id: DatabaseId) -> MetaResult<String> {
671        self.catalog_controller
672            .get_database_resource_group(database_id)
673            .await
674    }
675
676    pub fn cluster_id(&self) -> &ClusterId {
677        self.cluster_controller.cluster_id()
678    }
679
680    pub async fn list_rate_limits(&self) -> MetaResult<Vec<RateLimitInfo>> {
681        let rate_limits = self.catalog_controller.list_rate_limits().await?;
682        Ok(rate_limits)
683    }
684
685    pub async fn get_job_backfill_scan_types(
686        &self,
687        job_id: JobId,
688    ) -> MetaResult<HashMap<FragmentId, PbStreamScanType>> {
689        let backfill_types = self
690            .catalog_controller
691            .get_job_fragment_backfill_scan_type(job_id)
692            .await?;
693        Ok(backfill_types)
694    }
695
696    pub async fn collect_unreschedulable_backfill_jobs(
697        &self,
698        job_ids: impl IntoIterator<Item = &JobId>,
699        is_online: bool,
700    ) -> MetaResult<HashSet<JobId>> {
701        let mut unreschedulable = HashSet::new();
702
703        for job_id in job_ids {
704            let scan_types = self
705                .catalog_controller
706                .get_job_fragment_backfill_scan_type(*job_id)
707                .await?;
708            if scan_types
709                .values()
710                .any(|scan_type| !scan_type.is_reschedulable(is_online))
711            {
712                unreschedulable.insert(*job_id);
713            }
714        }
715
716        Ok(unreschedulable)
717    }
718
719    pub async fn collect_reschedule_blocked_jobs_for_creating_jobs(
720        &self,
721        creating_job_ids: impl IntoIterator<Item = &JobId>,
722        is_online: bool,
723    ) -> MetaResult<HashSet<JobId>> {
724        let creating_job_ids: HashSet<_> = creating_job_ids.into_iter().copied().collect();
725        if creating_job_ids.is_empty() {
726            return Ok(HashSet::new());
727        }
728
729        let inner = self.catalog_controller.inner.read().await;
730        let txn = inner.db.begin().await?;
731
732        let mut initial_fragment_ids = HashSet::new();
733        for job_id in &creating_job_ids {
734            let scan_types = self
735                .catalog_controller
736                .get_job_fragment_backfill_scan_type_in_txn(&txn, *job_id)
737                .await?;
738            initial_fragment_ids.extend(scan_types.into_iter().filter_map(
739                |(fragment_id, scan_type)| {
740                    (!scan_type.is_reschedulable(is_online)).then_some(fragment_id)
741                },
742            ));
743        }
744
745        if !initial_fragment_ids.is_empty() {
746            let upstream_fragments = self
747                .catalog_controller
748                .upstream_fragments_in_txn(&txn, initial_fragment_ids.iter().copied())
749                .await?;
750            initial_fragment_ids.extend(upstream_fragments.into_values().flatten());
751        }
752
753        let mut blocked_fragment_ids = initial_fragment_ids.clone();
754        if !initial_fragment_ids.is_empty() {
755            let initial_fragment_ids = initial_fragment_ids.into_iter().collect_vec();
756            let ensembles =
757                find_fragment_no_shuffle_dags_detailed(&txn, &initial_fragment_ids).await?;
758            for ensemble in ensembles {
759                blocked_fragment_ids.extend(ensemble.fragments());
760            }
761        }
762
763        let mut blocked_job_ids = HashSet::new();
764        if !blocked_fragment_ids.is_empty() {
765            let fragment_ids = blocked_fragment_ids.into_iter().collect_vec();
766            let fragment_job_ids = self
767                .catalog_controller
768                .get_fragment_job_id_in_txn(&txn, fragment_ids)
769                .await?;
770            blocked_job_ids.extend(
771                fragment_job_ids
772                    .into_iter()
773                    .map(|job_id| job_id.as_job_id()),
774            );
775        }
776
777        txn.commit().await?;
778
779        Ok(blocked_job_ids)
780    }
781}
782
783impl MetadataManager {
784    /// Wait for job finishing notification in `TrackingJob::finish`.
785    /// The progress is updated per barrier.
786    #[await_tree::instrument]
787    pub async fn wait_streaming_job_finished(
788        &self,
789        database_id: DatabaseId,
790        id: JobId,
791    ) -> MetaResult<NotificationVersion> {
792        tracing::debug!("wait_streaming_job_finished: {id:?}");
793        let mut mgr = self.catalog_controller.get_inner_write_guard().await;
794        if mgr.streaming_job_is_finished(id).await? {
795            return Ok(self.catalog_controller.notify_frontend_trivial().await);
796        }
797        let (tx, rx) = oneshot::channel();
798
799        mgr.register_finish_notifier(database_id, id, tx);
800        drop(mgr);
801        rx.await
802            .map_err(|_| "no received reason".to_owned())
803            .and_then(|result| result)
804            .map_err(|reason| anyhow!("failed to wait streaming job finish: {}", reason).into())
805    }
806
807    pub(crate) async fn notify_finish_failed(&self, database_id: Option<DatabaseId>, err: String) {
808        let mut mgr = self.catalog_controller.get_inner_write_guard().await;
809        mgr.notify_finish_failed(database_id, err);
810    }
811}