risingwave_meta/controller/catalog/
mod.rs

1// Copyright 2024 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15mod alter_op;
16mod create_op;
17mod drop_op;
18mod get_op;
19mod list_op;
20mod test;
21mod util;
22
23use std::collections::{BTreeSet, HashMap, HashSet};
24use std::iter;
25use std::mem::take;
26use std::sync::Arc;
27
28use anyhow::anyhow;
29use itertools::Itertools;
30use risingwave_common::catalog::{
31    DEFAULT_SCHEMA_NAME, FragmentTypeFlag, FragmentTypeMask, SYSTEM_SCHEMAS, TableOption,
32};
33use risingwave_common::current_cluster_version;
34use risingwave_common::id::JobId;
35use risingwave_common::secret::LocalSecretManager;
36use risingwave_common::util::stream_graph_visitor::visit_stream_node_cont_mut;
37use risingwave_connector::source::UPSTREAM_SOURCE_KEY;
38use risingwave_connector::source::cdc::build_cdc_table_id;
39use risingwave_meta_model::object::ObjectType;
40use risingwave_meta_model::prelude::*;
41use risingwave_meta_model::table::TableType;
42use risingwave_meta_model::{
43    ActorId, ColumnCatalogArray, ConnectionId, CreateType, DatabaseId, FragmentId, I32Array,
44    IndexId, JobStatus, ObjectId, Property, SchemaId, SecretId, SinkFormatDesc, SinkId, SourceId,
45    StreamNode, StreamSourceInfo, StreamingParallelism, SubscriptionId, TableId, TableIdArray,
46    UserId, ViewId, connection, database, fragment, function, index, object, object_dependency,
47    pending_sink_state, schema, secret, sink, source, streaming_job, subscription, table,
48    user_privilege, view,
49};
50use risingwave_pb::catalog::connection::Info as ConnectionInfo;
51use risingwave_pb::catalog::subscription::SubscriptionState;
52use risingwave_pb::catalog::table::PbTableType;
53use risingwave_pb::catalog::{
54    PbComment, PbConnection, PbDatabase, PbFunction, PbIndex, PbSchema, PbSecret, PbSink, PbSource,
55    PbSubscription, PbTable, PbView,
56};
57use risingwave_pb::meta::cancel_creating_jobs_request::PbCreatingJobInfo;
58use risingwave_pb::meta::list_object_dependencies_response::PbObjectDependencies;
59use risingwave_pb::meta::object::PbObjectInfo;
60use risingwave_pb::meta::subscribe_response::{
61    Info as NotificationInfo, Info, Operation as NotificationOperation, Operation,
62};
63use risingwave_pb::meta::{PbObject, PbObjectGroup};
64use risingwave_pb::stream_plan::stream_node::NodeBody;
65use risingwave_pb::telemetry::PbTelemetryEventStage;
66use risingwave_pb::user::PbUserInfo;
67use sea_orm::ActiveValue::Set;
68use sea_orm::sea_query::{Expr, Query, SimpleExpr};
69use sea_orm::{
70    ActiveModelTrait, ColumnTrait, DatabaseConnection, DatabaseTransaction, EntityTrait,
71    IntoActiveModel, JoinType, PaginatorTrait, QueryFilter, QuerySelect, RelationTrait,
72    SelectColumns, TransactionTrait, Value,
73};
74use tokio::sync::oneshot::Sender;
75use tokio::sync::{RwLock, RwLockReadGuard, RwLockWriteGuard};
76use tracing::info;
77
78use super::utils::{
79    check_subscription_name_duplicate, get_internal_tables_by_id, load_streaming_jobs_by_ids,
80    rename_relation, rename_relation_refer,
81};
82use crate::controller::ObjectModel;
83use crate::controller::catalog::util::update_internal_tables;
84use crate::controller::fragment::FragmentTypeMaskExt;
85use crate::controller::utils::*;
86use crate::manager::{
87    IGNORED_NOTIFICATION_VERSION, MetaSrvEnv, NotificationVersion,
88    get_referred_connection_ids_from_source, get_referred_secret_ids_from_source,
89};
90use crate::rpc::ddl_controller::DropMode;
91use crate::telemetry::{MetaTelemetryJobDesc, report_event};
92use crate::{MetaError, MetaResult};
93
94pub type Catalog = (
95    Vec<PbDatabase>,
96    Vec<PbSchema>,
97    Vec<PbTable>,
98    Vec<PbSource>,
99    Vec<PbSink>,
100    Vec<PbSubscription>,
101    Vec<PbIndex>,
102    Vec<PbView>,
103    Vec<PbFunction>,
104    Vec<PbConnection>,
105    Vec<PbSecret>,
106);
107
108pub type CatalogControllerRef = Arc<CatalogController>;
109
110/// `CatalogController` is the controller for catalog related operations, including database, schema, table, view, etc.
111pub struct CatalogController {
112    pub(crate) env: MetaSrvEnv,
113    pub(crate) inner: RwLock<CatalogControllerInner>,
114}
115
116#[derive(Clone, Default, Debug)]
117pub struct DropTableConnectorContext {
118    // we only apply one drop connector action for one table each time, so no need to vector here
119    pub(crate) to_change_streaming_job_id: JobId,
120    pub(crate) to_remove_state_table_id: TableId,
121    pub(crate) to_remove_source_id: SourceId,
122}
123
124#[derive(Clone, Default, Debug)]
125pub struct ReleaseContext {
126    pub(crate) database_id: DatabaseId,
127    pub(crate) removed_streaming_job_ids: Vec<JobId>,
128    /// Dropped state table list, need to unregister from hummock.
129    pub(crate) removed_state_table_ids: Vec<TableId>,
130
131    /// Dropped secrets, need to remove from secret manager.
132    pub(crate) removed_secret_ids: Vec<SecretId>,
133    /// Dropped sources (when `DROP SOURCE`), need to unregister from source manager.
134    pub(crate) removed_source_ids: Vec<SourceId>,
135    /// Dropped Source fragments (when `DROP MATERIALIZED VIEW` referencing sources),
136    /// need to unregister from source manager.
137    pub(crate) removed_source_fragments: HashMap<SourceId, BTreeSet<FragmentId>>,
138
139    pub(crate) removed_actors: HashSet<ActorId>,
140    pub(crate) removed_fragments: HashSet<FragmentId>,
141
142    /// Removed sink fragment by target fragment.
143    pub(crate) removed_sink_fragment_by_targets: HashMap<FragmentId, Vec<FragmentId>>,
144
145    /// Dropped iceberg table sinks
146    pub(crate) removed_iceberg_table_sinks: Vec<PbSink>,
147}
148
149impl CatalogController {
150    pub async fn new(env: MetaSrvEnv) -> MetaResult<Self> {
151        let meta_store = env.meta_store();
152        let catalog_controller = Self {
153            env,
154            inner: RwLock::new(CatalogControllerInner {
155                db: meta_store.conn,
156                creating_table_finish_notifier: HashMap::new(),
157                dropped_tables: HashMap::new(),
158            }),
159        };
160
161        catalog_controller.init().await?;
162        Ok(catalog_controller)
163    }
164
165    /// Used in `NotificationService::subscribe`.
166    /// Need to pay attention to the order of acquiring locks to prevent deadlock problems.
167    pub async fn get_inner_read_guard(&self) -> RwLockReadGuard<'_, CatalogControllerInner> {
168        self.inner.read().await
169    }
170
171    pub async fn get_inner_write_guard(&self) -> RwLockWriteGuard<'_, CatalogControllerInner> {
172        self.inner.write().await
173    }
174}
175
176pub struct CatalogControllerInner {
177    pub(crate) db: DatabaseConnection,
178    /// Registered finish notifiers for creating tables.
179    ///
180    /// `DdlController` will update this map, and pass the `tx` side to `CatalogController`.
181    /// On notifying, we can remove the entry from this map.
182    #[expect(clippy::type_complexity)]
183    pub creating_table_finish_notifier:
184        HashMap<DatabaseId, HashMap<JobId, Vec<Sender<Result<NotificationVersion, String>>>>>,
185    /// Tables have been dropped from the meta store, but the corresponding barrier remains unfinished.
186    pub dropped_tables: HashMap<TableId, PbTable>,
187}
188
189impl CatalogController {
190    pub(crate) async fn notify_frontend(
191        &self,
192        operation: NotificationOperation,
193        info: NotificationInfo,
194    ) -> NotificationVersion {
195        self.env
196            .notification_manager()
197            .notify_frontend(operation, info)
198            .await
199    }
200
201    pub(crate) async fn notify_frontend_relation_info(
202        &self,
203        operation: NotificationOperation,
204        relation_info: PbObjectInfo,
205    ) -> NotificationVersion {
206        self.env
207            .notification_manager()
208            .notify_frontend_object_info(operation, relation_info)
209            .await
210    }
211
212    pub(crate) async fn current_notification_version(&self) -> NotificationVersion {
213        self.env.notification_manager().current_version().await
214    }
215}
216
217impl CatalogController {
218    pub async fn finish_create_subscription_catalog(
219        &self,
220        subscription_id: SubscriptionId,
221    ) -> MetaResult<()> {
222        let inner = self.inner.write().await;
223        let txn = inner.db.begin().await?;
224
225        // update `created_at` as now() and `created_at_cluster_version` as current cluster version.
226        let res = Object::update_many()
227            .col_expr(object::Column::CreatedAt, Expr::current_timestamp().into())
228            .col_expr(
229                object::Column::CreatedAtClusterVersion,
230                current_cluster_version().into(),
231            )
232            .filter(object::Column::Oid.eq(subscription_id))
233            .exec(&txn)
234            .await?;
235        if res.rows_affected == 0 {
236            return Err(MetaError::catalog_id_not_found(
237                "subscription",
238                subscription_id,
239            ));
240        }
241
242        // mark the target subscription as `Create`.
243        let job = subscription::ActiveModel {
244            subscription_id: Set(subscription_id),
245            subscription_state: Set(SubscriptionState::Created.into()),
246            ..Default::default()
247        };
248        Subscription::update(job).exec(&txn).await?;
249
250        let _ = grant_default_privileges_automatically(&txn, subscription_id).await?;
251
252        txn.commit().await?;
253
254        Ok(())
255    }
256
257    pub async fn notify_create_subscription(
258        &self,
259        subscription_id: SubscriptionId,
260    ) -> MetaResult<NotificationVersion> {
261        let inner = self.inner.read().await;
262        let (subscription, obj) = Subscription::find_by_id(subscription_id)
263            .find_also_related(Object)
264            .filter(subscription::Column::SubscriptionState.eq(SubscriptionState::Created as i32))
265            .one(&inner.db)
266            .await?
267            .ok_or_else(|| MetaError::catalog_id_not_found("subscription", subscription_id))?;
268
269        let mut version = self
270            .notify_frontend(
271                NotificationOperation::Add,
272                NotificationInfo::ObjectGroup(PbObjectGroup {
273                    objects: vec![PbObject {
274                        object_info: PbObjectInfo::Subscription(
275                            ObjectModel(subscription, obj.unwrap(), None).into(),
276                        )
277                        .into(),
278                    }],
279                }),
280            )
281            .await;
282
283        // notify default privileges about the new subscription
284        let updated_user_ids: Vec<UserId> = UserPrivilege::find()
285            .select_only()
286            .distinct()
287            .column(user_privilege::Column::UserId)
288            .filter(user_privilege::Column::Oid.eq(subscription_id.as_object_id()))
289            .into_tuple()
290            .all(&inner.db)
291            .await?;
292
293        if !updated_user_ids.is_empty() {
294            let updated_user_infos = list_user_info_by_ids(updated_user_ids, &inner.db).await?;
295            version = self.notify_users_update(updated_user_infos).await;
296        }
297
298        Ok(version)
299    }
300
301    // for telemetry
302    pub async fn get_connector_usage(&self) -> MetaResult<jsonbb::Value> {
303        // get connector usage by source/sink
304        // the expect format is like:
305        // {
306        //     "source": [{
307        //         "$source_id": {
308        //             "connector": "kafka",
309        //             "format": "plain",
310        //             "encode": "json"
311        //         },
312        //     }],
313        //     "sink": [{
314        //         "$sink_id": {
315        //             "connector": "pulsar",
316        //             "format": "upsert",
317        //             "encode": "avro"
318        //         },
319        //     }],
320        // }
321
322        let inner = self.inner.read().await;
323        let source_props_and_info: Vec<(i32, Property, Option<StreamSourceInfo>)> = Source::find()
324            .select_only()
325            .column(source::Column::SourceId)
326            .column(source::Column::WithProperties)
327            .column(source::Column::SourceInfo)
328            .into_tuple()
329            .all(&inner.db)
330            .await?;
331        let sink_props_and_info: Vec<(i32, Property, Option<SinkFormatDesc>)> = Sink::find()
332            .select_only()
333            .column(sink::Column::SinkId)
334            .column(sink::Column::Properties)
335            .column(sink::Column::SinkFormatDesc)
336            .into_tuple()
337            .all(&inner.db)
338            .await?;
339        drop(inner);
340
341        let get_connector_from_property = |property: &Property| -> String {
342            property
343                .0
344                .get(UPSTREAM_SOURCE_KEY)
345                .cloned()
346                .unwrap_or_default()
347        };
348
349        let source_report: Vec<jsonbb::Value> = source_props_and_info
350            .iter()
351            .map(|(oid, property, info)| {
352                let connector_name = get_connector_from_property(property);
353                let mut format = None;
354                let mut encode = None;
355                if let Some(info) = info {
356                    let pb_info = info.to_protobuf();
357                    format = Some(pb_info.format().as_str_name());
358                    encode = Some(pb_info.row_encode().as_str_name());
359                }
360                jsonbb::json!({
361                    oid.to_string(): {
362                        "connector": connector_name,
363                        "format": format,
364                        "encode": encode,
365                    },
366                })
367            })
368            .collect_vec();
369
370        let sink_report: Vec<jsonbb::Value> = sink_props_and_info
371            .iter()
372            .map(|(oid, property, info)| {
373                let connector_name = get_connector_from_property(property);
374                let mut format = None;
375                let mut encode = None;
376                if let Some(info) = info {
377                    let pb_info = info.to_protobuf();
378                    format = Some(pb_info.format().as_str_name());
379                    encode = Some(pb_info.encode().as_str_name());
380                }
381                jsonbb::json!({
382                    oid.to_string(): {
383                        "connector": connector_name,
384                        "format": format,
385                        "encode": encode,
386                    },
387                })
388            })
389            .collect_vec();
390
391        Ok(jsonbb::json!({
392                "source": source_report,
393                "sink": sink_report,
394        }))
395    }
396
397    pub async fn clean_dirty_subscription(
398        &self,
399        database_id: Option<DatabaseId>,
400    ) -> MetaResult<()> {
401        let inner = self.inner.write().await;
402        let txn = inner.db.begin().await?;
403        let filter_condition = object::Column::ObjType.eq(ObjectType::Subscription).and(
404            object::Column::Oid.not_in_subquery(
405                Query::select()
406                    .column(subscription::Column::SubscriptionId)
407                    .from(Subscription)
408                    .and_where(
409                        subscription::Column::SubscriptionState
410                            .eq(SubscriptionState::Created as i32),
411                    )
412                    .take(),
413            ),
414        );
415
416        let filter_condition = if let Some(database_id) = database_id {
417            filter_condition.and(object::Column::DatabaseId.eq(database_id))
418        } else {
419            filter_condition
420        };
421        Object::delete_many()
422            .filter(filter_condition)
423            .exec(&txn)
424            .await?;
425        txn.commit().await?;
426        // We don't need to notify the frontend, because the Init subscription is not send to frontend.
427        Ok(())
428    }
429
430    /// `clean_dirty_creating_jobs` cleans up creating jobs that are creating in Foreground mode or in Initial status.
431    pub async fn clean_dirty_creating_jobs(
432        &self,
433        database_id: Option<DatabaseId>,
434    ) -> MetaResult<Vec<SourceId>> {
435        let inner = self.inner.write().await;
436        let txn = inner.db.begin().await?;
437
438        let filter_condition = streaming_job::Column::JobStatus.eq(JobStatus::Initial).or(
439            streaming_job::Column::JobStatus
440                .eq(JobStatus::Creating)
441                .and(streaming_job::Column::CreateType.eq(CreateType::Foreground)),
442        );
443
444        let filter_condition = if let Some(database_id) = database_id {
445            filter_condition.and(object::Column::DatabaseId.eq(database_id))
446        } else {
447            filter_condition
448        };
449
450        let mut dirty_job_objs: Vec<PartialObject> = streaming_job::Entity::find()
451            .select_only()
452            .column(streaming_job::Column::JobId)
453            .columns([
454                object::Column::Oid,
455                object::Column::ObjType,
456                object::Column::SchemaId,
457                object::Column::DatabaseId,
458            ])
459            .join(JoinType::InnerJoin, streaming_job::Relation::Object.def())
460            .filter(filter_condition)
461            .into_partial_model()
462            .all(&txn)
463            .await?;
464
465        // Check if there are any pending iceberg table jobs.
466        let dirty_iceberg_jobs = find_dirty_iceberg_table_jobs(&txn, database_id).await?;
467        if !dirty_iceberg_jobs.is_empty() {
468            dirty_job_objs.extend(dirty_iceberg_jobs);
469        }
470
471        Self::clean_dirty_sink_downstreams(&txn).await?;
472
473        if dirty_job_objs.is_empty() {
474            return Ok(vec![]);
475        }
476
477        self.log_cleaned_dirty_jobs(&dirty_job_objs, &txn).await?;
478
479        let dirty_job_ids = dirty_job_objs.iter().map(|obj| obj.oid).collect::<Vec<_>>();
480
481        // Filter out dummy objs for replacement.
482        // todo: we'd better introduce a new dummy object type for replacement.
483        let all_dirty_table_ids = dirty_job_objs
484            .iter()
485            .filter(|obj| obj.obj_type == ObjectType::Table)
486            .map(|obj| obj.oid)
487            .collect_vec();
488        let dirty_table_type_map: HashMap<ObjectId, TableType> = Table::find()
489            .select_only()
490            .column(table::Column::TableId)
491            .column(table::Column::TableType)
492            .filter(table::Column::TableId.is_in(all_dirty_table_ids))
493            .into_tuple::<(ObjectId, TableType)>()
494            .all(&txn)
495            .await?
496            .into_iter()
497            .collect();
498
499        let dirty_background_jobs: HashSet<ObjectId> = streaming_job::Entity::find()
500            .select_only()
501            .column(streaming_job::Column::JobId)
502            .filter(
503                streaming_job::Column::JobId
504                    .is_in(dirty_job_ids.clone())
505                    .and(streaming_job::Column::CreateType.eq(CreateType::Background)),
506            )
507            .into_tuple()
508            .all(&txn)
509            .await?
510            .into_iter()
511            .collect();
512
513        // notify delete for failed materialized views and background jobs.
514        let to_notify_objs = dirty_job_objs
515            .into_iter()
516            .filter(|obj| {
517                matches!(
518                    dirty_table_type_map.get(&obj.oid),
519                    Some(TableType::MaterializedView)
520                ) || dirty_background_jobs.contains(&obj.oid)
521            })
522            .collect_vec();
523
524        // The source ids for dirty tables with connector.
525        // FIXME: we should also clean dirty sources.
526        let dirty_associated_source_ids: Vec<SourceId> = Table::find()
527            .select_only()
528            .column(table::Column::OptionalAssociatedSourceId)
529            .filter(
530                table::Column::TableId
531                    .is_in(dirty_job_ids.clone())
532                    .and(table::Column::OptionalAssociatedSourceId.is_not_null()),
533            )
534            .into_tuple()
535            .all(&txn)
536            .await?;
537
538        let dirty_state_table_ids: Vec<TableId> = Table::find()
539            .select_only()
540            .column(table::Column::TableId)
541            .filter(table::Column::BelongsToJobId.is_in(dirty_job_ids.clone()))
542            .into_tuple()
543            .all(&txn)
544            .await?;
545
546        let dirty_internal_table_objs = Object::find()
547            .select_only()
548            .columns([
549                object::Column::Oid,
550                object::Column::ObjType,
551                object::Column::SchemaId,
552                object::Column::DatabaseId,
553            ])
554            .join(JoinType::InnerJoin, object::Relation::Table.def())
555            .filter(table::Column::BelongsToJobId.is_in(to_notify_objs.iter().map(|obj| obj.oid)))
556            .into_partial_model()
557            .all(&txn)
558            .await?;
559
560        let to_delete_objs: HashSet<ObjectId> = dirty_job_ids
561            .clone()
562            .into_iter()
563            .chain(
564                dirty_state_table_ids
565                    .into_iter()
566                    .map(|table_id| table_id.as_object_id()),
567            )
568            .chain(
569                dirty_associated_source_ids
570                    .iter()
571                    .map(|source_id| source_id.as_object_id()),
572            )
573            .collect();
574
575        let res = Object::delete_many()
576            .filter(object::Column::Oid.is_in(to_delete_objs))
577            .exec(&txn)
578            .await?;
579        assert!(res.rows_affected > 0);
580
581        txn.commit().await?;
582
583        let object_group = build_object_group_for_delete(
584            to_notify_objs
585                .into_iter()
586                .chain(dirty_internal_table_objs.into_iter())
587                .collect_vec(),
588        );
589
590        let _version = self
591            .notify_frontend(NotificationOperation::Delete, object_group)
592            .await;
593
594        Ok(dirty_associated_source_ids)
595    }
596
597    pub async fn comment_on(&self, comment: PbComment) -> MetaResult<NotificationVersion> {
598        let inner = self.inner.write().await;
599        let txn = inner.db.begin().await?;
600        ensure_object_id(ObjectType::Database, comment.database_id, &txn).await?;
601        ensure_object_id(ObjectType::Schema, comment.schema_id, &txn).await?;
602        let (table_obj, streaming_job) = Object::find_by_id(comment.table_id)
603            .find_also_related(StreamingJob)
604            .one(&txn)
605            .await?
606            .ok_or_else(|| MetaError::catalog_id_not_found("object", comment.table_id))?;
607
608        let table = if let Some(col_idx) = comment.column_index {
609            let columns: ColumnCatalogArray = Table::find_by_id(comment.table_id)
610                .select_only()
611                .column(table::Column::Columns)
612                .into_tuple()
613                .one(&txn)
614                .await?
615                .ok_or_else(|| MetaError::catalog_id_not_found("table", comment.table_id))?;
616            let mut pb_columns = columns.to_protobuf();
617
618            let column = pb_columns
619                .get_mut(col_idx as usize)
620                .ok_or_else(|| MetaError::catalog_id_not_found("column", col_idx))?;
621            let column_desc = column.column_desc.as_mut().ok_or_else(|| {
622                anyhow!(
623                    "column desc at index {} for table id {} not found",
624                    col_idx,
625                    comment.table_id
626                )
627            })?;
628            column_desc.description = comment.description;
629            table::ActiveModel {
630                table_id: Set(comment.table_id),
631                columns: Set(pb_columns.into()),
632                ..Default::default()
633            }
634            .update(&txn)
635            .await?
636        } else {
637            table::ActiveModel {
638                table_id: Set(comment.table_id),
639                description: Set(comment.description),
640                ..Default::default()
641            }
642            .update(&txn)
643            .await?
644        };
645        txn.commit().await?;
646
647        let version = self
648            .notify_frontend_relation_info(
649                NotificationOperation::Update,
650                PbObjectInfo::Table(ObjectModel(table, table_obj, streaming_job).into()),
651            )
652            .await;
653
654        Ok(version)
655    }
656
657    async fn notify_hummock_dropped_tables(&self, tables: Vec<PbTable>) {
658        if tables.is_empty() {
659            return;
660        }
661        let objects = tables
662            .into_iter()
663            .map(|t| PbObject {
664                object_info: Some(PbObjectInfo::Table(t)),
665            })
666            .collect();
667        let group = NotificationInfo::ObjectGroup(PbObjectGroup { objects });
668        self.env
669            .notification_manager()
670            .notify_hummock(NotificationOperation::Delete, group.clone())
671            .await;
672        self.env
673            .notification_manager()
674            .notify_compactor(NotificationOperation::Delete, group)
675            .await;
676    }
677
678    pub async fn complete_dropped_tables(&self, table_ids: impl IntoIterator<Item = TableId>) {
679        let mut inner = self.inner.write().await;
680        let tables = inner.complete_dropped_tables(table_ids);
681        self.notify_hummock_dropped_tables(tables).await;
682    }
683
684    pub async fn cleanup_dropped_tables(&self) {
685        let mut inner = self.inner.write().await;
686        let tables = inner.dropped_tables.drain().map(|(_, t)| t).collect();
687        self.notify_hummock_dropped_tables(tables).await;
688    }
689
690    pub async fn stats(&self) -> MetaResult<CatalogStats> {
691        let inner = self.inner.read().await;
692
693        let mut table_num_map: HashMap<_, _> = Table::find()
694            .select_only()
695            .column(table::Column::TableType)
696            .column_as(table::Column::TableId.count(), "num")
697            .group_by(table::Column::TableType)
698            .having(table::Column::TableType.ne(TableType::Internal))
699            .into_tuple::<(TableType, i64)>()
700            .all(&inner.db)
701            .await?
702            .into_iter()
703            .map(|(table_type, num)| (table_type, num as u64))
704            .collect();
705
706        let source_num = Source::find().count(&inner.db).await?;
707        let sink_num = Sink::find().count(&inner.db).await?;
708        let function_num = Function::find().count(&inner.db).await?;
709        let streaming_job_num = StreamingJob::find().count(&inner.db).await?;
710
711        let actor_num = {
712            let guard = self.env.shared_actor_info.read_guard();
713            guard
714                .iter_over_fragments()
715                .map(|(_, fragment)| fragment.actors.len() as u64)
716                .sum::<u64>()
717        };
718        let database_num = Database::find().count(&inner.db).await?;
719
720        Ok(CatalogStats {
721            table_num: table_num_map.remove(&TableType::Table).unwrap_or(0),
722            mview_num: table_num_map
723                .remove(&TableType::MaterializedView)
724                .unwrap_or(0),
725            index_num: table_num_map.remove(&TableType::Index).unwrap_or(0),
726            source_num,
727            sink_num,
728            function_num,
729            streaming_job_num,
730            actor_num,
731            database_num,
732        })
733    }
734
735    pub async fn fetch_sink_with_state_table_ids(
736        &self,
737        sink_ids: HashSet<SinkId>,
738    ) -> MetaResult<HashMap<SinkId, Vec<TableId>>> {
739        let inner = self.inner.read().await;
740
741        let query = Fragment::find()
742            .select_only()
743            .columns([fragment::Column::JobId, fragment::Column::StateTableIds])
744            .filter(
745                fragment::Column::JobId
746                    .is_in(sink_ids)
747                    .and(FragmentTypeMask::intersects(FragmentTypeFlag::Sink)),
748            );
749
750        let rows: Vec<(JobId, TableIdArray)> = query.into_tuple().all(&inner.db).await?;
751
752        debug_assert!(rows.iter().map(|(job_id, _)| job_id).all_unique());
753
754        let result = rows
755            .into_iter()
756            .map(|(job_id, table_id_array)| (job_id.as_sink_id(), table_id_array.0))
757            .collect::<HashMap<_, _>>();
758
759        Ok(result)
760    }
761
762    pub async fn list_all_pending_sinks(
763        &self,
764        database_id: Option<DatabaseId>,
765    ) -> MetaResult<HashSet<SinkId>> {
766        let inner = self.inner.read().await;
767
768        let mut query = pending_sink_state::Entity::find()
769            .select_only()
770            .columns([pending_sink_state::Column::SinkId])
771            .filter(
772                pending_sink_state::Column::SinkState.eq(pending_sink_state::SinkState::Pending),
773            )
774            .distinct();
775
776        if let Some(db_id) = database_id {
777            query = query
778                .join(
779                    JoinType::InnerJoin,
780                    pending_sink_state::Relation::Object.def(),
781                )
782                .filter(object::Column::DatabaseId.eq(db_id));
783        }
784
785        let result: Vec<SinkId> = query.into_tuple().all(&inner.db).await?;
786
787        Ok(result.into_iter().collect())
788    }
789
790    pub async fn abort_pending_sink_epochs(
791        &self,
792        sink_committed_epoch: HashMap<SinkId, u64>,
793    ) -> MetaResult<()> {
794        let inner = self.inner.write().await;
795        let txn = inner.db.begin().await?;
796
797        for (sink_id, committed_epoch) in sink_committed_epoch {
798            pending_sink_state::Entity::update_many()
799                .col_expr(
800                    pending_sink_state::Column::SinkState,
801                    Expr::value(pending_sink_state::SinkState::Aborted),
802                )
803                .filter(
804                    pending_sink_state::Column::SinkId
805                        .eq(sink_id)
806                        .and(pending_sink_state::Column::Epoch.gt(committed_epoch as i64)),
807                )
808                .exec(&txn)
809                .await?;
810        }
811
812        txn.commit().await?;
813        Ok(())
814    }
815}
816
817/// `CatalogStats` is a struct to store the statistics of all catalogs.
818pub struct CatalogStats {
819    pub table_num: u64,
820    pub mview_num: u64,
821    pub index_num: u64,
822    pub source_num: u64,
823    pub sink_num: u64,
824    pub function_num: u64,
825    pub streaming_job_num: u64,
826    pub actor_num: u64,
827    pub database_num: u64,
828}
829
830impl CatalogControllerInner {
831    pub async fn snapshot(&self) -> MetaResult<(Catalog, Vec<PbUserInfo>)> {
832        let databases = self.list_databases().await?;
833        let schemas = self.list_schemas().await?;
834        let tables = self.list_tables().await?;
835        let sources = self.list_sources().await?;
836        let sinks = self.list_sinks().await?;
837        let subscriptions = self.list_subscriptions().await?;
838        let indexes = self.list_indexes().await?;
839        let views = self.list_views().await?;
840        let functions = self.list_functions().await?;
841        let connections = self.list_connections().await?;
842        let secrets = self.list_secrets().await?;
843
844        let users = self.list_users().await?;
845
846        Ok((
847            (
848                databases,
849                schemas,
850                tables,
851                sources,
852                sinks,
853                subscriptions,
854                indexes,
855                views,
856                functions,
857                connections,
858                secrets,
859            ),
860            users,
861        ))
862    }
863
864    async fn list_databases(&self) -> MetaResult<Vec<PbDatabase>> {
865        let db_objs = Database::find()
866            .find_also_related(Object)
867            .all(&self.db)
868            .await?;
869        Ok(db_objs
870            .into_iter()
871            .map(|(db, obj)| ObjectModel(db, obj.unwrap(), None).into())
872            .collect())
873    }
874
875    async fn list_schemas(&self) -> MetaResult<Vec<PbSchema>> {
876        let schema_objs = Schema::find()
877            .find_also_related(Object)
878            .all(&self.db)
879            .await?;
880
881        Ok(schema_objs
882            .into_iter()
883            .map(|(schema, obj)| ObjectModel(schema, obj.unwrap(), None).into())
884            .collect())
885    }
886
887    async fn list_users(&self) -> MetaResult<Vec<PbUserInfo>> {
888        let mut user_infos: Vec<PbUserInfo> = User::find()
889            .all(&self.db)
890            .await?
891            .into_iter()
892            .map(Into::into)
893            .collect();
894
895        for user_info in &mut user_infos {
896            user_info.grant_privileges = get_user_privilege(user_info.id as _, &self.db).await?;
897        }
898        Ok(user_infos)
899    }
900
901    /// `list_all_tables` return all tables and internal tables.
902    pub async fn list_all_state_tables(&self) -> MetaResult<Vec<PbTable>> {
903        let table_objs = Table::find()
904            .find_also_related(Object)
905            .all(&self.db)
906            .await?;
907        let streaming_jobs = load_streaming_jobs_by_ids(
908            &self.db,
909            table_objs.iter().map(|(table, _)| table.job_id()),
910        )
911        .await?;
912
913        Ok(table_objs
914            .into_iter()
915            .map(|(table, obj)| {
916                let job_id = table.job_id();
917                let streaming_job = streaming_jobs.get(&job_id).cloned();
918                ObjectModel(table, obj.unwrap(), streaming_job).into()
919            })
920            .collect())
921    }
922
923    /// `list_tables` return all `CREATED` tables, `CREATING` materialized views/ `BACKGROUND` jobs and internal tables that belong to them and sinks.
924    async fn list_tables(&self) -> MetaResult<Vec<PbTable>> {
925        let mut table_objs = Table::find()
926            .find_also_related(Object)
927            .all(&self.db)
928            .await?;
929
930        let all_streaming_jobs: HashMap<JobId, streaming_job::Model> = StreamingJob::find()
931            .all(&self.db)
932            .await?
933            .into_iter()
934            .map(|job| (job.job_id, job))
935            .collect();
936
937        let sink_ids: HashSet<SinkId> = Sink::find()
938            .select_only()
939            .column(sink::Column::SinkId)
940            .into_tuple::<SinkId>()
941            .all(&self.db)
942            .await?
943            .into_iter()
944            .collect();
945
946        let mview_job_ids: HashSet<JobId> = table_objs
947            .iter()
948            .filter_map(|(table, _)| {
949                if table.table_type == TableType::MaterializedView {
950                    Some(table.table_id.as_job_id())
951                } else {
952                    None
953                }
954            })
955            .collect();
956
957        table_objs.retain(|(table, _)| {
958            let job_id = table.job_id();
959
960            if sink_ids.contains(&job_id.as_sink_id()) || mview_job_ids.contains(&job_id) {
961                return true;
962            }
963            if let Some(streaming_job) = all_streaming_jobs.get(&job_id) {
964                return streaming_job.job_status == JobStatus::Created
965                    || (streaming_job.create_type == CreateType::Background);
966            }
967            false
968        });
969
970        let mut tables = Vec::with_capacity(table_objs.len());
971        for (table, obj) in table_objs {
972            let job_id = table.job_id();
973            let streaming_job = all_streaming_jobs.get(&job_id).cloned();
974            let pb_table: PbTable = ObjectModel(table, obj.unwrap(), streaming_job).into();
975            tables.push(pb_table);
976        }
977        Ok(tables)
978    }
979
980    /// `list_sources` return all sources and `CREATED` ones if contains any streaming jobs.
981    async fn list_sources(&self) -> MetaResult<Vec<PbSource>> {
982        let mut source_objs = Source::find()
983            .find_also_related(Object)
984            .join(JoinType::LeftJoin, object::Relation::StreamingJob.def())
985            .filter(
986                streaming_job::Column::JobStatus
987                    .is_null()
988                    .or(streaming_job::Column::JobStatus.eq(JobStatus::Created)),
989            )
990            .all(&self.db)
991            .await?;
992
993        // filter out inner connector sources that are still under creating.
994        let created_table_ids: HashSet<TableId> = Table::find()
995            .select_only()
996            .column(table::Column::TableId)
997            .join(JoinType::InnerJoin, table::Relation::Object1.def())
998            .join(JoinType::LeftJoin, object::Relation::StreamingJob.def())
999            .filter(
1000                table::Column::OptionalAssociatedSourceId
1001                    .is_not_null()
1002                    .and(streaming_job::Column::JobStatus.eq(JobStatus::Created)),
1003            )
1004            .into_tuple()
1005            .all(&self.db)
1006            .await?
1007            .into_iter()
1008            .collect();
1009        source_objs.retain_mut(|(source, _)| {
1010            source.optional_associated_table_id.is_none()
1011                || created_table_ids.contains(&source.optional_associated_table_id.unwrap())
1012        });
1013
1014        Ok(source_objs
1015            .into_iter()
1016            .map(|(source, obj)| ObjectModel(source, obj.unwrap(), None).into())
1017            .collect())
1018    }
1019
1020    /// `list_sinks` return all sinks.
1021    async fn list_sinks(&self) -> MetaResult<Vec<PbSink>> {
1022        let sink_objs = Sink::find()
1023            .find_also_related(Object)
1024            .join(JoinType::LeftJoin, object::Relation::StreamingJob.def())
1025            .all(&self.db)
1026            .await?;
1027        let streaming_jobs = load_streaming_jobs_by_ids(
1028            &self.db,
1029            sink_objs.iter().map(|(sink, _)| sink.sink_id.as_job_id()),
1030        )
1031        .await?;
1032
1033        Ok(sink_objs
1034            .into_iter()
1035            .map(|(sink, obj)| {
1036                let streaming_job = streaming_jobs.get(&sink.sink_id.as_job_id()).cloned();
1037                ObjectModel(sink, obj.unwrap(), streaming_job).into()
1038            })
1039            .collect())
1040    }
1041
1042    /// `list_subscriptions` return all `CREATED` subscriptions.
1043    async fn list_subscriptions(&self) -> MetaResult<Vec<PbSubscription>> {
1044        let subscription_objs = Subscription::find()
1045            .find_also_related(Object)
1046            .filter(subscription::Column::SubscriptionState.eq(SubscriptionState::Created as i32))
1047            .all(&self.db)
1048            .await?;
1049
1050        Ok(subscription_objs
1051            .into_iter()
1052            .map(|(subscription, obj)| ObjectModel(subscription, obj.unwrap(), None).into())
1053            .collect())
1054    }
1055
1056    async fn list_views(&self) -> MetaResult<Vec<PbView>> {
1057        let view_objs = View::find().find_also_related(Object).all(&self.db).await?;
1058
1059        Ok(view_objs
1060            .into_iter()
1061            .map(|(view, obj)| ObjectModel(view, obj.unwrap(), None).into())
1062            .collect())
1063    }
1064
1065    /// `list_indexes` return all `CREATED` and `BACKGROUND` indexes.
1066    async fn list_indexes(&self) -> MetaResult<Vec<PbIndex>> {
1067        let index_objs = Index::find()
1068            .find_also_related(Object)
1069            .join(JoinType::LeftJoin, object::Relation::StreamingJob.def())
1070            .filter(
1071                streaming_job::Column::JobStatus
1072                    .eq(JobStatus::Created)
1073                    .or(streaming_job::Column::CreateType.eq(CreateType::Background)),
1074            )
1075            .all(&self.db)
1076            .await?;
1077        let streaming_jobs = load_streaming_jobs_by_ids(
1078            &self.db,
1079            index_objs
1080                .iter()
1081                .map(|(index, _)| index.index_id.as_job_id()),
1082        )
1083        .await?;
1084
1085        Ok(index_objs
1086            .into_iter()
1087            .map(|(index, obj)| {
1088                let streaming_job = streaming_jobs.get(&index.index_id.as_job_id()).cloned();
1089                ObjectModel(index, obj.unwrap(), streaming_job).into()
1090            })
1091            .collect())
1092    }
1093
1094    async fn list_connections(&self) -> MetaResult<Vec<PbConnection>> {
1095        let conn_objs = Connection::find()
1096            .find_also_related(Object)
1097            .all(&self.db)
1098            .await?;
1099
1100        Ok(conn_objs
1101            .into_iter()
1102            .map(|(conn, obj)| ObjectModel(conn, obj.unwrap(), None).into())
1103            .collect())
1104    }
1105
1106    pub async fn list_secrets(&self) -> MetaResult<Vec<PbSecret>> {
1107        let secret_objs = Secret::find()
1108            .find_also_related(Object)
1109            .all(&self.db)
1110            .await?;
1111        Ok(secret_objs
1112            .into_iter()
1113            .map(|(secret, obj)| ObjectModel(secret, obj.unwrap(), None).into())
1114            .collect())
1115    }
1116
1117    async fn list_functions(&self) -> MetaResult<Vec<PbFunction>> {
1118        let func_objs = Function::find()
1119            .find_also_related(Object)
1120            .all(&self.db)
1121            .await?;
1122
1123        Ok(func_objs
1124            .into_iter()
1125            .map(|(func, obj)| ObjectModel(func, obj.unwrap(), None).into())
1126            .collect())
1127    }
1128
1129    pub(crate) fn register_finish_notifier(
1130        &mut self,
1131        database_id: DatabaseId,
1132        id: JobId,
1133        sender: Sender<Result<NotificationVersion, String>>,
1134    ) {
1135        self.creating_table_finish_notifier
1136            .entry(database_id)
1137            .or_default()
1138            .entry(id)
1139            .or_default()
1140            .push(sender);
1141    }
1142
1143    pub(crate) async fn streaming_job_is_finished(&mut self, id: JobId) -> MetaResult<bool> {
1144        let status = StreamingJob::find()
1145            .select_only()
1146            .column(streaming_job::Column::JobStatus)
1147            .filter(streaming_job::Column::JobId.eq(id))
1148            .into_tuple::<JobStatus>()
1149            .one(&self.db)
1150            .await?;
1151
1152        status
1153            .map(|status| status == JobStatus::Created)
1154            .ok_or_else(|| {
1155                MetaError::catalog_id_not_found("streaming job", "may have been cancelled/dropped")
1156            })
1157    }
1158
1159    pub(crate) fn notify_finish_failed(&mut self, database_id: Option<DatabaseId>, err: String) {
1160        if let Some(database_id) = database_id {
1161            if let Some(creating_tables) = self.creating_table_finish_notifier.remove(&database_id)
1162            {
1163                for tx in creating_tables.into_values().flatten() {
1164                    let _ = tx.send(Err(err.clone()));
1165                }
1166            }
1167        } else {
1168            for tx in take(&mut self.creating_table_finish_notifier)
1169                .into_values()
1170                .flatten()
1171                .flat_map(|(_, txs)| txs.into_iter())
1172            {
1173                let _ = tx.send(Err(err.clone()));
1174            }
1175        }
1176    }
1177
1178    pub async fn list_time_travel_table_ids(&self) -> MetaResult<Vec<TableId>> {
1179        let table_ids: Vec<TableId> = Table::find()
1180            .select_only()
1181            .filter(table::Column::TableType.is_in(vec![
1182                TableType::Table,
1183                TableType::MaterializedView,
1184                TableType::Index,
1185            ]))
1186            .column(table::Column::TableId)
1187            .into_tuple()
1188            .all(&self.db)
1189            .await?;
1190        Ok(table_ids)
1191    }
1192
1193    /// Since the tables have been dropped from both meta store and streaming jobs, this method removes those table copies.
1194    /// Returns the removed table copies.
1195    pub(crate) fn complete_dropped_tables(
1196        &mut self,
1197        table_ids: impl IntoIterator<Item = TableId>,
1198    ) -> Vec<PbTable> {
1199        table_ids
1200            .into_iter()
1201            .filter_map(|table_id| {
1202                self.dropped_tables.remove(&table_id).map_or_else(
1203                    || {
1204                        tracing::warn!(%table_id, "table not found");
1205                        None
1206                    },
1207                    Some,
1208                )
1209            })
1210            .collect()
1211    }
1212}