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::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#[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 pub fn create_sql(&self) -> String {
61 self.definition.clone()
62 }
63
64 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 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
112impl SourceCatalog {
113 pub fn create_sql_purified(&self) -> String {
115 self.create_sql_ast_purified()
116 .and_then(|stmt| stmt.try_to_string().map_err(Into::into))
117 .unwrap_or_else(|_| self.create_sql())
118 }
119
120 pub fn create_sql_ast_purified(&self) -> Result<ast::Statement> {
124 match try_purify_table_source_create_sql_ast(
125 self.create_sql_ast()?,
126 &self.columns,
127 self.row_id_index,
128 &self.pk_col_ids,
129 ) {
130 Ok(stmt) => return Ok(stmt),
131 Err(e) => notice_to_user(format!(
132 "error occurred while purifying definition for source \"{}\", \
133 results may be inaccurate: {}",
134 self.name,
135 e.as_report()
136 )),
137 }
138
139 self.create_sql_ast()
140 }
141
142 pub fn fill_purified_create_sql(&mut self) {
148 self.definition = self.create_sql_purified();
149 }
150}
151
152impl From<&PbSource> for SourceCatalog {
153 fn from(prost: &PbSource) -> Self {
154 let id = prost.id;
155 let name = prost.name.clone();
156 let database_id = prost.database_id;
157 let schema_id = prost.schema_id;
158 let prost_columns = prost.columns.clone();
159 let pk_col_ids = prost
160 .pk_column_ids
161 .clone()
162 .into_iter()
163 .map(Into::into)
164 .collect();
165 let connector_props_with_secrets =
166 WithOptionsSecResolved::new(prost.with_properties.clone(), prost.secret_refs.clone());
167 let columns = prost_columns.into_iter().map(ColumnCatalog::from).collect();
168 let row_id_index = prost.row_id_index.map(|idx| idx as _);
169
170 let append_only = row_id_index.is_some();
171 let owner = prost.owner;
172 let watermark_descs = prost.get_watermark_descs().clone();
173
174 let associated_table_id = prost.optional_associated_table_id.map(|id| match id {
175 OptionalAssociatedTableId::AssociatedTableId(id) => id,
176 });
177 let version = prost.version;
178
179 let connection_id = prost.connection_id;
180 let rate_limit = prost.rate_limit;
181
182 Self {
183 id,
184 name,
185 schema_id,
186 database_id,
187 columns,
188 pk_col_ids,
189 append_only,
190 owner,
191 info: prost.info.clone().unwrap(),
192 row_id_index,
193 with_properties: connector_props_with_secrets,
194 watermark_descs,
195 associated_table_id: associated_table_id.map(|x| x.into()),
196 definition: prost.definition.clone(),
197 connection_id,
198 created_at_epoch: prost.created_at_epoch.map(Epoch::from),
199 initialized_at_epoch: prost.initialized_at_epoch.map(Epoch::from),
200 version,
201 created_at_cluster_version: prost.created_at_cluster_version.clone(),
202 initialized_at_cluster_version: prost.initialized_at_cluster_version.clone(),
203 rate_limit,
204 }
205 }
206}
207
208impl OwnedByUserCatalog for SourceCatalog {
209 fn owner(&self) -> UserId {
210 self.owner
211 }
212}