1use std::cmp::Ordering;
16use std::collections::{HashMap, HashSet};
17use std::num::NonZeroUsize;
18use std::sync::Arc;
19use std::sync::atomic::AtomicU64;
20use std::time::Duration;
21
22use anyhow::{Context, anyhow};
23use await_tree::InstrumentAwait;
24use either::Either;
25use itertools::Itertools;
26use risingwave_common::catalog::{
27    AlterDatabaseParam, ColumnCatalog, ColumnId, Field, FragmentTypeFlag,
28};
29use risingwave_common::config::DefaultParallelism;
30use risingwave_common::hash::VnodeCountCompat;
31use risingwave_common::secret::{LocalSecretManager, SecretEncryption};
32use risingwave_common::system_param::reader::SystemParamsRead;
33use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont_mut;
34use risingwave_common::{bail, bail_not_implemented, catalog};
35use risingwave_connector::WithOptionsSecResolved;
36use risingwave_connector::connector_common::validate_connection;
37use risingwave_connector::source::cdc::CdcScanOptions;
38use risingwave_connector::source::{
39    ConnectorProperties, SourceEnumeratorContext, UPSTREAM_SOURCE_KEY,
40};
41use risingwave_meta_model::exactly_once_iceberg_sink::{Column, Entity};
42use risingwave_meta_model::object::ObjectType;
43use risingwave_meta_model::{
44    ConnectionId, DatabaseId, DispatcherType, FragmentId, FunctionId, IndexId, JobStatus, ObjectId,
45    SchemaId, SecretId, SinkId, SourceId, SubscriptionId, TableId, UserId, ViewId, WorkerId,
46};
47use risingwave_pb::catalog::{
48    Comment, Connection, CreateType, Database, Function, PbSink, PbTable, Schema, Secret, Source,
49    Subscription, Table, View,
50};
51use risingwave_pb::common::PbActorLocation;
52use risingwave_pb::ddl_service::alter_owner_request::Object;
53use risingwave_pb::ddl_service::{
54    DdlProgress, TableJobType, WaitVersion, alter_name_request, alter_set_schema_request,
55    alter_swap_rename_request,
56};
57use risingwave_pb::meta::table_fragments::PbActorStatus;
58use risingwave_pb::meta::table_fragments::actor_status::PbActorState;
59use risingwave_pb::stream_plan::stream_node::NodeBody;
60use risingwave_pb::stream_plan::{
61    PbDispatchOutputMapping, PbStreamFragmentGraph, PbStreamNode, PbUpstreamSinkInfo,
62    StreamFragmentGraph as StreamFragmentGraphProto,
63};
64use risingwave_pb::telemetry::{PbTelemetryDatabaseObject, PbTelemetryEventStage};
65use sea_orm::{ColumnTrait, DatabaseConnection, EntityTrait, QueryFilter};
66use strum::Display;
67use thiserror_ext::AsReport;
68use tokio::sync::{OwnedSemaphorePermit, Semaphore};
69use tokio::time::sleep;
70use tracing::Instrument;
71
72use crate::barrier::BarrierManagerRef;
73use crate::controller::catalog::{DropTableConnectorContext, ReleaseContext};
74use crate::controller::cluster::StreamingClusterInfo;
75use crate::controller::streaming_job::{FinishAutoRefreshSchemaSinkContext, SinkIntoTableContext};
76use crate::controller::utils::build_select_node_list;
77use crate::error::{MetaErrorInner, bail_invalid_parameter, bail_unavailable};
78use crate::manager::{
79    IGNORED_NOTIFICATION_VERSION, LocalNotification, MetaSrvEnv, MetadataManager,
80    NotificationVersion, StreamingJob, StreamingJobType,
81};
82use crate::model::{
83    DownstreamFragmentRelation, Fragment, FragmentDownstreamRelation, StreamContext,
84    StreamJobFragments, StreamJobFragmentsToCreate, TableParallelism,
85};
86use crate::stream::cdc::{
87    is_parallelized_backfill_enabled, try_init_parallel_cdc_table_snapshot_splits,
88};
89use crate::stream::{
90    ActorGraphBuildResult, ActorGraphBuilder, AutoRefreshSchemaSinkContext,
91    CompleteStreamFragmentGraph, CreateStreamingJobContext, CreateStreamingJobOption,
92    FragmentGraphDownstreamContext, FragmentGraphUpstreamContext, GlobalStreamManagerRef,
93    ReplaceStreamJobContext, ReschedulePolicy, SourceChange, SourceManagerRef, StreamFragmentGraph,
94    UpstreamSinkInfo, check_sink_fragments_support_refresh_schema, create_source_worker,
95    rewrite_refresh_schema_sink_fragment, state_match, validate_sink,
96};
97use crate::telemetry::report_event;
98use crate::{MetaError, MetaResult};
99
100#[derive(PartialEq)]
101pub enum DropMode {
102    Restrict,
103    Cascade,
104}
105
106impl DropMode {
107    pub fn from_request_setting(cascade: bool) -> DropMode {
108        if cascade {
109            DropMode::Cascade
110        } else {
111            DropMode::Restrict
112        }
113    }
114}
115
116#[derive(strum::AsRefStr)]
117pub enum StreamingJobId {
118    MaterializedView(TableId),
119    Sink(SinkId),
120    Table(Option<SourceId>, TableId),
121    Index(IndexId),
122}
123
124impl std::fmt::Display for StreamingJobId {
125    fn fmt(&self, f: &mut std::fmt::Formatter<'_>) -> std::fmt::Result {
126        write!(f, "{}", self.as_ref())?;
127        write!(f, "({})", self.id())
128    }
129}
130
131impl StreamingJobId {
132    #[allow(dead_code)]
133    fn id(&self) -> TableId {
134        match self {
135            StreamingJobId::MaterializedView(id)
136            | StreamingJobId::Sink(id)
137            | StreamingJobId::Table(_, id)
138            | StreamingJobId::Index(id) => *id,
139        }
140    }
141}
142
143pub struct ReplaceStreamJobInfo {
146    pub streaming_job: StreamingJob,
147    pub fragment_graph: StreamFragmentGraphProto,
148}
149
150#[derive(Display)]
151pub enum DdlCommand {
152    CreateDatabase(Database),
153    DropDatabase(DatabaseId),
154    CreateSchema(Schema),
155    DropSchema(SchemaId, DropMode),
156    CreateNonSharedSource(Source),
157    DropSource(SourceId, DropMode),
158    CreateFunction(Function),
159    DropFunction(FunctionId, DropMode),
160    CreateView(View, HashSet<ObjectId>),
161    DropView(ViewId, DropMode),
162    CreateStreamingJob {
163        stream_job: StreamingJob,
164        fragment_graph: StreamFragmentGraphProto,
165        dependencies: HashSet<ObjectId>,
166        specific_resource_group: Option<String>, if_not_exists: bool,
168    },
169    DropStreamingJob {
170        job_id: StreamingJobId,
171        drop_mode: DropMode,
172    },
173    AlterName(alter_name_request::Object, String),
174    AlterSwapRename(alter_swap_rename_request::Object),
175    ReplaceStreamJob(ReplaceStreamJobInfo),
176    AlterNonSharedSource(Source),
177    AlterObjectOwner(Object, UserId),
178    AlterSetSchema(alter_set_schema_request::Object, SchemaId),
179    CreateConnection(Connection),
180    DropConnection(ConnectionId, DropMode),
181    CreateSecret(Secret),
182    AlterSecret(Secret),
183    DropSecret(SecretId),
184    CommentOn(Comment),
185    CreateSubscription(Subscription),
186    DropSubscription(SubscriptionId, DropMode),
187    AlterDatabaseParam(DatabaseId, AlterDatabaseParam),
188}
189
190impl DdlCommand {
191    fn object(&self) -> Either<String, ObjectId> {
193        use Either::*;
194        match self {
195            DdlCommand::CreateDatabase(database) => Left(database.name.clone()),
196            DdlCommand::DropDatabase(id) => Right(*id),
197            DdlCommand::CreateSchema(schema) => Left(schema.name.clone()),
198            DdlCommand::DropSchema(id, _) => Right(*id),
199            DdlCommand::CreateNonSharedSource(source) => Left(source.name.clone()),
200            DdlCommand::DropSource(id, _) => Right(*id),
201            DdlCommand::CreateFunction(function) => Left(function.name.clone()),
202            DdlCommand::DropFunction(id, _) => Right(*id),
203            DdlCommand::CreateView(view, _) => Left(view.name.clone()),
204            DdlCommand::DropView(id, _) => Right(*id),
205            DdlCommand::CreateStreamingJob { stream_job, .. } => Left(stream_job.name()),
206            DdlCommand::DropStreamingJob { job_id, .. } => Right(job_id.id()),
207            DdlCommand::AlterName(object, _) => Left(format!("{object:?}")),
208            DdlCommand::AlterSwapRename(object) => Left(format!("{object:?}")),
209            DdlCommand::ReplaceStreamJob(info) => Left(info.streaming_job.name()),
210            DdlCommand::AlterNonSharedSource(source) => Left(source.name.clone()),
211            DdlCommand::AlterObjectOwner(object, _) => Left(format!("{object:?}")),
212            DdlCommand::AlterSetSchema(object, _) => Left(format!("{object:?}")),
213            DdlCommand::CreateConnection(connection) => Left(connection.name.clone()),
214            DdlCommand::DropConnection(id, _) => Right(*id),
215            DdlCommand::CreateSecret(secret) => Left(secret.name.clone()),
216            DdlCommand::AlterSecret(secret) => Left(secret.name.clone()),
217            DdlCommand::DropSecret(id) => Right(*id),
218            DdlCommand::CommentOn(comment) => Right(comment.table_id as _),
219            DdlCommand::CreateSubscription(subscription) => Left(subscription.name.clone()),
220            DdlCommand::DropSubscription(id, _) => Right(*id),
221            DdlCommand::AlterDatabaseParam(id, _) => Right(*id),
222        }
223    }
224
225    fn allow_in_recovery(&self) -> bool {
226        match self {
227            DdlCommand::DropDatabase(_)
228            | DdlCommand::DropSchema(_, _)
229            | DdlCommand::DropSource(_, _)
230            | DdlCommand::DropFunction(_, _)
231            | DdlCommand::DropView(_, _)
232            | DdlCommand::DropStreamingJob { .. }
233            | DdlCommand::DropConnection(_, _)
234            | DdlCommand::DropSecret(_)
235            | DdlCommand::DropSubscription(_, _)
236            | DdlCommand::AlterName(_, _)
237            | DdlCommand::AlterObjectOwner(_, _)
238            | DdlCommand::AlterSetSchema(_, _)
239            | DdlCommand::CreateDatabase(_)
240            | DdlCommand::CreateSchema(_)
241            | DdlCommand::CreateFunction(_)
242            | DdlCommand::CreateView(_, _)
243            | DdlCommand::CreateConnection(_)
244            | DdlCommand::CommentOn(_)
245            | DdlCommand::CreateSecret(_)
246            | DdlCommand::AlterSecret(_)
247            | DdlCommand::AlterSwapRename(_)
248            | DdlCommand::AlterDatabaseParam(_, _) => true,
249            DdlCommand::CreateStreamingJob { .. }
250            | DdlCommand::CreateNonSharedSource(_)
251            | DdlCommand::ReplaceStreamJob(_)
252            | DdlCommand::AlterNonSharedSource(_)
253            | DdlCommand::CreateSubscription(_) => false,
254        }
255    }
256}
257
258#[derive(Clone)]
259pub struct DdlController {
260    pub(crate) env: MetaSrvEnv,
261
262    pub(crate) metadata_manager: MetadataManager,
263    pub(crate) stream_manager: GlobalStreamManagerRef,
264    pub(crate) source_manager: SourceManagerRef,
265    barrier_manager: BarrierManagerRef,
266
267    pub(crate) creating_streaming_job_permits: Arc<CreatingStreamingJobPermit>,
269
270    seq: Arc<AtomicU64>,
272}
273
274#[derive(Clone)]
275pub struct CreatingStreamingJobPermit {
276    pub(crate) semaphore: Arc<Semaphore>,
277}
278
279impl CreatingStreamingJobPermit {
280    async fn new(env: &MetaSrvEnv) -> Self {
281        let mut permits = env
282            .system_params_reader()
283            .await
284            .max_concurrent_creating_streaming_jobs() as usize;
285        if permits == 0 {
286            permits = Semaphore::MAX_PERMITS;
288        }
289        let semaphore = Arc::new(Semaphore::new(permits));
290
291        let (local_notification_tx, mut local_notification_rx) =
292            tokio::sync::mpsc::unbounded_channel();
293        env.notification_manager()
294            .insert_local_sender(local_notification_tx);
295        let semaphore_clone = semaphore.clone();
296        tokio::spawn(async move {
297            while let Some(notification) = local_notification_rx.recv().await {
298                let LocalNotification::SystemParamsChange(p) = ¬ification else {
299                    continue;
300                };
301                let mut new_permits = p.max_concurrent_creating_streaming_jobs() as usize;
302                if new_permits == 0 {
303                    new_permits = Semaphore::MAX_PERMITS;
304                }
305                match permits.cmp(&new_permits) {
306                    Ordering::Less => {
307                        semaphore_clone.add_permits(new_permits - permits);
308                    }
309                    Ordering::Equal => continue,
310                    Ordering::Greater => {
311                        let to_release = permits - new_permits;
312                        let reduced = semaphore_clone.forget_permits(to_release);
313                        if reduced != to_release {
315                            tracing::warn!(
316                                "no enough permits to release, expected {}, but reduced {}",
317                                to_release,
318                                reduced
319                            );
320                        }
321                    }
322                }
323                tracing::info!(
324                    "max_concurrent_creating_streaming_jobs changed from {} to {}",
325                    permits,
326                    new_permits
327                );
328                permits = new_permits;
329            }
330        });
331
332        Self { semaphore }
333    }
334}
335
336impl DdlController {
337    pub async fn new(
338        env: MetaSrvEnv,
339        metadata_manager: MetadataManager,
340        stream_manager: GlobalStreamManagerRef,
341        source_manager: SourceManagerRef,
342        barrier_manager: BarrierManagerRef,
343    ) -> Self {
344        let creating_streaming_job_permits = Arc::new(CreatingStreamingJobPermit::new(&env).await);
345        Self {
346            env,
347            metadata_manager,
348            stream_manager,
349            source_manager,
350            barrier_manager,
351            creating_streaming_job_permits,
352            seq: Arc::new(AtomicU64::new(0)),
353        }
354    }
355
356    pub fn next_seq(&self) -> u64 {
358        self.seq.fetch_add(1, std::sync::atomic::Ordering::Relaxed)
360    }
361
362    pub async fn run_command(&self, command: DdlCommand) -> MetaResult<Option<WaitVersion>> {
369        if !command.allow_in_recovery() {
370            self.barrier_manager.check_status_running()?;
371        }
372
373        let await_tree_key = format!("DDL Command {}", self.next_seq());
374        let await_tree_span = await_tree::span!("{command}({})", command.object());
375
376        let ctrl = self.clone();
377        let fut = async move {
378            match command {
379                DdlCommand::CreateDatabase(database) => ctrl.create_database(database).await,
380                DdlCommand::DropDatabase(database_id) => ctrl.drop_database(database_id).await,
381                DdlCommand::CreateSchema(schema) => ctrl.create_schema(schema).await,
382                DdlCommand::DropSchema(schema_id, drop_mode) => {
383                    ctrl.drop_schema(schema_id, drop_mode).await
384                }
385                DdlCommand::CreateNonSharedSource(source) => {
386                    ctrl.create_non_shared_source(source).await
387                }
388                DdlCommand::DropSource(source_id, drop_mode) => {
389                    ctrl.drop_source(source_id, drop_mode).await
390                }
391                DdlCommand::CreateFunction(function) => ctrl.create_function(function).await,
392                DdlCommand::DropFunction(function_id, drop_mode) => {
393                    ctrl.drop_function(function_id, drop_mode).await
394                }
395                DdlCommand::CreateView(view, dependencies) => {
396                    ctrl.create_view(view, dependencies).await
397                }
398                DdlCommand::DropView(view_id, drop_mode) => {
399                    ctrl.drop_view(view_id, drop_mode).await
400                }
401                DdlCommand::CreateStreamingJob {
402                    stream_job,
403                    fragment_graph,
404                    dependencies,
405                    specific_resource_group,
406                    if_not_exists,
407                } => {
408                    ctrl.create_streaming_job(
409                        stream_job,
410                        fragment_graph,
411                        dependencies,
412                        specific_resource_group,
413                        if_not_exists,
414                    )
415                    .await
416                }
417                DdlCommand::DropStreamingJob { job_id, drop_mode } => {
418                    ctrl.drop_streaming_job(job_id, drop_mode).await
419                }
420                DdlCommand::ReplaceStreamJob(ReplaceStreamJobInfo {
421                    streaming_job,
422                    fragment_graph,
423                }) => ctrl.replace_job(streaming_job, fragment_graph).await,
424                DdlCommand::AlterName(relation, name) => ctrl.alter_name(relation, &name).await,
425                DdlCommand::AlterObjectOwner(object, owner_id) => {
426                    ctrl.alter_owner(object, owner_id).await
427                }
428                DdlCommand::AlterSetSchema(object, new_schema_id) => {
429                    ctrl.alter_set_schema(object, new_schema_id).await
430                }
431                DdlCommand::CreateConnection(connection) => {
432                    ctrl.create_connection(connection).await
433                }
434                DdlCommand::DropConnection(connection_id, drop_mode) => {
435                    ctrl.drop_connection(connection_id, drop_mode).await
436                }
437                DdlCommand::CreateSecret(secret) => ctrl.create_secret(secret).await,
438                DdlCommand::DropSecret(secret_id) => ctrl.drop_secret(secret_id).await,
439                DdlCommand::AlterSecret(secret) => ctrl.alter_secret(secret).await,
440                DdlCommand::AlterNonSharedSource(source) => {
441                    ctrl.alter_non_shared_source(source).await
442                }
443                DdlCommand::CommentOn(comment) => ctrl.comment_on(comment).await,
444                DdlCommand::CreateSubscription(subscription) => {
445                    ctrl.create_subscription(subscription).await
446                }
447                DdlCommand::DropSubscription(subscription_id, drop_mode) => {
448                    ctrl.drop_subscription(subscription_id, drop_mode).await
449                }
450                DdlCommand::AlterSwapRename(objects) => ctrl.alter_swap_rename(objects).await,
451                DdlCommand::AlterDatabaseParam(database_id, param) => {
452                    ctrl.alter_database_param(database_id, param).await
453                }
454            }
455        }
456        .in_current_span();
457        let fut = (self.env.await_tree_reg())
458            .register(await_tree_key, await_tree_span)
459            .instrument(Box::pin(fut));
460        let notification_version = tokio::spawn(fut).await.map_err(|e| anyhow!(e))??;
461        Ok(Some(WaitVersion {
462            catalog_version: notification_version,
463            hummock_version_id: self.barrier_manager.get_hummock_version_id().await.to_u64(),
464        }))
465    }
466
467    pub async fn get_ddl_progress(&self) -> MetaResult<Vec<DdlProgress>> {
468        self.barrier_manager.get_ddl_progress().await
469    }
470
471    async fn create_database(&self, database: Database) -> MetaResult<NotificationVersion> {
472        let (version, updated_db) = self
473            .metadata_manager
474            .catalog_controller
475            .create_database(database)
476            .await?;
477        self.barrier_manager
479            .update_database_barrier(
480                updated_db.database_id,
481                updated_db.barrier_interval_ms.map(|v| v as u32),
482                updated_db.checkpoint_frequency.map(|v| v as u64),
483            )
484            .await?;
485        Ok(version)
486    }
487
488    #[tracing::instrument(skip(self), level = "debug")]
489    pub async fn reschedule_streaming_job(
490        &self,
491        job_id: u32,
492        target: ReschedulePolicy,
493        mut deferred: bool,
494    ) -> MetaResult<()> {
495        tracing::info!("altering parallelism for job {}", job_id);
496        if self.barrier_manager.check_status_running().is_err() {
497            tracing::info!(
498                "alter parallelism is set to deferred mode because the system is in recovery state"
499            );
500            deferred = true;
501        }
502
503        self.stream_manager
504            .reschedule_streaming_job(job_id, target, deferred)
505            .await
506    }
507
508    pub async fn reschedule_cdc_table_backfill(
509        &self,
510        job_id: u32,
511        target: ReschedulePolicy,
512    ) -> MetaResult<()> {
513        tracing::info!("alter CDC table backfill parallelism");
514        if self.barrier_manager.check_status_running().is_err() {
515            return Err(anyhow::anyhow!("CDC table backfill reschedule is unavailable because the system is in recovery state").into());
516        }
517        self.stream_manager
518            .reschedule_cdc_table_backfill(job_id, target)
519            .await
520    }
521
522    async fn drop_database(&self, database_id: DatabaseId) -> MetaResult<NotificationVersion> {
523        self.drop_object(ObjectType::Database, database_id as _, DropMode::Cascade)
524            .await
525    }
526
527    async fn create_schema(&self, schema: Schema) -> MetaResult<NotificationVersion> {
528        self.metadata_manager
529            .catalog_controller
530            .create_schema(schema)
531            .await
532    }
533
534    async fn drop_schema(
535        &self,
536        schema_id: SchemaId,
537        drop_mode: DropMode,
538    ) -> MetaResult<NotificationVersion> {
539        self.drop_object(ObjectType::Schema, schema_id as _, drop_mode)
540            .await
541    }
542
543    async fn create_non_shared_source(&self, source: Source) -> MetaResult<NotificationVersion> {
545        let handle = create_source_worker(&source, self.source_manager.metrics.clone())
546            .await
547            .context("failed to create source worker")?;
548
549        let (source_id, version) = self
550            .metadata_manager
551            .catalog_controller
552            .create_source(source)
553            .await?;
554        self.source_manager
555            .register_source_with_handle(source_id, handle)
556            .await?;
557        Ok(version)
558    }
559
560    async fn drop_source(
561        &self,
562        source_id: SourceId,
563        drop_mode: DropMode,
564    ) -> MetaResult<NotificationVersion> {
565        self.drop_object(ObjectType::Source, source_id as _, drop_mode)
566            .await
567    }
568
569    async fn alter_non_shared_source(&self, source: Source) -> MetaResult<NotificationVersion> {
572        self.metadata_manager
573            .catalog_controller
574            .alter_non_shared_source(source)
575            .await
576    }
577
578    async fn create_function(&self, function: Function) -> MetaResult<NotificationVersion> {
579        self.metadata_manager
580            .catalog_controller
581            .create_function(function)
582            .await
583    }
584
585    async fn drop_function(
586        &self,
587        function_id: FunctionId,
588        drop_mode: DropMode,
589    ) -> MetaResult<NotificationVersion> {
590        self.drop_object(ObjectType::Function, function_id as _, drop_mode)
591            .await
592    }
593
594    async fn create_view(
595        &self,
596        view: View,
597        dependencies: HashSet<ObjectId>,
598    ) -> MetaResult<NotificationVersion> {
599        self.metadata_manager
600            .catalog_controller
601            .create_view(view, dependencies)
602            .await
603    }
604
605    async fn drop_view(
606        &self,
607        view_id: ViewId,
608        drop_mode: DropMode,
609    ) -> MetaResult<NotificationVersion> {
610        self.drop_object(ObjectType::View, view_id as _, drop_mode)
611            .await
612    }
613
614    async fn create_connection(&self, connection: Connection) -> MetaResult<NotificationVersion> {
615        validate_connection(&connection).await?;
616        self.metadata_manager
617            .catalog_controller
618            .create_connection(connection)
619            .await
620    }
621
622    async fn drop_connection(
623        &self,
624        connection_id: ConnectionId,
625        drop_mode: DropMode,
626    ) -> MetaResult<NotificationVersion> {
627        self.drop_object(ObjectType::Connection, connection_id as _, drop_mode)
628            .await
629    }
630
631    async fn alter_database_param(
632        &self,
633        database_id: DatabaseId,
634        param: AlterDatabaseParam,
635    ) -> MetaResult<NotificationVersion> {
636        let (version, updated_db) = self
637            .metadata_manager
638            .catalog_controller
639            .alter_database_param(database_id, param)
640            .await?;
641        self.barrier_manager
643            .update_database_barrier(
644                database_id,
645                updated_db.barrier_interval_ms.map(|v| v as u32),
646                updated_db.checkpoint_frequency.map(|v| v as u64),
647            )
648            .await?;
649        Ok(version)
650    }
651
652    fn get_encrypted_payload(&self, secret: &Secret) -> MetaResult<Vec<u8>> {
655        let secret_store_private_key = self
656            .env
657            .opts
658            .secret_store_private_key
659            .clone()
660            .ok_or_else(|| anyhow!("secret_store_private_key is not configured"))?;
661
662        let encrypted_payload = SecretEncryption::encrypt(
663            secret_store_private_key.as_slice(),
664            secret.get_value().as_slice(),
665        )
666        .context(format!("failed to encrypt secret {}", secret.name))?;
667        Ok(encrypted_payload
668            .serialize()
669            .context(format!("failed to serialize secret {}", secret.name))?)
670    }
671
672    async fn create_secret(&self, mut secret: Secret) -> MetaResult<NotificationVersion> {
673        let secret_plain_payload = secret.value.clone();
676        let encrypted_payload = self.get_encrypted_payload(&secret)?;
677        secret.value = encrypted_payload;
678
679        self.metadata_manager
680            .catalog_controller
681            .create_secret(secret, secret_plain_payload)
682            .await
683    }
684
685    async fn drop_secret(&self, secret_id: SecretId) -> MetaResult<NotificationVersion> {
686        self.drop_object(ObjectType::Secret, secret_id as _, DropMode::Restrict)
687            .await
688    }
689
690    async fn alter_secret(&self, mut secret: Secret) -> MetaResult<NotificationVersion> {
691        let secret_plain_payload = secret.value.clone();
692        let encrypted_payload = self.get_encrypted_payload(&secret)?;
693        secret.value = encrypted_payload;
694        self.metadata_manager
695            .catalog_controller
696            .alter_secret(secret, secret_plain_payload)
697            .await
698    }
699
700    async fn create_subscription(
701        &self,
702        mut subscription: Subscription,
703    ) -> MetaResult<NotificationVersion> {
704        tracing::debug!("create subscription");
705        let _permit = self
706            .creating_streaming_job_permits
707            .semaphore
708            .acquire()
709            .await
710            .unwrap();
711        let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await;
712        self.metadata_manager
713            .catalog_controller
714            .create_subscription_catalog(&mut subscription)
715            .await?;
716        if let Err(err) = self.stream_manager.create_subscription(&subscription).await {
717            tracing::debug!(error = %err.as_report(), "failed to create subscription");
718            let _ = self
719                .metadata_manager
720                .catalog_controller
721                .try_abort_creating_subscription(subscription.id as _)
722                .await
723                .inspect_err(|e| {
724                    tracing::error!(
725                        error = %e.as_report(),
726                        "failed to abort create subscription after failure"
727                    );
728                });
729            return Err(err);
730        }
731
732        let version = self
733            .metadata_manager
734            .catalog_controller
735            .notify_create_subscription(subscription.id)
736            .await?;
737        tracing::debug!("finish create subscription");
738        Ok(version)
739    }
740
741    async fn drop_subscription(
742        &self,
743        subscription_id: SubscriptionId,
744        drop_mode: DropMode,
745    ) -> MetaResult<NotificationVersion> {
746        tracing::debug!("preparing drop subscription");
747        let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await;
748        let subscription = self
749            .metadata_manager
750            .catalog_controller
751            .get_subscription_by_id(subscription_id)
752            .await?;
753        let table_id = subscription.dependent_table_id;
754        let database_id = subscription.database_id.into();
755        let (_, version) = self
756            .metadata_manager
757            .catalog_controller
758            .drop_object(ObjectType::Subscription, subscription_id as _, drop_mode)
759            .await?;
760        self.stream_manager
761            .drop_subscription(database_id, subscription_id as _, table_id)
762            .await;
763        tracing::debug!("finish drop subscription");
764        Ok(version)
765    }
766
767    #[await_tree::instrument]
769    pub(crate) async fn validate_cdc_table(
770        &self,
771        table: &Table,
772        table_fragments: &StreamJobFragments,
773    ) -> MetaResult<()> {
774        let stream_scan_fragment = table_fragments
775            .fragments
776            .values()
777            .filter(|f| {
778                f.fragment_type_mask.contains(FragmentTypeFlag::StreamScan)
779                    || f.fragment_type_mask
780                        .contains(FragmentTypeFlag::StreamCdcScan)
781            })
782            .exactly_one()
783            .ok()
784            .with_context(|| {
785                format!(
786                    "expect exactly one stream scan fragment, got: {:?}",
787                    table_fragments.fragments
788                )
789            })?;
790        fn assert_parallelism(stream_scan_fragment: &Fragment, node_body: &Option<NodeBody>) {
791            if let Some(NodeBody::StreamCdcScan(node)) = node_body {
792                if let Some(o) = node.options
793                    && CdcScanOptions::from_proto(&o).is_parallelized_backfill()
794                {
795                    } else {
797                    assert_eq!(
798                        stream_scan_fragment.actors.len(),
799                        1,
800                        "Stream scan fragment should have only one actor"
801                    );
802                }
803            }
804        }
805        let mut found_cdc_scan = false;
806        match &stream_scan_fragment.nodes.node_body {
807            Some(NodeBody::StreamCdcScan(_)) => {
808                assert_parallelism(stream_scan_fragment, &stream_scan_fragment.nodes.node_body);
809                if self
810                    .validate_cdc_table_inner(&stream_scan_fragment.nodes.node_body, table.id)
811                    .await?
812                {
813                    found_cdc_scan = true;
814                }
815            }
816            Some(NodeBody::Project(_)) => {
818                for input in &stream_scan_fragment.nodes.input {
819                    assert_parallelism(stream_scan_fragment, &input.node_body);
820                    if self
821                        .validate_cdc_table_inner(&input.node_body, table.id)
822                        .await?
823                    {
824                        found_cdc_scan = true;
825                    }
826                }
827            }
828            _ => {
829                bail!("Unexpected node body for stream cdc scan");
830            }
831        };
832        if !found_cdc_scan {
833            bail!("No stream cdc scan node found in stream scan fragment");
834        }
835        Ok(())
836    }
837
838    async fn validate_cdc_table_inner(
839        &self,
840        node_body: &Option<NodeBody>,
841        table_id: u32,
842    ) -> MetaResult<bool> {
843        let meta_store = self.env.meta_store_ref();
844        if let Some(NodeBody::StreamCdcScan(stream_cdc_scan)) = node_body
845            && let Some(ref cdc_table_desc) = stream_cdc_scan.cdc_table_desc
846        {
847            let options_with_secret = WithOptionsSecResolved::new(
848                cdc_table_desc.connect_properties.clone(),
849                cdc_table_desc.secret_refs.clone(),
850            );
851
852            let mut props = ConnectorProperties::extract(options_with_secret, true)?;
853            props.init_from_pb_cdc_table_desc(cdc_table_desc);
854
855            let _enumerator = props
857                .create_split_enumerator(SourceEnumeratorContext::dummy().into())
858                .await?;
859
860            if is_parallelized_backfill_enabled(stream_cdc_scan) {
861                try_init_parallel_cdc_table_snapshot_splits(
863                    table_id,
864                    cdc_table_desc,
865                    meta_store,
866                    &stream_cdc_scan.options,
867                    self.env.opts.cdc_table_split_init_insert_batch_size,
868                    self.env.opts.cdc_table_split_init_sleep_interval_splits,
869                    self.env.opts.cdc_table_split_init_sleep_duration_millis,
870                )
871                .await?;
872            }
873
874            tracing::debug!(?table_id, "validate cdc table success");
875            Ok(true)
876        } else {
877            Ok(false)
878        }
879    }
880
881    pub async fn validate_table_for_sink(&self, table_id: TableId) -> MetaResult<()> {
882        let migrated = self
883            .metadata_manager
884            .catalog_controller
885            .has_table_been_migrated(table_id)
886            .await?;
887        if !migrated {
888            Err(anyhow::anyhow!("Creating sink into table is not allowed for unmigrated table {}. Please migrate it first.", table_id).into())
889        } else {
890            Ok(())
891        }
892    }
893
894    #[await_tree::instrument(boxed, "create_streaming_job({streaming_job})")]
897    pub async fn create_streaming_job(
898        &self,
899        mut streaming_job: StreamingJob,
900        fragment_graph: StreamFragmentGraphProto,
901        dependencies: HashSet<ObjectId>,
902        specific_resource_group: Option<String>,
903        if_not_exists: bool,
904    ) -> MetaResult<NotificationVersion> {
905        if let StreamingJob::Sink(sink) = &streaming_job
906            && let Some(target_table) = sink.target_table
907        {
908            self.validate_table_for_sink(target_table as _).await?;
909        }
910        let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap());
911        let check_ret = self
912            .metadata_manager
913            .catalog_controller
914            .create_job_catalog(
915                &mut streaming_job,
916                &ctx,
917                &fragment_graph.parallelism,
918                fragment_graph.max_parallelism as _,
919                dependencies,
920                specific_resource_group.clone(),
921            )
922            .await;
923        if let Err(meta_err) = check_ret {
924            if !if_not_exists {
925                return Err(meta_err);
926            }
927            return if let MetaErrorInner::Duplicated(_, _, Some(job_id)) = meta_err.inner() {
928                if streaming_job.create_type() == CreateType::Foreground {
929                    let database_id = streaming_job.database_id();
930                    self.metadata_manager
931                        .wait_streaming_job_finished(database_id.into(), *job_id)
932                        .await
933                } else {
934                    Ok(IGNORED_NOTIFICATION_VERSION)
935                }
936            } else {
937                Err(meta_err)
938            };
939        }
940        let job_id = streaming_job.id();
941        tracing::debug!(
942            id = job_id,
943            definition = streaming_job.definition(),
944            create_type = streaming_job.create_type().as_str_name(),
945            job_type = ?streaming_job.job_type(),
946            "starting streaming job",
947        );
948        let permit = self
950            .creating_streaming_job_permits
951            .semaphore
952            .clone()
953            .acquire_owned()
954            .instrument_await("acquire_creating_streaming_job_permit")
955            .await
956            .unwrap();
957        let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await;
958
959        let name = streaming_job.name();
960        let definition = streaming_job.definition();
961        let source_id = match &streaming_job {
962            StreamingJob::Table(Some(src), _, _) | StreamingJob::Source(src) => Some(src.id),
963            _ => None,
964        };
965
966        match self
968            .create_streaming_job_inner(
969                ctx,
970                streaming_job,
971                fragment_graph,
972                specific_resource_group,
973                permit,
974            )
975            .await
976        {
977            Ok(version) => Ok(version),
978            Err(err) => {
979                tracing::error!(id = job_id, error = %err.as_report(), "failed to create streaming job");
980                let event = risingwave_pb::meta::event_log::EventCreateStreamJobFail {
981                    id: job_id,
982                    name,
983                    definition,
984                    error: err.as_report().to_string(),
985                };
986                self.env.event_log_manager_ref().add_event_logs(vec![
987                    risingwave_pb::meta::event_log::Event::CreateStreamJobFail(event),
988                ]);
989                let (aborted, _) = self
990                    .metadata_manager
991                    .catalog_controller
992                    .try_abort_creating_streaming_job(job_id as _, false)
993                    .await?;
994                if aborted {
995                    tracing::warn!(id = job_id, "aborted streaming job");
996                    if let Some(source_id) = source_id {
998                        self.source_manager
999                            .apply_source_change(SourceChange::DropSource {
1000                                dropped_source_ids: vec![source_id as SourceId],
1001                            })
1002                            .await;
1003                    }
1004                }
1005                Err(err)
1006            }
1007        }
1008    }
1009
1010    #[await_tree::instrument(boxed)]
1011    async fn create_streaming_job_inner(
1012        &self,
1013        ctx: StreamContext,
1014        mut streaming_job: StreamingJob,
1015        fragment_graph: StreamFragmentGraphProto,
1016        specific_resource_group: Option<String>,
1017        permit: OwnedSemaphorePermit,
1018    ) -> MetaResult<NotificationVersion> {
1019        let mut fragment_graph =
1020            StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?;
1021        streaming_job.set_info_from_graph(&fragment_graph);
1022
1023        let incomplete_internal_tables = fragment_graph
1025            .incomplete_internal_tables()
1026            .into_values()
1027            .collect_vec();
1028        let table_id_map = self
1029            .metadata_manager
1030            .catalog_controller
1031            .create_internal_table_catalog(&streaming_job, incomplete_internal_tables)
1032            .await?;
1033        fragment_graph.refill_internal_table_ids(table_id_map);
1034
1035        tracing::debug!(id = streaming_job.id(), "building streaming job");
1037        let (ctx, stream_job_fragments) = self
1038            .build_stream_job(ctx, streaming_job, fragment_graph, specific_resource_group)
1039            .await?;
1040
1041        let streaming_job = &ctx.streaming_job;
1042
1043        match streaming_job {
1044            StreamingJob::Table(None, table, TableJobType::SharedCdcSource) => {
1045                self.validate_cdc_table(table, &stream_job_fragments)
1046                    .await?;
1047            }
1048            StreamingJob::Table(Some(source), ..) => {
1049                self.source_manager.register_source(source).await?;
1051                let connector_name = source
1052                    .get_with_properties()
1053                    .get(UPSTREAM_SOURCE_KEY)
1054                    .cloned();
1055                let attr = source.info.as_ref().map(|source_info| {
1056                    jsonbb::json!({
1057                            "format": source_info.format().as_str_name(),
1058                            "encode": source_info.row_encode().as_str_name(),
1059                    })
1060                });
1061                report_create_object(
1062                    streaming_job.id(),
1063                    "source",
1064                    PbTelemetryDatabaseObject::Source,
1065                    connector_name,
1066                    attr,
1067                );
1068            }
1069            StreamingJob::Sink(sink) => {
1070                if sink.auto_refresh_schema_from_table.is_some() {
1071                    check_sink_fragments_support_refresh_schema(&stream_job_fragments.fragments)?
1072                }
1073                validate_sink(sink).await?;
1075                let connector_name = sink.get_properties().get(UPSTREAM_SOURCE_KEY).cloned();
1076                let attr = sink.format_desc.as_ref().map(|sink_info| {
1077                    jsonbb::json!({
1078                        "format": sink_info.format().as_str_name(),
1079                        "encode": sink_info.encode().as_str_name(),
1080                    })
1081                });
1082                report_create_object(
1083                    streaming_job.id(),
1084                    "sink",
1085                    PbTelemetryDatabaseObject::Sink,
1086                    connector_name,
1087                    attr,
1088                );
1089            }
1090            StreamingJob::Source(source) => {
1091                self.source_manager.register_source(source).await?;
1093                let connector_name = source
1094                    .get_with_properties()
1095                    .get(UPSTREAM_SOURCE_KEY)
1096                    .cloned();
1097                let attr = source.info.as_ref().map(|source_info| {
1098                    jsonbb::json!({
1099                            "format": source_info.format().as_str_name(),
1100                            "encode": source_info.row_encode().as_str_name(),
1101                    })
1102                });
1103                report_create_object(
1104                    streaming_job.id(),
1105                    "source",
1106                    PbTelemetryDatabaseObject::Source,
1107                    connector_name,
1108                    attr,
1109                );
1110            }
1111            _ => {}
1112        }
1113
1114        self.metadata_manager
1115            .catalog_controller
1116            .prepare_stream_job_fragments(&stream_job_fragments, streaming_job, false)
1117            .await?;
1118
1119        let version = self
1121            .stream_manager
1122            .create_streaming_job(stream_job_fragments, ctx, permit)
1123            .await?;
1124
1125        Ok(version)
1126    }
1127
1128    pub async fn drop_object(
1130        &self,
1131        object_type: ObjectType,
1132        object_id: ObjectId,
1133        drop_mode: DropMode,
1134    ) -> MetaResult<NotificationVersion> {
1135        let (release_ctx, version) = self
1136            .metadata_manager
1137            .catalog_controller
1138            .drop_object(object_type, object_id, drop_mode)
1139            .await?;
1140
1141        let ReleaseContext {
1142            database_id,
1143            removed_streaming_job_ids,
1144            removed_state_table_ids,
1145            removed_source_ids,
1146            removed_secret_ids: secret_ids,
1147            removed_source_fragments,
1148            removed_actors,
1149            removed_fragments,
1150            removed_sink_fragment_by_targets,
1151        } = release_ctx;
1152
1153        let _guard = self.source_manager.pause_tick().await;
1154        self.stream_manager
1155            .drop_streaming_jobs(
1156                catalog::DatabaseId::new(database_id as _),
1157                removed_actors.iter().map(|id| *id as _).collect(),
1158                removed_streaming_job_ids,
1159                removed_state_table_ids,
1160                removed_fragments.iter().map(|id| *id as _).collect(),
1161                removed_sink_fragment_by_targets
1162                    .into_iter()
1163                    .map(|(target, sinks)| {
1164                        (target as _, sinks.into_iter().map(|id| id as _).collect())
1165                    })
1166                    .collect(),
1167            )
1168            .await;
1169
1170        self.source_manager
1173            .apply_source_change(SourceChange::DropSource {
1174                dropped_source_ids: removed_source_ids.into_iter().map(|id| id as _).collect(),
1175            })
1176            .await;
1177
1178        let dropped_source_fragments = removed_source_fragments
1181            .into_iter()
1182            .map(|(source_id, fragments)| {
1183                (
1184                    source_id,
1185                    fragments.into_iter().map(|id| id as u32).collect(),
1186                )
1187            })
1188            .collect();
1189        self.source_manager
1190            .apply_source_change(SourceChange::DropMv {
1191                dropped_source_fragments,
1192            })
1193            .await;
1194
1195        for secret in secret_ids {
1197            LocalSecretManager::global().remove_secret(secret as _);
1198        }
1199        Ok(version)
1200    }
1201
1202    #[await_tree::instrument(boxed, "replace_streaming_job({streaming_job})")]
1204    pub async fn replace_job(
1205        &self,
1206        mut streaming_job: StreamingJob,
1207        fragment_graph: StreamFragmentGraphProto,
1208    ) -> MetaResult<NotificationVersion> {
1209        match &streaming_job {
1210            StreamingJob::Table(..)
1211            | StreamingJob::Source(..)
1212            | StreamingJob::MaterializedView(..) => {}
1213            StreamingJob::Sink(..) | StreamingJob::Index(..) => {
1214                bail_not_implemented!("schema change for {}", streaming_job.job_type_str())
1215            }
1216        }
1217
1218        let job_id = streaming_job.id();
1219
1220        let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await;
1221        let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap());
1222
1223        let original_max_parallelism = self
1225            .metadata_manager
1226            .get_job_max_parallelism(streaming_job.id().into())
1227            .await?;
1228        let fragment_graph = PbStreamFragmentGraph {
1229            max_parallelism: original_max_parallelism as _,
1230            ..fragment_graph
1231        };
1232
1233        let fragment_graph = StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?;
1235        streaming_job.set_info_from_graph(&fragment_graph);
1236
1237        let streaming_job = streaming_job;
1239
1240        let auto_refresh_schema_sinks = if let StreamingJob::Table(_, table, _) = &streaming_job {
1241            let auto_refresh_schema_sinks = self
1242                .metadata_manager
1243                .catalog_controller
1244                .get_sink_auto_refresh_schema_from(table.id as _)
1245                .await?;
1246            if !auto_refresh_schema_sinks.is_empty() {
1247                let original_table_columns = self
1248                    .metadata_manager
1249                    .catalog_controller
1250                    .get_table_columns(table.id as _)
1251                    .await?;
1252                let mut original_table_column_ids: HashSet<_> = original_table_columns
1254                    .iter()
1255                    .map(|col| col.column_id())
1256                    .collect();
1257                let newly_added_columns = table
1258                    .columns
1259                    .iter()
1260                    .filter(|col| {
1261                        !original_table_column_ids.remove(&ColumnId::new(
1262                            col.column_desc.as_ref().unwrap().column_id as _,
1263                        ))
1264                    })
1265                    .map(|col| ColumnCatalog::from(col.clone()))
1266                    .collect_vec();
1267                if !original_table_column_ids.is_empty() {
1268                    return Err(anyhow!("new table columns does not contains all original columns. new: {:?}, original: {:?}, not included: {:?}", table.columns, original_table_columns, original_table_column_ids).into());
1269                }
1270                let mut sinks = Vec::with_capacity(auto_refresh_schema_sinks.len());
1271                for sink in auto_refresh_schema_sinks {
1272                    let sink_job_fragments = self
1273                        .metadata_manager
1274                        .get_job_fragments_by_id(&catalog::TableId::new(sink.id))
1275                        .await?;
1276                    if sink_job_fragments.fragments.len() != 1 {
1277                        return Err(anyhow!(
1278                            "auto schema refresh sink must have only one fragment, but got {}",
1279                            sink_job_fragments.fragments.len()
1280                        )
1281                        .into());
1282                    }
1283                    let original_sink_fragment =
1284                        sink_job_fragments.fragments.into_values().next().unwrap();
1285                    let (new_sink_fragment, new_schema, new_log_store_table) =
1286                        rewrite_refresh_schema_sink_fragment(
1287                            &original_sink_fragment,
1288                            &sink,
1289                            &newly_added_columns,
1290                            table,
1291                            fragment_graph.table_fragment_id(),
1292                            self.env.id_gen_manager(),
1293                            self.env.actor_id_generator(),
1294                        )?;
1295
1296                    assert_eq!(
1297                        original_sink_fragment.actors.len(),
1298                        new_sink_fragment.actors.len()
1299                    );
1300                    let actor_status = (0..original_sink_fragment.actors.len())
1301                        .map(|i| {
1302                            let worker_node_id = sink_job_fragments.actor_status
1303                                [&original_sink_fragment.actors[i].actor_id]
1304                                .location
1305                                .as_ref()
1306                                .unwrap()
1307                                .worker_node_id;
1308                            (
1309                                new_sink_fragment.actors[i].actor_id,
1310                                PbActorStatus {
1311                                    location: Some(PbActorLocation { worker_node_id }),
1312                                    state: PbActorState::Inactive as _,
1313                                },
1314                            )
1315                        })
1316                        .collect();
1317
1318                    let streaming_job = StreamingJob::Sink(sink);
1319
1320                    let tmp_sink_id = self
1321                        .metadata_manager
1322                        .catalog_controller
1323                        .create_job_catalog_for_replace(&streaming_job, None, None, None)
1324                        .await?;
1325                    let StreamingJob::Sink(sink) = streaming_job else {
1326                        unreachable!()
1327                    };
1328
1329                    sinks.push(AutoRefreshSchemaSinkContext {
1330                        tmp_sink_id,
1331                        original_sink: sink,
1332                        original_fragment: original_sink_fragment,
1333                        new_schema,
1334                        newly_add_fields: newly_added_columns
1335                            .iter()
1336                            .map(|col| Field::from(&col.column_desc))
1337                            .collect(),
1338                        new_fragment: new_sink_fragment,
1339                        new_log_store_table,
1340                        actor_status,
1341                    });
1342                }
1343                Some(sinks)
1344            } else {
1345                None
1346            }
1347        } else {
1348            None
1349        };
1350
1351        let tmp_id = self
1352            .metadata_manager
1353            .catalog_controller
1354            .create_job_catalog_for_replace(
1355                &streaming_job,
1356                Some(&ctx),
1357                fragment_graph.specified_parallelism().as_ref(),
1358                Some(fragment_graph.max_parallelism()),
1359            )
1360            .await?;
1361
1362        let tmp_sink_ids = auto_refresh_schema_sinks
1363            .as_ref()
1364            .map(|sinks| sinks.iter().map(|sink| sink.tmp_sink_id).collect_vec());
1365
1366        tracing::debug!(id = job_id, "building replace streaming job");
1367        let mut updated_sink_catalogs = vec![];
1368
1369        let mut drop_table_connector_ctx = None;
1370        let result: MetaResult<_> = try {
1371            let (mut ctx, mut stream_job_fragments) = self
1372                .build_replace_job(
1373                    ctx,
1374                    &streaming_job,
1375                    fragment_graph,
1376                    tmp_id as _,
1377                    auto_refresh_schema_sinks,
1378                )
1379                .await?;
1380            drop_table_connector_ctx = ctx.drop_table_connector_ctx.clone();
1381            let auto_refresh_schema_sink_finish_ctx =
1382                ctx.auto_refresh_schema_sinks.as_ref().map(|sinks| {
1383                    sinks
1384                        .iter()
1385                        .map(|sink| FinishAutoRefreshSchemaSinkContext {
1386                            tmp_sink_id: sink.tmp_sink_id,
1387                            original_sink_id: sink.original_sink.id as _,
1388                            columns: sink.new_schema.clone(),
1389                            new_log_store_table: sink
1390                                .new_log_store_table
1391                                .as_ref()
1392                                .map(|table| (table.id as _, table.columns.clone())),
1393                        })
1394                        .collect()
1395                });
1396
1397            if let StreamingJob::Table(_, table, ..) = &streaming_job {
1399                let union_fragment = stream_job_fragments.inner.union_fragment_for_table();
1400                let upstream_infos = self
1401                    .metadata_manager
1402                    .catalog_controller
1403                    .get_all_upstream_sink_infos(table, union_fragment.fragment_id as _)
1404                    .await?;
1405                refill_upstream_sink_union_in_table(&mut union_fragment.nodes, &upstream_infos);
1406
1407                for upstream_info in &upstream_infos {
1408                    let upstream_fragment_id = upstream_info.sink_fragment_id;
1409                    ctx.upstream_fragment_downstreams
1410                        .entry(upstream_fragment_id)
1411                        .or_default()
1412                        .push(upstream_info.new_sink_downstream.clone());
1413                    if upstream_info.sink_original_target_columns.is_empty() {
1414                        updated_sink_catalogs.push(upstream_info.sink_id);
1415                    }
1416                }
1417            }
1418
1419            let replace_upstream = ctx.replace_upstream.clone();
1420
1421            if let Some(sinks) = &ctx.auto_refresh_schema_sinks {
1422                let empty_downstreams = FragmentDownstreamRelation::default();
1423                for sink in sinks {
1424                    self.metadata_manager
1425                        .catalog_controller
1426                        .prepare_streaming_job(
1427                            sink.tmp_sink_id,
1428                            || [&sink.new_fragment].into_iter(),
1429                            &empty_downstreams,
1430                            true,
1431                            None,
1432                        )
1433                        .await?;
1434                }
1435            }
1436
1437            self.metadata_manager
1438                .catalog_controller
1439                .prepare_stream_job_fragments(&stream_job_fragments, &streaming_job, true)
1440                .await?;
1441
1442            self.stream_manager
1443                .replace_stream_job(stream_job_fragments, ctx)
1444                .await?;
1445            (replace_upstream, auto_refresh_schema_sink_finish_ctx)
1446        };
1447
1448        match result {
1449            Ok((replace_upstream, auto_refresh_schema_sink_finish_ctx)) => {
1450                let version = self
1451                    .metadata_manager
1452                    .catalog_controller
1453                    .finish_replace_streaming_job(
1454                        tmp_id,
1455                        streaming_job,
1456                        replace_upstream,
1457                        SinkIntoTableContext {
1458                            updated_sink_catalogs,
1459                        },
1460                        drop_table_connector_ctx.as_ref(),
1461                        auto_refresh_schema_sink_finish_ctx,
1462                    )
1463                    .await?;
1464                if let Some(drop_table_connector_ctx) = &drop_table_connector_ctx {
1465                    self.source_manager
1466                        .apply_source_change(SourceChange::DropSource {
1467                            dropped_source_ids: vec![drop_table_connector_ctx.to_remove_source_id],
1468                        })
1469                        .await;
1470                }
1471                Ok(version)
1472            }
1473            Err(err) => {
1474                tracing::error!(id = job_id, error = ?err.as_report(), "failed to replace job");
1475                let _ = self.metadata_manager
1476                    .catalog_controller
1477                    .try_abort_replacing_streaming_job(tmp_id, tmp_sink_ids)
1478                    .await.inspect_err(|err| {
1479                    tracing::error!(id = job_id, error = ?err.as_report(), "failed to abort replacing job");
1480                });
1481                Err(err)
1482            }
1483        }
1484    }
1485
1486    #[await_tree::instrument(boxed, "drop_streaming_job{}({job_id})", if let DropMode::Cascade = drop_mode { "_cascade" } else { "" }
1487    )]
1488    async fn drop_streaming_job(
1489        &self,
1490        job_id: StreamingJobId,
1491        drop_mode: DropMode,
1492    ) -> MetaResult<NotificationVersion> {
1493        let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await;
1494
1495        let (object_id, object_type) = match job_id {
1496            StreamingJobId::MaterializedView(id) => (id as _, ObjectType::Table),
1497            StreamingJobId::Sink(id) => (id as _, ObjectType::Sink),
1498            StreamingJobId::Table(_, id) => (id as _, ObjectType::Table),
1499            StreamingJobId::Index(idx) => (idx as _, ObjectType::Index),
1500        };
1501
1502        let job_status = self
1503            .metadata_manager
1504            .catalog_controller
1505            .get_streaming_job_status(job_id.id())
1506            .await?;
1507        let version = match job_status {
1508            JobStatus::Initial => {
1509                unreachable!(
1510                    "Job with Initial status should not notify frontend and therefore should not arrive here"
1511                );
1512            }
1513            JobStatus::Creating => {
1514                let canceled_jobs = self
1515                    .stream_manager
1516                    .cancel_streaming_jobs(vec![(job_id.id() as u32).into()])
1517                    .await;
1518                if canceled_jobs.is_empty() {
1519                    tracing::warn!(job_id = job_id.id(), "failed to cancel streaming job");
1520                }
1521                IGNORED_NOTIFICATION_VERSION
1522            }
1523            JobStatus::Created => {
1524                let version = self.drop_object(object_type, object_id, drop_mode).await?;
1525                #[cfg(not(madsim))]
1526                if let StreamingJobId::Sink(sink_id) = job_id {
1527                    let db = self.env.meta_store_ref().conn.clone();
1530                    clean_all_rows_by_sink_id(&db, sink_id).await?;
1531                }
1532                version
1533            }
1534        };
1535
1536        Ok(version)
1537    }
1538
1539    fn resolve_stream_parallelism(
1543        &self,
1544        specified: Option<NonZeroUsize>,
1545        max: NonZeroUsize,
1546        cluster_info: &StreamingClusterInfo,
1547        resource_group: String,
1548    ) -> MetaResult<NonZeroUsize> {
1549        let available = cluster_info.parallelism(&resource_group);
1550        let Some(available) = NonZeroUsize::new(available) else {
1551            bail_unavailable!(
1552                "no available slots to schedule in resource group \"{}\", \
1553                 have you allocated any compute nodes within this resource group?",
1554                resource_group
1555            );
1556        };
1557
1558        if let Some(specified) = specified {
1559            if specified > max {
1560                bail_invalid_parameter!(
1561                    "specified parallelism {} should not exceed max parallelism {}",
1562                    specified,
1563                    max,
1564                );
1565            }
1566            if specified > available {
1567                bail_unavailable!(
1568                    "insufficient parallelism to schedule in resource group \"{}\", \
1569                     required: {}, available: {}",
1570                    resource_group,
1571                    specified,
1572                    available,
1573                );
1574            }
1575            Ok(specified)
1576        } else {
1577            let default_parallelism = match self.env.opts.default_parallelism {
1579                DefaultParallelism::Full => available,
1580                DefaultParallelism::Default(num) => {
1581                    if num > available {
1582                        bail_unavailable!(
1583                            "insufficient parallelism to schedule in resource group \"{}\", \
1584                            required: {}, available: {}",
1585                            resource_group,
1586                            num,
1587                            available,
1588                        );
1589                    }
1590                    num
1591                }
1592            };
1593
1594            if default_parallelism > max {
1595                tracing::warn!(
1596                    max_parallelism = max.get(),
1597                    resource_group,
1598                    "too many parallelism available, use max parallelism instead",
1599                );
1600            }
1601            Ok(default_parallelism.min(max))
1602        }
1603    }
1604
1605    #[await_tree::instrument]
1611    pub(crate) async fn build_stream_job(
1612        &self,
1613        stream_ctx: StreamContext,
1614        mut stream_job: StreamingJob,
1615        fragment_graph: StreamFragmentGraph,
1616        specific_resource_group: Option<String>,
1617    ) -> MetaResult<(CreateStreamingJobContext, StreamJobFragmentsToCreate)> {
1618        let id = stream_job.id();
1619        let specified_parallelism = fragment_graph.specified_parallelism();
1620        let expr_context = stream_ctx.to_expr_context();
1621        let max_parallelism = NonZeroUsize::new(fragment_graph.max_parallelism()).unwrap();
1622
1623        let fragment_backfill_ordering = fragment_graph.create_fragment_backfill_ordering();
1625
1626        let (snapshot_backfill_info, cross_db_snapshot_backfill_info) =
1630            fragment_graph.collect_snapshot_backfill_info()?;
1631        assert!(
1632            snapshot_backfill_info
1633                .iter()
1634                .chain([&cross_db_snapshot_backfill_info])
1635                .flat_map(|info| info.upstream_mv_table_id_to_backfill_epoch.values())
1636                .all(|backfill_epoch| backfill_epoch.is_none()),
1637            "should not set backfill epoch when initially build the job: {:?} {:?}",
1638            snapshot_backfill_info,
1639            cross_db_snapshot_backfill_info
1640        );
1641
1642        let locality_fragment_state_table_mapping =
1643            fragment_graph.find_locality_provider_fragment_state_table_mapping();
1644
1645        self.metadata_manager
1647            .catalog_controller
1648            .validate_cross_db_snapshot_backfill(&cross_db_snapshot_backfill_info)
1649            .await?;
1650
1651        let upstream_table_ids = fragment_graph
1652            .dependent_table_ids()
1653            .iter()
1654            .filter(|id| {
1655                !cross_db_snapshot_backfill_info
1656                    .upstream_mv_table_id_to_backfill_epoch
1657                    .contains_key(id)
1658            })
1659            .cloned()
1660            .collect();
1661
1662        let (upstream_root_fragments, existing_actor_location) = self
1663            .metadata_manager
1664            .get_upstream_root_fragments(&upstream_table_ids)
1665            .await?;
1666
1667        if snapshot_backfill_info.is_some() {
1668            match stream_job {
1669                StreamingJob::MaterializedView(_)
1670                | StreamingJob::Sink(_)
1671                | StreamingJob::Index(_, _) => {}
1672                StreamingJob::Table(_, _, _) | StreamingJob::Source(_) => {
1673                    return Err(
1674                        anyhow!("snapshot_backfill not enabled for table and source").into(),
1675                    );
1676                }
1677            }
1678        }
1679
1680        let upstream_actors = upstream_root_fragments
1681            .values()
1682            .map(|(fragment, _)| {
1683                (
1684                    fragment.fragment_id,
1685                    fragment.actors.keys().copied().collect(),
1686                )
1687            })
1688            .collect();
1689
1690        let complete_graph = CompleteStreamFragmentGraph::with_upstreams(
1691            fragment_graph,
1692            FragmentGraphUpstreamContext {
1693                upstream_root_fragments,
1694                upstream_actor_location: existing_actor_location,
1695            },
1696            (&stream_job).into(),
1697        )?;
1698        let resource_group = match specific_resource_group {
1699            None => {
1700                self.metadata_manager
1701                    .get_database_resource_group(stream_job.database_id() as ObjectId)
1702                    .await?
1703            }
1704            Some(resource_group) => resource_group,
1705        };
1706
1707        let cluster_info = self.metadata_manager.get_streaming_cluster_info().await?;
1709
1710        let parallelism = self.resolve_stream_parallelism(
1711            specified_parallelism,
1712            max_parallelism,
1713            &cluster_info,
1714            resource_group.clone(),
1715        )?;
1716
1717        let parallelism = self
1718            .env
1719            .system_params_reader()
1720            .await
1721            .adaptive_parallelism_strategy()
1722            .compute_target_parallelism(parallelism.get());
1723
1724        let parallelism = NonZeroUsize::new(parallelism).expect("parallelism must be positive");
1725        let actor_graph_builder = ActorGraphBuilder::new(
1726            id,
1727            resource_group,
1728            complete_graph,
1729            cluster_info,
1730            parallelism,
1731        )?;
1732
1733        let ActorGraphBuildResult {
1734            graph,
1735            downstream_fragment_relations,
1736            building_locations,
1737            upstream_fragment_downstreams,
1738            new_no_shuffle,
1739            replace_upstream,
1740            ..
1741        } = actor_graph_builder.generate_graph(&self.env, &stream_job, expr_context)?;
1742        assert!(replace_upstream.is_empty());
1743
1744        let table_parallelism = match (specified_parallelism, &self.env.opts.default_parallelism) {
1751            (None, DefaultParallelism::Full) => TableParallelism::Adaptive,
1752            _ => TableParallelism::Fixed(parallelism.get()),
1753        };
1754
1755        let stream_job_fragments = StreamJobFragments::new(
1756            id.into(),
1757            graph,
1758            &building_locations.actor_locations,
1759            stream_ctx.clone(),
1760            table_parallelism,
1761            max_parallelism.get(),
1762        );
1763
1764        if let Some(mview_fragment) = stream_job_fragments.mview_fragment() {
1765            stream_job.set_table_vnode_count(mview_fragment.vnode_count());
1766        }
1767
1768        let new_upstream_sink = if let StreamingJob::Sink(sink) = &stream_job
1769            && let Ok(table_id) = sink.get_target_table()
1770        {
1771            let tables = self
1772                .metadata_manager
1773                .get_table_catalog_by_ids(&[*table_id as _])
1774                .await?;
1775            let target_table = tables
1776                .first()
1777                .ok_or_else(|| MetaError::catalog_id_not_found("table", *table_id))?;
1778            let sink_fragment = stream_job_fragments
1779                .sink_fragment()
1780                .ok_or_else(|| anyhow::anyhow!("sink fragment not found for sink {}", sink.id))?;
1781            let mview_fragment_id = self
1782                .metadata_manager
1783                .catalog_controller
1784                .get_mview_fragment_by_id(*table_id as _)
1785                .await?;
1786            let upstream_sink_info = build_upstream_sink_info(
1787                sink,
1788                sink_fragment.fragment_id as _,
1789                target_table,
1790                mview_fragment_id,
1791            )?;
1792            Some(upstream_sink_info)
1793        } else {
1794            None
1795        };
1796
1797        let ctx = CreateStreamingJobContext {
1798            upstream_fragment_downstreams,
1799            new_no_shuffle,
1800            upstream_actors,
1801            building_locations,
1802            definition: stream_job.definition(),
1803            mv_table_id: stream_job.mv_table(),
1804            create_type: stream_job.create_type(),
1805            job_type: (&stream_job).into(),
1806            streaming_job: stream_job,
1807            new_upstream_sink,
1808            option: CreateStreamingJobOption {},
1809            snapshot_backfill_info,
1810            cross_db_snapshot_backfill_info,
1811            fragment_backfill_ordering,
1812            locality_fragment_state_table_mapping,
1813        };
1814
1815        Ok((
1816            ctx,
1817            StreamJobFragmentsToCreate {
1818                inner: stream_job_fragments,
1819                downstreams: downstream_fragment_relations,
1820            },
1821        ))
1822    }
1823
1824    pub(crate) async fn build_replace_job(
1830        &self,
1831        stream_ctx: StreamContext,
1832        stream_job: &StreamingJob,
1833        mut fragment_graph: StreamFragmentGraph,
1834        tmp_job_id: TableId,
1835        auto_refresh_schema_sinks: Option<Vec<AutoRefreshSchemaSinkContext>>,
1836    ) -> MetaResult<(ReplaceStreamJobContext, StreamJobFragmentsToCreate)> {
1837        match &stream_job {
1838            StreamingJob::Table(..)
1839            | StreamingJob::Source(..)
1840            | StreamingJob::MaterializedView(..) => {}
1841            StreamingJob::Sink(..) | StreamingJob::Index(..) => {
1842                bail_not_implemented!("schema change for {}", stream_job.job_type_str())
1843            }
1844        }
1845
1846        let id = stream_job.id();
1847        let expr_context = stream_ctx.to_expr_context();
1848
1849        let mut drop_table_associated_source_id = None;
1851        if let StreamingJob::Table(None, _, _) = &stream_job {
1852            drop_table_associated_source_id = self
1853                .metadata_manager
1854                .get_table_associated_source_id(id as _)
1855                .await?;
1856        }
1857
1858        let old_fragments = self
1859            .metadata_manager
1860            .get_job_fragments_by_id(&id.into())
1861            .await?;
1862        let old_internal_table_ids = old_fragments.internal_table_ids();
1863
1864        let mut drop_table_connector_ctx = None;
1866        if let Some(to_remove_source_id) = drop_table_associated_source_id {
1867            debug_assert!(old_internal_table_ids.len() == 1);
1869
1870            drop_table_connector_ctx = Some(DropTableConnectorContext {
1871                to_change_streaming_job_id: id as i32,
1874                to_remove_state_table_id: old_internal_table_ids[0] as i32, to_remove_source_id,
1876            });
1877        } else if stream_job.is_materialized_view() {
1878            let old_fragments_upstreams = self
1881                .metadata_manager
1882                .catalog_controller
1883                .upstream_fragments(old_fragments.fragment_ids())
1884                .await?;
1885
1886            let old_state_graph =
1887                state_match::Graph::from_existing(&old_fragments, &old_fragments_upstreams);
1888            let new_state_graph = state_match::Graph::from_building(&fragment_graph);
1889            let mapping =
1890                state_match::match_graph_internal_tables(&new_state_graph, &old_state_graph)
1891                    .context("incompatible altering on the streaming job states")?;
1892
1893            fragment_graph.fit_internal_table_ids_with_mapping(mapping);
1894        } else {
1895            let old_internal_tables = self
1898                .metadata_manager
1899                .get_table_catalog_by_ids(&old_internal_table_ids)
1900                .await?;
1901            fragment_graph.fit_internal_tables_trivial(old_internal_tables)?;
1902        }
1903
1904        let original_root_fragment = old_fragments
1907            .root_fragment()
1908            .expect("root fragment not found");
1909
1910        let job_type = StreamingJobType::from(stream_job);
1911
1912        let (mut downstream_fragments, mut downstream_actor_location) =
1914            self.metadata_manager.get_downstream_fragments(id).await?;
1915
1916        if let Some(auto_refresh_schema_sinks) = &auto_refresh_schema_sinks {
1917            let mut remaining_fragment: HashSet<_> = auto_refresh_schema_sinks
1918                .iter()
1919                .map(|sink| sink.original_fragment.fragment_id)
1920                .collect();
1921            for (_, downstream_fragment, _) in &mut downstream_fragments {
1922                if let Some(sink) = auto_refresh_schema_sinks.iter().find(|sink| {
1923                    sink.original_fragment.fragment_id == downstream_fragment.fragment_id
1924                }) {
1925                    assert!(remaining_fragment.remove(&downstream_fragment.fragment_id));
1926                    for actor_id in downstream_fragment.actors.keys() {
1927                        downstream_actor_location.remove(actor_id);
1928                    }
1929                    for (actor_id, status) in &sink.actor_status {
1930                        downstream_actor_location.insert(
1931                            *actor_id,
1932                            status.location.as_ref().unwrap().worker_node_id as WorkerId,
1933                        );
1934                    }
1935
1936                    *downstream_fragment = (
1937                        &sink.new_fragment_info(),
1938                        catalog::TableId::from(stream_job.id()),
1939                    )
1940                        .into();
1941                }
1942            }
1943            assert!(remaining_fragment.is_empty());
1944        }
1945
1946        let complete_graph = match &job_type {
1948            StreamingJobType::Table(TableJobType::General) | StreamingJobType::Source => {
1949                CompleteStreamFragmentGraph::with_downstreams(
1950                    fragment_graph,
1951                    FragmentGraphDownstreamContext {
1952                        original_root_fragment_id: original_root_fragment.fragment_id,
1953                        downstream_fragments,
1954                        downstream_actor_location,
1955                    },
1956                    job_type,
1957                )?
1958            }
1959            StreamingJobType::Table(TableJobType::SharedCdcSource)
1960            | StreamingJobType::MaterializedView => {
1961                let (upstream_root_fragments, upstream_actor_location) = self
1963                    .metadata_manager
1964                    .get_upstream_root_fragments(fragment_graph.dependent_table_ids())
1965                    .await?;
1966
1967                CompleteStreamFragmentGraph::with_upstreams_and_downstreams(
1968                    fragment_graph,
1969                    FragmentGraphUpstreamContext {
1970                        upstream_root_fragments,
1971                        upstream_actor_location,
1972                    },
1973                    FragmentGraphDownstreamContext {
1974                        original_root_fragment_id: original_root_fragment.fragment_id,
1975                        downstream_fragments,
1976                        downstream_actor_location,
1977                    },
1978                    job_type,
1979                )?
1980            }
1981            _ => unreachable!(),
1982        };
1983
1984        let resource_group = self
1985            .metadata_manager
1986            .get_existing_job_resource_group(id as ObjectId)
1987            .await?;
1988
1989        let cluster_info = self.metadata_manager.get_streaming_cluster_info().await?;
1991
1992        let parallelism = NonZeroUsize::new(original_root_fragment.actors.len())
1995            .expect("The number of actors in the original table fragment should be greater than 0");
1996
1997        let actor_graph_builder = ActorGraphBuilder::new(
1998            id,
1999            resource_group,
2000            complete_graph,
2001            cluster_info,
2002            parallelism,
2003        )?;
2004
2005        let ActorGraphBuildResult {
2006            graph,
2007            downstream_fragment_relations,
2008            building_locations,
2009            upstream_fragment_downstreams,
2010            mut replace_upstream,
2011            new_no_shuffle,
2012            ..
2013        } = actor_graph_builder.generate_graph(&self.env, stream_job, expr_context)?;
2014
2015        if matches!(
2017            job_type,
2018            StreamingJobType::Source | StreamingJobType::Table(TableJobType::General)
2019        ) {
2020            assert!(upstream_fragment_downstreams.is_empty());
2021        }
2022
2023        let stream_job_fragments = StreamJobFragments::new(
2027            (tmp_job_id as u32).into(),
2028            graph,
2029            &building_locations.actor_locations,
2030            stream_ctx,
2031            old_fragments.assigned_parallelism,
2032            old_fragments.max_parallelism,
2033        );
2034
2035        if let Some(sinks) = &auto_refresh_schema_sinks {
2036            for sink in sinks {
2037                replace_upstream
2038                    .remove(&sink.new_fragment.fragment_id)
2039                    .expect("should exist");
2040            }
2041        }
2042
2043        let ctx = ReplaceStreamJobContext {
2047            old_fragments,
2048            replace_upstream,
2049            new_no_shuffle,
2050            upstream_fragment_downstreams,
2051            building_locations,
2052            streaming_job: stream_job.clone(),
2053            tmp_id: tmp_job_id as _,
2054            drop_table_connector_ctx,
2055            auto_refresh_schema_sinks,
2056        };
2057
2058        Ok((
2059            ctx,
2060            StreamJobFragmentsToCreate {
2061                inner: stream_job_fragments,
2062                downstreams: downstream_fragment_relations,
2063            },
2064        ))
2065    }
2066
2067    async fn alter_name(
2068        &self,
2069        relation: alter_name_request::Object,
2070        new_name: &str,
2071    ) -> MetaResult<NotificationVersion> {
2072        let (obj_type, id) = match relation {
2073            alter_name_request::Object::TableId(id) => (ObjectType::Table, id as ObjectId),
2074            alter_name_request::Object::ViewId(id) => (ObjectType::View, id as ObjectId),
2075            alter_name_request::Object::IndexId(id) => (ObjectType::Index, id as ObjectId),
2076            alter_name_request::Object::SinkId(id) => (ObjectType::Sink, id as ObjectId),
2077            alter_name_request::Object::SourceId(id) => (ObjectType::Source, id as ObjectId),
2078            alter_name_request::Object::SchemaId(id) => (ObjectType::Schema, id as ObjectId),
2079            alter_name_request::Object::DatabaseId(id) => (ObjectType::Database, id as ObjectId),
2080            alter_name_request::Object::SubscriptionId(id) => {
2081                (ObjectType::Subscription, id as ObjectId)
2082            }
2083        };
2084        self.metadata_manager
2085            .catalog_controller
2086            .alter_name(obj_type, id, new_name)
2087            .await
2088    }
2089
2090    async fn alter_swap_rename(
2091        &self,
2092        object: alter_swap_rename_request::Object,
2093    ) -> MetaResult<NotificationVersion> {
2094        let (obj_type, src_id, dst_id) = match object {
2095            alter_swap_rename_request::Object::Schema(_) => unimplemented!("schema swap"),
2096            alter_swap_rename_request::Object::Table(objs) => {
2097                let (src_id, dst_id) = (
2098                    objs.src_object_id as ObjectId,
2099                    objs.dst_object_id as ObjectId,
2100                );
2101                (ObjectType::Table, src_id, dst_id)
2102            }
2103            alter_swap_rename_request::Object::View(objs) => {
2104                let (src_id, dst_id) = (
2105                    objs.src_object_id as ObjectId,
2106                    objs.dst_object_id as ObjectId,
2107                );
2108                (ObjectType::View, src_id, dst_id)
2109            }
2110            alter_swap_rename_request::Object::Source(objs) => {
2111                let (src_id, dst_id) = (
2112                    objs.src_object_id as ObjectId,
2113                    objs.dst_object_id as ObjectId,
2114                );
2115                (ObjectType::Source, src_id, dst_id)
2116            }
2117            alter_swap_rename_request::Object::Sink(objs) => {
2118                let (src_id, dst_id) = (
2119                    objs.src_object_id as ObjectId,
2120                    objs.dst_object_id as ObjectId,
2121                );
2122                (ObjectType::Sink, src_id, dst_id)
2123            }
2124            alter_swap_rename_request::Object::Subscription(objs) => {
2125                let (src_id, dst_id) = (
2126                    objs.src_object_id as ObjectId,
2127                    objs.dst_object_id as ObjectId,
2128                );
2129                (ObjectType::Subscription, src_id, dst_id)
2130            }
2131        };
2132
2133        self.metadata_manager
2134            .catalog_controller
2135            .alter_swap_rename(obj_type, src_id, dst_id)
2136            .await
2137    }
2138
2139    async fn alter_owner(
2140        &self,
2141        object: Object,
2142        owner_id: UserId,
2143    ) -> MetaResult<NotificationVersion> {
2144        let (obj_type, id) = match object {
2145            Object::TableId(id) => (ObjectType::Table, id as ObjectId),
2146            Object::ViewId(id) => (ObjectType::View, id as ObjectId),
2147            Object::SourceId(id) => (ObjectType::Source, id as ObjectId),
2148            Object::SinkId(id) => (ObjectType::Sink, id as ObjectId),
2149            Object::SchemaId(id) => (ObjectType::Schema, id as ObjectId),
2150            Object::DatabaseId(id) => (ObjectType::Database, id as ObjectId),
2151            Object::SubscriptionId(id) => (ObjectType::Subscription, id as ObjectId),
2152            Object::ConnectionId(id) => (ObjectType::Connection, id as ObjectId),
2153        };
2154        self.metadata_manager
2155            .catalog_controller
2156            .alter_owner(obj_type, id, owner_id as _)
2157            .await
2158    }
2159
2160    async fn alter_set_schema(
2161        &self,
2162        object: alter_set_schema_request::Object,
2163        new_schema_id: SchemaId,
2164    ) -> MetaResult<NotificationVersion> {
2165        let (obj_type, id) = match object {
2166            alter_set_schema_request::Object::TableId(id) => (ObjectType::Table, id as ObjectId),
2167            alter_set_schema_request::Object::ViewId(id) => (ObjectType::View, id as ObjectId),
2168            alter_set_schema_request::Object::SourceId(id) => (ObjectType::Source, id as ObjectId),
2169            alter_set_schema_request::Object::SinkId(id) => (ObjectType::Sink, id as ObjectId),
2170            alter_set_schema_request::Object::FunctionId(id) => {
2171                (ObjectType::Function, id as ObjectId)
2172            }
2173            alter_set_schema_request::Object::ConnectionId(id) => {
2174                (ObjectType::Connection, id as ObjectId)
2175            }
2176            alter_set_schema_request::Object::SubscriptionId(id) => {
2177                (ObjectType::Subscription, id as ObjectId)
2178            }
2179        };
2180        self.metadata_manager
2181            .catalog_controller
2182            .alter_schema(obj_type, id, new_schema_id as _)
2183            .await
2184    }
2185
2186    pub async fn wait(&self) -> MetaResult<()> {
2187        let timeout_ms = 30 * 60 * 1000;
2188        for _ in 0..timeout_ms {
2189            if self
2190                .metadata_manager
2191                .catalog_controller
2192                .list_background_creating_jobs(true, None)
2193                .await?
2194                .is_empty()
2195            {
2196                return Ok(());
2197            }
2198
2199            sleep(Duration::from_millis(1)).await;
2200        }
2201        Err(MetaError::cancelled(format!(
2202            "timeout after {timeout_ms}ms"
2203        )))
2204    }
2205
2206    async fn comment_on(&self, comment: Comment) -> MetaResult<NotificationVersion> {
2207        self.metadata_manager
2208            .catalog_controller
2209            .comment_on(comment)
2210            .await
2211    }
2212}
2213
2214fn report_create_object(
2215    catalog_id: u32,
2216    event_name: &str,
2217    obj_type: PbTelemetryDatabaseObject,
2218    connector_name: Option<String>,
2219    attr_info: Option<jsonbb::Value>,
2220) {
2221    report_event(
2222        PbTelemetryEventStage::CreateStreamJob,
2223        event_name,
2224        catalog_id.into(),
2225        connector_name,
2226        Some(obj_type),
2227        attr_info,
2228    );
2229}
2230
2231async fn clean_all_rows_by_sink_id(db: &DatabaseConnection, sink_id: i32) -> MetaResult<()> {
2232    match Entity::delete_many()
2233        .filter(Column::SinkId.eq(sink_id))
2234        .exec(db)
2235        .await
2236    {
2237        Ok(result) => {
2238            let deleted_count = result.rows_affected;
2239
2240            tracing::info!(
2241                "Deleted {} items for sink_id = {} in iceberg exactly once system table.",
2242                deleted_count,
2243                sink_id
2244            );
2245            Ok(())
2246        }
2247        Err(e) => {
2248            tracing::error!(
2249                "Error deleting records for sink_id = {} from iceberg exactly once system table: {:?}",
2250                sink_id,
2251                e.as_report()
2252            );
2253            Err(e.into())
2254        }
2255    }
2256}
2257
2258pub fn build_upstream_sink_info(
2259    sink: &PbSink,
2260    sink_fragment_id: FragmentId,
2261    target_table: &PbTable,
2262    target_fragment_id: FragmentId,
2263) -> MetaResult<UpstreamSinkInfo> {
2264    let sink_columns = if !sink.original_target_columns.is_empty() {
2265        sink.original_target_columns.clone()
2266    } else {
2267        target_table.columns.clone()
2272    };
2273
2274    let sink_output_fields = sink_columns
2275        .iter()
2276        .map(|col| Field::from(col.column_desc.as_ref().unwrap()).to_prost())
2277        .collect_vec();
2278    let output_indices = (0..sink_output_fields.len())
2279        .map(|i| i as u32)
2280        .collect_vec();
2281
2282    let dist_key_indices: anyhow::Result<Vec<u32>> = try {
2283        let sink_idx_by_col_id = sink_columns
2284            .iter()
2285            .enumerate()
2286            .map(|(idx, col)| {
2287                let column_id = col.column_desc.as_ref().unwrap().column_id;
2288                (column_id, idx as u32)
2289            })
2290            .collect::<HashMap<_, _>>();
2291        target_table
2292            .distribution_key
2293            .iter()
2294            .map(|dist_idx| {
2295                let column_id = target_table.columns[*dist_idx as usize]
2296                    .column_desc
2297                    .as_ref()
2298                    .unwrap()
2299                    .column_id;
2300                let sink_idx = sink_idx_by_col_id
2301                    .get(&column_id)
2302                    .ok_or_else(|| anyhow::anyhow!("column id {} not found in sink", column_id))?;
2303                Ok(*sink_idx)
2304            })
2305            .collect::<anyhow::Result<Vec<_>>>()?
2306    };
2307    let dist_key_indices =
2308        dist_key_indices.map_err(|e| e.context("failed to get distribution key indices"))?;
2309    let downstream_fragment_id = target_fragment_id as _;
2310    let new_downstream_relation = DownstreamFragmentRelation {
2311        downstream_fragment_id,
2312        dispatcher_type: DispatcherType::Hash,
2313        dist_key_indices,
2314        output_mapping: PbDispatchOutputMapping::simple(output_indices),
2315    };
2316    let current_target_columns = target_table.get_columns();
2317    let project_exprs = build_select_node_list(&sink_columns, current_target_columns)?;
2318    Ok(UpstreamSinkInfo {
2319        sink_id: sink.id as _,
2320        sink_fragment_id: sink_fragment_id as _,
2321        sink_output_fields,
2322        sink_original_target_columns: sink.get_original_target_columns().clone(),
2323        project_exprs,
2324        new_sink_downstream: new_downstream_relation,
2325    })
2326}
2327
2328pub fn refill_upstream_sink_union_in_table(
2329    union_fragment_root: &mut PbStreamNode,
2330    upstream_sink_infos: &Vec<UpstreamSinkInfo>,
2331) {
2332    visit_stream_node_cont_mut(union_fragment_root, |node| {
2333        if let Some(NodeBody::UpstreamSinkUnion(upstream_sink_union)) = &mut node.node_body {
2334            let init_upstreams = upstream_sink_infos
2335                .iter()
2336                .map(|info| PbUpstreamSinkInfo {
2337                    upstream_fragment_id: info.sink_fragment_id,
2338                    sink_output_schema: info.sink_output_fields.clone(),
2339                    project_exprs: info.project_exprs.clone(),
2340                })
2341                .collect();
2342            upstream_sink_union.init_upstreams = init_upstreams;
2343            false
2344        } else {
2345            true
2346        }
2347    });
2348}