risingwave_frontend/catalog/
source_catalog.rs

1// Copyright 2025 RisingWave Labs
2//
3// Licensed under the Apache License, Version 2.0 (the "License");
4// you may not use this file except in compliance with the License.
5// You may obtain a copy of the License at
6//
7//     http://www.apache.org/licenses/LICENSE-2.0
8//
9// Unless required by applicable law or agreed to in writing, software
10// distributed under the License is distributed on an "AS IS" BASIS,
11// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12// See the License for the specific language governing permissions and
13// limitations under the License.
14
15use risingwave_common::catalog::{ColumnCatalog, SourceVersionId};
16use risingwave_common::util::epoch::Epoch;
17use risingwave_connector::{WithOptionsSecResolved, WithPropertiesExt};
18use risingwave_pb::catalog::source::OptionalAssociatedTableId;
19use risingwave_pb::catalog::{PbSource, StreamSourceInfo, WatermarkDesc};
20use risingwave_sqlparser::ast;
21use risingwave_sqlparser::parser::Parser;
22use thiserror_ext::AsReport as _;
23
24use super::purify::try_purify_table_source_create_sql_ast;
25use super::{ColumnId, ConnectionId, DatabaseId, OwnedByUserCatalog, SchemaId, SourceId};
26use crate::catalog::TableId;
27use crate::error::Result;
28use crate::session::current::notice_to_user;
29use crate::user::UserId;
30
31/// This struct `SourceCatalog` is used in frontend.
32/// Compared with `PbSource`, it only maintains information used during optimization.
33#[derive(Clone, Debug, PartialEq, Eq, Hash)]
34pub struct SourceCatalog {
35    pub id: SourceId,
36    pub name: String,
37    pub schema_id: SchemaId,
38    pub database_id: DatabaseId,
39    pub columns: Vec<ColumnCatalog>,
40    pub pk_col_ids: Vec<ColumnId>,
41    pub append_only: bool,
42    pub owner: UserId,
43    pub info: StreamSourceInfo,
44    pub row_id_index: Option<usize>,
45    pub with_properties: WithOptionsSecResolved,
46    pub watermark_descs: Vec<WatermarkDesc>,
47    pub associated_table_id: Option<TableId>,
48    pub definition: String,
49    pub connection_id: Option<ConnectionId>,
50    pub created_at_epoch: Option<Epoch>,
51    pub initialized_at_epoch: Option<Epoch>,
52    pub version: SourceVersionId,
53    pub created_at_cluster_version: Option<String>,
54    pub initialized_at_cluster_version: Option<String>,
55    pub rate_limit: Option<u32>,
56}
57
58impl SourceCatalog {
59    /// Returns the SQL definition when the source was created.
60    pub fn create_sql(&self) -> String {
61        self.definition.clone()
62    }
63
64    /// Returns the parsed SQL definition when the source was created.
65    ///
66    /// Returns error if it's invalid.
67    pub fn create_sql_ast(&self) -> Result<ast::Statement> {
68        Ok(Parser::parse_exactly_one(&self.definition)?)
69    }
70
71    pub fn to_prost(&self) -> PbSource {
72        let (with_properties, secret_refs) = self.with_properties.clone().into_parts();
73        PbSource {
74            id: self.id,
75            schema_id: self.schema_id,
76            database_id: self.database_id,
77            name: self.name.clone(),
78            row_id_index: self.row_id_index.map(|idx| idx as _),
79            columns: self.columns.iter().map(|c| c.to_protobuf()).collect(),
80            pk_column_ids: self.pk_col_ids.iter().map(Into::into).collect(),
81            with_properties,
82            owner: self.owner,
83            info: Some(self.info.clone()),
84            watermark_descs: self.watermark_descs.clone(),
85            definition: self.definition.clone(),
86            connection_id: self.connection_id,
87            initialized_at_epoch: self.initialized_at_epoch.map(|x| x.0),
88            created_at_epoch: self.created_at_epoch.map(|x| x.0),
89            optional_associated_table_id: self
90                .associated_table_id
91                .map(|id| OptionalAssociatedTableId::AssociatedTableId(id.table_id)),
92            version: self.version,
93            created_at_cluster_version: self.created_at_cluster_version.clone(),
94            initialized_at_cluster_version: self.initialized_at_cluster_version.clone(),
95            secret_refs,
96            rate_limit: self.rate_limit,
97        }
98    }
99
100    /// Get a reference to the source catalog's version.
101    pub fn version(&self) -> SourceVersionId {
102        self.version
103    }
104
105    pub fn connector_name(&self) -> String {
106        self.with_properties
107            .get_connector()
108            .expect("connector name is missing")
109    }
110
111    pub fn is_iceberg_connector(&self) -> bool {
112        self.with_properties.is_iceberg_connector()
113    }
114}
115
116impl SourceCatalog {
117    /// Returns the SQL definition when the source was created, purified with best effort.
118    pub fn create_sql_purified(&self) -> String {
119        self.create_sql_ast_purified()
120            .and_then(|stmt| stmt.try_to_string().map_err(Into::into))
121            .unwrap_or_else(|_| self.create_sql())
122    }
123
124    /// Returns the parsed SQL definition when the source was created, purified with best effort.
125    ///
126    /// Returns error if it's invalid.
127    pub fn create_sql_ast_purified(&self) -> Result<ast::Statement> {
128        if self.with_properties.is_cdc_connector() {
129            // For CDC sources, we should not purify the SQL definition to add column definitions
130            // or constraints.
131            return self.create_sql_ast();
132        }
133
134        match try_purify_table_source_create_sql_ast(
135            self.create_sql_ast()?,
136            &self.columns,
137            self.row_id_index,
138            &self.pk_col_ids,
139        ) {
140            Ok(stmt) => return Ok(stmt),
141            Err(e) => notice_to_user(format!(
142                "error occurred while purifying definition for source \"{}\", \
143                     results may be inaccurate: {}",
144                self.name,
145                e.as_report()
146            )),
147        }
148
149        self.create_sql_ast()
150    }
151
152    /// Fills the `definition` field with the purified SQL definition.
153    ///
154    /// There's no need to call this method for correctness because we automatically purify the
155    /// SQL definition at the time of querying. However, this helps to maintain more accurate
156    /// `definition` field in the catalog when directly inspected for debugging purposes.
157    pub fn fill_purified_create_sql(&mut self) {
158        self.definition = self.create_sql_purified();
159    }
160}
161
162impl From<&PbSource> for SourceCatalog {
163    fn from(prost: &PbSource) -> Self {
164        let id = prost.id;
165        let name = prost.name.clone();
166        let database_id = prost.database_id;
167        let schema_id = prost.schema_id;
168        let prost_columns = prost.columns.clone();
169        let pk_col_ids = prost
170            .pk_column_ids
171            .clone()
172            .into_iter()
173            .map(Into::into)
174            .collect();
175        let connector_props_with_secrets =
176            WithOptionsSecResolved::new(prost.with_properties.clone(), prost.secret_refs.clone());
177        let columns = prost_columns.into_iter().map(ColumnCatalog::from).collect();
178        let row_id_index = prost.row_id_index.map(|idx| idx as _);
179
180        let append_only = row_id_index.is_some();
181        let owner = prost.owner;
182        let watermark_descs = prost.get_watermark_descs().clone();
183
184        let associated_table_id = prost.optional_associated_table_id.map(|id| match id {
185            OptionalAssociatedTableId::AssociatedTableId(id) => id,
186        });
187        let version = prost.version;
188
189        let connection_id = prost.connection_id;
190        let rate_limit = prost.rate_limit;
191
192        Self {
193            id,
194            name,
195            schema_id,
196            database_id,
197            columns,
198            pk_col_ids,
199            append_only,
200            owner,
201            info: prost.info.clone().unwrap(),
202            row_id_index,
203            with_properties: connector_props_with_secrets,
204            watermark_descs,
205            associated_table_id: associated_table_id.map(|x| x.into()),
206            definition: prost.definition.clone(),
207            connection_id,
208            created_at_epoch: prost.created_at_epoch.map(Epoch::from),
209            initialized_at_epoch: prost.initialized_at_epoch.map(Epoch::from),
210            version,
211            created_at_cluster_version: prost.created_at_cluster_version.clone(),
212            initialized_at_cluster_version: prost.initialized_at_cluster_version.clone(),
213            rate_limit,
214        }
215    }
216}
217
218impl OwnedByUserCatalog for SourceCatalog {
219    fn owner(&self) -> UserId {
220        self.owner
221    }
222}