risingwave_meta/manager/
metadata.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
// 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::collections::{BTreeMap, HashMap, HashSet};
use std::fmt::{Debug, Formatter};
use std::pin::pin;
use std::time::Duration;

use anyhow::anyhow;
use futures::future::{select, Either};
use risingwave_common::catalog::{DatabaseId, TableId, TableOption};
use risingwave_meta_model::{ObjectId, SourceId, WorkerId};
use risingwave_pb::catalog::{PbSink, PbSource, PbTable};
use risingwave_pb::common::worker_node::{PbResource, State};
use risingwave_pb::common::{HostAddress, PbWorkerNode, PbWorkerType, WorkerNode, WorkerType};
use risingwave_pb::meta::add_worker_node_request::Property as AddNodeProperty;
use risingwave_pb::meta::list_rate_limits_response::RateLimitInfo;
use risingwave_pb::meta::table_fragments::{Fragment, PbFragment};
use risingwave_pb::stream_plan::{PbDispatchStrategy, StreamActor};
use tokio::sync::mpsc::{unbounded_channel, UnboundedReceiver};
use tokio::sync::oneshot;
use tokio::time::{sleep, Instant};
use tracing::warn;

use crate::barrier::Reschedule;
use crate::controller::catalog::CatalogControllerRef;
use crate::controller::cluster::{ClusterControllerRef, StreamingClusterInfo, WorkerExtraInfo};
use crate::controller::fragment::FragmentParallelismInfo;
use crate::manager::{LocalNotification, NotificationVersion};
use crate::model::{
    ActorId, ClusterId, FragmentId, StreamJobFragments, SubscriptionId, TableParallelism,
};
use crate::stream::SplitAssignment;
use crate::telemetry::MetaTelemetryJobDesc;
use crate::{MetaError, MetaResult};

#[derive(Clone)]
pub struct MetadataManager {
    pub cluster_controller: ClusterControllerRef,
    pub catalog_controller: CatalogControllerRef,
}

#[derive(Debug)]
pub(crate) enum ActiveStreamingWorkerChange {
    Add(WorkerNode),
    #[expect(dead_code)]
    Remove(WorkerNode),
    Update(WorkerNode),
}

pub struct ActiveStreamingWorkerNodes {
    worker_nodes: HashMap<WorkerId, WorkerNode>,
    rx: UnboundedReceiver<LocalNotification>,
    #[cfg_attr(not(debug_assertions), expect(dead_code))]
    meta_manager: MetadataManager,
}

impl Debug for ActiveStreamingWorkerNodes {
    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
        f.debug_struct("ActiveStreamingWorkerNodes")
            .field("worker_nodes", &self.worker_nodes)
            .finish()
    }
}

impl ActiveStreamingWorkerNodes {
    pub(crate) fn uninitialized(meta_manager: MetadataManager) -> Self {
        Self {
            worker_nodes: Default::default(),
            rx: unbounded_channel().1,
            meta_manager,
        }
    }

    /// Return an uninitialized one as a placeholder for future initialized
    pub(crate) async fn new_snapshot(meta_manager: MetadataManager) -> MetaResult<Self> {
        let (nodes, rx) = meta_manager
            .subscribe_active_streaming_compute_nodes()
            .await?;
        Ok(Self {
            worker_nodes: nodes.into_iter().map(|node| (node.id as _, node)).collect(),
            rx,
            meta_manager,
        })
    }

    pub(crate) fn current(&self) -> &HashMap<WorkerId, WorkerNode> {
        &self.worker_nodes
    }

    pub(crate) async fn wait_changed(
        &mut self,
        verbose_internal: Duration,
        verbose_timeout: Duration,
        verbose_fn: impl Fn(&Self),
    ) -> Option<ActiveStreamingWorkerChange> {
        let start = Instant::now();
        loop {
            if let Either::Left((change, _)) =
                select(pin!(self.changed()), pin!(sleep(verbose_internal))).await
            {
                break Some(change);
            }

            if start.elapsed() > verbose_timeout {
                break None;
            }

            verbose_fn(self)
        }
    }

