1use std::collections::{BTreeMap, HashMap, HashSet};
16use std::rc::Rc;
17use std::sync::LazyLock;
18
19use anyhow::{Context, anyhow};
20use either::Either;
21use external_schema::debezium::extract_debezium_avro_table_pk_columns;
22use external_schema::nexmark::check_nexmark_schema;
23use itertools::Itertools;
24use maplit::{convert_args, hashmap, hashset};
25use pgwire::pg_response::{PgResponse, StatementType};
26use rand::Rng;
27use risingwave_common::array::arrow::{IcebergArrowConvert, arrow_schema_iceberg};
28use risingwave_common::bail_not_implemented;
29use risingwave_common::catalog::{
30 ColumnCatalog, ColumnDesc, ColumnId, INITIAL_SOURCE_VERSION_ID, KAFKA_TIMESTAMP_COLUMN_NAME,
31 ROW_ID_COLUMN_NAME, TableId, debug_assert_column_ids_distinct,
32};
33use risingwave_common::license::Feature;
34use risingwave_common::secret::LocalSecretManager;
35use risingwave_common::system_param::reader::SystemParamsRead;
36use risingwave_common::types::DataType;
37use risingwave_common::util::iter_util::ZipEqFast;
38use risingwave_connector::WithPropertiesExt;
39use risingwave_connector::parser::additional_columns::{
40 build_additional_column_desc, get_supported_additional_columns,
41 source_add_partition_offset_cols,
42};
43use risingwave_connector::parser::{
44 AvroParserConfig, DEBEZIUM_IGNORE_KEY, DebeziumAvroParserConfig, ProtobufParserConfig,
45 SchemaLocation, SpecificParserConfig, TimestamptzHandling,
46 fetch_json_schema_and_map_to_columns,
47};
48use risingwave_connector::schema::AWS_GLUE_SCHEMA_ARN_KEY;
49use risingwave_connector::schema::schema_registry::{
50 SCHEMA_REGISTRY_BACKOFF_DURATION_KEY, SCHEMA_REGISTRY_BACKOFF_FACTOR_KEY,
51 SCHEMA_REGISTRY_MAX_DELAY_KEY, SCHEMA_REGISTRY_PASSWORD, SCHEMA_REGISTRY_RETRIES_MAX_KEY,
52 SCHEMA_REGISTRY_USERNAME, SchemaRegistryConfig, name_strategy_from_str,
53};
54use risingwave_connector::source::cdc::{
55 CDC_AUTO_SCHEMA_CHANGE_KEY, CDC_MONGODB_STRONG_SCHEMA_KEY, CDC_SHARING_MODE_KEY,
56 CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY, CDC_TRANSACTIONAL_KEY,
57 CDC_WAIT_FOR_STREAMING_START_TIMEOUT, CITUS_CDC_CONNECTOR, MONGODB_CDC_CONNECTOR,
58 MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, SQL_SERVER_CDC_CONNECTOR,
59};
60use risingwave_connector::source::datagen::DATAGEN_CONNECTOR;
61use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR;
62use risingwave_connector::source::nexmark::source::{EventType, get_event_data_types_with_names};
63use risingwave_connector::source::test_source::TEST_CONNECTOR;
64use risingwave_connector::source::{
65 AZBLOB_CONNECTOR, ConnectorProperties, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR,
66 KINESIS_CONNECTOR, LEGACY_S3_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR,
67 OPENDAL_S3_CONNECTOR, POSIX_FS_CONNECTOR, PULSAR_CONNECTOR,
68};
69pub use risingwave_connector::source::{UPSTREAM_SOURCE_KEY, WEBHOOK_CONNECTOR};
70use risingwave_pb::catalog::connection_params::PbConnectionType;
71use risingwave_pb::catalog::{PbSchemaRegistryNameStrategy, StreamSourceInfo, WatermarkDesc};
72use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType;
73use risingwave_pb::plan_common::{EncodeType, FormatType};
74use risingwave_pb::stream_plan::PbStreamFragmentGraph;
75use risingwave_pb::telemetry::TelemetryDatabaseObject;
76use risingwave_sqlparser::ast::{
77 AstString, ColumnDef, ColumnOption, CreateSourceStatement, Encode, Format, FormatEncodeOptions,
78 ObjectName, SourceWatermark, SqlOptionValue, TableConstraint, Value, get_delimiter,
79};
80use risingwave_sqlparser::parser::{IncludeOption, IncludeOptionItem};
81use thiserror_ext::AsReport;
82
83use super::RwPgResponse;
84use crate::binder::Binder;
85use crate::catalog::CatalogError;
86use crate::catalog::source_catalog::SourceCatalog;
87use crate::error::ErrorCode::{self, Deprecated, InvalidInputSyntax, NotSupported, ProtocolError};
88use crate::error::{Result, RwError};
89use crate::expr::Expr;
90use crate::handler::HandlerArgs;
91use crate::handler::create_table::{
92 ColumnIdGenerator, bind_pk_and_row_id_on_relation, bind_sql_column_constraints,
93 bind_sql_columns, bind_sql_pk_names, bind_table_constraints,
94};
95use crate::handler::util::{
96 SourceSchemaCompatExt, check_connector_match_connection_type, ensure_connection_type_allowed,
97};
98use crate::optimizer::plan_node::generic::SourceNodeKind;
99use crate::optimizer::plan_node::{LogicalSource, ToStream, ToStreamContext};
100use crate::session::SessionImpl;
101use crate::session::current::notice_to_user;
102use crate::utils::{
103 OverwriteOptions, resolve_connection_ref_and_secret_ref, resolve_privatelink_in_with_option,
104 resolve_secret_ref_in_with_options,
105};
106use crate::{OptimizerContext, WithOptions, WithOptionsSecResolved, bind_data_type, build_graph};
107
108mod external_schema;
109pub use external_schema::{
110 bind_columns_from_source, get_schema_location, schema_has_schema_registry,
111};
112mod validate;
113pub use validate::validate_compatibility;
114use validate::{SOURCE_ALLOWED_CONNECTION_CONNECTOR, SOURCE_ALLOWED_CONNECTION_SCHEMA_REGISTRY};
115mod additional_column;
116use additional_column::check_and_add_timestamp_column;
117pub use additional_column::handle_addition_columns;
118
119use crate::stream_fragmenter::GraphJobType;
120
121fn non_generated_sql_columns(columns: &[ColumnDef]) -> Vec<ColumnDef> {
122 columns
123 .iter()
124 .filter(|c| !c.is_generated())
125 .cloned()
126 .collect()
127}
128
129fn try_consume_string_from_options(
130 format_encode_options: &mut BTreeMap<String, String>,
131 key: &str,
132) -> Option<AstString> {
133 format_encode_options.remove(key).map(AstString)
134}
135
136fn try_consume_schema_registry_config_from_options(
137 format_encode_options: &mut BTreeMap<String, String>,
138) {
139 [
140 SCHEMA_REGISTRY_USERNAME,
141 SCHEMA_REGISTRY_PASSWORD,
142 SCHEMA_REGISTRY_MAX_DELAY_KEY,
143 SCHEMA_REGISTRY_BACKOFF_DURATION_KEY,
144 SCHEMA_REGISTRY_BACKOFF_FACTOR_KEY,
145 SCHEMA_REGISTRY_RETRIES_MAX_KEY,
146 ]
147 .iter()
148 .for_each(|key| {
149 try_consume_string_from_options(format_encode_options, key);
150 });
151}
152
153fn consume_string_from_options(
154 format_encode_options: &mut BTreeMap<String, String>,
155 key: &str,
156) -> Result<AstString> {
157 try_consume_string_from_options(format_encode_options, key).ok_or(RwError::from(ProtocolError(
158 format!("missing field {} in options", key),
159 )))
160}
161
162fn consume_aws_config_from_options(format_encode_options: &mut BTreeMap<String, String>) {
163 format_encode_options.retain(|key, _| !key.starts_with("aws."))
164}
165
166#[derive(Debug, Clone, Copy, PartialEq, Eq)]
167pub enum CreateSourceType {
168 SharedCdc,
169 SharedNonCdc,
171 NonShared,
172 Table,
174}
175
176impl CreateSourceType {
177 pub fn for_newly_created(
179 session: &SessionImpl,
180 with_properties: &impl WithPropertiesExt,
181 ) -> Self {
182 if with_properties.is_shareable_cdc_connector() {
183 CreateSourceType::SharedCdc
184 } else if with_properties.is_shareable_non_cdc_connector()
185 && session
186 .env()
187 .streaming_config()
188 .developer
189 .enable_shared_source
190 && session.config().streaming_use_shared_source()
191 {
192 CreateSourceType::SharedNonCdc
193 } else {
194 CreateSourceType::NonShared
195 }
196 }
197
198 pub fn for_replace(catalog: &SourceCatalog) -> Self {
199 if !catalog.info.is_shared() {
200 CreateSourceType::NonShared
201 } else if catalog.with_properties.is_shareable_cdc_connector() {
202 CreateSourceType::SharedCdc
203 } else {
204 CreateSourceType::SharedNonCdc
205 }
206 }
207
208 pub fn is_shared(&self) -> bool {
209 matches!(
210 self,
211 CreateSourceType::SharedCdc | CreateSourceType::SharedNonCdc
212 )
213 }
214}
215
216pub(crate) fn bind_all_columns(
218 format_encode: &FormatEncodeOptions,
219 cols_from_source: Option<Vec<ColumnCatalog>>,
220 cols_from_sql: Vec<ColumnCatalog>,
221 col_defs_from_sql: &[ColumnDef],
222 wildcard_idx: Option<usize>,
223 sql_column_strategy: SqlColumnStrategy,
224) -> Result<Vec<ColumnCatalog>> {
225 if let Some(cols_from_source) = cols_from_source {
226 let generated_cols_from_sql = cols_from_sql
229 .iter()
230 .filter(|c| {
231 col_defs_from_sql
232 .iter()
233 .find(|d| d.name.real_value() == c.name())
234 .unwrap()
235 .is_generated()
236 })
237 .cloned()
238 .collect_vec();
239
240 #[allow(clippy::collapsible_else_if)]
241 match sql_column_strategy {
242 SqlColumnStrategy::FollowUnchecked => {
244 assert!(
245 wildcard_idx.is_none(),
246 "wildcard still exists while strategy is Follows, not correctly purified?"
247 );
248 return Ok(cols_from_sql);
249 }
250
251 SqlColumnStrategy::Ignore => {}
253
254 SqlColumnStrategy::FollowChecked => {
255 let has_regular_cols_from_sql =
256 generated_cols_from_sql.len() != cols_from_sql.len();
257
258 if has_regular_cols_from_sql {
259 if wildcard_idx.is_some() {
260 return Err(RwError::from(NotSupported(
262 "When there's a wildcard (\"*\"), \
263 only generated columns are allowed in user-defined schema from SQL"
264 .to_owned(),
265 "Remove the non-generated columns".to_owned(),
266 )));
267 } else {
268 for col in &cols_from_sql {
271 if generated_cols_from_sql.contains(col) {
272 continue;
273 }
274 let Some(col_from_source) =
275 cols_from_source.iter().find(|c| c.name() == col.name())
276 else {
277 return Err(RwError::from(ProtocolError(format!(
278 "Column \"{}\" is defined in SQL but not found in the source",
279 col.name()
280 ))));
281 };
282
283 if col_from_source.data_type() != col.data_type() {
284 return Err(RwError::from(ProtocolError(format!(
285 "Data type mismatch for column \"{}\". \
286 Defined in SQL as \"{}\", but found in the source as \"{}\"",
287 col.name(),
288 col.data_type(),
289 col_from_source.data_type()
290 ))));
291 }
292 }
293 return Ok(cols_from_sql);
294 }
295 } else {
296 if wildcard_idx.is_some() {
297 } else {
301 notice_to_user("\
309 Neither wildcard (\"*\") nor regular (non-generated) columns appear in the user-defined schema from SQL. \
310 For backward compatibility, all columns from the source will be included at the beginning. \
311 For clarity, consider adding a wildcard (\"*\") to indicate where the columns from the source should be included, \
312 or specifying the columns you want to include from the source.
313 ");
314 }
315 }
316 }
317 }
318
319 let wildcard_idx = wildcard_idx.unwrap_or(0).min(generated_cols_from_sql.len());
324
325 let mut merged_cols = generated_cols_from_sql;
327 let merged_cols_r = merged_cols.split_off(wildcard_idx);
328 merged_cols.extend(cols_from_source);
329 merged_cols.extend(merged_cols_r);
330
331 Ok(merged_cols)
332 } else {
333 if wildcard_idx.is_some() {
334 return Err(RwError::from(NotSupported(
335 "Wildcard in user-defined schema is only allowed when there exists columns from external schema".to_owned(),
336 "Remove the wildcard or use a source with external schema".to_owned(),
337 )));
338 }
339 let non_generated_sql_defined_columns = non_generated_sql_columns(col_defs_from_sql);
340
341 match (&format_encode.format, &format_encode.row_encode) {
342 (Format::DebeziumMongo, Encode::Json) => {
343 let strong_schema = format_encode
344 .row_options
345 .iter()
346 .find(|k| k.name.real_value().to_lowercase() == CDC_MONGODB_STRONG_SCHEMA_KEY)
347 .map(|k| matches!(k.value, SqlOptionValue::Value(Value::Boolean(true))))
348 .unwrap_or(false);
349
350 if strong_schema {
352 let (_, id_column) = non_generated_sql_defined_columns
353 .iter()
354 .enumerate()
355 .find(|(idx, col)| *idx == 0 && col.name.real_value() == "_id")
356 .ok_or_else(|| {
357 RwError::from(ProtocolError(
358 "The `_id` column of the source with row format DebeziumMongoJson must be defined as the first column in SQL".to_owned(),
359 ))
360 })?;
361
362 let id_data_type = bind_data_type(id_column.data_type.as_ref().unwrap())?;
363 if !matches!(
364 id_data_type,
365 DataType::Varchar | DataType::Int32 | DataType::Int64 | DataType::Jsonb
366 ) {
367 return Err(RwError::from(ProtocolError(
368 "the `_id` column of the source with row format DebeziumMongoJson must be [Jsonb | Varchar | Int32 | Int64]".to_owned(),
369 )));
370 }
371
372 let mut columns = Vec::with_capacity(non_generated_sql_defined_columns.len());
373 columns.push(
374 ColumnCatalog {
376 column_desc: ColumnDesc::named("_id", 0.into(), id_data_type),
377 is_hidden: false,
378 },
379 );
380
381 for (idx, col) in non_generated_sql_defined_columns
383 .into_iter()
384 .skip(1)
386 .enumerate()
387 {
388 columns.push(ColumnCatalog {
389 column_desc: ColumnDesc::named(
390 col.name.real_value(),
391 (idx as i32).into(),
392 bind_data_type(col.data_type.as_ref().unwrap())?,
393 ),
394 is_hidden: false,
395 });
396 }
397
398 return Ok(columns);
399 }
400
401 let mut columns = vec![
402 ColumnCatalog {
403 column_desc: ColumnDesc::named("_id", 0.into(), DataType::Varchar),
404 is_hidden: false,
405 },
406 ColumnCatalog {
407 column_desc: ColumnDesc::named("payload", 0.into(), DataType::Jsonb),
408 is_hidden: false,
409 },
410 ];
411
412 if non_generated_sql_defined_columns.len() != 2
413 || non_generated_sql_defined_columns[0].name.real_value() != columns[0].name()
414 || non_generated_sql_defined_columns[1].name.real_value() != columns[1].name()
415 {
416 return Err(RwError::from(ProtocolError(
417 "the not generated columns of the source with row format DebeziumMongoJson
418 must be (_id [Jsonb | Varchar | Int32 | Int64], payload jsonb)."
419 .to_owned(),
420 )));
421 }
422 let key_data_type = bind_data_type(
424 non_generated_sql_defined_columns[0]
425 .data_type
426 .as_ref()
427 .unwrap(),
428 )?;
429 match key_data_type {
430 DataType::Jsonb | DataType::Varchar | DataType::Int32 | DataType::Int64 => {
431 columns[0].column_desc.data_type = key_data_type.clone();
432 }
433 _ => {
434 return Err(RwError::from(ProtocolError(
435 "the `_id` column of the source with row format DebeziumMongoJson
436 must be [Jsonb | Varchar | Int32 | Int64]"
437 .to_owned(),
438 )));
439 }
440 }
441
442 let value_data_type = bind_data_type(
444 non_generated_sql_defined_columns[1]
445 .data_type
446 .as_ref()
447 .unwrap(),
448 )?;
449 if !matches!(value_data_type, DataType::Jsonb) {
450 return Err(RwError::from(ProtocolError(
451 "the `payload` column of the source with row format DebeziumMongoJson
452 must be Jsonb datatype"
453 .to_owned(),
454 )));
455 }
456 Ok(columns)
457 }
458 (Format::Plain, Encode::Bytes) => {
459 let err = Err(RwError::from(ProtocolError(
460 "ENCODE BYTES only accepts one BYTEA type column".to_owned(),
461 )));
462 if non_generated_sql_defined_columns.len() == 1 {
463 let col_data_type = bind_data_type(
465 non_generated_sql_defined_columns[0]
466 .data_type
467 .as_ref()
468 .unwrap(),
469 )?;
470 if col_data_type == DataType::Bytea {
471 Ok(cols_from_sql)
472 } else {
473 err
474 }
475 } else {
476 err
477 }
478 }
479 (_, _) => Ok(cols_from_sql),
480 }
481 }
482}
483
484fn hint_format_encode(format_encode: &FormatEncodeOptions) -> String {
486 format!(
487 r#"Hint: For FORMAT {0} ENCODE {1}, INCLUDE KEY must be specified and the key column must be used as primary key.
488example:
489 CREATE TABLE <table_name> ( PRIMARY KEY ([rw_key | <key_name>]) )
490 INCLUDE KEY [AS <key_name>]
491 WITH (...)
492 FORMAT {0} ENCODE {1}{2}
493"#,
494 format_encode.format,
495 format_encode.row_encode,
496 if format_encode.row_encode == Encode::Json || format_encode.row_encode == Encode::Bytes {
497 "".to_owned()
498 } else {
499 " (...)".to_owned()
500 }
501 )
502}
503
504pub(crate) async fn bind_source_pk(
507 format_encode: &FormatEncodeOptions,
508 source_info: &StreamSourceInfo,
509 columns: &mut [ColumnCatalog],
510 sql_defined_pk_names: Vec<String>,
511 with_properties: &WithOptionsSecResolved,
512) -> Result<Vec<String>> {
513 let sql_defined_pk = !sql_defined_pk_names.is_empty();
514 let include_key_column_name: Option<String> = {
515 columns.iter().find_map(|catalog| {
518 if matches!(
519 catalog.column_desc.additional_column.column_type,
520 Some(AdditionalColumnType::Key(_))
521 ) {
522 Some(catalog.name().to_owned())
523 } else {
524 None
525 }
526 })
527 };
528 let additional_column_names = columns
529 .iter()
530 .filter_map(|col| {
531 if col.column_desc.additional_column.column_type.is_some() {
532 Some(col.name().to_owned())
533 } else {
534 None
535 }
536 })
537 .collect_vec();
538
539 let res = match (&format_encode.format, &format_encode.row_encode) {
540 (Format::Native, Encode::Native) | (Format::None, Encode::None) | (Format::Plain, _) => {
541 sql_defined_pk_names
542 }
543
544 (Format::Upsert, Encode::Json | Encode::Avro | Encode::Protobuf) => {
547 if let Some(ref key_column_name) = include_key_column_name
548 && sql_defined_pk
549 {
550 if sql_defined_pk_names.len() != 1
555 || !key_column_name.eq(sql_defined_pk_names[0].as_str())
556 {
557 return Err(RwError::from(ProtocolError(format!(
558 "Only \"{}\" can be used as primary key\n\n{}",
559 key_column_name,
560 hint_format_encode(format_encode)
561 ))));
562 }
563 sql_defined_pk_names
564 } else {
565 return if let Some(include_key_column_name) = include_key_column_name {
567 Err(RwError::from(ProtocolError(format!(
568 "Primary key must be specified to {}\n\n{}",
569 include_key_column_name,
570 hint_format_encode(format_encode)
571 ))))
572 } else {
573 Err(RwError::from(ProtocolError(format!(
574 "INCLUDE KEY clause not set\n\n{}",
575 hint_format_encode(format_encode)
576 ))))
577 };
578 }
579 }
580
581 (Format::Debezium, Encode::Json) => {
582 if !additional_column_names.is_empty() {
583 return Err(RwError::from(ProtocolError(format!(
584 "FORMAT DEBEZIUM forbids additional columns, but got {:?}",
585 additional_column_names
586 ))));
587 }
588 if !sql_defined_pk {
589 return Err(RwError::from(ProtocolError(
590 "Primary key must be specified when creating source with FORMAT DEBEZIUM."
591 .to_owned(),
592 )));
593 }
594 sql_defined_pk_names
595 }
596 (Format::Debezium, Encode::Avro) => {
597 if !additional_column_names.is_empty() {
598 return Err(RwError::from(ProtocolError(format!(
599 "FORMAT DEBEZIUM forbids additional columns, but got {:?}",
600 additional_column_names
601 ))));
602 }
603 if sql_defined_pk {
604 sql_defined_pk_names
605 } else {
606 let pk_names =
607 extract_debezium_avro_table_pk_columns(source_info, with_properties).await?;
608 for pk_name in &pk_names {
610 columns
611 .iter()
612 .find(|c: &&ColumnCatalog| c.name().eq(pk_name))
613 .ok_or_else(|| {
614 RwError::from(ProtocolError(format!(
615 "avro's key column {} not exists in avro's row schema",
616 pk_name
617 )))
618 })?;
619 }
620 pk_names
621 }
622 }
623 (Format::DebeziumMongo, Encode::Json) => {
624 if sql_defined_pk {
625 sql_defined_pk_names
626 } else {
627 vec!["_id".to_owned()]
628 }
629 }
630
631 (Format::Maxwell, Encode::Json) => {
632 if !additional_column_names.is_empty() {
633 return Err(RwError::from(ProtocolError(format!(
634 "FORMAT MAXWELL forbids additional columns, but got {:?}",
635 additional_column_names
636 ))));
637 }
638 if !sql_defined_pk {
639 return Err(RwError::from(ProtocolError(
640 "Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON.".to_owned(),
641 )));
642 }
643 sql_defined_pk_names
644 }
645
646 (Format::Canal, Encode::Json) => {
647 if !additional_column_names.is_empty() {
648 return Err(RwError::from(ProtocolError(format!(
649 "FORMAT CANAL forbids additional columns, but got {:?}",
650 additional_column_names
651 ))));
652 }
653 if !sql_defined_pk {
654 return Err(RwError::from(ProtocolError(
655 "Primary key must be specified when creating source with FORMAT CANAL ENCODE JSON.".to_owned(),
656 )));
657 }
658 sql_defined_pk_names
659 }
660 (format, encoding) => {
661 return Err(RwError::from(ProtocolError(format!(
662 "Unknown combination {:?} {:?}",
663 format, encoding
664 ))));
665 }
666 };
667 Ok(res)
668}
669
670pub(super) fn bind_source_watermark(
671 session: &SessionImpl,
672 name: String,
673 source_watermarks: Vec<SourceWatermark>,
674 column_catalogs: &[ColumnCatalog],
675) -> Result<Vec<WatermarkDesc>> {
676 let mut binder = Binder::new_for_ddl(session);
677 binder.bind_columns_to_context(name.clone(), column_catalogs)?;
678
679 let watermark_descs = source_watermarks
680 .into_iter()
681 .map(|source_watermark| {
682 let col_name = source_watermark.column.real_value();
683 let watermark_idx = binder.get_column_binding_index(name.clone(), &col_name)?;
684
685 let expr = binder.bind_expr(source_watermark.expr)?;
686 let watermark_col_type = column_catalogs[watermark_idx].data_type();
687 let watermark_expr_type = &expr.return_type();
688 if watermark_col_type != watermark_expr_type {
689 Err(RwError::from(ErrorCode::BindError(
690 format!("The return value type of the watermark expression must be identical to the watermark column data type. Current data type of watermark return value: `{}`, column `{}`",watermark_expr_type, watermark_col_type),
691 )))
692 } else {
693 let expr_proto = expr.to_expr_proto();
694 Ok::<_, RwError>(WatermarkDesc {
695 watermark_idx: watermark_idx as u32,
696 expr: Some(expr_proto),
697 })
698 }
699 })
700 .try_collect()?;
701 Ok(watermark_descs)
702}
703
704pub(super) fn check_format_encode(
710 props: &WithOptionsSecResolved,
711 row_id_index: Option<usize>,
712 columns: &[ColumnCatalog],
713) -> Result<()> {
714 let Some(connector) = props.get_connector() else {
715 return Ok(());
716 };
717
718 if connector == NEXMARK_CONNECTOR {
719 check_nexmark_schema(props, row_id_index, columns)
720 } else {
721 Ok(())
722 }
723}
724
725pub fn bind_connector_props(
726 handler_args: &HandlerArgs,
727 format_encode: &FormatEncodeOptions,
728 is_create_source: bool,
729) -> Result<WithOptions> {
730 let mut with_properties = handler_args.with_options.clone().into_connector_props();
731 validate_compatibility(format_encode, &mut with_properties)?;
732 let create_cdc_source_job = with_properties.is_shareable_cdc_connector();
733
734 if !is_create_source && with_properties.is_shareable_only_cdc_connector() {
735 return Err(RwError::from(ProtocolError(format!(
736 "connector {} does not support `CREATE TABLE`, please use `CREATE SOURCE` instead",
737 with_properties.get_connector().unwrap(),
738 ))));
739 }
740 if is_create_source && create_cdc_source_job {
741 if let Some(value) = with_properties.get(CDC_AUTO_SCHEMA_CHANGE_KEY)
742 && value
743 .parse::<bool>()
744 .map_err(|_| anyhow!("invalid value of '{}' option", CDC_AUTO_SCHEMA_CHANGE_KEY))?
745 {
746 Feature::CdcAutoSchemaChange
747 .check_available()
748 .map_err(|e| anyhow::anyhow!(e))?;
749 }
750
751 with_properties.insert(CDC_SNAPSHOT_MODE_KEY.into(), CDC_SNAPSHOT_BACKFILL.into());
753 with_properties.insert(CDC_SHARING_MODE_KEY.into(), "true".into());
755 if with_properties.enable_transaction_metadata() {
757 with_properties.insert(CDC_TRANSACTIONAL_KEY.into(), "true".into());
758 }
759 with_properties.insert(
760 CDC_WAIT_FOR_STREAMING_START_TIMEOUT.into(),
761 handler_args
762 .session
763 .config()
764 .cdc_source_wait_streaming_start_timeout()
765 .to_string(),
766 );
767 }
768 if with_properties.is_mysql_cdc_connector() {
769 with_properties
773 .entry("server.id".to_owned())
774 .or_insert(rand::rng().random_range(1..u32::MAX).to_string());
775 }
776 Ok(with_properties)
777}
778
779pub enum SqlColumnStrategy {
782 FollowUnchecked,
787
788 FollowChecked,
795
796 Ignore,
801}
802
803#[allow(clippy::too_many_arguments)]
804pub async fn bind_create_source_or_table_with_connector(
805 handler_args: HandlerArgs,
806 full_name: ObjectName,
807 format_encode: FormatEncodeOptions,
808 with_properties: WithOptions,
809 sql_columns_defs: &[ColumnDef],
810 constraints: Vec<TableConstraint>,
811 wildcard_idx: Option<usize>,
812 source_watermarks: Vec<SourceWatermark>,
813 columns_from_resolve_source: Option<Vec<ColumnCatalog>>,
814 source_info: StreamSourceInfo,
815 include_column_options: IncludeOption,
816 col_id_gen: &mut ColumnIdGenerator,
817 create_source_type: CreateSourceType,
818 source_rate_limit: Option<u32>,
819 sql_column_strategy: SqlColumnStrategy,
820) -> Result<SourceCatalog> {
821 let session = &handler_args.session;
822 let db_name: &str = &session.database();
823 let (schema_name, source_name) = Binder::resolve_schema_qualified_name(db_name, full_name)?;
824 let (database_id, schema_id) =
825 session.get_database_and_schema_id_for_create(schema_name.clone())?;
826
827 let is_create_source = create_source_type != CreateSourceType::Table;
828 if !is_create_source && with_properties.is_iceberg_connector() {
829 return Err(ErrorCode::BindError(
830 "can't CREATE TABLE with iceberg connector\n\nHint: use CREATE SOURCE instead"
831 .to_owned(),
832 )
833 .into());
834 }
835
836 if is_create_source {
837 match format_encode.format {
838 Format::Upsert
839 | Format::Debezium
840 | Format::DebeziumMongo
841 | Format::Maxwell
842 | Format::Canal => {
843 return Err(ErrorCode::BindError(format!(
844 "can't CREATE SOURCE with FORMAT {}.\n\nHint: use CREATE TABLE instead\n\n{}",
845 format_encode.format,
846 hint_format_encode(&format_encode)
847 ))
848 .into());
849 }
850 _ => {
851 }
853 }
854 }
855
856 let sql_pk_names = bind_sql_pk_names(sql_columns_defs, bind_table_constraints(&constraints)?)?;
857
858 let columns_from_sql = bind_sql_columns(sql_columns_defs, false)?;
859
860 let mut columns = bind_all_columns(
861 &format_encode,
862 columns_from_resolve_source,
863 columns_from_sql,
864 sql_columns_defs,
865 wildcard_idx,
866 sql_column_strategy,
867 )?;
868
869 handle_addition_columns(
871 Some(&format_encode),
872 &with_properties,
873 include_column_options,
874 &mut columns,
875 false,
876 )?;
877
878 if columns.is_empty() {
879 return Err(RwError::from(ProtocolError(
880 "Schema definition is required, either from SQL or schema registry.".to_owned(),
881 )));
882 }
883
884 if is_create_source {
886 check_and_add_timestamp_column(&with_properties, &mut columns);
888
889 if create_source_type == CreateSourceType::SharedNonCdc {
892 let (columns_exist, additional_columns) = source_add_partition_offset_cols(
893 &columns,
894 &with_properties.get_connector().unwrap(),
895 true, );
897 for (existed, c) in columns_exist.into_iter().zip_eq_fast(additional_columns) {
898 if !existed {
899 columns.push(ColumnCatalog::hidden(c));
900 }
901 }
902 }
903 }
904
905 let mut with_properties = with_properties;
907 resolve_privatelink_in_with_option(&mut with_properties)?;
908
909 if session
911 .env()
912 .system_params_manager()
913 .get_params()
914 .load()
915 .enforce_secret()
916 && Feature::SecretManagement.check_available().is_ok()
917 {
918 ConnectorProperties::enforce_secret_source(&with_properties)?;
920 }
921
922 let (with_properties, connection_type, connector_conn_ref) =
923 resolve_connection_ref_and_secret_ref(
924 with_properties,
925 session,
926 Some(TelemetryDatabaseObject::Source),
927 )?;
928 ensure_connection_type_allowed(connection_type, &SOURCE_ALLOWED_CONNECTION_CONNECTOR)?;
929
930 if !matches!(connection_type, PbConnectionType::Unspecified) {
932 let connector = with_properties.get_connector().unwrap();
933 check_connector_match_connection_type(connector.as_str(), &connection_type)?;
934 }
935
936 let pk_names = bind_source_pk(
937 &format_encode,
938 &source_info,
939 &mut columns,
940 sql_pk_names,
941 &with_properties,
942 )
943 .await?;
944
945 if is_create_source && !pk_names.is_empty() {
946 return Err(ErrorCode::InvalidInputSyntax(
947 "Source does not support PRIMARY KEY constraint, please use \"CREATE TABLE\" instead"
948 .to_owned(),
949 )
950 .into());
951 }
952
953 if let Some(duplicated_name) = columns.iter().map(|c| c.name()).duplicates().next() {
956 return Err(ErrorCode::InvalidInputSyntax(format!(
957 "column \"{}\" specified more than once",
958 duplicated_name
959 ))
960 .into());
961 }
962
963 for c in &mut columns {
965 let original_data_type = c.data_type().clone();
966 col_id_gen.generate(c)?;
967 if is_create_source {
971 c.column_desc.data_type = original_data_type;
972 }
973 }
974 debug_assert_column_ids_distinct(&columns);
975
976 let (mut columns, pk_col_ids, row_id_index) =
977 bind_pk_and_row_id_on_relation(columns, pk_names, true)?;
978
979 let watermark_descs =
980 bind_source_watermark(session, source_name.clone(), source_watermarks, &columns)?;
981 assert!(watermark_descs.len() <= 1);
983
984 bind_sql_column_constraints(
985 session,
986 source_name.clone(),
987 &mut columns,
988 sql_columns_defs.to_vec(),
990 &pk_col_ids,
991 )?;
992 check_format_encode(&with_properties, row_id_index, &columns)?;
993
994 let definition = handler_args.normalized_sql.clone();
995
996 let associated_table_id = if is_create_source {
997 None
998 } else {
999 Some(TableId::placeholder())
1000 };
1001 let source = SourceCatalog {
1002 id: TableId::placeholder().table_id,
1003 name: source_name,
1004 schema_id,
1005 database_id,
1006 columns,
1007 pk_col_ids,
1008 append_only: row_id_index.is_some(),
1009 owner: session.user_id(),
1010 info: source_info,
1011 row_id_index,
1012 with_properties,
1013 watermark_descs,
1014 associated_table_id,
1015 definition,
1016 connection_id: connector_conn_ref,
1017 created_at_epoch: None,
1018 initialized_at_epoch: None,
1019 version: INITIAL_SOURCE_VERSION_ID,
1020 created_at_cluster_version: None,
1021 initialized_at_cluster_version: None,
1022 rate_limit: source_rate_limit,
1023 };
1024 Ok(source)
1025}
1026
1027pub async fn handle_create_source(
1028 mut handler_args: HandlerArgs,
1029 stmt: CreateSourceStatement,
1030) -> Result<RwPgResponse> {
1031 let session = handler_args.session.clone();
1032 let overwrite_options = OverwriteOptions::new(&mut handler_args);
1033
1034 if let Either::Right(resp) = session.check_relation_name_duplicated(
1035 stmt.source_name.clone(),
1036 StatementType::CREATE_SOURCE,
1037 stmt.if_not_exists,
1038 )? {
1039 return Ok(resp);
1040 }
1041
1042 if handler_args.with_options.is_empty() {
1043 return Err(RwError::from(InvalidInputSyntax(
1044 "missing WITH clause".to_owned(),
1045 )));
1046 }
1047
1048 let format_encode = stmt.format_encode.into_v2_with_warning();
1049 let with_properties = bind_connector_props(&handler_args, &format_encode, true)?;
1050
1051 let create_source_type = CreateSourceType::for_newly_created(&session, &*with_properties);
1052 let (columns_from_resolve_source, source_info) = bind_columns_from_source(
1053 &session,
1054 &format_encode,
1055 Either::Left(&with_properties),
1056 create_source_type,
1057 )
1058 .await?;
1059 let mut col_id_gen = ColumnIdGenerator::new_initial();
1060
1061 if stmt.columns.iter().any(|col| {
1062 col.options
1063 .iter()
1064 .any(|def| matches!(def.option, ColumnOption::NotNull))
1065 }) {
1066 return Err(RwError::from(InvalidInputSyntax(
1067 "NOT NULL constraint is not supported in source schema".to_owned(),
1068 )));
1069 }
1070
1071 let source_catalog = bind_create_source_or_table_with_connector(
1072 handler_args.clone(),
1073 stmt.source_name,
1074 format_encode,
1075 with_properties,
1076 &stmt.columns,
1077 stmt.constraints,
1078 stmt.wildcard_idx,
1079 stmt.source_watermarks,
1080 columns_from_resolve_source,
1081 source_info,
1082 stmt.include_column_options,
1083 &mut col_id_gen,
1084 create_source_type,
1085 overwrite_options.source_rate_limit,
1086 SqlColumnStrategy::FollowChecked,
1087 )
1088 .await?;
1089
1090 if stmt.temporary {
1092 if session.get_temporary_source(&source_catalog.name).is_some() {
1093 return Err(CatalogError::duplicated("source", source_catalog.name.clone()).into());
1094 }
1095 session.create_temporary_source(source_catalog);
1096 return Ok(PgResponse::empty_result(StatementType::CREATE_SOURCE));
1097 }
1098
1099 let source = source_catalog.to_prost();
1100
1101 let catalog_writer = session.catalog_writer()?;
1102
1103 if create_source_type.is_shared() {
1104 let graph = generate_stream_graph_for_source(handler_args, source_catalog)?;
1105 catalog_writer.create_source(source, Some(graph)).await?;
1106 } else {
1107 catalog_writer.create_source(source, None).await?;
1109 }
1110
1111 Ok(PgResponse::empty_result(StatementType::CREATE_SOURCE))
1112}
1113
1114pub(super) fn generate_stream_graph_for_source(
1115 handler_args: HandlerArgs,
1116 source_catalog: SourceCatalog,
1117) -> Result<PbStreamFragmentGraph> {
1118 let context = OptimizerContext::from_handler_args(handler_args);
1119 let source_node = LogicalSource::with_catalog(
1120 Rc::new(source_catalog),
1121 SourceNodeKind::CreateSharedSource,
1122 context.into(),
1123 None,
1124 )?;
1125
1126 let stream_plan = source_node.to_stream(&mut ToStreamContext::new(false))?;
1127 let graph = build_graph(stream_plan, Some(GraphJobType::Source))?;
1128 Ok(graph)
1129}
1130
1131#[cfg(test)]
1132pub mod tests {
1133 use std::collections::HashMap;
1134 use std::sync::Arc;
1135
1136 use risingwave_common::catalog::{
1137 DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, ROW_ID_COLUMN_NAME,
1138 };
1139 use risingwave_common::types::{DataType, StructType};
1140
1141 use crate::catalog::root_catalog::SchemaPath;
1142 use crate::catalog::source_catalog::SourceCatalog;
1143 use crate::test_utils::{LocalFrontend, PROTO_FILE_DATA, create_proto_file};
1144
1145 const GET_COLUMN_FROM_CATALOG: fn(&Arc<SourceCatalog>) -> HashMap<&str, DataType> =
1146 |catalog: &Arc<SourceCatalog>| -> HashMap<&str, DataType> {
1147 catalog
1148 .columns
1149 .iter()
1150 .map(|col| (col.name(), col.data_type().clone()))
1151 .collect::<HashMap<&str, DataType>>()
1152 };
1153
1154 #[tokio::test]
1155 async fn test_create_source_handler() {
1156 let proto_file = create_proto_file(PROTO_FILE_DATA);
1157 let sql = format!(
1158 r#"CREATE SOURCE t
1159 WITH (connector = 'kinesis')
1160 FORMAT PLAIN ENCODE PROTOBUF (message = '.test.TestRecord', schema.location = 'file://{}')"#,
1161 proto_file.path().to_str().unwrap()
1162 );
1163 let frontend = LocalFrontend::new(Default::default()).await;
1164 frontend.run_sql(sql).await.unwrap();
1165
1166 let session = frontend.session_ref();
1167 let catalog_reader = session.env().catalog_reader().read_guard();
1168 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
1169
1170 let (source, _) = catalog_reader
1172 .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
1173 .unwrap();
1174 assert_eq!(source.name, "t");
1175
1176 let columns = GET_COLUMN_FROM_CATALOG(source);
1177
1178 let city_type = StructType::new(vec![
1179 ("address", DataType::Varchar),
1180 ("zipcode", DataType::Varchar),
1181 ])
1182 .into();
1184 let expected_columns = maplit::hashmap! {
1185 ROW_ID_COLUMN_NAME => DataType::Serial,
1186 "id" => DataType::Int32,
1187 "zipcode" => DataType::Int64,
1188 "rate" => DataType::Float32,
1189 "country" => StructType::new(
1190 vec![("address", DataType::Varchar),("city", city_type),("zipcode", DataType::Varchar)],
1191 )
1192 .into(),
1194 };
1195 assert_eq!(columns, expected_columns, "{columns:#?}");
1196 }
1197
1198 #[tokio::test]
1199 async fn test_duplicate_props_options() {
1200 let proto_file = create_proto_file(PROTO_FILE_DATA);
1201 let sql = format!(
1202 r#"CREATE SOURCE t
1203 WITH (
1204 connector = 'kinesis',
1205 aws.region='user_test_topic',
1206 endpoint='172.10.1.1:9090,172.10.1.2:9090',
1207 aws.credentials.access_key_id = 'your_access_key_1',
1208 aws.credentials.secret_access_key = 'your_secret_key_1'
1209 )
1210 FORMAT PLAIN ENCODE PROTOBUF (
1211 message = '.test.TestRecord',
1212 aws.credentials.access_key_id = 'your_access_key_2',
1213 aws.credentials.secret_access_key = 'your_secret_key_2',
1214 schema.location = 'file://{}',
1215 )"#,
1216 proto_file.path().to_str().unwrap()
1217 );
1218 let frontend = LocalFrontend::new(Default::default()).await;
1219 frontend.run_sql(sql).await.unwrap();
1220
1221 let session = frontend.session_ref();
1222 let catalog_reader = session.env().catalog_reader().read_guard();
1223 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
1224
1225 let (source, _) = catalog_reader
1227 .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
1228 .unwrap();
1229 assert_eq!(source.name, "t");
1230
1231 assert_eq!(
1233 source
1234 .info
1235 .format_encode_options
1236 .get("aws.credentials.access_key_id")
1237 .unwrap(),
1238 "your_access_key_2"
1239 );
1240 assert_eq!(
1241 source
1242 .info
1243 .format_encode_options
1244 .get("aws.credentials.secret_access_key")
1245 .unwrap(),
1246 "your_secret_key_2"
1247 );
1248
1249 assert_eq!(
1251 source
1252 .with_properties
1253 .get("aws.credentials.access_key_id")
1254 .unwrap(),
1255 "your_access_key_1"
1256 );
1257 assert_eq!(
1258 source
1259 .with_properties
1260 .get("aws.credentials.secret_access_key")
1261 .unwrap(),
1262 "your_secret_key_1"
1263 );
1264
1265 assert!(!source.with_properties.contains_key("schema.location"));
1267 }
1268
1269 #[tokio::test]
1270 async fn test_multi_table_cdc_create_source_handler() {
1271 let sql =
1272 "CREATE SOURCE t2 WITH (connector = 'mysql-cdc') FORMAT PLAIN ENCODE JSON".to_owned();
1273 let frontend = LocalFrontend::new(Default::default()).await;
1274 let session = frontend.session_ref();
1275
1276 frontend
1277 .run_sql_with_session(session.clone(), sql)
1278 .await
1279 .unwrap();
1280 let catalog_reader = session.env().catalog_reader().read_guard();
1281 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
1282
1283 let (source, _) = catalog_reader
1285 .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t2")
1286 .unwrap();
1287 assert_eq!(source.name, "t2");
1288
1289 let columns = source
1290 .columns
1291 .iter()
1292 .map(|col| (col.name(), col.data_type().clone()))
1293 .collect::<Vec<(&str, DataType)>>();
1294
1295 expect_test::expect![[r#"
1296 [
1297 (
1298 "payload",
1299 Jsonb,
1300 ),
1301 (
1302 "_rw_offset",
1303 Varchar,
1304 ),
1305 (
1306 "_rw_table_name",
1307 Varchar,
1308 ),
1309 (
1310 "_row_id",
1311 Serial,
1312 ),
1313 ]
1314 "#]]
1315 .assert_debug_eq(&columns);
1316 }
1317
1318 #[tokio::test]
1319 async fn test_source_addition_columns() {
1320 let sql =
1322 "CREATE SOURCE s (v1 int) include key as _rw_kafka_key with (connector = 'kafka') format plain encode json".to_owned();
1323 let frontend = LocalFrontend::new(Default::default()).await;
1324 frontend.run_sql(sql).await.unwrap();
1325 let session = frontend.session_ref();
1326 let catalog_reader = session.env().catalog_reader().read_guard();
1327 let (source, _) = catalog_reader
1328 .get_source_by_name(
1329 DEFAULT_DATABASE_NAME,
1330 SchemaPath::Name(DEFAULT_SCHEMA_NAME),
1331 "s",
1332 )
1333 .unwrap();
1334 assert_eq!(source.name, "s");
1335
1336 let columns = source
1337 .columns
1338 .iter()
1339 .map(|col| (col.name(), col.data_type().clone()))
1340 .collect::<Vec<(&str, DataType)>>();
1341
1342 expect_test::expect![[r#"
1343 [
1344 (
1345 "v1",
1346 Int32,
1347 ),
1348 (
1349 "_rw_kafka_key",
1350 Bytea,
1351 ),
1352 (
1353 "_rw_kafka_timestamp",
1354 Timestamptz,
1355 ),
1356 (
1357 "_rw_kafka_partition",
1358 Varchar,
1359 ),
1360 (
1361 "_rw_kafka_offset",
1362 Varchar,
1363 ),
1364 (
1365 "_row_id",
1366 Serial,
1367 ),
1368 ]
1369 "#]]
1370 .assert_debug_eq(&columns);
1371
1372 let sql =
1373 "CREATE SOURCE s3 (v1 int) include timestamp 'header1' as header_col with (connector = 'kafka') format plain encode json".to_owned();
1374 match frontend.run_sql(sql).await {
1375 Err(e) => {
1376 assert_eq!(
1377 e.to_string(),
1378 "Protocol error: Only header column can have inner field, but got \"timestamp\""
1379 )
1380 }
1381 _ => unreachable!(),
1382 }
1383 }
1384}