1use std::collections::{BTreeMap, HashMap, HashSet};
16use std::rc::Rc;
17use std::sync::Arc;
18
19use anyhow::{Context, anyhow};
20use clap::ValueEnum;
21use either::Either;
22use fixedbitset::FixedBitSet;
23use itertools::Itertools;
24use percent_encoding::percent_decode_str;
25use pgwire::pg_response::{PgResponse, StatementType};
26use prost::Message as _;
27use risingwave_common::catalog::{
28 CdcTableDesc, ColumnCatalog, ColumnDesc, ConflictBehavior, DEFAULT_SCHEMA_NAME, Engine,
29 ICEBERG_SINK_PREFIX, ICEBERG_SOURCE_PREFIX, RISINGWAVE_ICEBERG_ROW_ID, ROW_ID_COLUMN_NAME,
30 TableId,
31};
32use risingwave_common::config::MetaBackend;
33use risingwave_common::global_jvm::Jvm;
34use risingwave_common::session_config::sink_decouple::SinkDecouple;
35use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
36use risingwave_common::util::value_encoding::DatumToProtoExt;
37use risingwave_common::{bail, bail_not_implemented};
38use risingwave_connector::sink::decouple_checkpoint_log_sink::COMMIT_CHECKPOINT_INTERVAL;
39use risingwave_connector::source::cdc::build_cdc_table_id;
40use risingwave_connector::source::cdc::external::{
41 DATABASE_NAME_KEY, ExternalCdcTableType, ExternalTableConfig, ExternalTableImpl,
42 SCHEMA_NAME_KEY, TABLE_NAME_KEY,
43};
44use risingwave_connector::{WithOptionsSecResolved, WithPropertiesExt, source};
45use risingwave_pb::catalog::connection::Info as ConnectionInfo;
46use risingwave_pb::catalog::connection_params::ConnectionType;
47use risingwave_pb::catalog::{PbSource, PbWebhookSourceInfo, WatermarkDesc};
48use risingwave_pb::ddl_service::{PbTableJobType, TableJobType};
49use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn;
50use risingwave_pb::plan_common::{
51 AdditionalColumn, ColumnDescVersion, DefaultColumnDesc, GeneratedColumnDesc,
52};
53use risingwave_pb::secret::PbSecretRef;
54use risingwave_pb::secret::secret_ref::PbRefAsType;
55use risingwave_pb::stream_plan::StreamFragmentGraph;
56use risingwave_sqlparser::ast::{
57 CdcTableInfo, ColumnDef, ColumnOption, CompatibleFormatEncode, ConnectionRefValue, CreateSink,
58 CreateSinkStatement, CreateSourceStatement, DataType as AstDataType, ExplainOptions, Format,
59 FormatEncodeOptions, Ident, ObjectName, OnConflict, SecretRefAsType, SourceWatermark,
60 Statement, TableConstraint, WebhookSourceInfo, WithProperties,
61};
62use risingwave_sqlparser::parser::{IncludeOption, Parser};
63use thiserror_ext::AsReport;
64
65use super::RwPgResponse;
66use super::create_source::{CreateSourceType, SqlColumnStrategy, bind_columns_from_source};
67use crate::binder::{Clause, SecureCompareContext, bind_data_type};
68use crate::catalog::root_catalog::SchemaPath;
69use crate::catalog::source_catalog::SourceCatalog;
70use crate::catalog::table_catalog::TableVersion;
71use crate::catalog::{ColumnId, DatabaseId, SchemaId, SourceId, check_column_name_not_reserved};
72use crate::error::{ErrorCode, Result, RwError, bail_bind_error};
73use crate::expr::{Expr, ExprImpl, ExprRewriter};
74use crate::handler::HandlerArgs;
75use crate::handler::create_source::{
76 UPSTREAM_SOURCE_KEY, bind_connector_props, bind_create_source_or_table_with_connector,
77 bind_source_watermark, handle_addition_columns,
78};
79use crate::handler::util::{
80 LongRunningNotificationAction, SourceSchemaCompatExt, execute_with_long_running_notification,
81};
82use crate::optimizer::plan_node::generic::{SourceNodeKind, build_cdc_scan_options_with_options};
83use crate::optimizer::plan_node::{
84 LogicalCdcScan, LogicalPlanRef, LogicalSource, StreamPlanRef as PlanRef,
85};
86use crate::optimizer::property::{Order, RequiredDist};
87use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRoot};
88use crate::session::SessionImpl;
89use crate::session::current::notice_to_user;
90use crate::stream_fragmenter::{GraphJobType, build_graph};
91use crate::utils::OverwriteOptions;
92use crate::{Binder, Explain, TableCatalog, WithOptions};
93
94mod col_id_gen;
95pub use col_id_gen::*;
96use risingwave_connector::sink::SinkParam;
97use risingwave_connector::sink::iceberg::{
98 COMPACTION_DELETE_FILES_COUNT_THRESHOLD, COMPACTION_INTERVAL_SEC, COMPACTION_MAX_SNAPSHOTS_NUM,
99 COMPACTION_SMALL_FILES_THRESHOLD_MB, COMPACTION_TARGET_FILE_SIZE_MB,
100 COMPACTION_TRIGGER_SNAPSHOT_COUNT, COMPACTION_TYPE, CompactionType, ENABLE_COMPACTION,
101 ENABLE_SNAPSHOT_EXPIRATION, ICEBERG_WRITE_MODE_COPY_ON_WRITE, ICEBERG_WRITE_MODE_MERGE_ON_READ,
102 IcebergSink, IcebergWriteMode, SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_FILES,
103 SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_META_DATA, SNAPSHOT_EXPIRATION_MAX_AGE_MILLIS,
104 SNAPSHOT_EXPIRATION_RETAIN_LAST, WRITE_MODE, parse_partition_by_exprs,
105};
106use risingwave_pb::ddl_service::create_iceberg_table_request::{PbSinkJobInfo, PbTableJobInfo};
107
108use crate::handler::create_sink::{SinkPlanContext, gen_sink_plan};
109
110fn ensure_column_options_supported(c: &ColumnDef) -> Result<()> {
111 for option_def in &c.options {
112 match option_def.option {
113 ColumnOption::GeneratedColumns(_) => {}
114 ColumnOption::DefaultValue(_) => {}
115 ColumnOption::DefaultValueInternal { .. } => {}
116 ColumnOption::Unique { is_primary: true } => {}
117 ColumnOption::Null => {}
118 ColumnOption::NotNull => {}
119 _ => bail_not_implemented!("column constraints \"{}\"", option_def),
120 }
121 }
122 Ok(())
123}
124
125pub fn bind_sql_columns(
129 column_defs: &[ColumnDef],
130 is_for_drop_table_connector: bool,
131) -> Result<Vec<ColumnCatalog>> {
132 let mut columns = Vec::with_capacity(column_defs.len());
133
134 for column in column_defs {
135 ensure_column_options_supported(column)?;
136 let ColumnDef {
140 name,
141 data_type,
142 collation,
143 options,
144 ..
145 } = column;
146
147 let data_type = data_type
148 .clone()
149 .ok_or_else(|| ErrorCode::InvalidInputSyntax("data type is not specified".into()))?;
150 if let Some(collation) = collation {
151 if !["C", "POSIX"].contains(&collation.real_value().as_str()) {
157 bail_not_implemented!(
158 "Collate collation other than `C` or `POSIX` is not implemented"
159 );
160 }
161
162 match data_type {
163 AstDataType::Text | AstDataType::Varchar | AstDataType::Char(_) => {}
164 _ => {
165 return Err(ErrorCode::NotSupported(
166 format!("{} is not a collatable data type", data_type),
167 "The only built-in collatable data types are `varchar`, please check your type".into(),
168 ).into());
169 }
170 }
171 }
172
173 if !is_for_drop_table_connector {
174 check_column_name_not_reserved(&name.real_value())?;
178 }
179
180 let nullable: bool = !options
181 .iter()
182 .any(|def| matches!(def.option, ColumnOption::NotNull));
183
184 columns.push(ColumnCatalog {
185 column_desc: ColumnDesc {
186 data_type: bind_data_type(&data_type)?,
187 column_id: ColumnId::placeholder(),
188 name: name.real_value(),
189 generated_or_default_column: None,
190 description: None,
191 additional_column: AdditionalColumn { column_type: None },
192 version: ColumnDescVersion::LATEST,
193 system_column: None,
194 nullable,
195 },
196 is_hidden: false,
197 });
198 }
199
200 Ok(columns)
201}
202
203fn check_generated_column_constraints(
204 column_name: &String,
205 column_id: ColumnId,
206 expr: &ExprImpl,
207 column_catalogs: &[ColumnCatalog],
208 generated_column_names: &[String],
209 pk_column_ids: &[ColumnId],
210) -> Result<()> {
211 let input_refs = expr.collect_input_refs(column_catalogs.len());
212 for idx in input_refs.ones() {
213 let referred_generated_column = &column_catalogs[idx].column_desc.name;
214 if generated_column_names
215 .iter()
216 .any(|c| c == referred_generated_column)
217 {
218 return Err(ErrorCode::BindError(format!(
219 "Generated can not reference another generated column. \
220 But here generated column \"{}\" referenced another generated column \"{}\"",
221 column_name, referred_generated_column
222 ))
223 .into());
224 }
225 }
226
227 if pk_column_ids.contains(&column_id) && expr.is_impure() {
228 return Err(ErrorCode::BindError(format!(
229 "Generated columns with impure expressions should not be part of the primary key. \
230 Here column \"{}\" is defined as part of the primary key.",
231 column_name
232 ))
233 .into());
234 }
235
236 Ok(())
237}
238
239pub fn bind_sql_column_constraints(
242 session: &SessionImpl,
243 table_name: String,
244 column_catalogs: &mut [ColumnCatalog],
245 columns: &[ColumnDef],
246 pk_column_ids: &[ColumnId],
247) -> Result<()> {
248 let generated_column_names = {
249 let mut names = vec![];
250 for column in columns {
251 for option_def in &column.options {
252 if let ColumnOption::GeneratedColumns(_) = option_def.option {
253 names.push(column.name.real_value());
254 break;
255 }
256 }
257 }
258 names
259 };
260
261 let mut binder = Binder::new_for_ddl(session);
262 binder.bind_columns_to_context(table_name, column_catalogs)?;
263
264 for column in columns {
265 let Some(idx) = column_catalogs
266 .iter()
267 .position(|c| c.name() == column.name.real_value())
268 else {
269 continue;
272 };
273
274 for option_def in &column.options {
275 match &option_def.option {
276 ColumnOption::GeneratedColumns(expr) => {
277 binder.set_clause(Some(Clause::GeneratedColumn));
278
279 let expr_impl = binder.bind_expr(expr).with_context(|| {
280 format!(
281 "fail to bind expression in generated column \"{}\"",
282 column.name.real_value()
283 )
284 })?;
285
286 check_generated_column_constraints(
287 &column.name.real_value(),
288 column_catalogs[idx].column_id(),
289 &expr_impl,
290 column_catalogs,
291 &generated_column_names,
292 pk_column_ids,
293 )?;
294
295 column_catalogs[idx].column_desc.generated_or_default_column = Some(
296 GeneratedOrDefaultColumn::GeneratedColumn(GeneratedColumnDesc {
297 expr: Some(expr_impl.to_expr_proto()),
298 }),
299 );
300 binder.set_clause(None);
301 }
302 ColumnOption::DefaultValue(expr) => {
303 let expr_impl = binder
304 .bind_expr(expr)?
305 .cast_assign(column_catalogs[idx].data_type())?;
306
307 let rewritten_expr_impl = session
315 .pinned_snapshot()
316 .inline_now_proc_time()
317 .rewrite_expr(expr_impl.clone());
318
319 if let Some(snapshot_value) = rewritten_expr_impl.try_fold_const() {
320 let snapshot_value = snapshot_value?;
321
322 column_catalogs[idx].column_desc.generated_or_default_column =
323 Some(GeneratedOrDefaultColumn::DefaultColumn(DefaultColumnDesc {
324 snapshot_value: Some(snapshot_value.to_protobuf()),
325 expr: Some(expr_impl.to_expr_proto()),
326 }));
328 } else {
329 return Err(ErrorCode::BindError(format!(
330 "Default expression used in column `{}` cannot be evaluated. \
331 Use generated columns instead if you mean to reference other columns.",
332 column.name
333 ))
334 .into());
335 }
336 }
337 ColumnOption::DefaultValueInternal { persisted, expr: _ } => {
338 if persisted.is_empty() {
343 bail_bind_error!(
344 "DEFAULT INTERNAL is only used for internal purposes, \
345 please specify a concrete default value"
346 );
347 }
348
349 let desc = DefaultColumnDesc::decode(&**persisted)
350 .expect("failed to decode persisted `DefaultColumnDesc`");
351
352 column_catalogs[idx].column_desc.generated_or_default_column =
353 Some(GeneratedOrDefaultColumn::DefaultColumn(desc));
354 }
355 _ => {}
356 }
357 }
358 }
359 Ok(())
360}
361
362pub fn bind_table_constraints(table_constraints: &[TableConstraint]) -> Result<Vec<String>> {
364 let mut pk_column_names = vec![];
365
366 for constraint in table_constraints {
367 match constraint {
368 TableConstraint::Unique {
369 name: _,
370 columns,
371 is_primary: true,
372 } => {
373 if !pk_column_names.is_empty() {
374 return Err(multiple_pk_definition_err());
375 }
376 pk_column_names = columns.iter().map(|c| c.real_value()).collect_vec();
377 }
378 _ => bail_not_implemented!("table constraint \"{}\"", constraint),
379 }
380 }
381 Ok(pk_column_names)
382}
383
384pub fn bind_sql_pk_names(
385 columns_defs: &[ColumnDef],
386 pk_names_from_table_constraints: Vec<String>,
387) -> Result<Vec<String>> {
388 let mut pk_column_names = pk_names_from_table_constraints;
389
390 for column in columns_defs {
391 for option_def in &column.options {
392 if let ColumnOption::Unique { is_primary: true } = option_def.option {
393 if !pk_column_names.is_empty() {
394 return Err(multiple_pk_definition_err());
395 }
396 pk_column_names.push(column.name.real_value());
397 };
398 }
399 }
400
401 Ok(pk_column_names)
402}
403
404fn multiple_pk_definition_err() -> RwError {
405 ErrorCode::BindError("multiple primary keys are not allowed".into()).into()
406}
407
408pub fn bind_pk_and_row_id_on_relation(
413 mut columns: Vec<ColumnCatalog>,
414 pk_names: Vec<String>,
415 must_need_pk: bool,
416) -> Result<(Vec<ColumnCatalog>, Vec<ColumnId>, Option<usize>)> {
417 for c in &columns {
418 assert!(c.column_id() != ColumnId::placeholder());
419 }
420
421 let name_to_id = columns
423 .iter()
424 .map(|c| (c.name(), c.column_id()))
425 .collect::<HashMap<_, _>>();
426
427 let mut pk_column_ids: Vec<_> = pk_names
428 .iter()
429 .map(|name| {
430 name_to_id.get(name.as_str()).copied().ok_or_else(|| {
431 ErrorCode::BindError(format!("column \"{name}\" named in key does not exist"))
432 })
433 })
434 .try_collect()?;
435
436 let need_row_id = pk_column_ids.is_empty() && must_need_pk;
438
439 let row_id_index = need_row_id.then(|| {
440 let column = ColumnCatalog::row_id_column();
441 let index = columns.len();
442 pk_column_ids = vec![column.column_id()];
443 columns.push(column);
444 index
445 });
446
447 if let Some(col) = columns.iter().map(|c| c.name()).duplicates().next() {
448 Err(ErrorCode::InvalidInputSyntax(format!(
449 "column \"{col}\" specified more than once"
450 )))?;
451 }
452
453 Ok((columns, pk_column_ids, row_id_index))
454}
455
456#[allow(clippy::too_many_arguments)]
459pub(crate) async fn gen_create_table_plan_with_source(
460 mut handler_args: HandlerArgs,
461 explain_options: ExplainOptions,
462 table_name: ObjectName,
463 column_defs: Vec<ColumnDef>,
464 wildcard_idx: Option<usize>,
465 constraints: Vec<TableConstraint>,
466 format_encode: FormatEncodeOptions,
467 source_watermarks: Vec<SourceWatermark>,
468 mut col_id_gen: ColumnIdGenerator,
469 include_column_options: IncludeOption,
470 props: CreateTableProps,
471 sql_column_strategy: SqlColumnStrategy,
472) -> Result<(PlanRef, Option<SourceCatalog>, TableCatalog)> {
473 if props.append_only
474 && format_encode.format != Format::Plain
475 && format_encode.format != Format::Native
476 {
477 return Err(ErrorCode::BindError(format!(
478 "Append only table does not support format {}.",
479 format_encode.format
480 ))
481 .into());
482 }
483
484 let session = &handler_args.session;
485 let (with_properties, refresh_mode) =
486 bind_connector_props(&handler_args, &format_encode, false)?;
487 if with_properties.is_shareable_cdc_connector() {
488 generated_columns_check_for_cdc_table(&column_defs)?;
489 not_null_check_for_cdc_table(&wildcard_idx, &column_defs)?;
490 } else if column_defs.iter().any(|col| {
491 col.options
492 .iter()
493 .any(|def| matches!(def.option, ColumnOption::NotNull))
494 }) {
495 notice_to_user(
497 "The table contains columns with NOT NULL constraints. Any rows from upstream violating the constraints will be ignored silently.",
498 );
499 }
500
501 let db_name: &str = &session.database();
502 let (schema_name, _) = Binder::resolve_schema_qualified_name(db_name, &table_name)?;
503
504 let (columns_from_resolve_source, source_info) = bind_columns_from_source(
506 session,
507 &format_encode,
508 Either::Left(&with_properties),
509 CreateSourceType::Table,
510 )
511 .await?;
512
513 let overwrite_options = OverwriteOptions::new(&mut handler_args);
514 let rate_limit = overwrite_options.source_rate_limit;
515 let source = bind_create_source_or_table_with_connector(
516 handler_args.clone(),
517 table_name,
518 format_encode,
519 with_properties,
520 &column_defs,
521 constraints,
522 wildcard_idx,
523 source_watermarks,
524 columns_from_resolve_source,
525 source_info,
526 include_column_options,
527 &mut col_id_gen,
528 CreateSourceType::Table,
529 rate_limit,
530 sql_column_strategy,
531 refresh_mode,
532 )
533 .await?;
534
535 let context = OptimizerContext::new(handler_args, explain_options);
536
537 let (plan, table) = gen_table_plan_with_source(
538 context.into(),
539 schema_name,
540 source.clone(),
541 col_id_gen.into_version(),
542 props,
543 )?;
544
545 Ok((plan, Some(source), table))
546}
547
548#[allow(clippy::too_many_arguments)]
551pub(crate) fn gen_create_table_plan(
552 context: OptimizerContext,
553 table_name: ObjectName,
554 column_defs: Vec<ColumnDef>,
555 constraints: Vec<TableConstraint>,
556 mut col_id_gen: ColumnIdGenerator,
557 source_watermarks: Vec<SourceWatermark>,
558 props: CreateTableProps,
559 is_for_replace_plan: bool,
560) -> Result<(PlanRef, TableCatalog)> {
561 let mut columns = bind_sql_columns(&column_defs, is_for_replace_plan)?;
562 for c in &mut columns {
563 col_id_gen.generate(c)?;
564 }
565
566 let (_, secret_refs, connection_refs) = context.with_options().clone().into_parts();
567 if !secret_refs.is_empty() || !connection_refs.is_empty() {
568 return Err(crate::error::ErrorCode::InvalidParameterValue("Secret reference and Connection reference are not allowed in options when creating table without external source".to_owned()).into());
569 }
570
571 gen_create_table_plan_without_source(
572 context,
573 table_name,
574 columns,
575 column_defs,
576 constraints,
577 source_watermarks,
578 col_id_gen.into_version(),
579 props,
580 )
581}
582
583#[allow(clippy::too_many_arguments)]
584pub(crate) fn gen_create_table_plan_without_source(
585 context: OptimizerContext,
586 table_name: ObjectName,
587 columns: Vec<ColumnCatalog>,
588 column_defs: Vec<ColumnDef>,
589 constraints: Vec<TableConstraint>,
590 source_watermarks: Vec<SourceWatermark>,
591 version: TableVersion,
592 props: CreateTableProps,
593) -> Result<(PlanRef, TableCatalog)> {
594 let pk_names = bind_sql_pk_names(&column_defs, bind_table_constraints(&constraints)?)?;
596 let (mut columns, pk_column_ids, row_id_index) =
597 bind_pk_and_row_id_on_relation(columns, pk_names, true)?;
598
599 let watermark_descs: Vec<WatermarkDesc> = bind_source_watermark(
600 context.session_ctx(),
601 table_name.real_value(),
602 source_watermarks,
603 &columns,
604 )?;
605
606 bind_sql_column_constraints(
607 context.session_ctx(),
608 table_name.real_value(),
609 &mut columns,
610 &column_defs,
611 &pk_column_ids,
612 )?;
613 let session = context.session_ctx().clone();
614
615 let db_name = &session.database();
616 let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, &table_name)?;
617
618 let info = CreateTableInfo {
619 columns,
620 pk_column_ids,
621 row_id_index,
622 watermark_descs,
623 source_catalog: None,
624 version,
625 };
626
627 gen_table_plan_inner(context.into(), schema_name, table_name, info, props)
628}
629
630fn gen_table_plan_with_source(
631 context: OptimizerContextRef,
632 schema_name: Option<String>,
633 source_catalog: SourceCatalog,
634 version: TableVersion,
635 props: CreateTableProps,
636) -> Result<(PlanRef, TableCatalog)> {
637 let table_name = source_catalog.name.clone();
638
639 let info = CreateTableInfo {
640 columns: source_catalog.columns.clone(),
641 pk_column_ids: source_catalog.pk_col_ids.clone(),
642 row_id_index: source_catalog.row_id_index,
643 watermark_descs: source_catalog.watermark_descs.clone(),
644 source_catalog: Some(source_catalog),
645 version,
646 };
647
648 gen_table_plan_inner(context, schema_name, table_name, info, props)
649}
650
651#[derive(Clone, Copy)]
653pub enum EitherOnConflict {
654 Ast(Option<OnConflict>),
655 Resolved(ConflictBehavior),
656}
657
658impl From<Option<OnConflict>> for EitherOnConflict {
659 fn from(v: Option<OnConflict>) -> Self {
660 Self::Ast(v)
661 }
662}
663
664impl From<ConflictBehavior> for EitherOnConflict {
665 fn from(v: ConflictBehavior) -> Self {
666 Self::Resolved(v)
667 }
668}
669
670impl EitherOnConflict {
671 pub fn to_behavior(self, append_only: bool, row_id_as_pk: bool) -> Result<ConflictBehavior> {
673 let conflict_behavior = match self {
674 EitherOnConflict::Ast(on_conflict) => {
675 if append_only {
676 if row_id_as_pk {
677 ConflictBehavior::NoCheck
679 } else {
680 if let Some(on_conflict) = on_conflict
682 && on_conflict != OnConflict::Nothing
683 {
684 return Err(ErrorCode::InvalidInputSyntax(
685 "When PRIMARY KEY constraint applied to an APPEND ONLY table, \
686 the ON CONFLICT behavior must be DO NOTHING."
687 .to_owned(),
688 )
689 .into());
690 }
691 ConflictBehavior::IgnoreConflict
692 }
693 } else {
694 match on_conflict.unwrap_or(OnConflict::UpdateFull) {
696 OnConflict::UpdateFull => ConflictBehavior::Overwrite,
697 OnConflict::Nothing => ConflictBehavior::IgnoreConflict,
698 OnConflict::UpdateIfNotNull => ConflictBehavior::DoUpdateIfNotNull,
699 }
700 }
701 }
702 EitherOnConflict::Resolved(b) => b,
703 };
704
705 Ok(conflict_behavior)
706 }
707}
708
709pub struct CreateTableInfo {
714 pub columns: Vec<ColumnCatalog>,
715 pub pk_column_ids: Vec<ColumnId>,
716 pub row_id_index: Option<usize>,
717 pub watermark_descs: Vec<WatermarkDesc>,
718 pub source_catalog: Option<SourceCatalog>,
719 pub version: TableVersion,
720}
721
722pub struct CreateTableProps {
727 pub definition: String,
728 pub append_only: bool,
729 pub on_conflict: EitherOnConflict,
730 pub with_version_columns: Vec<String>,
731 pub webhook_info: Option<PbWebhookSourceInfo>,
732 pub engine: Engine,
733}
734
735#[allow(clippy::too_many_arguments)]
736fn gen_table_plan_inner(
737 context: OptimizerContextRef,
738 schema_name: Option<String>,
739 table_name: String,
740 info: CreateTableInfo,
741 props: CreateTableProps,
742) -> Result<(PlanRef, TableCatalog)> {
743 let CreateTableInfo {
744 ref columns,
745 row_id_index,
746 ref watermark_descs,
747 ref source_catalog,
748 ..
749 } = info;
750 let CreateTableProps { append_only, .. } = props;
751
752 let (database_id, schema_id) = context
753 .session_ctx()
754 .get_database_and_schema_id_for_create(schema_name)?;
755
756 let session = context.session_ctx().clone();
757 let retention_seconds = context.with_options().retention_seconds();
758
759 let source_node: LogicalPlanRef = LogicalSource::new(
760 source_catalog.clone().map(Rc::new),
761 columns.clone(),
762 row_id_index,
763 SourceNodeKind::CreateTable,
764 context.clone(),
765 None,
766 )?
767 .into();
768
769 let required_cols = FixedBitSet::with_capacity(columns.len());
770 let plan_root = PlanRoot::new_with_logical_plan(
771 source_node,
772 RequiredDist::Any,
773 Order::any(),
774 required_cols,
775 vec![],
776 );
777
778 if !append_only && !watermark_descs.is_empty() {
779 return Err(ErrorCode::NotSupported(
780 "Defining watermarks on table requires the table to be append only.".to_owned(),
781 "Use the key words `APPEND ONLY`".to_owned(),
782 )
783 .into());
784 }
785
786 if !append_only && retention_seconds.is_some() {
787 if session
788 .config()
789 .unsafe_enable_storage_retention_for_non_append_only_tables()
790 {
791 tracing::warn!(
792 "Storage retention is enabled for non-append-only table {}. This may lead to stream inconsistency.",
793 table_name
794 );
795 const NOTICE: &str = "Storage retention is enabled for non-append-only table. \
796 This may lead to stream inconsistency and unrecoverable \
797 node failure if there is any row INSERT/UPDATE/DELETE operation \
798 corresponding to the TTLed primary keys";
799 session.notice_to_user(NOTICE);
800 } else {
801 return Err(ErrorCode::NotSupported(
802 "Defining retention seconds on table requires the table to be append only."
803 .to_owned(),
804 "Use the key words `APPEND ONLY`".to_owned(),
805 )
806 .into());
807 }
808 }
809
810 let materialize =
811 plan_root.gen_table_plan(context, table_name, database_id, schema_id, info, props)?;
812
813 let mut table = materialize.table().clone();
814 table.owner = session.user_id();
815
816 Ok((materialize.into(), table))
817}
818
819#[allow(clippy::too_many_arguments)]
823pub(crate) fn gen_create_table_plan_for_cdc_table(
824 context: OptimizerContextRef,
825 source: Arc<SourceCatalog>,
826 external_table_name: String,
827 column_defs: Vec<ColumnDef>,
828 mut columns: Vec<ColumnCatalog>,
829 pk_names: Vec<String>,
830 cdc_with_options: WithOptionsSecResolved,
831 mut col_id_gen: ColumnIdGenerator,
832 on_conflict: Option<OnConflict>,
833 with_version_columns: Vec<String>,
834 include_column_options: IncludeOption,
835 table_name: ObjectName,
836 resolved_table_name: String, database_id: DatabaseId,
838 schema_id: SchemaId,
839 table_id: TableId,
840 engine: Engine,
841) -> Result<(PlanRef, TableCatalog)> {
842 let session = context.session_ctx().clone();
843
844 handle_addition_columns(
846 None,
847 &cdc_with_options,
848 include_column_options,
849 &mut columns,
850 true,
851 )?;
852
853 for c in &mut columns {
854 col_id_gen.generate(c)?;
855 }
856
857 let (mut columns, pk_column_ids, _row_id_index) =
858 bind_pk_and_row_id_on_relation(columns, pk_names, true)?;
859
860 bind_sql_column_constraints(
862 context.session_ctx(),
863 table_name.real_value(),
864 &mut columns,
865 &column_defs,
866 &pk_column_ids,
867 )?;
868
869 let definition = context.normalized_sql().to_owned();
870
871 let pk_column_indices = {
872 let mut id_to_idx = HashMap::new();
873 columns.iter().enumerate().for_each(|(idx, c)| {
874 id_to_idx.insert(c.column_id(), idx);
875 });
876 pk_column_ids
878 .iter()
879 .map(|c| id_to_idx.get(c).copied().unwrap())
880 .collect_vec()
881 };
882 let table_pk = pk_column_indices
883 .iter()
884 .map(|idx| ColumnOrder::new(*idx, OrderType::ascending()))
885 .collect();
886
887 let (options, secret_refs) = cdc_with_options.into_parts();
888
889 let non_generated_column_descs = columns
890 .iter()
891 .filter(|&c| !c.is_generated())
892 .map(|c| c.column_desc.clone())
893 .collect_vec();
894 let non_generated_column_num = non_generated_column_descs.len();
895 let cdc_table_type = ExternalCdcTableType::from_properties(&options);
896 let cdc_table_desc = CdcTableDesc {
897 table_id,
898 source_id: source.id, external_table_name: external_table_name.clone(),
900 pk: table_pk,
901 columns: non_generated_column_descs,
902 stream_key: pk_column_indices,
903 connect_properties: options,
904 secret_refs,
905 };
906
907 tracing::debug!(?cdc_table_desc, "create cdc table");
908 let options = build_cdc_scan_options_with_options(context.with_options(), &cdc_table_type)?;
909
910 let logical_scan = LogicalCdcScan::create(
911 external_table_name.clone(),
912 Rc::new(cdc_table_desc),
913 context.clone(),
914 options,
915 );
916
917 let scan_node: LogicalPlanRef = logical_scan.into();
918 let required_cols = FixedBitSet::with_capacity(non_generated_column_num);
919 let plan_root = PlanRoot::new_with_logical_plan(
920 scan_node,
921 RequiredDist::Any,
922 Order::any(),
923 required_cols,
924 vec![],
925 );
926
927 let cdc_table_id = build_cdc_table_id(source.id, &external_table_name);
928 let materialize = plan_root.gen_table_plan(
929 context,
930 resolved_table_name,
931 database_id,
932 schema_id,
933 CreateTableInfo {
934 columns,
935 pk_column_ids,
936 row_id_index: None,
937 watermark_descs: vec![],
938 source_catalog: Some((*source).clone()),
939 version: col_id_gen.into_version(),
940 },
941 CreateTableProps {
942 definition,
943 append_only: false,
944 on_conflict: on_conflict.into(),
945 with_version_columns,
946 webhook_info: None,
947 engine,
948 },
949 )?;
950
951 let mut table = materialize.table().clone();
952 table.owner = session.user_id();
953 table.cdc_table_id = Some(cdc_table_id);
954 table.cdc_table_type = Some(cdc_table_type);
955 Ok((materialize.into(), table))
956}
957
958fn derive_with_options_for_cdc_table(
959 source_with_properties: &WithOptionsSecResolved,
960 external_table_name: String,
961) -> Result<WithOptionsSecResolved> {
962 use source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, SQL_SERVER_CDC_CONNECTOR};
963 let source_database_name: &str = source_with_properties
965 .get("database.name")
966 .ok_or_else(|| anyhow!("The source with properties does not contain 'database.name'"))?
967 .as_str();
968 let mut with_options = source_with_properties.clone();
969 if let Some(connector) = source_with_properties.get(UPSTREAM_SOURCE_KEY) {
970 match connector.as_str() {
971 MYSQL_CDC_CONNECTOR => {
972 let (db_name, table_name) = external_table_name.split_once('.').ok_or_else(|| {
975 anyhow!("The upstream table name must contain database name prefix, e.g. 'database.table'")
976 })?;
977 if !source_database_name
979 .split(',')
980 .map(|s| s.trim())
981 .any(|name| name == db_name)
982 {
983 return Err(anyhow!(
984 "The database name `{}` in the FROM clause is not included in the database name `{}` in source definition",
985 db_name,
986 source_database_name
987 ).into());
988 }
989 with_options.insert(DATABASE_NAME_KEY.into(), db_name.into());
990 with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
991 }
992 POSTGRES_CDC_CONNECTOR => {
993 let (schema_name, table_name) = external_table_name
994 .split_once('.')
995 .ok_or_else(|| anyhow!("The upstream table name must contain schema name prefix, e.g. 'public.table'"))?;
996
997 with_options.insert(SCHEMA_NAME_KEY.into(), schema_name.into());
999 with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
1000 }
1001 SQL_SERVER_CDC_CONNECTOR => {
1002 let parts: Vec<&str> = external_table_name.split('.').collect();
1008 let (_, schema_name, table_name) = match parts.len() {
1009 3 => {
1010 let db_name = parts[0];
1012 let schema_name = parts[1];
1013 let table_name = parts[2];
1014
1015 if db_name != source_database_name {
1017 return Err(anyhow!(
1018 "The database name `{}` in the FROM clause is not the same as the database name `{}` in source definition",
1019 db_name,
1020 source_database_name
1021 ).into());
1022 }
1023 (db_name, schema_name, table_name)
1024 }
1025 2 => {
1026 let schema_name = parts[0];
1028 let table_name = parts[1];
1029 (source_database_name, schema_name, table_name)
1030 }
1031 1 => {
1032 let table_name = parts[0];
1034 (source_database_name, "dbo", table_name)
1035 }
1036 _ => {
1037 return Err(anyhow!(
1038 "The upstream table name must be in one of these formats: 'database.schema.table', 'schema.table', or 'table'"
1039 ).into());
1040 }
1041 };
1042
1043 with_options.insert(SCHEMA_NAME_KEY.into(), schema_name.into());
1045 with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
1046 }
1047 _ => {
1048 return Err(RwError::from(anyhow!(
1049 "connector {} is not supported for cdc table",
1050 connector
1051 )));
1052 }
1053 };
1054 }
1055 Ok(with_options)
1056}
1057
1058#[allow(clippy::too_many_arguments)]
1059pub(super) async fn handle_create_table_plan(
1060 handler_args: HandlerArgs,
1061 explain_options: ExplainOptions,
1062 format_encode: Option<FormatEncodeOptions>,
1063 cdc_table_info: Option<CdcTableInfo>,
1064 table_name: &ObjectName,
1065 column_defs: Vec<ColumnDef>,
1066 wildcard_idx: Option<usize>,
1067 constraints: Vec<TableConstraint>,
1068 source_watermarks: Vec<SourceWatermark>,
1069 append_only: bool,
1070 on_conflict: Option<OnConflict>,
1071 with_version_columns: Vec<String>,
1072 include_column_options: IncludeOption,
1073 webhook_info: Option<WebhookSourceInfo>,
1074 engine: Engine,
1075) -> Result<(
1076 PlanRef,
1077 Option<SourceCatalog>,
1078 TableCatalog,
1079 TableJobType,
1080 Option<SourceId>,
1081)> {
1082 let col_id_gen = ColumnIdGenerator::new_initial();
1083 let format_encode = check_create_table_with_source(
1084 &handler_args.with_options,
1085 format_encode,
1086 &include_column_options,
1087 &cdc_table_info,
1088 )?;
1089 let webhook_info = webhook_info
1090 .map(|info| bind_webhook_info(&handler_args.session, &column_defs, info))
1091 .transpose()?;
1092
1093 let props = CreateTableProps {
1094 definition: handler_args.normalized_sql.clone(),
1095 append_only,
1096 on_conflict: on_conflict.into(),
1097 with_version_columns: with_version_columns.clone(),
1098 webhook_info,
1099 engine,
1100 };
1101
1102 let ((plan, source, table), job_type, shared_shource_id) = match (
1103 format_encode,
1104 cdc_table_info.as_ref(),
1105 ) {
1106 (Some(format_encode), None) => (
1107 gen_create_table_plan_with_source(
1108 handler_args,
1109 explain_options,
1110 table_name.clone(),
1111 column_defs,
1112 wildcard_idx,
1113 constraints,
1114 format_encode,
1115 source_watermarks,
1116 col_id_gen,
1117 include_column_options,
1118 props,
1119 SqlColumnStrategy::FollowChecked,
1120 )
1121 .await?,
1122 TableJobType::General,
1123 None,
1124 ),
1125 (None, None) => {
1126 let context = OptimizerContext::new(handler_args, explain_options);
1127 let (plan, table) = gen_create_table_plan(
1128 context,
1129 table_name.clone(),
1130 column_defs,
1131 constraints,
1132 col_id_gen,
1133 source_watermarks,
1134 props,
1135 false,
1136 )?;
1137
1138 ((plan, None, table), TableJobType::General, None)
1139 }
1140
1141 (None, Some(cdc_table)) => {
1142 sanity_check_for_table_on_cdc_source(
1143 append_only,
1144 &column_defs,
1145 &wildcard_idx,
1146 &constraints,
1147 &source_watermarks,
1148 )?;
1149
1150 generated_columns_check_for_cdc_table(&column_defs)?;
1151 not_null_check_for_cdc_table(&wildcard_idx, &column_defs)?;
1152
1153 let session = &handler_args.session;
1154 let db_name = &session.database();
1155 let user_name = &session.user_name();
1156 let search_path = session.config().search_path();
1157 let (schema_name, resolved_table_name) =
1158 Binder::resolve_schema_qualified_name(db_name, table_name)?;
1159 let (database_id, schema_id) =
1160 session.get_database_and_schema_id_for_create(schema_name.clone())?;
1161
1162 let (source_schema, source_name) =
1164 Binder::resolve_schema_qualified_name(db_name, &cdc_table.source_name)?;
1165
1166 let source = {
1167 let catalog_reader = session.env().catalog_reader().read_guard();
1168 let schema_path =
1169 SchemaPath::new(source_schema.as_deref(), &search_path, user_name);
1170
1171 let (source, _) = catalog_reader.get_source_by_name(
1172 db_name,
1173 schema_path,
1174 source_name.as_str(),
1175 )?;
1176 source.clone()
1177 };
1178 let cdc_with_options: WithOptionsSecResolved = derive_with_options_for_cdc_table(
1179 &source.with_properties,
1180 cdc_table.external_table_name.clone(),
1181 )?;
1182
1183 let (columns, pk_names) = match wildcard_idx {
1184 Some(_) => bind_cdc_table_schema_externally(cdc_with_options.clone()).await?,
1185 None => {
1186 for column_def in &column_defs {
1187 for option_def in &column_def.options {
1188 if let ColumnOption::DefaultValue(_)
1189 | ColumnOption::DefaultValueInternal { .. } = option_def.option
1190 {
1191 return Err(ErrorCode::NotSupported(
1192 "Default value for columns defined on the table created from a CDC source".into(),
1193 "Remove the default value expression in the column definitions".into(),
1194 )
1195 .into());
1196 }
1197 }
1198 }
1199
1200 let (columns, pk_names) =
1201 bind_cdc_table_schema(&column_defs, &constraints, false)?;
1202 let (options, secret_refs) = cdc_with_options.clone().into_parts();
1204 let _config = ExternalTableConfig::try_from_btreemap(options, secret_refs)
1205 .context("failed to extract external table config")?;
1206
1207 (columns, pk_names)
1208 }
1209 };
1210
1211 let context: OptimizerContextRef =
1212 OptimizerContext::new(handler_args, explain_options).into();
1213 let shared_source_id = source.id;
1214 let (plan, table) = gen_create_table_plan_for_cdc_table(
1215 context,
1216 source,
1217 cdc_table.external_table_name.clone(),
1218 column_defs,
1219 columns,
1220 pk_names,
1221 cdc_with_options,
1222 col_id_gen,
1223 on_conflict,
1224 with_version_columns,
1225 include_column_options,
1226 table_name.clone(),
1227 resolved_table_name,
1228 database_id,
1229 schema_id,
1230 TableId::placeholder(),
1231 engine,
1232 )?;
1233
1234 (
1235 (plan, None, table),
1236 TableJobType::SharedCdcSource,
1237 Some(shared_source_id),
1238 )
1239 }
1240 (Some(_), Some(_)) => {
1241 return Err(ErrorCode::NotSupported(
1242 "Data format and encoding format doesn't apply to table created from a CDC source"
1243 .into(),
1244 "Remove the FORMAT and ENCODE specification".into(),
1245 )
1246 .into());
1247 }
1248 };
1249 Ok((plan, source, table, job_type, shared_shource_id))
1250}
1251
1252fn generated_columns_check_for_cdc_table(columns: &Vec<ColumnDef>) -> Result<()> {
1254 let mut found_generated_column = false;
1255 for column in columns {
1256 let mut is_generated = false;
1257
1258 for option_def in &column.options {
1259 if let ColumnOption::GeneratedColumns(_) = option_def.option {
1260 is_generated = true;
1261 break;
1262 }
1263 }
1264
1265 if is_generated {
1266 found_generated_column = true;
1267 } else if found_generated_column {
1268 return Err(ErrorCode::NotSupported(
1269 "Non-generated column found after a generated column.".into(),
1270 "Ensure that all generated columns appear at the end of the cdc table definition."
1271 .into(),
1272 )
1273 .into());
1274 }
1275 }
1276 Ok(())
1277}
1278
1279fn not_null_check_for_cdc_table(
1281 wildcard_idx: &Option<usize>,
1282 column_defs: &Vec<ColumnDef>,
1283) -> Result<()> {
1284 if !wildcard_idx.is_some()
1285 && column_defs.iter().any(|col| {
1286 col.options
1287 .iter()
1288 .any(|opt| matches!(opt.option, ColumnOption::NotNull))
1289 })
1290 {
1291 return Err(ErrorCode::NotSupported(
1292 "CDC table with NOT NULL constraint is not supported".to_owned(),
1293 "Please remove the NOT NULL constraint for columns".to_owned(),
1294 )
1295 .into());
1296 }
1297 Ok(())
1298}
1299
1300fn sanity_check_for_table_on_cdc_source(
1302 append_only: bool,
1303 column_defs: &Vec<ColumnDef>,
1304 wildcard_idx: &Option<usize>,
1305 constraints: &Vec<TableConstraint>,
1306 source_watermarks: &Vec<SourceWatermark>,
1307) -> Result<()> {
1308 if wildcard_idx.is_some() && !column_defs.is_empty() {
1310 return Err(ErrorCode::NotSupported(
1311 "wildcard(*) and column definitions cannot be used together".to_owned(),
1312 "Remove the wildcard or column definitions".to_owned(),
1313 )
1314 .into());
1315 }
1316
1317 if !wildcard_idx.is_some()
1319 && !constraints.iter().any(|c| {
1320 matches!(
1321 c,
1322 TableConstraint::Unique {
1323 is_primary: true,
1324 ..
1325 }
1326 )
1327 })
1328 && !column_defs.iter().any(|col| {
1329 col.options
1330 .iter()
1331 .any(|opt| matches!(opt.option, ColumnOption::Unique { is_primary: true }))
1332 })
1333 {
1334 return Err(ErrorCode::NotSupported(
1335 "CDC table without primary key constraint is not supported".to_owned(),
1336 "Please define a primary key".to_owned(),
1337 )
1338 .into());
1339 }
1340
1341 if append_only {
1342 return Err(ErrorCode::NotSupported(
1343 "append only modifier on the table created from a CDC source".into(),
1344 "Remove the APPEND ONLY clause".into(),
1345 )
1346 .into());
1347 }
1348
1349 if !source_watermarks.is_empty() {
1350 return Err(ErrorCode::NotSupported(
1351 "watermark defined on the table created from a CDC source".into(),
1352 "Remove the Watermark definitions".into(),
1353 )
1354 .into());
1355 }
1356
1357 Ok(())
1358}
1359
1360async fn bind_cdc_table_schema_externally(
1362 cdc_with_options: WithOptionsSecResolved,
1363) -> Result<(Vec<ColumnCatalog>, Vec<String>)> {
1364 let (options, secret_refs) = cdc_with_options.into_parts();
1366 let config = ExternalTableConfig::try_from_btreemap(options, secret_refs)
1367 .context("failed to extract external table config")?;
1368
1369 let table = ExternalTableImpl::connect(config)
1370 .await
1371 .context("failed to auto derive table schema")?;
1372
1373 Ok((
1374 table
1375 .column_descs()
1376 .iter()
1377 .cloned()
1378 .map(|column_desc| ColumnCatalog {
1379 column_desc,
1380 is_hidden: false,
1381 })
1382 .collect(),
1383 table.pk_names().clone(),
1384 ))
1385}
1386
1387fn bind_cdc_table_schema(
1389 column_defs: &Vec<ColumnDef>,
1390 constraints: &Vec<TableConstraint>,
1391 is_for_replace_plan: bool,
1392) -> Result<(Vec<ColumnCatalog>, Vec<String>)> {
1393 let columns = bind_sql_columns(column_defs, is_for_replace_plan)?;
1394
1395 let pk_names = bind_sql_pk_names(column_defs, bind_table_constraints(constraints)?)?;
1396 Ok((columns, pk_names))
1397}
1398
1399#[allow(clippy::too_many_arguments)]
1400pub async fn handle_create_table(
1401 handler_args: HandlerArgs,
1402 table_name: ObjectName,
1403 column_defs: Vec<ColumnDef>,
1404 wildcard_idx: Option<usize>,
1405 constraints: Vec<TableConstraint>,
1406 if_not_exists: bool,
1407 format_encode: Option<FormatEncodeOptions>,
1408 source_watermarks: Vec<SourceWatermark>,
1409 append_only: bool,
1410 on_conflict: Option<OnConflict>,
1411 with_version_columns: Vec<String>,
1412 cdc_table_info: Option<CdcTableInfo>,
1413 include_column_options: IncludeOption,
1414 webhook_info: Option<WebhookSourceInfo>,
1415 ast_engine: risingwave_sqlparser::ast::Engine,
1416) -> Result<RwPgResponse> {
1417 let session = handler_args.session.clone();
1418
1419 if append_only {
1420 session.notice_to_user("APPEND ONLY TABLE is currently an experimental feature.");
1421 }
1422
1423 session.check_cluster_limits().await?;
1424
1425 let engine = match ast_engine {
1426 risingwave_sqlparser::ast::Engine::Hummock => Engine::Hummock,
1427 risingwave_sqlparser::ast::Engine::Iceberg => Engine::Iceberg,
1428 };
1429
1430 if let Either::Right(resp) = session.check_relation_name_duplicated(
1431 table_name.clone(),
1432 StatementType::CREATE_TABLE,
1433 if_not_exists,
1434 )? {
1435 return Ok(resp);
1436 }
1437
1438 let (graph, source, hummock_table, job_type, shared_source_id) = {
1439 let (plan, source, table, job_type, shared_source_id) = handle_create_table_plan(
1440 handler_args.clone(),
1441 ExplainOptions::default(),
1442 format_encode,
1443 cdc_table_info,
1444 &table_name,
1445 column_defs.clone(),
1446 wildcard_idx,
1447 constraints.clone(),
1448 source_watermarks,
1449 append_only,
1450 on_conflict,
1451 with_version_columns,
1452 include_column_options,
1453 webhook_info,
1454 engine,
1455 )
1456 .await?;
1457 tracing::trace!("table_plan: {:?}", plan.explain_to_string());
1458
1459 let graph = build_graph(plan, Some(GraphJobType::Table))?;
1460
1461 (graph, source, table, job_type, shared_source_id)
1462 };
1463
1464 tracing::trace!(
1465 "name={}, graph=\n{}",
1466 table_name,
1467 serde_json::to_string_pretty(&graph).unwrap()
1468 );
1469
1470 let dependencies = shared_source_id
1471 .map(|id| HashSet::from([id.as_object_id()]))
1472 .unwrap_or_default();
1473
1474 match engine {
1476 Engine::Hummock => {
1477 let catalog_writer = session.catalog_writer()?;
1478 let action = match job_type {
1479 TableJobType::SharedCdcSource => LongRunningNotificationAction::MonitorBackfillJob,
1480 _ => LongRunningNotificationAction::DiagnoseBarrierLatency,
1481 };
1482 execute_with_long_running_notification(
1483 catalog_writer.create_table(
1484 source.map(|s| s.to_prost()),
1485 hummock_table.to_prost(),
1486 graph,
1487 job_type,
1488 if_not_exists,
1489 dependencies,
1490 ),
1491 &session,
1492 "CREATE TABLE",
1493 action,
1494 )
1495 .await?;
1496 }
1497 Engine::Iceberg => {
1498 let hummock_table_name = hummock_table.name.clone();
1499 session.create_staging_table(hummock_table.clone());
1500 let res = create_iceberg_engine_table(
1501 session.clone(),
1502 handler_args,
1503 source.map(|s| s.to_prost()),
1504 hummock_table,
1505 graph,
1506 table_name,
1507 job_type,
1508 if_not_exists,
1509 )
1510 .await;
1511 session.drop_staging_table(&hummock_table_name);
1512 res?
1513 }
1514 }
1515
1516 Ok(PgResponse::empty_result(StatementType::CREATE_TABLE))
1517}
1518
1519#[allow(clippy::too_many_arguments)]
1528pub async fn create_iceberg_engine_table(
1529 session: Arc<SessionImpl>,
1530 handler_args: HandlerArgs,
1531 mut source: Option<PbSource>,
1532 table: TableCatalog,
1533 graph: StreamFragmentGraph,
1534 table_name: ObjectName,
1535 job_type: PbTableJobType,
1536 if_not_exists: bool,
1537) -> Result<()> {
1538 let meta_client = session.env().meta_client();
1539 let meta_store_endpoint = meta_client.get_meta_store_endpoint().await?;
1540
1541 let meta_store_endpoint = url::Url::parse(&meta_store_endpoint).map_err(|_| {
1542 ErrorCode::InternalError("failed to parse the meta store endpoint".to_owned())
1543 })?;
1544 let meta_store_backend = meta_store_endpoint.scheme().to_owned();
1545 let meta_store_user = meta_store_endpoint.username().to_owned();
1546 let meta_store_password = match meta_store_endpoint.password() {
1547 Some(password) => percent_decode_str(password)
1548 .decode_utf8()
1549 .map_err(|_| {
1550 ErrorCode::InternalError(
1551 "failed to parse password from meta store endpoint".to_owned(),
1552 )
1553 })?
1554 .into_owned(),
1555 None => "".to_owned(),
1556 };
1557 let meta_store_host = meta_store_endpoint
1558 .host_str()
1559 .ok_or_else(|| {
1560 ErrorCode::InternalError("failed to parse host from meta store endpoint".to_owned())
1561 })?
1562 .to_owned();
1563 let meta_store_port = meta_store_endpoint.port().ok_or_else(|| {
1564 ErrorCode::InternalError("failed to parse port from meta store endpoint".to_owned())
1565 })?;
1566 let meta_store_database = meta_store_endpoint
1567 .path()
1568 .trim_start_matches('/')
1569 .to_owned();
1570
1571 let Ok(meta_backend) = MetaBackend::from_str(&meta_store_backend, true) else {
1572 bail!("failed to parse meta backend: {}", meta_store_backend);
1573 };
1574
1575 let catalog_uri = match meta_backend {
1576 MetaBackend::Postgres => {
1577 format!(
1578 "jdbc:postgresql://{}:{}/{}",
1579 meta_store_host.clone(),
1580 meta_store_port.clone(),
1581 meta_store_database.clone()
1582 )
1583 }
1584 MetaBackend::Mysql => {
1585 format!(
1586 "jdbc:mysql://{}:{}/{}",
1587 meta_store_host.clone(),
1588 meta_store_port.clone(),
1589 meta_store_database.clone()
1590 )
1591 }
1592 MetaBackend::Sqlite | MetaBackend::Sql | MetaBackend::Mem => {
1593 bail!(
1594 "Unsupported meta backend for iceberg engine table: {}",
1595 meta_store_backend
1596 );
1597 }
1598 };
1599
1600 let rw_db_name = session
1601 .env()
1602 .catalog_reader()
1603 .read_guard()
1604 .get_database_by_id(table.database_id)?
1605 .name()
1606 .to_owned();
1607 let rw_schema_name = session
1608 .env()
1609 .catalog_reader()
1610 .read_guard()
1611 .get_schema_by_id(table.database_id, table.schema_id)?
1612 .name()
1613 .clone();
1614 let iceberg_catalog_name = rw_db_name.clone();
1615 let iceberg_database_name = rw_schema_name.clone();
1616 let iceberg_table_name = table_name.0.last().unwrap().real_value();
1617
1618 let iceberg_engine_connection: String = session.config().iceberg_engine_connection();
1619 let sink_decouple = session.config().sink_decouple();
1620 if matches!(sink_decouple, SinkDecouple::Disable) {
1621 bail!(
1622 "Iceberg engine table only supports with sink decouple, try `set sink_decouple = true` to resolve it"
1623 );
1624 }
1625
1626 let mut connection_ref = BTreeMap::new();
1627 let with_common = if iceberg_engine_connection.is_empty() {
1628 bail!("to use iceberg engine table, the variable `iceberg_engine_connection` must be set.");
1629 } else {
1630 let parts: Vec<&str> = iceberg_engine_connection.split('.').collect();
1631 assert_eq!(parts.len(), 2);
1632 let connection_catalog =
1633 session.get_connection_by_name(Some(parts[0].to_owned()), parts[1])?;
1634 if let ConnectionInfo::ConnectionParams(params) = &connection_catalog.info {
1635 if params.connection_type == ConnectionType::Iceberg as i32 {
1636 connection_ref.insert(
1638 "connection".to_owned(),
1639 ConnectionRefValue {
1640 connection_name: ObjectName::from(vec![
1641 Ident::from(parts[0]),
1642 Ident::from(parts[1]),
1643 ]),
1644 },
1645 );
1646
1647 let mut with_common = BTreeMap::new();
1648 with_common.insert("connector".to_owned(), "iceberg".to_owned());
1649 with_common.insert("database.name".to_owned(), iceberg_database_name);
1650 with_common.insert("table.name".to_owned(), iceberg_table_name);
1651
1652 if let Some(s) = params.properties.get("hosted_catalog")
1653 && s.eq_ignore_ascii_case("true")
1654 {
1655 with_common.insert("catalog.type".to_owned(), "jdbc".to_owned());
1656 with_common.insert("catalog.uri".to_owned(), catalog_uri);
1657 with_common.insert("catalog.jdbc.user".to_owned(), meta_store_user);
1658 with_common.insert(
1659 "catalog.jdbc.password".to_owned(),
1660 meta_store_password.clone(),
1661 );
1662 with_common.insert("catalog.name".to_owned(), iceberg_catalog_name);
1663 }
1664
1665 with_common
1666 } else {
1667 return Err(RwError::from(ErrorCode::InvalidParameterValue(
1668 "Only iceberg connection could be used in iceberg engine".to_owned(),
1669 )));
1670 }
1671 } else {
1672 return Err(RwError::from(ErrorCode::InvalidParameterValue(
1673 "Private Link Service has been deprecated. Please create a new connection instead."
1674 .to_owned(),
1675 )));
1676 }
1677 };
1678
1679 let mut pks = table
1682 .pk_column_names()
1683 .iter()
1684 .map(|c| c.to_string())
1685 .collect::<Vec<String>>();
1686
1687 let sink_from = if pks.len() == 1 && pks[0].eq(ROW_ID_COLUMN_NAME) {
1689 pks = vec![RISINGWAVE_ICEBERG_ROW_ID.to_owned()];
1690 let [stmt]: [_; 1] = Parser::parse_sql(&format!(
1691 "select {} as {}, * from {}",
1692 ROW_ID_COLUMN_NAME, RISINGWAVE_ICEBERG_ROW_ID, table_name
1693 ))
1694 .context("unable to parse query")?
1695 .try_into()
1696 .unwrap();
1697
1698 let Statement::Query(query) = &stmt else {
1699 panic!("unexpected statement: {:?}", stmt);
1700 };
1701 CreateSink::AsQuery(query.clone())
1702 } else {
1703 CreateSink::From(table_name.clone())
1704 };
1705
1706 let mut sink_name = table_name.clone();
1707 *sink_name.0.last_mut().unwrap() = Ident::from(
1708 (ICEBERG_SINK_PREFIX.to_owned() + &sink_name.0.last().unwrap().real_value()).as_str(),
1709 );
1710 let create_sink_stmt = CreateSinkStatement {
1711 if_not_exists: false,
1712 sink_name,
1713 with_properties: WithProperties(vec![]),
1714 sink_from,
1715 columns: vec![],
1716 emit_mode: None,
1717 sink_schema: None,
1718 into_table_name: None,
1719 };
1720
1721 let mut sink_handler_args = handler_args.clone();
1722
1723 let mut sink_with = with_common.clone();
1724
1725 if table.append_only {
1726 sink_with.insert("type".to_owned(), "append-only".to_owned());
1727 } else {
1728 sink_with.insert("primary_key".to_owned(), pks.join(","));
1729 sink_with.insert("type".to_owned(), "upsert".to_owned());
1730 }
1731 let commit_checkpoint_interval = handler_args
1750 .with_options
1751 .get(COMMIT_CHECKPOINT_INTERVAL)
1752 .map(|v| v.to_owned())
1753 .unwrap_or_else(|| "60".to_owned());
1754 let commit_checkpoint_interval = commit_checkpoint_interval.parse::<u32>().map_err(|_| {
1755 ErrorCode::InvalidInputSyntax(format!(
1756 "commit_checkpoint_interval must be greater than 0: {}",
1757 commit_checkpoint_interval
1758 ))
1759 })?;
1760
1761 if commit_checkpoint_interval == 0 {
1762 bail!("commit_checkpoint_interval must be greater than 0");
1763 }
1764
1765 source
1767 .as_mut()
1768 .map(|x| x.with_properties.remove(COMMIT_CHECKPOINT_INTERVAL));
1769
1770 let sink_decouple = session.config().sink_decouple();
1771 if matches!(sink_decouple, SinkDecouple::Disable) && commit_checkpoint_interval > 1 {
1772 bail!(
1773 "config conflict: `commit_checkpoint_interval` larger than 1 means that sink decouple must be enabled, but session config sink_decouple is disabled"
1774 )
1775 }
1776
1777 sink_with.insert(
1778 COMMIT_CHECKPOINT_INTERVAL.to_owned(),
1779 commit_checkpoint_interval.to_string(),
1780 );
1781 sink_with.insert("create_table_if_not_exists".to_owned(), "true".to_owned());
1782
1783 sink_with.insert("is_exactly_once".to_owned(), "true".to_owned());
1784
1785 if let Some(enable_compaction) = handler_args.with_options.get(ENABLE_COMPACTION) {
1786 match enable_compaction.to_lowercase().as_str() {
1787 "true" => {
1788 sink_with.insert(ENABLE_COMPACTION.to_owned(), "true".to_owned());
1789 }
1790 "false" => {
1791 sink_with.insert(ENABLE_COMPACTION.to_owned(), "false".to_owned());
1792 }
1793 _ => {
1794 return Err(ErrorCode::InvalidInputSyntax(format!(
1795 "enable_compaction must be true or false: {}",
1796 enable_compaction
1797 ))
1798 .into());
1799 }
1800 }
1801
1802 source
1804 .as_mut()
1805 .map(|x| x.with_properties.remove("enable_compaction"));
1806 } else {
1807 sink_with.insert(ENABLE_COMPACTION.to_owned(), "true".to_owned());
1808 }
1809
1810 if let Some(compaction_interval_sec) = handler_args.with_options.get(COMPACTION_INTERVAL_SEC) {
1811 let compaction_interval_sec = compaction_interval_sec.parse::<u64>().map_err(|_| {
1812 ErrorCode::InvalidInputSyntax(format!(
1813 "compaction_interval_sec must be greater than 0: {}",
1814 commit_checkpoint_interval
1815 ))
1816 })?;
1817 if compaction_interval_sec == 0 {
1818 bail!("compaction_interval_sec must be greater than 0");
1819 }
1820 sink_with.insert(
1821 "compaction_interval_sec".to_owned(),
1822 compaction_interval_sec.to_string(),
1823 );
1824 source
1826 .as_mut()
1827 .map(|x| x.with_properties.remove("compaction_interval_sec"));
1828 }
1829
1830 let has_enabled_snapshot_expiration = if let Some(enable_snapshot_expiration) =
1831 handler_args.with_options.get(ENABLE_SNAPSHOT_EXPIRATION)
1832 {
1833 source
1835 .as_mut()
1836 .map(|x| x.with_properties.remove(ENABLE_SNAPSHOT_EXPIRATION));
1837 match enable_snapshot_expiration.to_lowercase().as_str() {
1838 "true" => {
1839 sink_with.insert(ENABLE_SNAPSHOT_EXPIRATION.to_owned(), "true".to_owned());
1840 true
1841 }
1842 "false" => {
1843 sink_with.insert(ENABLE_SNAPSHOT_EXPIRATION.to_owned(), "false".to_owned());
1844 false
1845 }
1846 _ => {
1847 return Err(ErrorCode::InvalidInputSyntax(format!(
1848 "enable_snapshot_expiration must be true or false: {}",
1849 enable_snapshot_expiration
1850 ))
1851 .into());
1852 }
1853 }
1854 } else {
1855 sink_with.insert(ENABLE_SNAPSHOT_EXPIRATION.to_owned(), "true".to_owned());
1856 true
1857 };
1858
1859 if has_enabled_snapshot_expiration {
1860 if let Some(snapshot_expiration_retain_last) = handler_args
1862 .with_options
1863 .get(SNAPSHOT_EXPIRATION_RETAIN_LAST)
1864 {
1865 sink_with.insert(
1866 SNAPSHOT_EXPIRATION_RETAIN_LAST.to_owned(),
1867 snapshot_expiration_retain_last.to_owned(),
1868 );
1869 source
1871 .as_mut()
1872 .map(|x| x.with_properties.remove(SNAPSHOT_EXPIRATION_RETAIN_LAST));
1873 }
1874
1875 if let Some(snapshot_expiration_max_age) = handler_args
1876 .with_options
1877 .get(SNAPSHOT_EXPIRATION_MAX_AGE_MILLIS)
1878 {
1879 sink_with.insert(
1880 SNAPSHOT_EXPIRATION_MAX_AGE_MILLIS.to_owned(),
1881 snapshot_expiration_max_age.to_owned(),
1882 );
1883 source
1885 .as_mut()
1886 .map(|x| x.with_properties.remove(SNAPSHOT_EXPIRATION_MAX_AGE_MILLIS));
1887 }
1888
1889 if let Some(snapshot_expiration_clear_expired_files) = handler_args
1890 .with_options
1891 .get(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_FILES)
1892 {
1893 sink_with.insert(
1894 SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_FILES.to_owned(),
1895 snapshot_expiration_clear_expired_files.to_owned(),
1896 );
1897 source.as_mut().map(|x| {
1899 x.with_properties
1900 .remove(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_FILES)
1901 });
1902 }
1903
1904 if let Some(snapshot_expiration_clear_expired_meta_data) = handler_args
1905 .with_options
1906 .get(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_META_DATA)
1907 {
1908 sink_with.insert(
1909 SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_META_DATA.to_owned(),
1910 snapshot_expiration_clear_expired_meta_data.to_owned(),
1911 );
1912 source.as_mut().map(|x| {
1914 x.with_properties
1915 .remove(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_META_DATA)
1916 });
1917 }
1918 }
1919
1920 if let Some(write_mode) = handler_args.with_options.get(WRITE_MODE) {
1921 let write_mode = IcebergWriteMode::try_from(write_mode.as_str()).map_err(|_| {
1922 ErrorCode::InvalidInputSyntax(format!(
1923 "invalid write_mode: {}, must be one of: {}, {}",
1924 write_mode, ICEBERG_WRITE_MODE_MERGE_ON_READ, ICEBERG_WRITE_MODE_COPY_ON_WRITE
1925 ))
1926 })?;
1927
1928 match write_mode {
1929 IcebergWriteMode::MergeOnRead => {
1930 sink_with.insert(WRITE_MODE.to_owned(), write_mode.as_str().to_owned());
1931 }
1932
1933 IcebergWriteMode::CopyOnWrite => {
1934 if table.append_only {
1935 return Err(ErrorCode::NotSupported(
1936 "COPY ON WRITE is not supported for append-only iceberg table".to_owned(),
1937 "Please use MERGE ON READ instead".to_owned(),
1938 )
1939 .into());
1940 }
1941
1942 sink_with.insert(WRITE_MODE.to_owned(), write_mode.as_str().to_owned());
1943 }
1944 }
1945
1946 source
1948 .as_mut()
1949 .map(|x| x.with_properties.remove("write_mode"));
1950 } else {
1951 sink_with.insert(
1952 WRITE_MODE.to_owned(),
1953 ICEBERG_WRITE_MODE_MERGE_ON_READ.to_owned(),
1954 );
1955 }
1956
1957 if let Some(max_snapshots_num_before_compaction) =
1958 handler_args.with_options.get(COMPACTION_MAX_SNAPSHOTS_NUM)
1959 {
1960 let max_snapshots_num_before_compaction = max_snapshots_num_before_compaction
1961 .parse::<u32>()
1962 .map_err(|_| {
1963 ErrorCode::InvalidInputSyntax(format!(
1964 "{} must be greater than 0: {}",
1965 COMPACTION_MAX_SNAPSHOTS_NUM, max_snapshots_num_before_compaction
1966 ))
1967 })?;
1968
1969 if max_snapshots_num_before_compaction == 0 {
1970 bail!(format!(
1971 "{} must be greater than 0",
1972 COMPACTION_MAX_SNAPSHOTS_NUM
1973 ));
1974 }
1975
1976 sink_with.insert(
1977 COMPACTION_MAX_SNAPSHOTS_NUM.to_owned(),
1978 max_snapshots_num_before_compaction.to_string(),
1979 );
1980
1981 source
1983 .as_mut()
1984 .map(|x| x.with_properties.remove(COMPACTION_MAX_SNAPSHOTS_NUM));
1985 }
1986
1987 if let Some(small_files_threshold_mb) = handler_args
1988 .with_options
1989 .get(COMPACTION_SMALL_FILES_THRESHOLD_MB)
1990 {
1991 let small_files_threshold_mb = small_files_threshold_mb.parse::<u64>().map_err(|_| {
1992 ErrorCode::InvalidInputSyntax(format!(
1993 "{} must be greater than 0: {}",
1994 COMPACTION_SMALL_FILES_THRESHOLD_MB, small_files_threshold_mb
1995 ))
1996 })?;
1997 if small_files_threshold_mb == 0 {
1998 bail!(format!(
1999 "{} must be a greater than 0",
2000 COMPACTION_SMALL_FILES_THRESHOLD_MB
2001 ));
2002 }
2003 sink_with.insert(
2004 COMPACTION_SMALL_FILES_THRESHOLD_MB.to_owned(),
2005 small_files_threshold_mb.to_string(),
2006 );
2007
2008 source.as_mut().map(|x| {
2010 x.with_properties
2011 .remove(COMPACTION_SMALL_FILES_THRESHOLD_MB)
2012 });
2013 }
2014
2015 if let Some(delete_files_count_threshold) = handler_args
2016 .with_options
2017 .get(COMPACTION_DELETE_FILES_COUNT_THRESHOLD)
2018 {
2019 let delete_files_count_threshold =
2020 delete_files_count_threshold.parse::<usize>().map_err(|_| {
2021 ErrorCode::InvalidInputSyntax(format!(
2022 "{} must be greater than 0: {}",
2023 COMPACTION_DELETE_FILES_COUNT_THRESHOLD, delete_files_count_threshold
2024 ))
2025 })?;
2026 if delete_files_count_threshold == 0 {
2027 bail!(format!(
2028 "{} must be greater than 0",
2029 COMPACTION_DELETE_FILES_COUNT_THRESHOLD
2030 ));
2031 }
2032 sink_with.insert(
2033 COMPACTION_DELETE_FILES_COUNT_THRESHOLD.to_owned(),
2034 delete_files_count_threshold.to_string(),
2035 );
2036
2037 source.as_mut().map(|x| {
2039 x.with_properties
2040 .remove(COMPACTION_DELETE_FILES_COUNT_THRESHOLD)
2041 });
2042 }
2043
2044 if let Some(trigger_snapshot_count) = handler_args
2045 .with_options
2046 .get(COMPACTION_TRIGGER_SNAPSHOT_COUNT)
2047 {
2048 let trigger_snapshot_count = trigger_snapshot_count.parse::<usize>().map_err(|_| {
2049 ErrorCode::InvalidInputSyntax(format!(
2050 "{} must be greater than 0: {}",
2051 COMPACTION_TRIGGER_SNAPSHOT_COUNT, trigger_snapshot_count
2052 ))
2053 })?;
2054 if trigger_snapshot_count == 0 {
2055 bail!(format!(
2056 "{} must be greater than 0",
2057 COMPACTION_TRIGGER_SNAPSHOT_COUNT
2058 ));
2059 }
2060 sink_with.insert(
2061 COMPACTION_TRIGGER_SNAPSHOT_COUNT.to_owned(),
2062 trigger_snapshot_count.to_string(),
2063 );
2064
2065 source
2067 .as_mut()
2068 .map(|x| x.with_properties.remove(COMPACTION_TRIGGER_SNAPSHOT_COUNT));
2069 }
2070
2071 if let Some(target_file_size_mb) = handler_args
2072 .with_options
2073 .get(COMPACTION_TARGET_FILE_SIZE_MB)
2074 {
2075 let target_file_size_mb = target_file_size_mb.parse::<u64>().map_err(|_| {
2076 ErrorCode::InvalidInputSyntax(format!(
2077 "{} must be greater than 0: {}",
2078 COMPACTION_TARGET_FILE_SIZE_MB, target_file_size_mb
2079 ))
2080 })?;
2081 if target_file_size_mb == 0 {
2082 bail!(format!(
2083 "{} must be greater than 0",
2084 COMPACTION_TARGET_FILE_SIZE_MB
2085 ));
2086 }
2087 sink_with.insert(
2088 COMPACTION_TARGET_FILE_SIZE_MB.to_owned(),
2089 target_file_size_mb.to_string(),
2090 );
2091 source
2093 .as_mut()
2094 .map(|x| x.with_properties.remove(COMPACTION_TARGET_FILE_SIZE_MB));
2095 }
2096
2097 if let Some(compaction_type) = handler_args.with_options.get(COMPACTION_TYPE) {
2098 let compaction_type = CompactionType::try_from(compaction_type.as_str()).map_err(|_| {
2099 ErrorCode::InvalidInputSyntax(format!(
2100 "invalid compaction_type: {}, must be one of {:?}",
2101 compaction_type,
2102 &[
2103 CompactionType::Full,
2104 CompactionType::SmallFiles,
2105 CompactionType::FilesWithDelete
2106 ]
2107 ))
2108 })?;
2109
2110 sink_with.insert(
2111 COMPACTION_TYPE.to_owned(),
2112 compaction_type.as_str().to_owned(),
2113 );
2114
2115 source
2117 .as_mut()
2118 .map(|x| x.with_properties.remove(COMPACTION_TYPE));
2119 }
2120
2121 let partition_by = handler_args
2122 .with_options
2123 .get("partition_by")
2124 .map(|v| v.to_owned());
2125
2126 if let Some(partition_by) = &partition_by {
2127 let mut partition_columns = vec![];
2128 for (column, _) in parse_partition_by_exprs(partition_by.clone())? {
2129 table
2130 .columns()
2131 .iter()
2132 .find(|col| col.name().eq_ignore_ascii_case(&column))
2133 .ok_or_else(|| {
2134 ErrorCode::InvalidInputSyntax(format!(
2135 "Partition source column does not exist in schema: {}",
2136 column
2137 ))
2138 })?;
2139
2140 partition_columns.push(column);
2141 }
2142
2143 ensure_partition_columns_are_prefix_of_primary_key(&partition_columns, &pks).map_err(
2144 |_| {
2145 ErrorCode::InvalidInputSyntax(
2146 "The partition columns should be the prefix of the primary key".to_owned(),
2147 )
2148 },
2149 )?;
2150
2151 sink_with.insert("partition_by".to_owned(), partition_by.to_owned());
2152
2153 source
2155 .as_mut()
2156 .map(|x| x.with_properties.remove("partition_by"));
2157 }
2158
2159 sink_handler_args.with_options =
2160 WithOptions::new(sink_with, Default::default(), connection_ref.clone());
2161 let SinkPlanContext {
2162 sink_plan,
2163 sink_catalog,
2164 ..
2165 } = gen_sink_plan(sink_handler_args, create_sink_stmt, None, true).await?;
2166 let sink_graph = build_graph(sink_plan, Some(GraphJobType::Sink))?;
2167
2168 let mut source_name = table_name.clone();
2169 *source_name.0.last_mut().unwrap() = Ident::from(
2170 (ICEBERG_SOURCE_PREFIX.to_owned() + &source_name.0.last().unwrap().real_value()).as_str(),
2171 );
2172 let create_source_stmt = CreateSourceStatement {
2173 temporary: false,
2174 if_not_exists: false,
2175 columns: vec![],
2176 source_name,
2177 wildcard_idx: Some(0),
2178 constraints: vec![],
2179 with_properties: WithProperties(vec![]),
2180 format_encode: CompatibleFormatEncode::V2(FormatEncodeOptions::none()),
2181 source_watermarks: vec![],
2182 include_column_options: vec![],
2183 };
2184
2185 let mut source_handler_args = handler_args.clone();
2186 let source_with = with_common;
2187 source_handler_args.with_options =
2188 WithOptions::new(source_with, Default::default(), connection_ref);
2189
2190 let overwrite_options = OverwriteOptions::new(&mut source_handler_args);
2191 let format_encode = create_source_stmt.format_encode.into_v2_with_warning();
2192 let (with_properties, refresh_mode) =
2193 bind_connector_props(&source_handler_args, &format_encode, true)?;
2194
2195 let (iceberg_catalog, table_identifier) = {
2198 let sink_param = SinkParam::try_from_sink_catalog(sink_catalog.clone())?;
2199 let iceberg_sink = IcebergSink::try_from(sink_param)?;
2200 iceberg_sink.create_table_if_not_exists().await?;
2201
2202 let iceberg_catalog = iceberg_sink.config.create_catalog().await?;
2203 let table_identifier = iceberg_sink.config.full_table_name()?;
2204 (iceberg_catalog, table_identifier)
2205 };
2206
2207 let create_source_type = CreateSourceType::for_newly_created(&session, &*with_properties);
2208 let (columns_from_resolve_source, source_info) = bind_columns_from_source(
2209 &session,
2210 &format_encode,
2211 Either::Left(&with_properties),
2212 create_source_type,
2213 )
2214 .await?;
2215 let mut col_id_gen = ColumnIdGenerator::new_initial();
2216
2217 let iceberg_source_catalog = bind_create_source_or_table_with_connector(
2218 source_handler_args,
2219 create_source_stmt.source_name,
2220 format_encode,
2221 with_properties,
2222 &create_source_stmt.columns,
2223 create_source_stmt.constraints,
2224 create_source_stmt.wildcard_idx,
2225 create_source_stmt.source_watermarks,
2226 columns_from_resolve_source,
2227 source_info,
2228 create_source_stmt.include_column_options,
2229 &mut col_id_gen,
2230 create_source_type,
2231 overwrite_options.source_rate_limit,
2232 SqlColumnStrategy::FollowChecked,
2233 refresh_mode,
2234 )
2235 .await?;
2236
2237 let _ = Jvm::get_or_init()?;
2240
2241 let catalog_writer = session.catalog_writer()?;
2242 let action = match job_type {
2243 TableJobType::SharedCdcSource => LongRunningNotificationAction::MonitorBackfillJob,
2244 _ => LongRunningNotificationAction::DiagnoseBarrierLatency,
2245 };
2246 let res = execute_with_long_running_notification(
2247 catalog_writer.create_iceberg_table(
2248 PbTableJobInfo {
2249 source,
2250 table: Some(table.to_prost()),
2251 fragment_graph: Some(graph),
2252 job_type: job_type as _,
2253 },
2254 PbSinkJobInfo {
2255 sink: Some(sink_catalog.to_proto()),
2256 fragment_graph: Some(sink_graph),
2257 },
2258 iceberg_source_catalog.to_prost(),
2259 if_not_exists,
2260 ),
2261 &session,
2262 "CREATE TABLE",
2263 action,
2264 )
2265 .await;
2266
2267 if res.is_err() {
2268 let _ = iceberg_catalog
2269 .drop_table(&table_identifier)
2270 .await
2271 .inspect_err(|err| {
2272 tracing::error!(
2273 "failed to drop iceberg table {} after create iceberg engine table failed: {}",
2274 table_identifier,
2275 err.as_report()
2276 );
2277 });
2278 res?
2279 }
2280
2281 Ok(())
2282}
2283
2284pub fn check_create_table_with_source(
2285 with_options: &WithOptions,
2286 format_encode: Option<FormatEncodeOptions>,
2287 include_column_options: &IncludeOption,
2288 cdc_table_info: &Option<CdcTableInfo>,
2289) -> Result<Option<FormatEncodeOptions>> {
2290 if cdc_table_info.is_some() {
2292 return Ok(format_encode);
2293 }
2294 let defined_source = with_options.is_source_connector();
2295
2296 if !include_column_options.is_empty() && !defined_source {
2297 return Err(ErrorCode::InvalidInputSyntax(
2298 "INCLUDE should be used with a connector".to_owned(),
2299 )
2300 .into());
2301 }
2302 if defined_source {
2303 format_encode.as_ref().ok_or_else(|| {
2304 ErrorCode::InvalidInputSyntax("Please specify a source schema using FORMAT".to_owned())
2305 })?;
2306 }
2307 Ok(format_encode)
2308}
2309
2310fn ensure_partition_columns_are_prefix_of_primary_key(
2311 partition_columns: &[String],
2312 primary_key_columns: &[String],
2313) -> std::result::Result<(), String> {
2314 if partition_columns.len() > primary_key_columns.len() {
2315 return Err("Partition columns cannot be longer than primary key columns.".to_owned());
2316 }
2317
2318 for (i, partition_col) in partition_columns.iter().enumerate() {
2319 if primary_key_columns.get(i) != Some(partition_col) {
2320 return Err(format!(
2321 "Partition column '{}' is not a prefix of the primary key.",
2322 partition_col
2323 ));
2324 }
2325 }
2326
2327 Ok(())
2328}
2329
2330#[allow(clippy::too_many_arguments)]
2331pub async fn generate_stream_graph_for_replace_table(
2332 _session: &Arc<SessionImpl>,
2333 table_name: ObjectName,
2334 original_catalog: &Arc<TableCatalog>,
2335 handler_args: HandlerArgs,
2336 statement: Statement,
2337 col_id_gen: ColumnIdGenerator,
2338 sql_column_strategy: SqlColumnStrategy,
2339) -> Result<(
2340 StreamFragmentGraph,
2341 TableCatalog,
2342 Option<SourceCatalog>,
2343 TableJobType,
2344)> {
2345 let Statement::CreateTable {
2346 columns,
2347 constraints,
2348 source_watermarks,
2349 append_only,
2350 on_conflict,
2351 with_version_columns,
2352 wildcard_idx,
2353 cdc_table_info,
2354 format_encode,
2355 include_column_options,
2356 engine,
2357 with_options,
2358 ..
2359 } = statement
2360 else {
2361 panic!("unexpected statement type: {:?}", statement);
2362 };
2363
2364 let format_encode = format_encode
2365 .clone()
2366 .map(|format_encode| format_encode.into_v2_with_warning());
2367
2368 let engine = match engine {
2369 risingwave_sqlparser::ast::Engine::Hummock => Engine::Hummock,
2370 risingwave_sqlparser::ast::Engine::Iceberg => Engine::Iceberg,
2371 };
2372
2373 let is_drop_connector =
2374 original_catalog.associated_source_id().is_some() && format_encode.is_none();
2375 if is_drop_connector {
2376 debug_assert!(
2377 source_watermarks.is_empty()
2378 && include_column_options.is_empty()
2379 && with_options
2380 .iter()
2381 .all(|opt| opt.name.real_value().to_lowercase() != "connector")
2382 );
2383 }
2384
2385 let props = CreateTableProps {
2386 definition: handler_args.normalized_sql.clone(),
2387 append_only,
2388 on_conflict: on_conflict.into(),
2389 with_version_columns: with_version_columns
2390 .iter()
2391 .map(|col| col.real_value())
2392 .collect(),
2393 webhook_info: original_catalog.webhook_info.clone(),
2394 engine,
2395 };
2396
2397 let ((plan, mut source, mut table), job_type) = match (format_encode, cdc_table_info.as_ref()) {
2398 (Some(format_encode), None) => (
2399 gen_create_table_plan_with_source(
2400 handler_args,
2401 ExplainOptions::default(),
2402 table_name,
2403 columns,
2404 wildcard_idx,
2405 constraints,
2406 format_encode,
2407 source_watermarks,
2408 col_id_gen,
2409 include_column_options,
2410 props,
2411 sql_column_strategy,
2412 )
2413 .await?,
2414 TableJobType::General,
2415 ),
2416 (None, None) => {
2417 let context = OptimizerContext::from_handler_args(handler_args);
2418 let (plan, table) = gen_create_table_plan(
2419 context,
2420 table_name,
2421 columns,
2422 constraints,
2423 col_id_gen,
2424 source_watermarks,
2425 props,
2426 true,
2427 )?;
2428 ((plan, None, table), TableJobType::General)
2429 }
2430 (None, Some(cdc_table)) => {
2431 let session = &handler_args.session;
2432 let (source, resolved_table_name) =
2433 get_source_and_resolved_table_name(session, cdc_table.clone(), table_name.clone())?;
2434
2435 let cdc_with_options = derive_with_options_for_cdc_table(
2436 &source.with_properties,
2437 cdc_table.external_table_name.clone(),
2438 )?;
2439
2440 let (column_catalogs, pk_names) = bind_cdc_table_schema(&columns, &constraints, true)?;
2441
2442 let context: OptimizerContextRef =
2443 OptimizerContext::new(handler_args, ExplainOptions::default()).into();
2444 let (plan, table) = gen_create_table_plan_for_cdc_table(
2445 context,
2446 source,
2447 cdc_table.external_table_name.clone(),
2448 columns,
2449 column_catalogs,
2450 pk_names,
2451 cdc_with_options,
2452 col_id_gen,
2453 on_conflict,
2454 with_version_columns
2455 .iter()
2456 .map(|col| col.real_value())
2457 .collect(),
2458 include_column_options,
2459 table_name,
2460 resolved_table_name,
2461 original_catalog.database_id,
2462 original_catalog.schema_id,
2463 original_catalog.id(),
2464 engine,
2465 )?;
2466
2467 ((plan, None, table), TableJobType::SharedCdcSource)
2468 }
2469 (Some(_), Some(_)) => {
2470 return Err(ErrorCode::NotSupported(
2471 "Data format and encoding format doesn't apply to table created from a CDC source"
2472 .into(),
2473 "Remove the FORMAT and ENCODE specification".into(),
2474 )
2475 .into());
2476 }
2477 };
2478
2479 if table.pk_column_ids() != original_catalog.pk_column_ids() {
2480 Err(ErrorCode::InvalidInputSyntax(
2481 "alter primary key of table is not supported".to_owned(),
2482 ))?
2483 }
2484
2485 let graph = build_graph(plan, Some(GraphJobType::Table))?;
2486
2487 table.id = original_catalog.id();
2489 if !is_drop_connector && let Some(source_id) = original_catalog.associated_source_id() {
2490 table.associated_source_id = Some(source_id);
2491
2492 let source = source.as_mut().unwrap();
2493 source.id = source_id;
2494 source.associated_table_id = Some(table.id());
2495 }
2496
2497 Ok((graph, table, source, job_type))
2498}
2499
2500fn get_source_and_resolved_table_name(
2501 session: &Arc<SessionImpl>,
2502 cdc_table: CdcTableInfo,
2503 table_name: ObjectName,
2504) -> Result<(Arc<SourceCatalog>, String)> {
2505 let db_name = &session.database();
2506 let (_, resolved_table_name) = Binder::resolve_schema_qualified_name(db_name, &table_name)?;
2507
2508 let (source_schema, source_name) =
2509 Binder::resolve_schema_qualified_name(db_name, &cdc_table.source_name)?;
2510
2511 let source = {
2512 let catalog_reader = session.env().catalog_reader().read_guard();
2513 let schema_name = source_schema.unwrap_or(DEFAULT_SCHEMA_NAME.to_owned());
2514 let (source, _) = catalog_reader.get_source_by_name(
2515 db_name,
2516 SchemaPath::Name(schema_name.as_str()),
2517 source_name.as_str(),
2518 )?;
2519 source.clone()
2520 };
2521
2522 Ok((source, resolved_table_name))
2523}
2524
2525fn bind_webhook_info(
2527 session: &Arc<SessionImpl>,
2528 columns_defs: &[ColumnDef],
2529 webhook_info: WebhookSourceInfo,
2530) -> Result<PbWebhookSourceInfo> {
2531 if columns_defs.len() != 1 || columns_defs[0].data_type.as_ref().unwrap() != &AstDataType::Jsonb
2533 {
2534 return Err(ErrorCode::InvalidInputSyntax(
2535 "Table with webhook source should have exactly one JSONB column".to_owned(),
2536 )
2537 .into());
2538 }
2539
2540 let WebhookSourceInfo {
2541 secret_ref,
2542 signature_expr,
2543 wait_for_persistence,
2544 is_batched,
2545 } = webhook_info;
2546
2547 let (pb_secret_ref, secret_name) = if let Some(secret_ref) = secret_ref {
2549 let db_name = &session.database();
2550 let (schema_name, secret_name) =
2551 Binder::resolve_schema_qualified_name(db_name, &secret_ref.secret_name)?;
2552 let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?;
2553 (
2554 Some(PbSecretRef {
2555 secret_id: secret_catalog.id,
2556 ref_as: match secret_ref.ref_as {
2557 SecretRefAsType::Text => PbRefAsType::Text,
2558 SecretRefAsType::File => PbRefAsType::File,
2559 }
2560 .into(),
2561 }),
2562 Some(secret_name),
2563 )
2564 } else {
2565 (None, None)
2566 };
2567
2568 let secure_compare_context = SecureCompareContext {
2569 column_name: columns_defs[0].name.real_value(),
2570 secret_name,
2571 };
2572 let mut binder = Binder::new_for_ddl(session).with_secure_compare(secure_compare_context);
2573 let expr = binder.bind_expr(&signature_expr)?;
2574
2575 if expr.as_function_call().is_none()
2577 || expr.as_function_call().unwrap().func_type()
2578 != crate::optimizer::plan_node::generic::ExprType::SecureCompare
2579 {
2580 return Err(ErrorCode::InvalidInputSyntax(
2581 "The signature verification function must be SECURE_COMPARE()".to_owned(),
2582 )
2583 .into());
2584 }
2585
2586 let pb_webhook_info = PbWebhookSourceInfo {
2587 secret_ref: pb_secret_ref,
2588 signature_expr: Some(expr.to_expr_proto()),
2589 wait_for_persistence,
2590 is_batched,
2591 };
2592
2593 Ok(pb_webhook_info)
2594}
2595
2596#[cfg(test)]
2597mod tests {
2598 use risingwave_common::catalog::{
2599 DEFAULT_DATABASE_NAME, ROW_ID_COLUMN_NAME, RW_TIMESTAMP_COLUMN_NAME,
2600 };
2601 use risingwave_common::types::{DataType, StructType};
2602
2603 use super::*;
2604 use crate::test_utils::{LocalFrontend, PROTO_FILE_DATA, create_proto_file};
2605
2606 #[tokio::test]
2607 async fn test_create_table_handler() {
2608 let sql =
2609 "create table t (v1 smallint, v2 struct<v3 bigint, v4 float, v5 double>) append only;";
2610 let frontend = LocalFrontend::new(Default::default()).await;
2611 frontend.run_sql(sql).await.unwrap();
2612
2613 let session = frontend.session_ref();
2614 let catalog_reader = session.env().catalog_reader().read_guard();
2615 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
2616
2617 let (table, _) = catalog_reader
2619 .get_created_table_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
2620 .unwrap();
2621 assert_eq!(table.name(), "t");
2622
2623 let columns = table
2624 .columns
2625 .iter()
2626 .map(|col| (col.name(), col.data_type().clone()))
2627 .collect::<HashMap<&str, DataType>>();
2628
2629 let expected_columns = maplit::hashmap! {
2630 ROW_ID_COLUMN_NAME => DataType::Serial,
2631 "v1" => DataType::Int16,
2632 "v2" => StructType::new(
2633 vec![("v3", DataType::Int64),("v4", DataType::Float64),("v5", DataType::Float64)],
2634 )
2635 .with_ids([3, 4, 5].map(ColumnId::new))
2636 .into(),
2637 RW_TIMESTAMP_COLUMN_NAME => DataType::Timestamptz,
2638 };
2639
2640 assert_eq!(columns, expected_columns, "{columns:#?}");
2641 }
2642
2643 #[test]
2644 fn test_bind_primary_key() {
2645 for (sql, expected) in [
2648 ("create table t (v1 int, v2 int)", Ok(&[0] as &[_])),
2649 ("create table t (v1 int primary key, v2 int)", Ok(&[1])),
2650 ("create table t (v1 int, v2 int primary key)", Ok(&[2])),
2651 (
2652 "create table t (v1 int primary key, v2 int primary key)",
2653 Err("multiple primary keys are not allowed"),
2654 ),
2655 (
2656 "create table t (v1 int primary key primary key, v2 int)",
2657 Err("multiple primary keys are not allowed"),
2658 ),
2659 (
2660 "create table t (v1 int, v2 int, primary key (v1))",
2661 Ok(&[1]),
2662 ),
2663 (
2664 "create table t (v1 int, primary key (v2), v2 int)",
2665 Ok(&[2]),
2666 ),
2667 (
2668 "create table t (primary key (v2, v1), v1 int, v2 int)",
2669 Ok(&[2, 1]),
2670 ),
2671 (
2672 "create table t (v1 int, primary key (v1), v2 int, primary key (v1))",
2673 Err("multiple primary keys are not allowed"),
2674 ),
2675 (
2676 "create table t (v1 int primary key, primary key (v1), v2 int)",
2677 Err("multiple primary keys are not allowed"),
2678 ),
2679 (
2680 "create table t (v1 int, primary key (V3), v2 int)",
2681 Err("column \"v3\" named in key does not exist"),
2682 ),
2683 ] {
2684 let mut ast = risingwave_sqlparser::parser::Parser::parse_sql(sql).unwrap();
2685 let risingwave_sqlparser::ast::Statement::CreateTable {
2686 columns: column_defs,
2687 constraints,
2688 ..
2689 } = ast.remove(0)
2690 else {
2691 panic!("test case should be create table")
2692 };
2693 let actual: Result<_> = (|| {
2694 let mut columns = bind_sql_columns(&column_defs, false)?;
2695 let mut col_id_gen = ColumnIdGenerator::new_initial();
2696 for c in &mut columns {
2697 col_id_gen.generate(c)?;
2698 }
2699
2700 let pk_names =
2701 bind_sql_pk_names(&column_defs, bind_table_constraints(&constraints)?)?;
2702 let (_, pk_column_ids, _) =
2703 bind_pk_and_row_id_on_relation(columns, pk_names, true)?;
2704 Ok(pk_column_ids)
2705 })();
2706 match (expected, actual) {
2707 (Ok(expected), Ok(actual)) => assert_eq!(
2708 expected.iter().copied().map(ColumnId::new).collect_vec(),
2709 actual,
2710 "sql: {sql}"
2711 ),
2712 (Ok(_), Err(actual)) => panic!("sql: {sql}\nunexpected error: {actual:?}"),
2713 (Err(_), Ok(actual)) => panic!("sql: {sql}\nexpects error but got: {actual:?}"),
2714 (Err(expected), Err(actual)) => assert!(
2715 actual.to_string().contains(expected),
2716 "sql: {sql}\nexpected: {expected:?}\nactual: {actual:?}"
2717 ),
2718 }
2719 }
2720 }
2721
2722 #[tokio::test]
2723 async fn test_duplicate_props_options() {
2724 let proto_file = create_proto_file(PROTO_FILE_DATA);
2725 let sql = format!(
2726 r#"CREATE TABLE t
2727 WITH (
2728 connector = 'kinesis',
2729 aws.region='user_test_topic',
2730 endpoint='172.10.1.1:9090,172.10.1.2:9090',
2731 aws.credentials.access_key_id = 'your_access_key_1',
2732 aws.credentials.secret_access_key = 'your_secret_key_1'
2733 )
2734 FORMAT PLAIN ENCODE PROTOBUF (
2735 message = '.test.TestRecord',
2736 aws.credentials.access_key_id = 'your_access_key_2',
2737 aws.credentials.secret_access_key = 'your_secret_key_2',
2738 schema.location = 'file://{}',
2739 )"#,
2740 proto_file.path().to_str().unwrap()
2741 );
2742 let frontend = LocalFrontend::new(Default::default()).await;
2743 frontend.run_sql(sql).await.unwrap();
2744
2745 let session = frontend.session_ref();
2746 let catalog_reader = session.env().catalog_reader().read_guard();
2747 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
2748
2749 let (source, _) = catalog_reader
2751 .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
2752 .unwrap();
2753 assert_eq!(source.name, "t");
2754
2755 assert_eq!(
2757 source
2758 .info
2759 .format_encode_options
2760 .get("aws.credentials.access_key_id")
2761 .unwrap(),
2762 "your_access_key_2"
2763 );
2764 assert_eq!(
2765 source
2766 .info
2767 .format_encode_options
2768 .get("aws.credentials.secret_access_key")
2769 .unwrap(),
2770 "your_secret_key_2"
2771 );
2772
2773 assert_eq!(
2775 source
2776 .with_properties
2777 .get("aws.credentials.access_key_id")
2778 .unwrap(),
2779 "your_access_key_1"
2780 );
2781 assert_eq!(
2782 source
2783 .with_properties
2784 .get("aws.credentials.secret_access_key")
2785 .unwrap(),
2786 "your_secret_key_1"
2787 );
2788
2789 assert!(!source.with_properties.contains_key("schema.location"));
2791 }
2792}