    pub(crate) async fn changed(&mut self) -> ActiveStreamingWorkerChange {
        let ret = loop {
            let notification = self
                .rx
                .recv()
                .await
                .expect("notification stopped or uninitialized");
            match notification {
                LocalNotification::WorkerNodeDeleted(worker) => {
                    let is_streaming_compute_node = worker.r#type == WorkerType::ComputeNode as i32
                        && worker.property.as_ref().unwrap().is_streaming;
                    let Some(prev_worker) = self.worker_nodes.remove(&(worker.id as _)) else {
                        if is_streaming_compute_node {
                            warn!(
                                ?worker,
                                "notify to delete an non-existing streaming compute worker"
                            );
                        }
                        continue;
                    };
                    if !is_streaming_compute_node {
                        warn!(
                            ?worker,
                            ?prev_worker,
                            "deleted worker has a different recent type"
                        );
                    }
                    if worker.state == State::Starting as i32 {
                        warn!(
                            id = worker.id,
                            host = ?worker.host,
                            state = worker.state,
                            "a starting streaming worker is deleted"
                        );
                    }
                    break ActiveStreamingWorkerChange::Remove(prev_worker);
                }
                LocalNotification::WorkerNodeActivated(worker) => {
                    if worker.r#type != WorkerType::ComputeNode as i32
                        || !worker.property.as_ref().unwrap().is_streaming
                    {
                        if let Some(prev_worker) = self.worker_nodes.remove(&(worker.id as _)) {
                            warn!(
                                ?worker,
                                ?prev_worker,
                                "the type of a streaming worker is changed"
                            );
                            break ActiveStreamingWorkerChange::Remove(prev_worker);
                        } else {
                            continue;
                        }
                    }
                    assert_eq!(
                        worker.state,
                        State::Running as i32,
                        "not started worker added: {:?}",
                        worker
                    );
                    if let Some(prev_worker) =
                        self.worker_nodes.insert(worker.id as _, worker.clone())
                    {
                        assert_eq!(prev_worker.host, worker.host);
                        assert_eq!(prev_worker.r#type, worker.r#type);
                        warn!(
                            ?prev_worker,
                            ?worker,
                            eq = prev_worker == worker,
                            "notify to update an existing active worker"
                        );
                        if prev_worker == worker {
                            continue;
                        } else {
                            break ActiveStreamingWorkerChange::Update(worker);
                        }
                    } else {
                        break ActiveStreamingWorkerChange::Add(worker);
                    }
                }
                _ => {
                    continue;
                }
            }
        };

        ret
    }

    #[cfg(debug_assertions)]
    pub(crate) async fn validate_change(&self) {
        use risingwave_pb::common::WorkerNode;
        use thiserror_ext::AsReport;
        match self
            .meta_manager
            .list_active_streaming_compute_nodes()
            .await
        {
            Ok(worker_nodes) => {
                let ignore_irrelevant_info = |node: &WorkerNode| {
                    (
                        node.id,
                        WorkerNode {
                            id: node.id,
                            r#type: node.r#type,
                            host: node.host.clone(),
                            parallelism: node.parallelism,
                            property: node.property.clone(),
                            resource: node.resource.clone(),
                            ..Default::default()
                        },
                    )
                };
                let worker_nodes: HashMap<_, _> =
                    worker_nodes.iter().map(ignore_irrelevant_info).collect();
                let curr_worker_nodes: HashMap<_, _> = self
                    .current()
                    .values()
                    .map(ignore_irrelevant_info)
                    .collect();
                if worker_nodes != curr_worker_nodes {
                    warn!(
                        ?worker_nodes,
                        ?curr_worker_nodes,
                        "different to global snapshot"
                    );
                }
            }
            Err(e) => {
                warn!(e = ?e.as_report(), "fail to list_active_streaming_compute_nodes to compare with local snapshot");
            }
        }
    }
}

impl MetadataManager {
    pub fn new(
        cluster_controller: ClusterControllerRef,
        catalog_controller: CatalogControllerRef,
    ) -> Self {
        Self {
            cluster_controller,
            catalog_controller,
        }
    }

    pub async fn get_worker_by_id(&self, worker_id: WorkerId) -> MetaResult<Option<PbWorkerNode>> {
        self.cluster_controller.get_worker_by_id(worker_id).await
    }

