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, span};
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};
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, oneshot};
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 GlobalStreamManagerRef, JobRescheduleTarget, ReplaceStreamJobContext, SourceChange,
93 SourceManagerRef, StreamFragmentGraph, UpstreamSinkInfo,
94 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: JobRescheduleTarget,
493 mut deferred: bool,
494 ) -> MetaResult<()> {
495 tracing::info!("alter parallelism");
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: JobRescheduleTarget,
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 stream_job_id = streaming_job.id();
1121 match streaming_job.create_type() {
1122 CreateType::Unspecified | CreateType::Foreground => {
1123 let version = self
1124 .stream_manager
1125 .create_streaming_job(stream_job_fragments, ctx, None)
1126 .await?;
1127 Ok(version)
1128 }
1129 CreateType::Background => {
1130 let await_tree_key = format!("Background DDL Worker ({})", stream_job_id);
1131 let await_tree_span =
1132 span!("{:?}({})", streaming_job.job_type(), streaming_job.name());
1133
1134 let ctrl = self.clone();
1135 let (tx, rx) = oneshot::channel();
1136 let fut = async move {
1137 let _ = ctrl
1138 .stream_manager
1139 .create_streaming_job(stream_job_fragments, ctx, Some(tx))
1140 .await
1141 .inspect_err(|err| {
1142 tracing::error!(id = stream_job_id, error = ?err.as_report(), "failed to create background streaming job");
1143 });
1144 drop(permit);
1146 };
1147
1148 let fut = (self.env.await_tree_reg())
1149 .register(await_tree_key, await_tree_span)
1150 .instrument(fut);
1151 tokio::spawn(fut);
1152
1153 rx.instrument_await("wait_background_streaming_job_creation_started")
1154 .await
1155 .map_err(|_| {
1156 anyhow!(
1157 "failed to receive create streaming job result of job: {}",
1158 stream_job_id
1159 )
1160 })??;
1161 Ok(IGNORED_NOTIFICATION_VERSION)
1162 }
1163 }
1164 }
1165
1166 pub async fn drop_object(
1168 &self,
1169 object_type: ObjectType,
1170 object_id: ObjectId,
1171 drop_mode: DropMode,
1172 ) -> MetaResult<NotificationVersion> {
1173 let (release_ctx, version) = self
1174 .metadata_manager
1175 .catalog_controller
1176 .drop_object(object_type, object_id, drop_mode)
1177 .await?;
1178
1179 let ReleaseContext {
1180 database_id,
1181 removed_streaming_job_ids,
1182 removed_state_table_ids,
1183 removed_source_ids,
1184 removed_secret_ids: secret_ids,
1185 removed_source_fragments,
1186 removed_actors,
1187 removed_fragments,
1188 removed_sink_fragment_by_targets,
1189 } = release_ctx;
1190
1191 let _guard = self.source_manager.pause_tick().await;
1192 self.stream_manager
1193 .drop_streaming_jobs(
1194 risingwave_common::catalog::DatabaseId::new(database_id as _),
1195 removed_actors.iter().map(|id| *id as _).collect(),
1196 removed_streaming_job_ids,
1197 removed_state_table_ids,
1198 removed_fragments.iter().map(|id| *id as _).collect(),
1199 removed_sink_fragment_by_targets
1200 .into_iter()
1201 .map(|(target, sinks)| {
1202 (target as _, sinks.into_iter().map(|id| id as _).collect())
1203 })
1204 .collect(),
1205 )
1206 .await;
1207
1208 self.source_manager
1211 .apply_source_change(SourceChange::DropSource {
1212 dropped_source_ids: removed_source_ids.into_iter().map(|id| id as _).collect(),
1213 })
1214 .await;
1215
1216 let dropped_source_fragments = removed_source_fragments
1219 .into_iter()
1220 .map(|(source_id, fragments)| {
1221 (
1222 source_id,
1223 fragments.into_iter().map(|id| id as u32).collect(),
1224 )
1225 })
1226 .collect();
1227 self.source_manager
1228 .apply_source_change(SourceChange::DropMv {
1229 dropped_source_fragments,
1230 })
1231 .await;
1232
1233 for secret in secret_ids {
1235 LocalSecretManager::global().remove_secret(secret as _);
1236 }
1237 Ok(version)
1238 }
1239
1240 #[await_tree::instrument(boxed, "replace_streaming_job({streaming_job})")]
1242 pub async fn replace_job(
1243 &self,
1244 mut streaming_job: StreamingJob,
1245 fragment_graph: StreamFragmentGraphProto,
1246 ) -> MetaResult<NotificationVersion> {
1247 match &streaming_job {
1248 StreamingJob::Table(..)
1249 | StreamingJob::Source(..)
1250 | StreamingJob::MaterializedView(..) => {}
1251 StreamingJob::Sink(..) | StreamingJob::Index(..) => {
1252 bail_not_implemented!("schema change for {}", streaming_job.job_type_str())
1253 }
1254 }
1255
1256 let job_id = streaming_job.id();
1257
1258 let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await;
1259 let ctx = StreamContext::from_protobuf(fragment_graph.get_ctx().unwrap());
1260
1261 let original_max_parallelism = self
1263 .metadata_manager
1264 .get_job_max_parallelism(streaming_job.id().into())
1265 .await?;
1266 let fragment_graph = PbStreamFragmentGraph {
1267 max_parallelism: original_max_parallelism as _,
1268 ..fragment_graph
1269 };
1270
1271 let fragment_graph = StreamFragmentGraph::new(&self.env, fragment_graph, &streaming_job)?;
1273 streaming_job.set_info_from_graph(&fragment_graph);
1274
1275 let streaming_job = streaming_job;
1277
1278 let auto_refresh_schema_sinks = if let StreamingJob::Table(_, table, _) = &streaming_job {
1279 let auto_refresh_schema_sinks = self
1280 .metadata_manager
1281 .catalog_controller
1282 .get_sink_auto_refresh_schema_from(table.id as _)
1283 .await?;
1284 if !auto_refresh_schema_sinks.is_empty() {
1285 let original_table_columns = self
1286 .metadata_manager
1287 .catalog_controller
1288 .get_table_columns(table.id as _)
1289 .await?;
1290 let mut original_table_column_ids: HashSet<_> = original_table_columns
1292 .iter()
1293 .map(|col| col.column_id())
1294 .collect();
1295 let newly_added_columns = table
1296 .columns
1297 .iter()
1298 .filter(|col| {
1299 !original_table_column_ids.remove(&ColumnId::new(
1300 col.column_desc.as_ref().unwrap().column_id as _,
1301 ))
1302 })
1303 .map(|col| ColumnCatalog::from(col.clone()))
1304 .collect_vec();
1305 if !original_table_column_ids.is_empty() {
1306 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());
1307 }
1308 let mut sinks = Vec::with_capacity(auto_refresh_schema_sinks.len());
1309 for sink in auto_refresh_schema_sinks {
1310 let sink_job_fragments = self
1311 .metadata_manager
1312 .get_job_fragments_by_id(&risingwave_common::catalog::TableId::new(sink.id))
1313 .await?;
1314 if sink_job_fragments.fragments.len() != 1 {
1315 return Err(anyhow!(
1316 "auto schema refresh sink must have only one fragment, but got {}",
1317 sink_job_fragments.fragments.len()
1318 )
1319 .into());
1320 }
1321 let original_sink_fragment =
1322 sink_job_fragments.fragments.into_values().next().unwrap();
1323 let (new_sink_fragment, new_schema, new_log_store_table) =
1324 rewrite_refresh_schema_sink_fragment(
1325 &original_sink_fragment,
1326 &sink,
1327 &newly_added_columns,
1328 table,
1329 fragment_graph.table_fragment_id(),
1330 self.env.id_gen_manager(),
1331 )?;
1332
1333 assert_eq!(
1334 original_sink_fragment.actors.len(),
1335 new_sink_fragment.actors.len()
1336 );
1337 let actor_status = (0..original_sink_fragment.actors.len())
1338 .map(|i| {
1339 let worker_node_id = sink_job_fragments.actor_status
1340 [&original_sink_fragment.actors[i].actor_id]
1341 .location
1342 .as_ref()
1343 .unwrap()
1344 .worker_node_id;
1345 (
1346 new_sink_fragment.actors[i].actor_id,
1347 PbActorStatus {
1348 location: Some(PbActorLocation { worker_node_id }),
1349 state: PbActorState::Inactive as _,
1350 },
1351 )
1352 })
1353 .collect();
1354
1355 let streaming_job = StreamingJob::Sink(sink);
1356
1357 let tmp_sink_id = self
1358 .metadata_manager
1359 .catalog_controller
1360 .create_job_catalog_for_replace(&streaming_job, None, None, None)
1361 .await?;
1362 let StreamingJob::Sink(sink) = streaming_job else {
1363 unreachable!()
1364 };
1365
1366 sinks.push(AutoRefreshSchemaSinkContext {
1367 tmp_sink_id,
1368 original_sink: sink,
1369 original_fragment: original_sink_fragment,
1370 new_schema,
1371 newly_add_fields: newly_added_columns
1372 .iter()
1373 .map(|col| Field::from(&col.column_desc))
1374 .collect(),
1375 new_fragment: new_sink_fragment,
1376 new_log_store_table,
1377 actor_status,
1378 });
1379 }
1380 Some(sinks)
1381 } else {
1382 None
1383 }
1384 } else {
1385 None
1386 };
1387
1388 let tmp_id = self
1389 .metadata_manager
1390 .catalog_controller
1391 .create_job_catalog_for_replace(
1392 &streaming_job,
1393 Some(&ctx),
1394 fragment_graph.specified_parallelism().as_ref(),
1395 Some(fragment_graph.max_parallelism()),
1396 )
1397 .await?;
1398
1399 let tmp_sink_ids = auto_refresh_schema_sinks
1400 .as_ref()
1401 .map(|sinks| sinks.iter().map(|sink| sink.tmp_sink_id).collect_vec());
1402
1403 tracing::debug!(id = job_id, "building replace streaming job");
1404 let mut updated_sink_catalogs = vec![];
1405
1406 let mut drop_table_connector_ctx = None;
1407 let result: MetaResult<_> = try {
1408 let (mut ctx, mut stream_job_fragments) = self
1409 .build_replace_job(
1410 ctx,
1411 &streaming_job,
1412 fragment_graph,
1413 tmp_id as _,
1414 auto_refresh_schema_sinks,
1415 )
1416 .await?;
1417 drop_table_connector_ctx = ctx.drop_table_connector_ctx.clone();
1418 let auto_refresh_schema_sink_finish_ctx =
1419 ctx.auto_refresh_schema_sinks.as_ref().map(|sinks| {
1420 sinks
1421 .iter()
1422 .map(|sink| FinishAutoRefreshSchemaSinkContext {
1423 tmp_sink_id: sink.tmp_sink_id,
1424 original_sink_id: sink.original_sink.id as _,
1425 columns: sink.new_schema.clone(),
1426 new_log_store_table: sink
1427 .new_log_store_table
1428 .as_ref()
1429 .map(|table| (table.id as _, table.columns.clone())),
1430 })
1431 .collect()
1432 });
1433
1434 if let StreamingJob::Table(_, table, ..) = &streaming_job {
1436 let union_fragment = stream_job_fragments.inner.union_fragment_for_table();
1437 let upstream_infos = self
1438 .metadata_manager
1439 .catalog_controller
1440 .get_all_upstream_sink_infos(table, union_fragment.fragment_id as _)
1441 .await?;
1442 refill_upstream_sink_union_in_table(&mut union_fragment.nodes, &upstream_infos);
1443
1444 for upstream_info in &upstream_infos {
1445 let upstream_fragment_id = upstream_info.sink_fragment_id;
1446 ctx.upstream_fragment_downstreams
1447 .entry(upstream_fragment_id)
1448 .or_default()
1449 .push(upstream_info.new_sink_downstream.clone());
1450 if upstream_info.sink_original_target_columns.is_empty() {
1451 updated_sink_catalogs.push(upstream_info.sink_id);
1452 }
1453 }
1454 }
1455
1456 let replace_upstream = ctx.replace_upstream.clone();
1457
1458 if let Some(sinks) = &ctx.auto_refresh_schema_sinks {
1459 let empty_downstreams = FragmentDownstreamRelation::default();
1460 for sink in sinks {
1461 self.metadata_manager
1462 .catalog_controller
1463 .prepare_streaming_job(
1464 sink.tmp_sink_id,
1465 || [&sink.new_fragment].into_iter(),
1466 &sink.actor_status,
1467 &empty_downstreams,
1468 true,
1469 None,
1470 )
1471 .await?;
1472 }
1473 }
1474
1475 self.metadata_manager
1476 .catalog_controller
1477 .prepare_stream_job_fragments(&stream_job_fragments, &streaming_job, true)
1478 .await?;
1479
1480 self.stream_manager
1481 .replace_stream_job(stream_job_fragments, ctx)
1482 .await?;
1483 (replace_upstream, auto_refresh_schema_sink_finish_ctx)
1484 };
1485
1486 match result {
1487 Ok((replace_upstream, auto_refresh_schema_sink_finish_ctx)) => {
1488 let version = self
1489 .metadata_manager
1490 .catalog_controller
1491 .finish_replace_streaming_job(
1492 tmp_id,
1493 streaming_job,
1494 replace_upstream,
1495 SinkIntoTableContext {
1496 updated_sink_catalogs,
1497 },
1498 drop_table_connector_ctx.as_ref(),
1499 auto_refresh_schema_sink_finish_ctx,
1500 )
1501 .await?;
1502 if let Some(drop_table_connector_ctx) = &drop_table_connector_ctx {
1503 self.source_manager
1504 .apply_source_change(SourceChange::DropSource {
1505 dropped_source_ids: vec![drop_table_connector_ctx.to_remove_source_id],
1506 })
1507 .await;
1508 }
1509 Ok(version)
1510 }
1511 Err(err) => {
1512 tracing::error!(id = job_id, error = ?err.as_report(), "failed to replace job");
1513 let _ = self.metadata_manager
1514 .catalog_controller
1515 .try_abort_replacing_streaming_job(tmp_id, tmp_sink_ids)
1516 .await.inspect_err(|err| {
1517 tracing::error!(id = job_id, error = ?err.as_report(), "failed to abort replacing job");
1518 });
1519 Err(err)
1520 }
1521 }
1522 }
1523
1524 #[await_tree::instrument(boxed, "drop_streaming_job{}({job_id})", if let DropMode::Cascade = drop_mode { "_cascade" } else { "" }
1525 )]
1526 async fn drop_streaming_job(
1527 &self,
1528 job_id: StreamingJobId,
1529 drop_mode: DropMode,
1530 ) -> MetaResult<NotificationVersion> {
1531 let _reschedule_job_lock = self.stream_manager.reschedule_lock_read_guard().await;
1532
1533 let (object_id, object_type) = match job_id {
1534 StreamingJobId::MaterializedView(id) => (id as _, ObjectType::Table),
1535 StreamingJobId::Sink(id) => (id as _, ObjectType::Sink),
1536 StreamingJobId::Table(_, id) => (id as _, ObjectType::Table),
1537 StreamingJobId::Index(idx) => (idx as _, ObjectType::Index),
1538 };
1539
1540 let job_status = self
1541 .metadata_manager
1542 .catalog_controller
1543 .get_streaming_job_status(job_id.id())
1544 .await?;
1545 let version = match job_status {
1546 JobStatus::Initial => {
1547 unreachable!(
1548 "Job with Initial status should not notify frontend and therefore should not arrive here"
1549 );
1550 }
1551 JobStatus::Creating => {
1552 let canceled_jobs = self
1553 .stream_manager
1554 .cancel_streaming_jobs(vec![(job_id.id() as u32).into()])
1555 .await;
1556 if canceled_jobs.is_empty() {
1557 tracing::warn!(job_id = job_id.id(), "failed to cancel streaming job");
1558 }
1559 IGNORED_NOTIFICATION_VERSION
1560 }
1561 JobStatus::Created => {
1562 let version = self.drop_object(object_type, object_id, drop_mode).await?;
1563 #[cfg(not(madsim))]
1564 if let StreamingJobId::Sink(sink_id) = job_id {
1565 let db = self.env.meta_store_ref().conn.clone();
1568 clean_all_rows_by_sink_id(&db, sink_id).await?;
1569 }
1570 version
1571 }
1572 };
1573
1574 Ok(version)
1575 }
1576
1577 fn resolve_stream_parallelism(
1581 &self,
1582 specified: Option<NonZeroUsize>,
1583 max: NonZeroUsize,
1584 cluster_info: &StreamingClusterInfo,
1585 resource_group: String,
1586 ) -> MetaResult<NonZeroUsize> {
1587 let available = cluster_info.parallelism(&resource_group);
1588 let Some(available) = NonZeroUsize::new(available) else {
1589 bail_unavailable!(
1590 "no available slots to schedule in resource group \"{}\", \
1591 have you allocated any compute nodes within this resource group?",
1592 resource_group
1593 );
1594 };
1595
1596 if let Some(specified) = specified {
1597 if specified > max {
1598 bail_invalid_parameter!(
1599 "specified parallelism {} should not exceed max parallelism {}",
1600 specified,
1601 max,
1602 );
1603 }
1604 if specified > available {
1605 bail_unavailable!(
1606 "insufficient parallelism to schedule in resource group \"{}\", \
1607 required: {}, available: {}",
1608 resource_group,
1609 specified,
1610 available,
1611 );
1612 }
1613 Ok(specified)
1614 } else {
1615 let default_parallelism = match self.env.opts.default_parallelism {
1617 DefaultParallelism::Full => available,
1618 DefaultParallelism::Default(num) => {
1619 if num > available {
1620 bail_unavailable!(
1621 "insufficient parallelism to schedule in resource group \"{}\", \
1622 required: {}, available: {}",
1623 resource_group,
1624 num,
1625 available,
1626 );
1627 }
1628 num
1629 }
1630 };
1631
1632 if default_parallelism > max {
1633 tracing::warn!(
1634 max_parallelism = max.get(),
1635 resource_group,
1636 "too many parallelism available, use max parallelism instead",
1637 );
1638 }
1639 Ok(default_parallelism.min(max))
1640 }
1641 }
1642
1643 #[await_tree::instrument]
1649 pub(crate) async fn build_stream_job(
1650 &self,
1651 stream_ctx: StreamContext,
1652 mut stream_job: StreamingJob,
1653 fragment_graph: StreamFragmentGraph,
1654 specific_resource_group: Option<String>,
1655 ) -> MetaResult<(CreateStreamingJobContext, StreamJobFragmentsToCreate)> {
1656 let id = stream_job.id();
1657 let specified_parallelism = fragment_graph.specified_parallelism();
1658 let expr_context = stream_ctx.to_expr_context();
1659 let max_parallelism = NonZeroUsize::new(fragment_graph.max_parallelism()).unwrap();
1660
1661 let fragment_backfill_ordering = fragment_graph.create_fragment_backfill_ordering();
1663
1664 let (snapshot_backfill_info, cross_db_snapshot_backfill_info) =
1668 fragment_graph.collect_snapshot_backfill_info()?;
1669 assert!(
1670 snapshot_backfill_info
1671 .iter()
1672 .chain([&cross_db_snapshot_backfill_info])
1673 .flat_map(|info| info.upstream_mv_table_id_to_backfill_epoch.values())
1674 .all(|backfill_epoch| backfill_epoch.is_none()),
1675 "should not set backfill epoch when initially build the job: {:?} {:?}",
1676 snapshot_backfill_info,
1677 cross_db_snapshot_backfill_info
1678 );
1679
1680 self.metadata_manager
1682 .catalog_controller
1683 .validate_cross_db_snapshot_backfill(&cross_db_snapshot_backfill_info)
1684 .await?;
1685
1686 let upstream_table_ids = fragment_graph
1687 .dependent_table_ids()
1688 .iter()
1689 .filter(|id| {
1690 !cross_db_snapshot_backfill_info
1691 .upstream_mv_table_id_to_backfill_epoch
1692 .contains_key(id)
1693 })
1694 .cloned()
1695 .collect();
1696
1697 let (upstream_root_fragments, existing_actor_location) = self
1698 .metadata_manager
1699 .get_upstream_root_fragments(&upstream_table_ids)
1700 .await?;
1701
1702 if snapshot_backfill_info.is_some() {
1703 match stream_job {
1704 StreamingJob::MaterializedView(_)
1705 | StreamingJob::Sink(_)
1706 | StreamingJob::Index(_, _) => {}
1707 StreamingJob::Table(_, _, _) | StreamingJob::Source(_) => {
1708 return Err(
1709 anyhow!("snapshot_backfill not enabled for table and source").into(),
1710 );
1711 }
1712 }
1713 }
1714
1715 let upstream_actors = upstream_root_fragments
1716 .values()
1717 .map(|fragment| {
1718 (
1719 fragment.fragment_id,
1720 fragment.actors.iter().map(|actor| actor.actor_id).collect(),
1721 )
1722 })
1723 .collect();
1724
1725 let complete_graph = CompleteStreamFragmentGraph::with_upstreams(
1726 fragment_graph,
1727 upstream_root_fragments,
1728 existing_actor_location,
1729 (&stream_job).into(),
1730 )?;
1731
1732 let resource_group = match specific_resource_group {
1733 None => {
1734 self.metadata_manager
1735 .get_database_resource_group(stream_job.database_id() as ObjectId)
1736 .await?
1737 }
1738 Some(resource_group) => resource_group,
1739 };
1740
1741 let cluster_info = self.metadata_manager.get_streaming_cluster_info().await?;
1743
1744 let parallelism = self.resolve_stream_parallelism(
1745 specified_parallelism,
1746 max_parallelism,
1747 &cluster_info,
1748 resource_group.clone(),
1749 )?;
1750
1751 let parallelism = self
1752 .env
1753 .system_params_reader()
1754 .await
1755 .adaptive_parallelism_strategy()
1756 .compute_target_parallelism(parallelism.get());
1757
1758 let parallelism = NonZeroUsize::new(parallelism).expect("parallelism must be positive");
1759 let actor_graph_builder = ActorGraphBuilder::new(
1760 id,
1761 resource_group,
1762 complete_graph,
1763 cluster_info,
1764 parallelism,
1765 )?;
1766
1767 let ActorGraphBuildResult {
1768 graph,
1769 downstream_fragment_relations,
1770 building_locations,
1771 upstream_fragment_downstreams,
1772 new_no_shuffle,
1773 replace_upstream,
1774 ..
1775 } = actor_graph_builder.generate_graph(&self.env, &stream_job, expr_context)?;
1776 assert!(replace_upstream.is_empty());
1777
1778 let table_parallelism = match (specified_parallelism, &self.env.opts.default_parallelism) {
1785 (None, DefaultParallelism::Full) => TableParallelism::Adaptive,
1786 _ => TableParallelism::Fixed(parallelism.get()),
1787 };
1788
1789 let stream_job_fragments = StreamJobFragments::new(
1790 id.into(),
1791 graph,
1792 &building_locations.actor_locations,
1793 stream_ctx.clone(),
1794 table_parallelism,
1795 max_parallelism.get(),
1796 );
1797
1798 if let Some(mview_fragment) = stream_job_fragments.mview_fragment() {
1799 stream_job.set_table_vnode_count(mview_fragment.vnode_count());
1800 }
1801
1802 let new_upstream_sink = if let StreamingJob::Sink(sink) = &stream_job
1803 && let Ok(table_id) = sink.get_target_table()
1804 {
1805 let tables = self
1806 .metadata_manager
1807 .get_table_catalog_by_ids(&[*table_id as _])
1808 .await?;
1809 let target_table = tables
1810 .first()
1811 .ok_or_else(|| MetaError::catalog_id_not_found("table", *table_id))?;
1812 let sink_fragment = stream_job_fragments
1813 .sink_fragment()
1814 .ok_or_else(|| anyhow::anyhow!("sink fragment not found for sink {}", sink.id))?;
1815 let mview_fragment_id = self
1816 .metadata_manager
1817 .catalog_controller
1818 .get_mview_fragment_by_id(*table_id as _)
1819 .await?;
1820 let upstream_sink_info = build_upstream_sink_info(
1821 sink,
1822 sink_fragment.fragment_id as _,
1823 target_table,
1824 mview_fragment_id,
1825 )?;
1826 Some(upstream_sink_info)
1827 } else {
1828 None
1829 };
1830
1831 let ctx = CreateStreamingJobContext {
1832 upstream_fragment_downstreams,
1833 new_no_shuffle,
1834 upstream_actors,
1835 building_locations,
1836 definition: stream_job.definition(),
1837 mv_table_id: stream_job.mv_table(),
1838 create_type: stream_job.create_type(),
1839 job_type: (&stream_job).into(),
1840 streaming_job: stream_job,
1841 new_upstream_sink,
1842 option: CreateStreamingJobOption {},
1843 snapshot_backfill_info,
1844 cross_db_snapshot_backfill_info,
1845 fragment_backfill_ordering,
1846 };
1847
1848 Ok((
1849 ctx,
1850 StreamJobFragmentsToCreate {
1851 inner: stream_job_fragments,
1852 downstreams: downstream_fragment_relations,
1853 },
1854 ))
1855 }
1856
1857 pub(crate) async fn build_replace_job(
1863 &self,
1864 stream_ctx: StreamContext,
1865 stream_job: &StreamingJob,
1866 mut fragment_graph: StreamFragmentGraph,
1867 tmp_job_id: TableId,
1868 auto_refresh_schema_sinks: Option<Vec<AutoRefreshSchemaSinkContext>>,
1869 ) -> MetaResult<(ReplaceStreamJobContext, StreamJobFragmentsToCreate)> {
1870 match &stream_job {
1871 StreamingJob::Table(..)
1872 | StreamingJob::Source(..)
1873 | StreamingJob::MaterializedView(..) => {}
1874 StreamingJob::Sink(..) | StreamingJob::Index(..) => {
1875 bail_not_implemented!("schema change for {}", stream_job.job_type_str())
1876 }
1877 }
1878
1879 let id = stream_job.id();
1880 let expr_context = stream_ctx.to_expr_context();
1881
1882 let mut drop_table_associated_source_id = None;
1884 if let StreamingJob::Table(None, _, _) = &stream_job {
1885 drop_table_associated_source_id = self
1886 .metadata_manager
1887 .get_table_associated_source_id(id as _)
1888 .await?;
1889 }
1890
1891 let old_fragments = self
1892 .metadata_manager
1893 .get_job_fragments_by_id(&id.into())
1894 .await?;
1895 let old_internal_table_ids = old_fragments.internal_table_ids();
1896
1897 let mut drop_table_connector_ctx = None;
1899 if let Some(to_remove_source_id) = drop_table_associated_source_id {
1900 debug_assert!(old_internal_table_ids.len() == 1);
1902
1903 drop_table_connector_ctx = Some(DropTableConnectorContext {
1904 to_change_streaming_job_id: id as i32,
1907 to_remove_state_table_id: old_internal_table_ids[0] as i32, to_remove_source_id,
1909 });
1910 } else if stream_job.is_materialized_view() {
1911 let old_fragments_upstreams = self
1914 .metadata_manager
1915 .catalog_controller
1916 .upstream_fragments(old_fragments.fragment_ids())
1917 .await?;
1918
1919 let old_state_graph =
1920 state_match::Graph::from_existing(&old_fragments, &old_fragments_upstreams);
1921 let new_state_graph = state_match::Graph::from_building(&fragment_graph);
1922 let mapping =
1923 state_match::match_graph_internal_tables(&new_state_graph, &old_state_graph)
1924 .context("incompatible altering on the streaming job states")?;
1925
1926 fragment_graph.fit_internal_table_ids_with_mapping(mapping);
1927 } else {
1928 let old_internal_tables = self
1931 .metadata_manager
1932 .get_table_catalog_by_ids(&old_internal_table_ids)
1933 .await?;
1934 fragment_graph.fit_internal_tables_trivial(old_internal_tables)?;
1935 }
1936
1937 let original_root_fragment = old_fragments
1940 .root_fragment()
1941 .expect("root fragment not found");
1942
1943 let job_type = StreamingJobType::from(stream_job);
1944
1945 let (mut downstream_fragments, mut downstream_actor_location) =
1947 self.metadata_manager.get_downstream_fragments(id).await?;
1948
1949 if let Some(auto_refresh_schema_sinks) = &auto_refresh_schema_sinks {
1950 let mut remaining_fragment: HashSet<_> = auto_refresh_schema_sinks
1951 .iter()
1952 .map(|sink| sink.original_fragment.fragment_id)
1953 .collect();
1954 for (_, downstream_fragment) in &mut downstream_fragments {
1955 if let Some(sink) = auto_refresh_schema_sinks.iter().find(|sink| {
1956 sink.original_fragment.fragment_id == downstream_fragment.fragment_id
1957 }) {
1958 assert!(remaining_fragment.remove(&downstream_fragment.fragment_id));
1959 for actor in &downstream_fragment.actors {
1960 downstream_actor_location.remove(&actor.actor_id);
1961 }
1962 for (actor_id, status) in &sink.actor_status {
1963 downstream_actor_location.insert(
1964 *actor_id,
1965 status.location.as_ref().unwrap().worker_node_id as WorkerId,
1966 );
1967 }
1968 *downstream_fragment = sink.new_fragment.clone();
1969 }
1970 }
1971 assert!(remaining_fragment.is_empty());
1972 }
1973
1974 let complete_graph = match &job_type {
1976 StreamingJobType::Table(TableJobType::General) | StreamingJobType::Source => {
1977 CompleteStreamFragmentGraph::with_downstreams(
1978 fragment_graph,
1979 original_root_fragment.fragment_id,
1980 downstream_fragments,
1981 downstream_actor_location,
1982 job_type,
1983 )?
1984 }
1985 StreamingJobType::Table(TableJobType::SharedCdcSource)
1986 | StreamingJobType::MaterializedView => {
1987 let (upstream_root_fragments, upstream_actor_location) = self
1989 .metadata_manager
1990 .get_upstream_root_fragments(fragment_graph.dependent_table_ids())
1991 .await?;
1992
1993 CompleteStreamFragmentGraph::with_upstreams_and_downstreams(
1994 fragment_graph,
1995 upstream_root_fragments,
1996 upstream_actor_location,
1997 original_root_fragment.fragment_id,
1998 downstream_fragments,
1999 downstream_actor_location,
2000 job_type,
2001 )?
2002 }
2003 _ => unreachable!(),
2004 };
2005
2006 let resource_group = self
2007 .metadata_manager
2008 .get_existing_job_resource_group(id as ObjectId)
2009 .await?;
2010
2011 let cluster_info = self.metadata_manager.get_streaming_cluster_info().await?;
2013
2014 let parallelism = NonZeroUsize::new(original_root_fragment.actors.len())
2017 .expect("The number of actors in the original table fragment should be greater than 0");
2018
2019 let actor_graph_builder = ActorGraphBuilder::new(
2020 id,
2021 resource_group,
2022 complete_graph,
2023 cluster_info,
2024 parallelism,
2025 )?;
2026
2027 let ActorGraphBuildResult {
2028 graph,
2029 downstream_fragment_relations,
2030 building_locations,
2031 upstream_fragment_downstreams,
2032 mut replace_upstream,
2033 new_no_shuffle,
2034 ..
2035 } = actor_graph_builder.generate_graph(&self.env, stream_job, expr_context)?;
2036
2037 if matches!(
2039 job_type,
2040 StreamingJobType::Source | StreamingJobType::Table(TableJobType::General)
2041 ) {
2042 assert!(upstream_fragment_downstreams.is_empty());
2043 }
2044
2045 let stream_job_fragments = StreamJobFragments::new(
2049 (tmp_job_id as u32).into(),
2050 graph,
2051 &building_locations.actor_locations,
2052 stream_ctx,
2053 old_fragments.assigned_parallelism,
2054 old_fragments.max_parallelism,
2055 );
2056
2057 if let Some(sinks) = &auto_refresh_schema_sinks {
2058 for sink in sinks {
2059 replace_upstream
2060 .remove(&sink.new_fragment.fragment_id)
2061 .expect("should exist");
2062 }
2063 }
2064
2065 let ctx = ReplaceStreamJobContext {
2069 old_fragments,
2070 replace_upstream,
2071 new_no_shuffle,
2072 upstream_fragment_downstreams,
2073 building_locations,
2074 streaming_job: stream_job.clone(),
2075 tmp_id: tmp_job_id as _,
2076 drop_table_connector_ctx,
2077 auto_refresh_schema_sinks,
2078 };
2079
2080 Ok((
2081 ctx,
2082 StreamJobFragmentsToCreate {
2083 inner: stream_job_fragments,
2084 downstreams: downstream_fragment_relations,
2085 },
2086 ))
2087 }
2088
2089 async fn alter_name(
2090 &self,
2091 relation: alter_name_request::Object,
2092 new_name: &str,
2093 ) -> MetaResult<NotificationVersion> {
2094 let (obj_type, id) = match relation {
2095 alter_name_request::Object::TableId(id) => (ObjectType::Table, id as ObjectId),
2096 alter_name_request::Object::ViewId(id) => (ObjectType::View, id as ObjectId),
2097 alter_name_request::Object::IndexId(id) => (ObjectType::Index, id as ObjectId),
2098 alter_name_request::Object::SinkId(id) => (ObjectType::Sink, id as ObjectId),
2099 alter_name_request::Object::SourceId(id) => (ObjectType::Source, id as ObjectId),
2100 alter_name_request::Object::SchemaId(id) => (ObjectType::Schema, id as ObjectId),
2101 alter_name_request::Object::DatabaseId(id) => (ObjectType::Database, id as ObjectId),
2102 alter_name_request::Object::SubscriptionId(id) => {
2103 (ObjectType::Subscription, id as ObjectId)
2104 }
2105 };
2106 self.metadata_manager
2107 .catalog_controller
2108 .alter_name(obj_type, id, new_name)
2109 .await
2110 }
2111
2112 async fn alter_swap_rename(
2113 &self,
2114 object: alter_swap_rename_request::Object,
2115 ) -> MetaResult<NotificationVersion> {
2116 let (obj_type, src_id, dst_id) = match object {
2117 alter_swap_rename_request::Object::Schema(_) => unimplemented!("schema swap"),
2118 alter_swap_rename_request::Object::Table(objs) => {
2119 let (src_id, dst_id) = (
2120 objs.src_object_id as ObjectId,
2121 objs.dst_object_id as ObjectId,
2122 );
2123 (ObjectType::Table, src_id, dst_id)
2124 }
2125 alter_swap_rename_request::Object::View(objs) => {
2126 let (src_id, dst_id) = (
2127 objs.src_object_id as ObjectId,
2128 objs.dst_object_id as ObjectId,
2129 );
2130 (ObjectType::View, src_id, dst_id)
2131 }
2132 alter_swap_rename_request::Object::Source(objs) => {
2133 let (src_id, dst_id) = (
2134 objs.src_object_id as ObjectId,
2135 objs.dst_object_id as ObjectId,
2136 );
2137 (ObjectType::Source, src_id, dst_id)
2138 }
2139 alter_swap_rename_request::Object::Sink(objs) => {
2140 let (src_id, dst_id) = (
2141 objs.src_object_id as ObjectId,
2142 objs.dst_object_id as ObjectId,
2143 );
2144 (ObjectType::Sink, src_id, dst_id)
2145 }
2146 alter_swap_rename_request::Object::Subscription(objs) => {
2147 let (src_id, dst_id) = (
2148 objs.src_object_id as ObjectId,
2149 objs.dst_object_id as ObjectId,
2150 );
2151 (ObjectType::Subscription, src_id, dst_id)
2152 }
2153 };
2154
2155 self.metadata_manager
2156 .catalog_controller
2157 .alter_swap_rename(obj_type, src_id, dst_id)
2158 .await
2159 }
2160
2161 async fn alter_owner(
2162 &self,
2163 object: Object,
2164 owner_id: UserId,
2165 ) -> MetaResult<NotificationVersion> {
2166 let (obj_type, id) = match object {
2167 Object::TableId(id) => (ObjectType::Table, id as ObjectId),
2168 Object::ViewId(id) => (ObjectType::View, id as ObjectId),
2169 Object::SourceId(id) => (ObjectType::Source, id as ObjectId),
2170 Object::SinkId(id) => (ObjectType::Sink, id as ObjectId),
2171 Object::SchemaId(id) => (ObjectType::Schema, id as ObjectId),
2172 Object::DatabaseId(id) => (ObjectType::Database, id as ObjectId),
2173 Object::SubscriptionId(id) => (ObjectType::Subscription, id as ObjectId),
2174 Object::ConnectionId(id) => (ObjectType::Connection, id as ObjectId),
2175 };
2176 self.metadata_manager
2177 .catalog_controller
2178 .alter_owner(obj_type, id, owner_id as _)
2179 .await
2180 }
2181
2182 async fn alter_set_schema(
2183 &self,
2184 object: alter_set_schema_request::Object,
2185 new_schema_id: SchemaId,
2186 ) -> MetaResult<NotificationVersion> {
2187 let (obj_type, id) = match object {
2188 alter_set_schema_request::Object::TableId(id) => (ObjectType::Table, id as ObjectId),
2189 alter_set_schema_request::Object::ViewId(id) => (ObjectType::View, id as ObjectId),
2190 alter_set_schema_request::Object::SourceId(id) => (ObjectType::Source, id as ObjectId),
2191 alter_set_schema_request::Object::SinkId(id) => (ObjectType::Sink, id as ObjectId),
2192 alter_set_schema_request::Object::FunctionId(id) => {
2193 (ObjectType::Function, id as ObjectId)
2194 }
2195 alter_set_schema_request::Object::ConnectionId(id) => {
2196 (ObjectType::Connection, id as ObjectId)
2197 }
2198 alter_set_schema_request::Object::SubscriptionId(id) => {
2199 (ObjectType::Subscription, id as ObjectId)
2200 }
2201 };
2202 self.metadata_manager
2203 .catalog_controller
2204 .alter_schema(obj_type, id, new_schema_id as _)
2205 .await
2206 }
2207
2208 pub async fn wait(&self) -> MetaResult<()> {
2209 let timeout_ms = 30 * 60 * 1000;
2210 for _ in 0..timeout_ms {
2211 if self
2212 .metadata_manager
2213 .catalog_controller
2214 .list_background_creating_jobs(true)
2215 .await?
2216 .is_empty()
2217 {
2218 return Ok(());
2219 }
2220
2221 sleep(Duration::from_millis(1)).await;
2222 }
2223 Err(MetaError::cancelled(format!(
2224 "timeout after {timeout_ms}ms"
2225 )))
2226 }
2227
2228 async fn comment_on(&self, comment: Comment) -> MetaResult<NotificationVersion> {
2229 self.metadata_manager
2230 .catalog_controller
2231 .comment_on(comment)
2232 .await
2233 }
2234}
2235
2236fn report_create_object(
2237 catalog_id: u32,
2238 event_name: &str,
2239 obj_type: PbTelemetryDatabaseObject,
2240 connector_name: Option<String>,
2241 attr_info: Option<jsonbb::Value>,
2242) {
2243 report_event(
2244 PbTelemetryEventStage::CreateStreamJob,
2245 event_name,
2246 catalog_id.into(),
2247 connector_name,
2248 Some(obj_type),
2249 attr_info,
2250 );
2251}
2252
2253async fn clean_all_rows_by_sink_id(db: &DatabaseConnection, sink_id: i32) -> MetaResult<()> {
2254 match Entity::delete_many()
2255 .filter(Column::SinkId.eq(sink_id))
2256 .exec(db)
2257 .await
2258 {
2259 Ok(result) => {
2260 let deleted_count = result.rows_affected;
2261
2262 tracing::info!(
2263 "Deleted {} items for sink_id = {} in iceberg exactly once system table.",
2264 deleted_count,
2265 sink_id
2266 );
2267 Ok(())
2268 }
2269 Err(e) => {
2270 tracing::error!(
2271 "Error deleting records for sink_id = {} from iceberg exactly once system table: {:?}",
2272 sink_id,
2273 e.as_report()
2274 );
2275 Err(e.into())
2276 }
2277 }
2278}
2279
2280pub fn build_upstream_sink_info(
2281 sink: &PbSink,
2282 sink_fragment_id: FragmentId,
2283 target_table: &PbTable,
2284 target_fragment_id: FragmentId,
2285) -> MetaResult<UpstreamSinkInfo> {
2286 let sink_columns = if !sink.original_target_columns.is_empty() {
2287 sink.original_target_columns.clone()
2288 } else {
2289 target_table.columns.clone()
2294 };
2295
2296 let sink_output_fields = sink_columns
2297 .iter()
2298 .map(|col| Field::from(col.column_desc.as_ref().unwrap()).to_prost())
2299 .collect_vec();
2300 let output_indices = (0..sink_output_fields.len())
2301 .map(|i| i as u32)
2302 .collect_vec();
2303
2304 let dist_key_indices: anyhow::Result<Vec<u32>> = try {
2305 let sink_idx_by_col_id = sink_columns
2306 .iter()
2307 .enumerate()
2308 .map(|(idx, col)| {
2309 let column_id = col.column_desc.as_ref().unwrap().column_id;
2310 (column_id, idx as u32)
2311 })
2312 .collect::<HashMap<_, _>>();
2313 target_table
2314 .distribution_key
2315 .iter()
2316 .map(|dist_idx| {
2317 let column_id = target_table.columns[*dist_idx as usize]
2318 .column_desc
2319 .as_ref()
2320 .unwrap()
2321 .column_id;
2322 let sink_idx = sink_idx_by_col_id
2323 .get(&column_id)
2324 .ok_or_else(|| anyhow::anyhow!("column id {} not found in sink", column_id))?;
2325 Ok(*sink_idx)
2326 })
2327 .collect::<anyhow::Result<Vec<_>>>()?
2328 };
2329 let dist_key_indices =
2330 dist_key_indices.map_err(|e| e.context("failed to get distribution key indices"))?;
2331 let downstream_fragment_id = target_fragment_id as _;
2332 let new_downstream_relation = DownstreamFragmentRelation {
2333 downstream_fragment_id,
2334 dispatcher_type: DispatcherType::Hash,
2335 dist_key_indices,
2336 output_mapping: PbDispatchOutputMapping::simple(output_indices),
2337 };
2338 let current_target_columns = target_table.get_columns();
2339 let project_exprs = build_select_node_list(&sink_columns, current_target_columns)?;
2340 Ok(UpstreamSinkInfo {
2341 sink_id: sink.id as _,
2342 sink_fragment_id: sink_fragment_id as _,
2343 sink_output_fields: sink_output_fields.clone(),
2344 sink_original_target_columns: sink.get_original_target_columns().clone(),
2345 project_exprs,
2346 new_sink_downstream: new_downstream_relation,
2347 })
2348}
2349
2350pub fn refill_upstream_sink_union_in_table(
2351 union_fragment_root: &mut PbStreamNode,
2352 upstream_sink_infos: &Vec<UpstreamSinkInfo>,
2353) {
2354 visit_stream_node_cont_mut(union_fragment_root, |node| {
2355 if let Some(NodeBody::UpstreamSinkUnion(upstream_sink_union)) = &mut node.node_body {
2356 let init_upstreams = upstream_sink_infos
2357 .iter()
2358 .map(|info| PbUpstreamSinkInfo {
2359 upstream_fragment_id: info.sink_fragment_id,
2360 sink_output_schema: info.sink_output_fields.clone(),
2361 project_exprs: info.project_exprs.clone(),
2362 })
2363 .collect();
2364 upstream_sink_union.init_upstreams = init_upstreams;
2365 false
2366 } else {
2367 true
2368 }
2369 });
2370}