risingwave_frontend/catalog/
source_catalog.rs1use risingwave_common::catalog::{ColumnCatalog, SourceVersionId};
16use risingwave_common::util::epoch::Epoch;
17use risingwave_connector::{WithOptionsSecResolved, WithPropertiesExt};
18use risingwave_pb::catalog::{PbSource, StreamSourceInfo, WatermarkDesc};
19use risingwave_pb::plan_common::SourceRefreshMode;
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#[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 pub refresh_mode: Option<SourceRefreshMode>,
57}
58
59impl SourceCatalog {
60 pub fn create_sql(&self) -> String {
62 self.definition.clone()
63 }
64
65 pub fn create_sql_ast(&self) -> Result<ast::Statement> {
69 Ok(Parser::parse_exactly_one(&self.definition)?)
70 }
71
72 pub fn to_prost(&self) -> PbSource {
73 let (with_properties, secret_refs) = self.with_properties.clone().into_parts();
74 PbSource {
75 id: self.id,
76 schema_id: self.schema_id,
77 database_id: self.database_id,
78 name: self.name.clone(),
79 row_id_index: self.row_id_index.map(|idx| idx as _),
80 columns: self.columns.iter().map(|c| c.to_protobuf()).collect(),
81 pk_column_ids: self.pk_col_ids.iter().map(Into::into).collect(),
82 with_properties,
83 owner: self.owner,
84 info: Some(self.info.clone()),
85 watermark_descs: self.watermark_descs.clone(),
86 definition: self.definition.clone(),
87 connection_id: self.connection_id,
88 initialized_at_epoch: self.initialized_at_epoch.map(|x| x.0),
89 created_at_epoch: self.created_at_epoch.map(|x| x.0),
90 optional_associated_table_id: self.associated_table_id.map(Into::into),
91 version: self.version,
92 created_at_cluster_version: self.created_at_cluster_version.clone(),
93 initialized_at_cluster_version: self.initialized_at_cluster_version.clone(),
94 secret_refs,
95 rate_limit: self.rate_limit,
96 refresh_mode: self.refresh_mode,
97 }
98 }
99
100 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 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 pub fn create_sql_ast_purified(&self) -> Result<ast::Statement> {
128 if self.with_properties.is_cdc_connector() {
129 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 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(Into::into);
185 let version = prost.version;
186
187 let connection_id = prost.connection_id;
188 let rate_limit = prost.rate_limit;
189
190 Self {
191 id,
192 name,
193 schema_id,
194 database_id,
195 columns,
196 pk_col_ids,
197 append_only,
198 owner,
199 info: prost.info.clone().unwrap(),
200 row_id_index,
201 with_properties: connector_props_with_secrets,
202 watermark_descs,
203 associated_table_id,
204 definition: prost.definition.clone(),
205 connection_id,
206 created_at_epoch: prost.created_at_epoch.map(Epoch::from),
207 initialized_at_epoch: prost.initialized_at_epoch.map(Epoch::from),
208 version,
209 created_at_cluster_version: prost.created_at_cluster_version.clone(),
210 initialized_at_cluster_version: prost.initialized_at_cluster_version.clone(),
211 rate_limit,
212 refresh_mode: prost.refresh_mode,
213 }
214 }
215}
216
217impl OwnedByUserCatalog for SourceCatalog {
218 fn owner(&self) -> UserId {
219 self.owner
220 }
221}