    pub async fn count_worker_node(&self) -> MetaResult<HashMap<WorkerType, u64>> {
        let node_map = self.cluster_controller.count_worker_by_type().await?;
        Ok(node_map
            .into_iter()
            .map(|(ty, cnt)| (ty.into(), cnt as u64))
            .collect())
    }

    pub async fn get_worker_info_by_id(&self, worker_id: WorkerId) -> Option<WorkerExtraInfo> {
        self.cluster_controller
            .get_worker_info_by_id(worker_id as _)
            .await
    }

    pub async fn add_worker_node(
        &self,
        r#type: PbWorkerType,
        host_address: HostAddress,
        property: AddNodeProperty,
        resource: PbResource,
    ) -> MetaResult<WorkerId> {
        self.cluster_controller
            .add_worker(r#type, host_address, property, resource)
            .await
            .map(|id| id as WorkerId)
    }

    pub async fn list_worker_node(
        &self,
        worker_type: Option<WorkerType>,
        worker_state: Option<State>,
    ) -> MetaResult<Vec<PbWorkerNode>> {
        self.cluster_controller
            .list_workers(worker_type.map(Into::into), worker_state.map(Into::into))
            .await
    }

    pub async fn subscribe_active_streaming_compute_nodes(
        &self,
    ) -> MetaResult<(Vec<WorkerNode>, UnboundedReceiver<LocalNotification>)> {
        self.cluster_controller
            .subscribe_active_streaming_compute_nodes()
            .await
    }

    pub async fn list_active_streaming_compute_nodes(&self) -> MetaResult<Vec<PbWorkerNode>> {
        self.cluster_controller
            .list_active_streaming_workers()
            .await
    }

    pub async fn list_active_serving_compute_nodes(&self) -> MetaResult<Vec<PbWorkerNode>> {
        self.cluster_controller.list_active_serving_workers().await
    }

    pub async fn list_active_database_ids(&self) -> MetaResult<HashSet<DatabaseId>> {
        Ok(self
            .catalog_controller
            .list_fragment_database_ids(None)
            .await?
            .into_iter()
            .map(|(_, database_id)| DatabaseId::new(database_id as _))
            .collect())
    }

    pub async fn split_fragment_map_by_database<T: Debug>(
        &self,
        fragment_map: HashMap<FragmentId, T>,
    ) -> MetaResult<HashMap<DatabaseId, HashMap<FragmentId, T>>> {
        let fragment_to_database_map: HashMap<_, _> = self
            .catalog_controller
            .list_fragment_database_ids(Some(
                fragment_map
                    .keys()
                    .map(|fragment_id| *fragment_id as _)
                    .collect(),
            ))
            .await?
            .into_iter()
            .map(|(fragment_id, database_id)| {
                (fragment_id as FragmentId, DatabaseId::new(database_id as _))
            })
            .collect();
        let mut ret: HashMap<_, HashMap<_, _>> = HashMap::new();
        for (fragment_id, value) in fragment_map {
            let database_id = *fragment_to_database_map
                .get(&fragment_id)
                .ok_or_else(|| anyhow!("cannot get database_id of fragment {fragment_id}"))?;
            ret.entry(database_id)
                .or_default()
                .try_insert(fragment_id, value)
                .expect("non duplicate");
        }
        Ok(ret)
    }

    pub async fn list_background_creating_jobs(&self) -> MetaResult<Vec<TableId>> {
        let tables = self
            .catalog_controller
            .list_background_creating_mviews(false)
            .await?;

        Ok(tables
            .into_iter()
            .map(|table| TableId::from(table.table_id as u32))
            .collect())
    }

    pub async fn list_sources(&self) -> MetaResult<Vec<PbSource>> {
        self.catalog_controller.list_sources().await
    }

    pub async fn post_apply_reschedules(
        &self,
        reschedules: HashMap<FragmentId, Reschedule>,
        table_parallelism_assignment: HashMap<TableId, TableParallelism>,
    ) -> MetaResult<()> {
        // temp convert u32 to i32
        let reschedules = reschedules.into_iter().map(|(k, v)| (k as _, v)).collect();

        self.catalog_controller
            .post_apply_reschedules(reschedules, table_parallelism_assignment)
            .await
    }

    pub async fn running_fragment_parallelisms(
        &self,
        id_filter: Option<HashSet<FragmentId>>,
    ) -> MetaResult<HashMap<FragmentId, FragmentParallelismInfo>> {
        let id_filter = id_filter.map(|ids| ids.into_iter().map(|id| id as _).collect());
        Ok(self
            .catalog_controller
            .running_fragment_parallelisms(id_filter)
            .await?
            .into_iter()
            .map(|(k, v)| (k as FragmentId, v))
            .collect())
    }

    /// Get and filter the "**root**" fragments of the specified relations.
    /// The root fragment is the bottom-most fragment of its fragment graph, and can be a `MView` or a `Source`.
    ///
    /// ## What can be the root fragment
    /// - For MV, it should have one `MView` fragment.
    /// - For table, it should have one `MView` fragment and one or two `Source` fragments. `MView` should be the root.
    /// - For source, it should have one `Source` fragment.
    ///
    /// In other words, it's the `MView` fragment if it exists, otherwise it's the `Source` fragment.
    ///
    /// ## What do we expect to get for different creating streaming job
    /// - MV/Sink/Index should have MV upstream fragments for upstream MV/Tables, and Source upstream fragments for upstream shared sources.
    /// - CDC Table has a Source upstream fragment.
    /// - Sources and other Tables shouldn't have an upstream fragment.
    pub async fn get_upstream_root_fragments(
        &self,
        upstream_table_ids: &HashSet<TableId>,
    ) -> MetaResult<(HashMap<TableId, Fragment>, HashMap<ActorId, WorkerId>)> {
        let (upstream_root_fragments, actors) = self
            .catalog_controller
            .get_upstream_root_fragments(
                upstream_table_ids
                    .iter()
                    .map(|id| id.table_id as _)
                    .collect(),
            )
            .await?;

        let actors = actors
            .into_iter()
            .map(|(actor, worker)| (actor as u32, worker))
            .collect();

        Ok((
            upstream_root_fragments
                .into_iter()
                .map(|(id, fragment)| ((id as u32).into(), fragment))
                .collect(),
            actors,
        ))
    }

    pub async fn get_streaming_cluster_info(&self) -> MetaResult<StreamingClusterInfo> {
        self.cluster_controller.get_streaming_cluster_info().await
    }

    pub async fn get_all_table_options(&self) -> MetaResult<HashMap<u32, TableOption>> {
        self.catalog_controller
            .get_all_table_options()
            .await
            .map(|tops| tops.into_iter().map(|(id, opt)| (id as u32, opt)).collect())
    }

    pub async fn get_table_name_type_mapping(&self) -> MetaResult<HashMap<u32, (String, String)>> {
        let mappings = self
            .catalog_controller
            .get_table_name_type_mapping()
            .await?;
        Ok(mappings
            .into_iter()
            .map(|(id, value)| (id as u32, value))
            .collect())
    }

    pub async fn get_created_table_ids(&self) -> MetaResult<Vec<u32>> {
        let table_ids = self.catalog_controller.get_created_table_ids().await?;
        Ok(table_ids.into_iter().map(|id| id as u32).collect())
    }

    pub async fn get_table_catalog_by_ids(&self, ids: Vec<u32>) -> MetaResult<Vec<PbTable>> {
        self.catalog_controller
            .get_table_by_ids(ids.into_iter().map(|id| id as _).collect())
            .await
    }

    pub async fn get_sink_catalog_by_ids(&self, ids: &[u32]) -> MetaResult<Vec<PbSink>> {
        self.catalog_controller
            .get_sink_by_ids(ids.iter().map(|id| *id as _).collect())
            .await
    }

    pub async fn get_table_catalog_by_cdc_table_id(
        &self,
        cdc_table_id: &String,
    ) -> MetaResult<Vec<PbTable>> {
        self.catalog_controller
            .get_table_by_cdc_table_id(cdc_table_id)
            .await
    }

    pub async fn get_downstream_chain_fragments(
        &self,
        job_id: u32,
    ) -> MetaResult<(
        Vec<(PbDispatchStrategy, PbFragment)>,
        HashMap<ActorId, WorkerId>,
    )> {
        let (fragments, actors) = self
            .catalog_controller
            .get_downstream_chain_fragments(job_id as _)
            .await?;

        let actors = actors
            .into_iter()
            .map(|(actor, worker)| (actor as u32, worker))
            .collect();

        Ok((fragments, actors))
    }

    pub async fn get_worker_actor_ids(
        &self,
        job_ids: HashSet<TableId>,
    ) -> MetaResult<BTreeMap<WorkerId, Vec<ActorId>>> {
        let worker_actors = self
            .catalog_controller
            .get_worker_actor_ids(job_ids.into_iter().map(|id| id.table_id as _).collect())
            .await?;
        Ok(worker_actors
            .into_iter()
            .map(|(id, actors)| {
                (
                    id as WorkerId,
                    actors.into_iter().map(|id| id as ActorId).collect(),
                )
            })
            .collect())
    }

    pub async fn get_job_id_to_internal_table_ids_mapping(&self) -> Option<Vec<(u32, Vec<u32>)>> {
        let job_internal_table_ids = self.catalog_controller.get_job_internal_table_ids().await;
        job_internal_table_ids.map(|ids| {
            ids.into_iter()
                .map(|(id, internal_ids)| {
                    (
                        id as u32,
                        internal_ids.into_iter().map(|id| id as u32).collect(),
                    )
                })
                .collect()
        })
    }

    pub async fn get_job_fragments_by_id(
        &self,
        job_id: &TableId,
    ) -> MetaResult<StreamJobFragments> {
        let pb_table_fragments = self
            .catalog_controller
            .get_job_fragments_by_id(job_id.table_id as _)
            .await?;
        Ok(StreamJobFragments::from_protobuf(pb_table_fragments))
    }

    pub async fn get_running_actors_of_fragment(
        &self,
        id: FragmentId,
    ) -> MetaResult<HashSet<ActorId>> {
        let actor_ids = self
            .catalog_controller
            .get_running_actors_of_fragment(id as _)
            .await?;
        Ok(actor_ids.into_iter().map(|id| id as ActorId).collect())
    }

    pub async fn get_running_actors_and_upstream_actors_of_fragment(
        &self,
        id: FragmentId,
    ) -> MetaResult<HashSet<(ActorId, Vec<ActorId>)>> {
        let actor_ids = self
            .catalog_controller
            .get_running_actors_and_upstream_of_fragment(id as _)
            .await?;
        Ok(actor_ids
            .into_iter()
            .map(|(id, actors)| {
                (
                    id as ActorId,
                    actors
                        .into_inner()
                        .into_iter()
                        .flat_map(|(_, ids)| ids.into_iter().map(|id| id as ActorId))
                        .collect(),
                )
            })
            .collect())
    }

    pub async fn get_job_fragments_by_ids(
        &self,
        ids: &[TableId],
    ) -> MetaResult<Vec<StreamJobFragments>> {
        let mut table_fragments = vec![];
        for id in ids {
            let pb_table_fragments = self
                .catalog_controller
                .get_job_fragments_by_id(id.table_id as _)
                .await?;
            table_fragments.push(StreamJobFragments::from_protobuf(pb_table_fragments));
        }
        Ok(table_fragments)
    }

    pub async fn all_active_actors(&self) -> MetaResult<HashMap<ActorId, StreamActor>> {
        let table_fragments = self.catalog_controller.table_fragments().await?;
        let mut actor_maps = HashMap::new();
        for (_, fragments) in table_fragments {
            let tf = StreamJobFragments::from_protobuf(fragments);
            for actor in tf.active_actors() {
                actor_maps
                    .try_insert(actor.actor_id, actor)
                    .expect("non duplicate");
            }
        }
        Ok(actor_maps)
    }

    pub async fn worker_actor_count(&self) -> MetaResult<HashMap<WorkerId, usize>> {
        let actor_cnt = self.catalog_controller.worker_actor_count().await?;
        Ok(actor_cnt
            .into_iter()
            .map(|(id, cnt)| (id as WorkerId, cnt))
            .collect())
    }

    pub async fn count_streaming_job(&self) -> MetaResult<usize> {
        self.catalog_controller
            .list_streaming_job_infos()
            .await
            .map(|x| x.len())
    }

    pub async fn list_stream_job_desc(&self) -> MetaResult<Vec<MetaTelemetryJobDesc>> {
        self.catalog_controller
            .list_stream_job_desc_for_telemetry()
            .await
    }

    pub async fn update_source_rate_limit_by_source_id(
        &self,
        source_id: SourceId,
        rate_limit: Option<u32>,
    ) -> MetaResult<HashMap<FragmentId, Vec<ActorId>>> {
        let fragment_actors = self
            .catalog_controller
            .update_source_rate_limit_by_source_id(source_id as _, rate_limit)
            .await?;
        Ok(fragment_actors
            .into_iter()
            .map(|(id, actors)| (id as _, actors.into_iter().map(|id| id as _).collect()))
            .collect())
    }

    pub async fn update_backfill_rate_limit_by_table_id(
        &self,
        table_id: TableId,
        rate_limit: Option<u32>,
    ) -> MetaResult<HashMap<FragmentId, Vec<ActorId>>> {
        let fragment_actors = self
            .catalog_controller
            .update_backfill_rate_limit_by_job_id(table_id.table_id as _, rate_limit)
            .await?;
        Ok(fragment_actors
            .into_iter()
            .map(|(id, actors)| (id as _, actors.into_iter().map(|id| id as _).collect()))
            .collect())
    }

    pub async fn update_actor_splits_by_split_assignment(
        &self,
        split_assignment: &SplitAssignment,
    ) -> MetaResult<()> {
        self.catalog_controller
            .update_actor_splits(split_assignment)
            .await
    }

    pub async fn get_mv_depended_subscriptions(
        &self,
    ) -> MetaResult<HashMap<DatabaseId, HashMap<TableId, HashMap<SubscriptionId, u64>>>> {
        Ok(self
            .catalog_controller
            .get_mv_depended_subscriptions()
            .await?
            .into_iter()
            .map(|(database_id, mv_depended_subscriptions)| {
                (
                    DatabaseId::new(database_id as _),
                    mv_depended_subscriptions
                        .into_iter()
                        .map(|(table_id, subscriptions)| {
                            (
                                TableId::new(table_id as _),
                                subscriptions
                                    .into_iter()
                                    .map(|(subscription_id, retention_time)| {
                                        (subscription_id as SubscriptionId, retention_time)
                                    })
                                    .collect(),
                            )
                        })
                        .collect(),
                )
            })
            .collect())
    }

    pub async fn get_job_max_parallelism(&self, table_id: TableId) -> MetaResult<usize> {
        self.catalog_controller
            .get_max_parallelism_by_id(table_id.table_id as _)
            .await
    }

    pub fn cluster_id(&self) -> &ClusterId {
        self.cluster_controller.cluster_id()
    }

    pub async fn list_rate_limits(&self) -> MetaResult<Vec<RateLimitInfo>> {
        let rate_limits = self.catalog_controller.list_rate_limits().await?;
        Ok(rate_limits)
    }
}

impl MetadataManager {
    /// Wait for job finishing notification in `TrackingJob::finish`.
    /// The progress is updated per barrier.
    pub(crate) async fn wait_streaming_job_finished(
        &self,
        id: ObjectId,
    ) -> MetaResult<NotificationVersion> {
        tracing::debug!("wait_streaming_job_finished: {id:?}");
        let mut mgr = self.catalog_controller.get_inner_write_guard().await;
        if mgr.streaming_job_is_finished(id).await? {
            return Ok(self.catalog_controller.current_notification_version().await);
        }
        let (tx, rx) = oneshot::channel();

        mgr.register_finish_notifier(id, tx);
        drop(mgr);
        rx.await.map_err(|e| anyhow!(e))?
    }

    pub(crate) async fn notify_finish_failed(&self, err: &MetaError) {
        let mut mgr = self.catalog_controller.get_inner_write_guard().await;
        mgr.notify_finish_failed(err);
    }
}