risingwave_frontend/handler/
alter_source_with_sr.rs1use std::sync::Arc;
16
17use either::Either;
18use itertools::Itertools;
19use pgwire::pg_response::StatementType;
20use risingwave_common::bail_not_implemented;
21use risingwave_common::catalog::{ColumnCatalog, max_column_id};
22use risingwave_connector::WithPropertiesExt;
23use risingwave_pb::catalog::StreamSourceInfo;
24use risingwave_pb::plan_common::{EncodeType, FormatType};
25use risingwave_sqlparser::ast::{
26 CompatibleFormatEncode, CreateSourceStatement, Encode, Format, FormatEncodeOptions, ObjectName,
27 SqlOption, Statement,
28};
29
30use super::create_source::{
31 generate_stream_graph_for_source, schema_has_schema_registry, validate_compatibility,
32};
33use super::util::SourceSchemaCompatExt;
34use super::{HandlerArgs, RwPgResponse};
35use crate::catalog::root_catalog::SchemaPath;
36use crate::catalog::source_catalog::SourceCatalog;
37use crate::error::{ErrorCode, Result};
38use crate::handler::create_source::{CreateSourceType, bind_columns_from_source};
39use crate::session::SessionImpl;
40use crate::utils::resolve_secret_ref_in_with_options;
41use crate::{Binder, WithOptions};
42
43fn format_type_to_format(from: FormatType) -> Option<Format> {
44 Some(match from {
45 FormatType::Unspecified => return None,
46 FormatType::Native => Format::Native,
47 FormatType::Debezium => Format::Debezium,
48 FormatType::DebeziumMongo => Format::DebeziumMongo,
49 FormatType::Maxwell => Format::Maxwell,
50 FormatType::Canal => Format::Canal,
51 FormatType::Upsert => Format::Upsert,
52 FormatType::Plain => Format::Plain,
53 FormatType::None => Format::None,
54 })
55}
56
57fn encode_type_to_encode(from: EncodeType) -> Option<Encode> {
58 Some(match from {
59 EncodeType::Unspecified => return None,
60 EncodeType::Native => Encode::Native,
61 EncodeType::Avro => Encode::Avro,
62 EncodeType::Csv => Encode::Csv,
63 EncodeType::Protobuf => Encode::Protobuf,
64 EncodeType::Json => Encode::Json,
65 EncodeType::Bytes => Encode::Bytes,
66 EncodeType::Template => Encode::Template,
67 EncodeType::Parquet => Encode::Parquet,
68 EncodeType::None => Encode::None,
69 EncodeType::Text => Encode::Text,
70 })
71}
72
73fn columns_minus(columns_a: &[ColumnCatalog], columns_b: &[ColumnCatalog]) -> Vec<ColumnCatalog> {
85 columns_a
86 .iter()
87 .filter(|col_a| {
88 !col_a.is_hidden()
89 && !col_a.is_connector_additional_column()
90 && !col_a.is_generated()
91 && !columns_b.iter().any(|col_b| {
92 col_a.name() == col_b.name() && col_a.data_type() == col_b.data_type()
93 })
94 })
95 .cloned()
96 .collect()
97}
98
99pub fn fetch_source_catalog_with_db_schema_id(
101 session: &SessionImpl,
102 name: &ObjectName,
103) -> Result<Arc<SourceCatalog>> {
104 let db_name = &session.database();
105 let (schema_name, real_source_name) =
106 Binder::resolve_schema_qualified_name(db_name, name.clone())?;
107 let search_path = session.config().search_path();
108 let user_name = &session.user_name();
109
110 let schema_path = SchemaPath::new(schema_name.as_deref(), &search_path, user_name);
111
112 let reader = session.env().catalog_reader().read_guard();
113 let (source, schema_name) =
114 reader.get_source_by_name(db_name, schema_path, &real_source_name)?;
115
116 session.check_privilege_for_drop_alter(schema_name, &**source)?;
117
118 Ok(Arc::clone(source))
119}
120
121pub fn check_format_encode(
124 original_source: &SourceCatalog,
125 new_format_encode: &FormatEncodeOptions,
126) -> Result<()> {
127 let StreamSourceInfo {
128 format, row_encode, ..
129 } = original_source.info;
130 let (Some(old_format), Some(old_row_encode)) = (
131 format_type_to_format(FormatType::try_from(format).unwrap()),
132 encode_type_to_encode(EncodeType::try_from(row_encode).unwrap()),
133 ) else {
134 return Err(ErrorCode::NotSupported(
135 "altering a legacy source which is not created using `FORMAT .. ENCODE ..` Clause"
136 .to_owned(),
137 "try this feature by creating a fresh source".to_owned(),
138 )
139 .into());
140 };
141
142 if new_format_encode.format != old_format || new_format_encode.row_encode != old_row_encode {
143 bail_not_implemented!(
144 "the original definition is FORMAT {:?} ENCODE {:?}, and altering them is not supported yet",
145 &old_format,
146 &old_row_encode,
147 );
148 }
149
150 Ok(())
151}
152
153pub async fn refresh_sr_and_get_columns_diff(
155 original_source: &SourceCatalog,
156 format_encode: &FormatEncodeOptions,
157 session: &Arc<SessionImpl>,
158) -> Result<(StreamSourceInfo, Vec<ColumnCatalog>, Vec<ColumnCatalog>)> {
159 let mut with_properties = original_source.with_properties.clone();
160 validate_compatibility(format_encode, &mut with_properties)?;
161
162 if with_properties.is_cdc_connector() {
163 bail_not_implemented!("altering a cdc source is not supported");
164 }
165
166 let (Some(columns_from_resolve_source), source_info) = bind_columns_from_source(
167 session,
168 format_encode,
169 Either::Right(&with_properties),
170 CreateSourceType::for_replace(original_source),
171 )
172 .await?
173 else {
174 unreachable!("source without schema registry is rejected")
176 };
177
178 let mut added_columns = columns_minus(&columns_from_resolve_source, &original_source.columns);
179 let mut next_col_id = max_column_id(&original_source.columns).next();
181 for col in &mut added_columns {
182 col.column_desc.column_id = next_col_id;
183 next_col_id = next_col_id.next();
184 }
185 let dropped_columns = columns_minus(&original_source.columns, &columns_from_resolve_source);
186 tracing::debug!(
187 ?added_columns,
188 ?dropped_columns,
189 ?columns_from_resolve_source,
190 original_source = ?original_source.columns
191 );
192
193 Ok((source_info, added_columns, dropped_columns))
194}
195
196fn get_format_encode_from_source(source: &SourceCatalog) -> Result<FormatEncodeOptions> {
197 let stmt = source.create_sql_ast()?;
198 let Statement::CreateSource {
199 stmt: CreateSourceStatement { format_encode, .. },
200 } = stmt
201 else {
202 unreachable!()
203 };
204 Ok(format_encode.into_v2_with_warning())
205}
206
207pub async fn handler_refresh_schema(
208 handler_args: HandlerArgs,
209 name: ObjectName,
210) -> Result<RwPgResponse> {
211 let source = fetch_source_catalog_with_db_schema_id(&handler_args.session, &name)?;
212 let format_encode = get_format_encode_from_source(&source)?;
213 handle_alter_source_with_sr(handler_args, name, format_encode).await
214}
215
216pub async fn handle_alter_source_with_sr(
217 handler_args: HandlerArgs,
218 name: ObjectName,
219 format_encode: FormatEncodeOptions,
220) -> Result<RwPgResponse> {
221 let session = handler_args.session.clone();
222 let source = fetch_source_catalog_with_db_schema_id(&session, &name)?;
223 let mut source = source.as_ref().clone();
224
225 if source.associated_table_id.is_some() {
226 return Err(ErrorCode::NotSupported(
227 "alter table with connector using ALTER SOURCE statement".to_owned(),
228 "try to use ALTER TABLE instead".to_owned(),
229 )
230 .into());
231 };
232
233 check_format_encode(&source, &format_encode)?;
234
235 if !schema_has_schema_registry(&format_encode) {
236 return Err(ErrorCode::NotSupported(
237 "altering a source without schema registry".to_owned(),
238 "try `ALTER SOURCE .. ADD COLUMN ...` instead".to_owned(),
239 )
240 .into());
241 }
242
243 let (source_info, added_columns, dropped_columns) =
244 refresh_sr_and_get_columns_diff(&source, &format_encode, &session).await?;
245
246 if !dropped_columns.is_empty() {
247 bail_not_implemented!(
248 "this altering statement will drop columns, which is not supported yet: {}",
249 dropped_columns
250 .iter()
251 .map(|col| format!("({}: {})", col.name(), col.data_type()))
252 .join(", ")
253 );
254 }
255
256 source.info = source_info;
257 source.columns.extend(added_columns);
258 source.definition = alter_definition_format_encode(
259 source.create_sql_ast_purified()?,
260 format_encode.row_options.clone(),
261 )?;
262
263 let (format_encode_options, format_encode_secret_ref) = resolve_secret_ref_in_with_options(
264 WithOptions::try_from(format_encode.row_options())?,
265 session.as_ref(),
266 )?
267 .into_parts();
268 source
269 .info
270 .format_encode_options
271 .extend(format_encode_options);
272
273 source
274 .info
275 .format_encode_secret_refs
276 .extend(format_encode_secret_ref);
277
278 source.version += 1;
280
281 let pb_source = source.to_prost();
282 let catalog_writer = session.catalog_writer()?;
283 if source.info.is_shared() {
284 let graph = generate_stream_graph_for_source(handler_args, source.clone())?;
285 catalog_writer.replace_source(pb_source, graph).await?
286 } else {
287 catalog_writer.alter_source(pb_source).await?;
288 }
289 Ok(RwPgResponse::empty_result(StatementType::ALTER_SOURCE))
290}
291
292pub fn alter_definition_format_encode(
294 mut stmt: Statement,
295 format_encode_options: Vec<SqlOption>,
296) -> Result<String> {
297 match &mut stmt {
298 Statement::CreateSource {
299 stmt: CreateSourceStatement { format_encode, .. },
300 }
301 | Statement::CreateTable {
302 format_encode: Some(format_encode),
303 ..
304 } => {
305 match format_encode {
306 CompatibleFormatEncode::V2(schema) => {
307 schema.row_options = format_encode_options;
308 }
309 CompatibleFormatEncode::RowFormat(_schema) => unreachable!(),
312 }
313 }
314 _ => unreachable!(),
315 }
316
317 Ok(stmt.to_string())
318}
319
320#[cfg(test)]
321pub mod tests {
322 use risingwave_common::catalog::{DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME};
323 use risingwave_common::types::DataType;
324
325 use crate::catalog::root_catalog::SchemaPath;
326 use crate::test_utils::{LocalFrontend, PROTO_FILE_DATA, create_proto_file};
327
328 #[tokio::test]
329 async fn test_alter_source_with_sr_handler() {
330 let proto_file = create_proto_file(PROTO_FILE_DATA);
331 let sql = format!(
332 r#"CREATE SOURCE src
333 WITH (
334 connector = 'kafka',
335 topic = 'test-topic',
336 properties.bootstrap.server = 'localhost:29092'
337 )
338 FORMAT PLAIN ENCODE PROTOBUF (
339 message = '.test.TestRecord',
340 schema.location = 'file://{}'
341 )"#,
342 proto_file.path().to_str().unwrap()
343 );
344 let frontend = LocalFrontend::new(Default::default()).await;
345 let session = frontend.session_ref();
346 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
347
348 frontend
349 .run_sql_with_session(session.clone(), "SET streaming_use_shared_source TO false;")
350 .await
351 .unwrap();
352 frontend
353 .run_sql_with_session(session.clone(), sql)
354 .await
355 .unwrap();
356
357 let get_source = || {
358 let catalog_reader = session.env().catalog_reader().read_guard();
359 catalog_reader
360 .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "src")
361 .unwrap()
362 .0
363 .clone()
364 };
365
366 let source = get_source();
367 expect_test::expect!["CREATE SOURCE src (id INT, country STRUCT<address CHARACTER VARYING, city STRUCT<address CHARACTER VARYING, zipcode CHARACTER VARYING>, zipcode CHARACTER VARYING>, zipcode BIGINT, rate REAL) WITH (connector = 'kafka', topic = 'test-topic', properties.bootstrap.server = 'localhost:29092') FORMAT PLAIN ENCODE PROTOBUF (message = '.test.TestRecord', schema.location = 'file://')"].assert_eq(&source.create_sql_purified().replace(proto_file.path().to_str().unwrap(), ""));
368
369 let sql = format!(
370 r#"ALTER SOURCE src FORMAT UPSERT ENCODE PROTOBUF (
371 message = '.test.TestRecord',
372 schema.location = 'file://{}'
373 )"#,
374 proto_file.path().to_str().unwrap()
375 );
376 assert!(
377 frontend
378 .run_sql(sql)
379 .await
380 .unwrap_err()
381 .to_string()
382 .contains("the original definition is FORMAT Plain ENCODE Protobuf")
383 );
384
385 let sql = format!(
386 r#"ALTER SOURCE src FORMAT PLAIN ENCODE PROTOBUF (
387 message = '.test.TestRecordAlterType',
388 schema.location = 'file://{}'
389 )"#,
390 proto_file.path().to_str().unwrap()
391 );
392 let res_str = frontend.run_sql(sql).await.unwrap_err().to_string();
393 assert!(res_str.contains("id: integer"));
394 assert!(res_str.contains("zipcode: bigint"));
395
396 let sql = format!(
397 r#"ALTER SOURCE src FORMAT PLAIN ENCODE PROTOBUF (
398 message = '.test.TestRecordExt',
399 schema.location = 'file://{}'
400 )"#,
401 proto_file.path().to_str().unwrap()
402 );
403 frontend.run_sql(sql).await.unwrap();
404
405 let altered_source = get_source();
406
407 let name_column = altered_source
408 .columns
409 .iter()
410 .find(|col| col.column_desc.name == "name")
411 .unwrap();
412 assert_eq!(name_column.column_desc.data_type, DataType::Varchar);
413
414 expect_test::expect!["CREATE SOURCE src (id INT, country STRUCT<address CHARACTER VARYING, city STRUCT<address CHARACTER VARYING, zipcode CHARACTER VARYING>, zipcode CHARACTER VARYING>, zipcode BIGINT, rate REAL, name CHARACTER VARYING) WITH (connector = 'kafka', topic = 'test-topic', properties.bootstrap.server = 'localhost:29092') FORMAT PLAIN ENCODE PROTOBUF (message = '.test.TestRecordExt', schema.location = 'file://')"].assert_eq(&altered_source.create_sql_purified().replace(proto_file.path().to_str().unwrap(), ""));
415 }
416}