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 = 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 let has_non_ttl_watermark = watermark_descs.iter().any(|d| !d.with_ttl);
779
780 if !append_only && has_non_ttl_watermark {
781 return Err(ErrorCode::NotSupported(
782 "Defining watermarks on table requires the table to be append only.".to_owned(),
783 "Use the key words `APPEND ONLY`".to_owned(),
784 )
785 .into());
786 }
787
788 if !append_only && retention_seconds.is_some() {
789 if session
790 .config()
791 .unsafe_enable_storage_retention_for_non_append_only_tables()
792 {
793 tracing::warn!(
794 "Storage retention is enabled for non-append-only table {}. This may lead to stream inconsistency.",
795 table_name
796 );
797 const NOTICE: &str = "Storage retention is enabled for non-append-only table. \
798 This may lead to stream inconsistency and unrecoverable \
799 node failure if there is any row INSERT/UPDATE/DELETE operation \
800 corresponding to the TTLed primary keys";
801 session.notice_to_user(NOTICE);
802 } else {
803 return Err(ErrorCode::NotSupported(
804 "Defining retention seconds on table requires the table to be append only."
805 .to_owned(),
806 "Use the key words `APPEND ONLY`".to_owned(),
807 )
808 .into());
809 }
810 }
811
812 let materialize =
813 plan_root.gen_table_plan(context, table_name, database_id, schema_id, info, props)?;
814
815 let mut table = materialize.table().clone();
816 table.owner = session.user_id();
817
818 Ok((materialize.into(), table))
819}
820
821#[allow(clippy::too_many_arguments)]
825pub(crate) fn gen_create_table_plan_for_cdc_table(
826 context: OptimizerContextRef,
827 source: Arc<SourceCatalog>,
828 external_table_name: String,
829 column_defs: Vec<ColumnDef>,
830 mut columns: Vec<ColumnCatalog>,
831 pk_names: Vec<String>,
832 cdc_with_options: WithOptionsSecResolved,
833 mut col_id_gen: ColumnIdGenerator,
834 on_conflict: Option<OnConflict>,
835 with_version_columns: Vec<String>,
836 include_column_options: IncludeOption,
837 table_name: ObjectName,
838 resolved_table_name: String, database_id: DatabaseId,
840 schema_id: SchemaId,
841 table_id: TableId,
842 engine: Engine,
843) -> Result<(PlanRef, TableCatalog)> {
844 let session = context.session_ctx().clone();
845
846 handle_addition_columns(
848 None,
849 &cdc_with_options,
850 include_column_options,
851 &mut columns,
852 true,
853 )?;
854
855 for c in &mut columns {
856 col_id_gen.generate(c)?;
857 }
858
859 let (mut columns, pk_column_ids, _row_id_index) =
860 bind_pk_and_row_id_on_relation(columns, pk_names, true)?;
861
862 bind_sql_column_constraints(
864 context.session_ctx(),
865 table_name.real_value(),
866 &mut columns,
867 &column_defs,
868 &pk_column_ids,
869 )?;
870
871 let definition = context.normalized_sql().to_owned();
872
873 let pk_column_indices = {
874 let mut id_to_idx = HashMap::new();
875 columns.iter().enumerate().for_each(|(idx, c)| {
876 id_to_idx.insert(c.column_id(), idx);
877 });
878 pk_column_ids
880 .iter()
881 .map(|c| id_to_idx.get(c).copied().unwrap())
882 .collect_vec()
883 };
884 let table_pk = pk_column_indices
885 .iter()
886 .map(|idx| ColumnOrder::new(*idx, OrderType::ascending()))
887 .collect();
888
889 let (options, secret_refs) = cdc_with_options.into_parts();
890
891 let non_generated_column_descs = columns
892 .iter()
893 .filter(|&c| !c.is_generated())
894 .map(|c| c.column_desc.clone())
895 .collect_vec();
896 let non_generated_column_num = non_generated_column_descs.len();
897 let cdc_table_type = ExternalCdcTableType::from_properties(&options);
898 let cdc_table_desc = CdcTableDesc {
899 table_id,
900 source_id: source.id, external_table_name: external_table_name.clone(),
902 pk: table_pk,
903 columns: non_generated_column_descs,
904 stream_key: pk_column_indices,
905 connect_properties: options,
906 secret_refs,
907 };
908
909 tracing::debug!(?cdc_table_desc, "create cdc table");
910 let options = build_cdc_scan_options_with_options(context.with_options(), &cdc_table_type)?;
911
912 let logical_scan = LogicalCdcScan::create(
913 external_table_name.clone(),
914 Rc::new(cdc_table_desc),
915 context.clone(),
916 options,
917 );
918
919 let scan_node: LogicalPlanRef = logical_scan.into();
920 let required_cols = FixedBitSet::with_capacity(non_generated_column_num);
921 let plan_root = PlanRoot::new_with_logical_plan(
922 scan_node,
923 RequiredDist::Any,
924 Order::any(),
925 required_cols,
926 vec![],
927 );
928
929 let cdc_table_id = build_cdc_table_id(source.id, &external_table_name);
930 let materialize = plan_root.gen_table_plan(
931 context,
932 resolved_table_name,
933 database_id,
934 schema_id,
935 CreateTableInfo {
936 columns,
937 pk_column_ids,
938 row_id_index: None,
939 watermark_descs: vec![],
940 source_catalog: Some((*source).clone()),
941 version: col_id_gen.into_version(),
942 },
943 CreateTableProps {
944 definition,
945 append_only: false,
946 on_conflict: on_conflict.into(),
947 with_version_columns,
948 webhook_info: None,
949 engine,
950 },
951 )?;
952
953 let mut table = materialize.table().clone();
954 table.owner = session.user_id();
955 table.cdc_table_id = Some(cdc_table_id);
956 table.cdc_table_type = Some(cdc_table_type);
957 Ok((materialize.into(), table))
958}
959
960fn derive_with_options_for_cdc_table(
961 source_with_properties: &WithOptionsSecResolved,
962 external_table_name: String,
963) -> Result<WithOptionsSecResolved> {
964 use source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, SQL_SERVER_CDC_CONNECTOR};
965 let source_database_name: &str = source_with_properties
967 .get("database.name")
968 .ok_or_else(|| anyhow!("The source with properties does not contain 'database.name'"))?
969 .as_str();
970 let mut with_options = source_with_properties.clone();
971 if let Some(connector) = source_with_properties.get(UPSTREAM_SOURCE_KEY) {
972 match connector.as_str() {
973 MYSQL_CDC_CONNECTOR => {
974 let (db_name, table_name) = external_table_name.split_once('.').ok_or_else(|| {
977 anyhow!("The upstream table name must contain database name prefix, e.g. 'database.table'")
978 })?;
979 if !source_database_name
981 .split(',')
982 .map(|s| s.trim())
983 .any(|name| name == db_name)
984 {
985 return Err(anyhow!(
986 "The database name `{}` in the FROM clause is not included in the database name `{}` in source definition",
987 db_name,
988 source_database_name
989 ).into());
990 }
991 with_options.insert(DATABASE_NAME_KEY.into(), db_name.into());
992 with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
993 }
994 POSTGRES_CDC_CONNECTOR => {
995 let (schema_name, table_name) = external_table_name
996 .split_once('.')
997 .ok_or_else(|| anyhow!("The upstream table name must contain schema name prefix, e.g. 'public.table'"))?;
998
999 with_options.insert(SCHEMA_NAME_KEY.into(), schema_name.into());
1001 with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
1002 }
1003 SQL_SERVER_CDC_CONNECTOR => {
1004 let parts: Vec<&str> = external_table_name.split('.').collect();
1010 let (_, schema_name, table_name) = match parts.len() {
1011 3 => {
1012 let db_name = parts[0];
1014 let schema_name = parts[1];
1015 let table_name = parts[2];
1016
1017 if db_name != source_database_name {
1019 return Err(anyhow!(
1020 "The database name `{}` in the FROM clause is not the same as the database name `{}` in source definition",
1021 db_name,
1022 source_database_name
1023 ).into());
1024 }
1025 (db_name, schema_name, table_name)
1026 }
1027 2 => {
1028 let schema_name = parts[0];
1030 let table_name = parts[1];
1031 (source_database_name, schema_name, table_name)
1032 }
1033 1 => {
1034 let table_name = parts[0];
1036 (source_database_name, "dbo", table_name)
1037 }
1038 _ => {
1039 return Err(anyhow!(
1040 "The upstream table name must be in one of these formats: 'database.schema.table', 'schema.table', or 'table'"
1041 ).into());
1042 }
1043 };
1044
1045 with_options.insert(SCHEMA_NAME_KEY.into(), schema_name.into());
1047 with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
1048 }
1049 _ => {
1050 return Err(RwError::from(anyhow!(
1051 "connector {} is not supported for cdc table",
1052 connector
1053 )));
1054 }
1055 };
1056 }
1057 Ok(with_options)
1058}
1059
1060#[allow(clippy::too_many_arguments)]
1061pub(super) async fn handle_create_table_plan(
1062 handler_args: HandlerArgs,
1063 explain_options: ExplainOptions,
1064 format_encode: Option<FormatEncodeOptions>,
1065 cdc_table_info: Option<CdcTableInfo>,
1066 table_name: &ObjectName,
1067 column_defs: Vec<ColumnDef>,
1068 wildcard_idx: Option<usize>,
1069 constraints: Vec<TableConstraint>,
1070 source_watermarks: Vec<SourceWatermark>,
1071 append_only: bool,
1072 on_conflict: Option<OnConflict>,
1073 with_version_columns: Vec<String>,
1074 include_column_options: IncludeOption,
1075 webhook_info: Option<WebhookSourceInfo>,
1076 engine: Engine,
1077) -> Result<(
1078 PlanRef,
1079 Option<SourceCatalog>,
1080 TableCatalog,
1081 TableJobType,
1082 Option<SourceId>,
1083)> {
1084 let col_id_gen = ColumnIdGenerator::new_initial();
1085 let format_encode = check_create_table_with_source(
1086 &handler_args.with_options,
1087 format_encode,
1088 &include_column_options,
1089 &cdc_table_info,
1090 )?;
1091 let webhook_info = webhook_info
1092 .map(|info| bind_webhook_info(&handler_args.session, &column_defs, info))
1093 .transpose()?;
1094
1095 let props = CreateTableProps {
1096 definition: handler_args.normalized_sql.clone(),
1097 append_only,
1098 on_conflict: on_conflict.into(),
1099 with_version_columns: with_version_columns.clone(),
1100 webhook_info,
1101 engine,
1102 };
1103
1104 let ((plan, source, table), job_type, shared_shource_id) = match (
1105 format_encode,
1106 cdc_table_info.as_ref(),
1107 ) {
1108 (Some(format_encode), None) => (
1109 gen_create_table_plan_with_source(
1110 handler_args,
1111 explain_options,
1112 table_name.clone(),
1113 column_defs,
1114 wildcard_idx,
1115 constraints,
1116 format_encode,
1117 source_watermarks,
1118 col_id_gen,
1119 include_column_options,
1120 props,
1121 SqlColumnStrategy::FollowChecked,
1122 )
1123 .await?,
1124 TableJobType::General,
1125 None,
1126 ),
1127 (None, None) => {
1128 let context = OptimizerContext::new(handler_args, explain_options);
1129 let (plan, table) = gen_create_table_plan(
1130 context,
1131 table_name.clone(),
1132 column_defs,
1133 constraints,
1134 col_id_gen,
1135 source_watermarks,
1136 props,
1137 false,
1138 )?;
1139
1140 ((plan, None, table), TableJobType::General, None)
1141 }
1142
1143 (None, Some(cdc_table)) => {
1144 sanity_check_for_table_on_cdc_source(
1145 append_only,
1146 &column_defs,
1147 &wildcard_idx,
1148 &constraints,
1149 &source_watermarks,
1150 )?;
1151
1152 generated_columns_check_for_cdc_table(&column_defs)?;
1153 not_null_check_for_cdc_table(&wildcard_idx, &column_defs)?;
1154
1155 let session = &handler_args.session;
1156 let db_name = &session.database();
1157 let user_name = &session.user_name();
1158 let search_path = session.config().search_path();
1159 let (schema_name, resolved_table_name) =
1160 Binder::resolve_schema_qualified_name(db_name, table_name)?;
1161 let (database_id, schema_id) =
1162 session.get_database_and_schema_id_for_create(schema_name.clone())?;
1163
1164 let (source_schema, source_name) =
1166 Binder::resolve_schema_qualified_name(db_name, &cdc_table.source_name)?;
1167
1168 let source = {
1169 let catalog_reader = session.env().catalog_reader().read_guard();
1170 let schema_path =
1171 SchemaPath::new(source_schema.as_deref(), &search_path, user_name);
1172
1173 let (source, _) = catalog_reader.get_source_by_name(
1174 db_name,
1175 schema_path,
1176 source_name.as_str(),
1177 )?;
1178 source.clone()
1179 };
1180 let cdc_with_options: WithOptionsSecResolved = derive_with_options_for_cdc_table(
1181 &source.with_properties,
1182 cdc_table.external_table_name.clone(),
1183 )?;
1184
1185 let (columns, pk_names) = match wildcard_idx {
1186 Some(_) => bind_cdc_table_schema_externally(cdc_with_options.clone()).await?,
1187 None => {
1188 for column_def in &column_defs {
1189 for option_def in &column_def.options {
1190 if let ColumnOption::DefaultValue(_)
1191 | ColumnOption::DefaultValueInternal { .. } = option_def.option
1192 {
1193 return Err(ErrorCode::NotSupported(
1194 "Default value for columns defined on the table created from a CDC source".into(),
1195 "Remove the default value expression in the column definitions".into(),
1196 )
1197 .into());
1198 }
1199 }
1200 }
1201
1202 let (columns, pk_names) =
1203 bind_cdc_table_schema(&column_defs, &constraints, false)?;
1204 let (options, secret_refs) = cdc_with_options.clone().into_parts();
1206 let _config = ExternalTableConfig::try_from_btreemap(options, secret_refs)
1207 .context("failed to extract external table config")?;
1208
1209 (columns, pk_names)
1210 }
1211 };
1212
1213 let context: OptimizerContextRef =
1214 OptimizerContext::new(handler_args, explain_options).into();
1215 let shared_source_id = source.id;
1216 let (plan, table) = gen_create_table_plan_for_cdc_table(
1217 context,
1218 source,
1219 cdc_table.external_table_name.clone(),
1220 column_defs,
1221 columns,
1222 pk_names,
1223 cdc_with_options,
1224 col_id_gen,
1225 on_conflict,
1226 with_version_columns,
1227 include_column_options,
1228 table_name.clone(),
1229 resolved_table_name,
1230 database_id,
1231 schema_id,
1232 TableId::placeholder(),
1233 engine,
1234 )?;
1235
1236 (
1237 (plan, None, table),
1238 TableJobType::SharedCdcSource,
1239 Some(shared_source_id),
1240 )
1241 }
1242 (Some(_), Some(_)) => {
1243 return Err(ErrorCode::NotSupported(
1244 "Data format and encoding format doesn't apply to table created from a CDC source"
1245 .into(),
1246 "Remove the FORMAT and ENCODE specification".into(),
1247 )
1248 .into());
1249 }
1250 };
1251 Ok((plan, source, table, job_type, shared_shource_id))
1252}
1253
1254fn generated_columns_check_for_cdc_table(columns: &Vec<ColumnDef>) -> Result<()> {
1256 let mut found_generated_column = false;
1257 for column in columns {
1258 let mut is_generated = false;
1259
1260 for option_def in &column.options {
1261 if let ColumnOption::GeneratedColumns(_) = option_def.option {
1262 is_generated = true;
1263 break;
1264 }
1265 }
1266
1267 if is_generated {
1268 found_generated_column = true;
1269 } else if found_generated_column {
1270 return Err(ErrorCode::NotSupported(
1271 "Non-generated column found after a generated column.".into(),
1272 "Ensure that all generated columns appear at the end of the cdc table definition."
1273 .into(),
1274 )
1275 .into());
1276 }
1277 }
1278 Ok(())
1279}
1280
1281fn not_null_check_for_cdc_table(
1283 wildcard_idx: &Option<usize>,
1284 column_defs: &Vec<ColumnDef>,
1285) -> Result<()> {
1286 if !wildcard_idx.is_some()
1287 && column_defs.iter().any(|col| {
1288 col.options
1289 .iter()
1290 .any(|opt| matches!(opt.option, ColumnOption::NotNull))
1291 })
1292 {
1293 return Err(ErrorCode::NotSupported(
1294 "CDC table with NOT NULL constraint is not supported".to_owned(),
1295 "Please remove the NOT NULL constraint for columns".to_owned(),
1296 )
1297 .into());
1298 }
1299 Ok(())
1300}
1301
1302fn sanity_check_for_table_on_cdc_source(
1304 append_only: bool,
1305 column_defs: &Vec<ColumnDef>,
1306 wildcard_idx: &Option<usize>,
1307 constraints: &Vec<TableConstraint>,
1308 source_watermarks: &Vec<SourceWatermark>,
1309) -> Result<()> {
1310 if wildcard_idx.is_some() && !column_defs.is_empty() {
1312 return Err(ErrorCode::NotSupported(
1313 "wildcard(*) and column definitions cannot be used together".to_owned(),
1314 "Remove the wildcard or column definitions".to_owned(),
1315 )
1316 .into());
1317 }
1318
1319 if !wildcard_idx.is_some()
1321 && !constraints.iter().any(|c| {
1322 matches!(
1323 c,
1324 TableConstraint::Unique {
1325 is_primary: true,
1326 ..
1327 }
1328 )
1329 })
1330 && !column_defs.iter().any(|col| {
1331 col.options
1332 .iter()
1333 .any(|opt| matches!(opt.option, ColumnOption::Unique { is_primary: true }))
1334 })
1335 {
1336 return Err(ErrorCode::NotSupported(
1337 "CDC table without primary key constraint is not supported".to_owned(),
1338 "Please define a primary key".to_owned(),
1339 )
1340 .into());
1341 }
1342
1343 if append_only {
1344 return Err(ErrorCode::NotSupported(
1345 "append only modifier on the table created from a CDC source".into(),
1346 "Remove the APPEND ONLY clause".into(),
1347 )
1348 .into());
1349 }
1350
1351 if !source_watermarks.is_empty() {
1352 return Err(ErrorCode::NotSupported(
1353 "watermark defined on the table created from a CDC source".into(),
1354 "Remove the Watermark definitions".into(),
1355 )
1356 .into());
1357 }
1358
1359 Ok(())
1360}
1361
1362async fn bind_cdc_table_schema_externally(
1364 cdc_with_options: WithOptionsSecResolved,
1365) -> Result<(Vec<ColumnCatalog>, Vec<String>)> {
1366 let (options, secret_refs) = cdc_with_options.into_parts();
1368 let config = ExternalTableConfig::try_from_btreemap(options, secret_refs)
1369 .context("failed to extract external table config")?;
1370
1371 let table = ExternalTableImpl::connect(config)
1372 .await
1373 .context("failed to auto derive table schema")?;
1374
1375 Ok((
1376 table
1377 .column_descs()
1378 .iter()
1379 .cloned()
1380 .map(|column_desc| ColumnCatalog {
1381 column_desc,
1382 is_hidden: false,
1383 })
1384 .collect(),
1385 table.pk_names().clone(),
1386 ))
1387}
1388
1389fn bind_cdc_table_schema(
1391 column_defs: &Vec<ColumnDef>,
1392 constraints: &Vec<TableConstraint>,
1393 is_for_replace_plan: bool,
1394) -> Result<(Vec<ColumnCatalog>, Vec<String>)> {
1395 let columns = bind_sql_columns(column_defs, is_for_replace_plan)?;
1396
1397 let pk_names = bind_sql_pk_names(column_defs, bind_table_constraints(constraints)?)?;
1398 Ok((columns, pk_names))
1399}
1400
1401#[allow(clippy::too_many_arguments)]
1402pub async fn handle_create_table(
1403 handler_args: HandlerArgs,
1404 table_name: ObjectName,
1405 column_defs: Vec<ColumnDef>,
1406 wildcard_idx: Option<usize>,
1407 constraints: Vec<TableConstraint>,
1408 if_not_exists: bool,
1409 format_encode: Option<FormatEncodeOptions>,
1410 source_watermarks: Vec<SourceWatermark>,
1411 append_only: bool,
1412 on_conflict: Option<OnConflict>,
1413 with_version_columns: Vec<String>,
1414 cdc_table_info: Option<CdcTableInfo>,
1415 include_column_options: IncludeOption,
1416 webhook_info: Option<WebhookSourceInfo>,
1417 ast_engine: risingwave_sqlparser::ast::Engine,
1418) -> Result<RwPgResponse> {
1419 let session = handler_args.session.clone();
1420
1421 if append_only {
1422 session.notice_to_user("APPEND ONLY TABLE is currently an experimental feature.");
1423 }
1424
1425 session.check_cluster_limits().await?;
1426
1427 let engine = match ast_engine {
1428 risingwave_sqlparser::ast::Engine::Hummock => Engine::Hummock,
1429 risingwave_sqlparser::ast::Engine::Iceberg => Engine::Iceberg,
1430 };
1431
1432 if let Either::Right(resp) = session.check_relation_name_duplicated(
1433 table_name.clone(),
1434 StatementType::CREATE_TABLE,
1435 if_not_exists,
1436 )? {
1437 return Ok(resp);
1438 }
1439
1440 let (graph, source, hummock_table, job_type, shared_source_id) = {
1441 let (plan, source, table, job_type, shared_source_id) = handle_create_table_plan(
1442 handler_args.clone(),
1443 ExplainOptions::default(),
1444 format_encode,
1445 cdc_table_info,
1446 &table_name,
1447 column_defs.clone(),
1448 wildcard_idx,
1449 constraints.clone(),
1450 source_watermarks,
1451 append_only,
1452 on_conflict,
1453 with_version_columns,
1454 include_column_options,
1455 webhook_info,
1456 engine,
1457 )
1458 .await?;
1459 tracing::trace!("table_plan: {:?}", plan.explain_to_string());
1460
1461 let graph = build_graph(plan, Some(GraphJobType::Table))?;
1462
1463 (graph, source, table, job_type, shared_source_id)
1464 };
1465
1466 tracing::trace!(
1467 "name={}, graph=\n{}",
1468 table_name,
1469 serde_json::to_string_pretty(&graph).unwrap()
1470 );
1471
1472 let dependencies = shared_source_id
1473 .map(|id| HashSet::from([id.as_object_id()]))
1474 .unwrap_or_default();
1475
1476 match engine {
1478 Engine::Hummock => {
1479 let catalog_writer = session.catalog_writer()?;
1480 let action = match job_type {
1481 TableJobType::SharedCdcSource => LongRunningNotificationAction::MonitorBackfillJob,
1482 _ => LongRunningNotificationAction::DiagnoseBarrierLatency,
1483 };
1484 execute_with_long_running_notification(
1485 catalog_writer.create_table(
1486 source.map(|s| s.to_prost()),
1487 hummock_table.to_prost(),
1488 graph,
1489 job_type,
1490 if_not_exists,
1491 dependencies,
1492 ),
1493 &session,
1494 "CREATE TABLE",
1495 action,
1496 )
1497 .await?;
1498 }
1499 Engine::Iceberg => {
1500 let hummock_table_name = hummock_table.name.clone();
1501 session.create_staging_table(hummock_table.clone());
1502 let res = create_iceberg_engine_table(
1503 session.clone(),
1504 handler_args,
1505 source.map(|s| s.to_prost()),
1506 hummock_table,
1507 graph,
1508 table_name,
1509 job_type,
1510 if_not_exists,
1511 )
1512 .await;
1513 session.drop_staging_table(&hummock_table_name);
1514 res?
1515 }
1516 }
1517
1518 Ok(PgResponse::empty_result(StatementType::CREATE_TABLE))
1519}
1520
1521#[allow(clippy::too_many_arguments)]
1530pub async fn create_iceberg_engine_table(
1531 session: Arc<SessionImpl>,
1532 handler_args: HandlerArgs,
1533 mut source: Option<PbSource>,
1534 table: TableCatalog,
1535 graph: StreamFragmentGraph,
1536 table_name: ObjectName,
1537 job_type: PbTableJobType,
1538 if_not_exists: bool,
1539) -> Result<()> {
1540 let rw_db_name = session
1541 .env()
1542 .catalog_reader()
1543 .read_guard()
1544 .get_database_by_id(table.database_id)?
1545 .name()
1546 .to_owned();
1547 let rw_schema_name = session
1548 .env()
1549 .catalog_reader()
1550 .read_guard()
1551 .get_schema_by_id(table.database_id, table.schema_id)?
1552 .name()
1553 .clone();
1554 let iceberg_catalog_name = rw_db_name.clone();
1555 let iceberg_database_name = rw_schema_name.clone();
1556 let iceberg_table_name = table_name.0.last().unwrap().real_value();
1557
1558 let iceberg_engine_connection: String = session.config().iceberg_engine_connection();
1559 let sink_decouple = session.config().sink_decouple();
1560 if matches!(sink_decouple, SinkDecouple::Disable) {
1561 bail!(
1562 "Iceberg engine table only supports with sink decouple, try `set sink_decouple = true` to resolve it"
1563 );
1564 }
1565
1566 let mut connection_ref = BTreeMap::new();
1567 let with_common = if iceberg_engine_connection.is_empty() {
1568 bail!("to use iceberg engine table, the variable `iceberg_engine_connection` must be set.");
1569 } else {
1570 let parts: Vec<&str> = iceberg_engine_connection.split('.').collect();
1571 assert_eq!(parts.len(), 2);
1572 let connection_catalog =
1573 session.get_connection_by_name(Some(parts[0].to_owned()), parts[1])?;
1574 if let ConnectionInfo::ConnectionParams(params) = &connection_catalog.info {
1575 if params.connection_type == ConnectionType::Iceberg as i32 {
1576 connection_ref.insert(
1578 "connection".to_owned(),
1579 ConnectionRefValue {
1580 connection_name: ObjectName::from(vec![
1581 Ident::from(parts[0]),
1582 Ident::from(parts[1]),
1583 ]),
1584 },
1585 );
1586
1587 let mut with_common = BTreeMap::new();
1588 with_common.insert("connector".to_owned(), "iceberg".to_owned());
1589 with_common.insert("database.name".to_owned(), iceberg_database_name);
1590 with_common.insert("table.name".to_owned(), iceberg_table_name);
1591
1592 let hosted_catalog = params
1593 .properties
1594 .get("hosted_catalog")
1595 .map(|s| s.eq_ignore_ascii_case("true"))
1596 .unwrap_or(false);
1597 if hosted_catalog {
1598 let meta_client = session.env().meta_client();
1599 let meta_store_endpoint = meta_client.get_meta_store_endpoint().await?;
1600
1601 let meta_store_endpoint =
1602 url::Url::parse(&meta_store_endpoint).map_err(|_| {
1603 ErrorCode::InternalError(
1604 "failed to parse the meta store endpoint".to_owned(),
1605 )
1606 })?;
1607 let meta_store_backend = meta_store_endpoint.scheme().to_owned();
1608 let meta_store_user = meta_store_endpoint.username().to_owned();
1609 let meta_store_password = match meta_store_endpoint.password() {
1610 Some(password) => percent_decode_str(password)
1611 .decode_utf8()
1612 .map_err(|_| {
1613 ErrorCode::InternalError(
1614 "failed to parse password from meta store endpoint".to_owned(),
1615 )
1616 })?
1617 .into_owned(),
1618 None => "".to_owned(),
1619 };
1620 let meta_store_host = meta_store_endpoint
1621 .host_str()
1622 .ok_or_else(|| {
1623 ErrorCode::InternalError(
1624 "failed to parse host from meta store endpoint".to_owned(),
1625 )
1626 })?
1627 .to_owned();
1628 let meta_store_port = meta_store_endpoint.port().ok_or_else(|| {
1629 ErrorCode::InternalError(
1630 "failed to parse port from meta store endpoint".to_owned(),
1631 )
1632 })?;
1633 let meta_store_database = meta_store_endpoint
1634 .path()
1635 .trim_start_matches('/')
1636 .to_owned();
1637
1638 let Ok(meta_backend) = MetaBackend::from_str(&meta_store_backend, true) else {
1639 bail!("failed to parse meta backend: {}", meta_store_backend);
1640 };
1641
1642 let catalog_uri = match meta_backend {
1643 MetaBackend::Postgres => {
1644 format!(
1645 "jdbc:postgresql://{}:{}/{}",
1646 meta_store_host, meta_store_port, meta_store_database
1647 )
1648 }
1649 MetaBackend::Mysql => {
1650 format!(
1651 "jdbc:mysql://{}:{}/{}",
1652 meta_store_host, meta_store_port, meta_store_database
1653 )
1654 }
1655 MetaBackend::Sqlite | MetaBackend::Sql | MetaBackend::Mem => {
1656 bail!(
1657 "Unsupported meta backend for iceberg engine table: {}",
1658 meta_store_backend
1659 );
1660 }
1661 };
1662
1663 with_common.insert("catalog.type".to_owned(), "jdbc".to_owned());
1664 with_common.insert("catalog.uri".to_owned(), catalog_uri);
1665 with_common.insert("catalog.jdbc.user".to_owned(), meta_store_user);
1666 with_common.insert("catalog.jdbc.password".to_owned(), meta_store_password);
1667 with_common.insert("catalog.name".to_owned(), iceberg_catalog_name);
1668 }
1669
1670 with_common
1671 } else {
1672 return Err(RwError::from(ErrorCode::InvalidParameterValue(
1673 "Only iceberg connection could be used in iceberg engine".to_owned(),
1674 )));
1675 }
1676 } else {
1677 return Err(RwError::from(ErrorCode::InvalidParameterValue(
1678 "Private Link Service has been deprecated. Please create a new connection instead."
1679 .to_owned(),
1680 )));
1681 }
1682 };
1683
1684 let mut pks = table
1687 .pk_column_names()
1688 .iter()
1689 .map(|c| c.to_string())
1690 .collect::<Vec<String>>();
1691
1692 let sink_from = if pks.len() == 1 && pks[0].eq(ROW_ID_COLUMN_NAME) {
1694 pks = vec![RISINGWAVE_ICEBERG_ROW_ID.to_owned()];
1695 let [stmt]: [_; 1] = Parser::parse_sql(&format!(
1696 "select {} as {}, * from {}",
1697 ROW_ID_COLUMN_NAME, RISINGWAVE_ICEBERG_ROW_ID, table_name
1698 ))
1699 .context("unable to parse query")?
1700 .try_into()
1701 .unwrap();
1702
1703 let Statement::Query(query) = &stmt else {
1704 panic!("unexpected statement: {:?}", stmt);
1705 };
1706 CreateSink::AsQuery(query.clone())
1707 } else {
1708 CreateSink::From(table_name.clone())
1709 };
1710
1711 let mut sink_name = table_name.clone();
1712 *sink_name.0.last_mut().unwrap() = Ident::from(
1713 (ICEBERG_SINK_PREFIX.to_owned() + &sink_name.0.last().unwrap().real_value()).as_str(),
1714 );
1715 let create_sink_stmt = CreateSinkStatement {
1716 if_not_exists: false,
1717 sink_name,
1718 with_properties: WithProperties(vec![]),
1719 sink_from,
1720 columns: vec![],
1721 emit_mode: None,
1722 sink_schema: None,
1723 into_table_name: None,
1724 };
1725
1726 let mut sink_handler_args = handler_args.clone();
1727
1728 let mut sink_with = with_common.clone();
1729
1730 if table.append_only {
1731 sink_with.insert("type".to_owned(), "append-only".to_owned());
1732 } else {
1733 sink_with.insert("primary_key".to_owned(), pks.join(","));
1734 sink_with.insert("type".to_owned(), "upsert".to_owned());
1735 }
1736 let commit_checkpoint_interval = handler_args
1755 .with_options
1756 .get(COMMIT_CHECKPOINT_INTERVAL)
1757 .map(|v| v.to_owned())
1758 .unwrap_or_else(|| "60".to_owned());
1759 let commit_checkpoint_interval = commit_checkpoint_interval.parse::<u32>().map_err(|_| {
1760 ErrorCode::InvalidInputSyntax(format!(
1761 "commit_checkpoint_interval must be greater than 0: {}",
1762 commit_checkpoint_interval
1763 ))
1764 })?;
1765
1766 if commit_checkpoint_interval == 0 {
1767 bail!("commit_checkpoint_interval must be greater than 0");
1768 }
1769
1770 source
1772 .as_mut()
1773 .map(|x| x.with_properties.remove(COMMIT_CHECKPOINT_INTERVAL));
1774
1775 let sink_decouple = session.config().sink_decouple();
1776 if matches!(sink_decouple, SinkDecouple::Disable) && commit_checkpoint_interval > 1 {
1777 bail!(
1778 "config conflict: `commit_checkpoint_interval` larger than 1 means that sink decouple must be enabled, but session config sink_decouple is disabled"
1779 )
1780 }
1781
1782 sink_with.insert(
1783 COMMIT_CHECKPOINT_INTERVAL.to_owned(),
1784 commit_checkpoint_interval.to_string(),
1785 );
1786 sink_with.insert("create_table_if_not_exists".to_owned(), "true".to_owned());
1787
1788 sink_with.insert("is_exactly_once".to_owned(), "true".to_owned());
1789
1790 if let Some(enable_compaction) = handler_args.with_options.get(ENABLE_COMPACTION) {
1791 match enable_compaction.to_lowercase().as_str() {
1792 "true" => {
1793 sink_with.insert(ENABLE_COMPACTION.to_owned(), "true".to_owned());
1794 }
1795 "false" => {
1796 sink_with.insert(ENABLE_COMPACTION.to_owned(), "false".to_owned());
1797 }
1798 _ => {
1799 return Err(ErrorCode::InvalidInputSyntax(format!(
1800 "enable_compaction must be true or false: {}",
1801 enable_compaction
1802 ))
1803 .into());
1804 }
1805 }
1806
1807 source
1809 .as_mut()
1810 .map(|x| x.with_properties.remove("enable_compaction"));
1811 } else {
1812 sink_with.insert(ENABLE_COMPACTION.to_owned(), "true".to_owned());
1813 }
1814
1815 if let Some(compaction_interval_sec) = handler_args.with_options.get(COMPACTION_INTERVAL_SEC) {
1816 let compaction_interval_sec = compaction_interval_sec.parse::<u64>().map_err(|_| {
1817 ErrorCode::InvalidInputSyntax(format!(
1818 "compaction_interval_sec must be greater than 0: {}",
1819 commit_checkpoint_interval
1820 ))
1821 })?;
1822 if compaction_interval_sec == 0 {
1823 bail!("compaction_interval_sec must be greater than 0");
1824 }
1825 sink_with.insert(
1826 "compaction_interval_sec".to_owned(),
1827 compaction_interval_sec.to_string(),
1828 );
1829 source
1831 .as_mut()
1832 .map(|x| x.with_properties.remove("compaction_interval_sec"));
1833 }
1834
1835 let has_enabled_snapshot_expiration = if let Some(enable_snapshot_expiration) =
1836 handler_args.with_options.get(ENABLE_SNAPSHOT_EXPIRATION)
1837 {
1838 source
1840 .as_mut()
1841 .map(|x| x.with_properties.remove(ENABLE_SNAPSHOT_EXPIRATION));
1842 match enable_snapshot_expiration.to_lowercase().as_str() {
1843 "true" => {
1844 sink_with.insert(ENABLE_SNAPSHOT_EXPIRATION.to_owned(), "true".to_owned());
1845 true
1846 }
1847 "false" => {
1848 sink_with.insert(ENABLE_SNAPSHOT_EXPIRATION.to_owned(), "false".to_owned());
1849 false
1850 }
1851 _ => {
1852 return Err(ErrorCode::InvalidInputSyntax(format!(
1853 "enable_snapshot_expiration must be true or false: {}",
1854 enable_snapshot_expiration
1855 ))
1856 .into());
1857 }
1858 }
1859 } else {
1860 sink_with.insert(ENABLE_SNAPSHOT_EXPIRATION.to_owned(), "true".to_owned());
1861 true
1862 };
1863
1864 if has_enabled_snapshot_expiration {
1865 if let Some(snapshot_expiration_retain_last) = handler_args
1867 .with_options
1868 .get(SNAPSHOT_EXPIRATION_RETAIN_LAST)
1869 {
1870 sink_with.insert(
1871 SNAPSHOT_EXPIRATION_RETAIN_LAST.to_owned(),
1872 snapshot_expiration_retain_last.to_owned(),
1873 );
1874 source
1876 .as_mut()
1877 .map(|x| x.with_properties.remove(SNAPSHOT_EXPIRATION_RETAIN_LAST));
1878 }
1879
1880 if let Some(snapshot_expiration_max_age) = handler_args
1881 .with_options
1882 .get(SNAPSHOT_EXPIRATION_MAX_AGE_MILLIS)
1883 {
1884 sink_with.insert(
1885 SNAPSHOT_EXPIRATION_MAX_AGE_MILLIS.to_owned(),
1886 snapshot_expiration_max_age.to_owned(),
1887 );
1888 source
1890 .as_mut()
1891 .map(|x| x.with_properties.remove(SNAPSHOT_EXPIRATION_MAX_AGE_MILLIS));
1892 }
1893
1894 if let Some(snapshot_expiration_clear_expired_files) = handler_args
1895 .with_options
1896 .get(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_FILES)
1897 {
1898 sink_with.insert(
1899 SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_FILES.to_owned(),
1900 snapshot_expiration_clear_expired_files.to_owned(),
1901 );
1902 source.as_mut().map(|x| {
1904 x.with_properties
1905 .remove(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_FILES)
1906 });
1907 }
1908
1909 if let Some(snapshot_expiration_clear_expired_meta_data) = handler_args
1910 .with_options
1911 .get(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_META_DATA)
1912 {
1913 sink_with.insert(
1914 SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_META_DATA.to_owned(),
1915 snapshot_expiration_clear_expired_meta_data.to_owned(),
1916 );
1917 source.as_mut().map(|x| {
1919 x.with_properties
1920 .remove(SNAPSHOT_EXPIRATION_CLEAR_EXPIRED_META_DATA)
1921 });
1922 }
1923 }
1924
1925 if let Some(write_mode) = handler_args.with_options.get(WRITE_MODE) {
1926 let write_mode = IcebergWriteMode::try_from(write_mode.as_str()).map_err(|_| {
1927 ErrorCode::InvalidInputSyntax(format!(
1928 "invalid write_mode: {}, must be one of: {}, {}",
1929 write_mode, ICEBERG_WRITE_MODE_MERGE_ON_READ, ICEBERG_WRITE_MODE_COPY_ON_WRITE
1930 ))
1931 })?;
1932
1933 match write_mode {
1934 IcebergWriteMode::MergeOnRead => {
1935 sink_with.insert(WRITE_MODE.to_owned(), write_mode.as_str().to_owned());
1936 }
1937
1938 IcebergWriteMode::CopyOnWrite => {
1939 if table.append_only {
1940 return Err(ErrorCode::NotSupported(
1941 "COPY ON WRITE is not supported for append-only iceberg table".to_owned(),
1942 "Please use MERGE ON READ instead".to_owned(),
1943 )
1944 .into());
1945 }
1946
1947 sink_with.insert(WRITE_MODE.to_owned(), write_mode.as_str().to_owned());
1948 }
1949 }
1950
1951 source
1953 .as_mut()
1954 .map(|x| x.with_properties.remove("write_mode"));
1955 } else {
1956 sink_with.insert(
1957 WRITE_MODE.to_owned(),
1958 ICEBERG_WRITE_MODE_MERGE_ON_READ.to_owned(),
1959 );
1960 }
1961
1962 if let Some(max_snapshots_num_before_compaction) =
1963 handler_args.with_options.get(COMPACTION_MAX_SNAPSHOTS_NUM)
1964 {
1965 let max_snapshots_num_before_compaction = max_snapshots_num_before_compaction
1966 .parse::<u32>()
1967 .map_err(|_| {
1968 ErrorCode::InvalidInputSyntax(format!(
1969 "{} must be greater than 0: {}",
1970 COMPACTION_MAX_SNAPSHOTS_NUM, max_snapshots_num_before_compaction
1971 ))
1972 })?;
1973
1974 if max_snapshots_num_before_compaction == 0 {
1975 bail!(format!(
1976 "{} must be greater than 0",
1977 COMPACTION_MAX_SNAPSHOTS_NUM
1978 ));
1979 }
1980
1981 sink_with.insert(
1982 COMPACTION_MAX_SNAPSHOTS_NUM.to_owned(),
1983 max_snapshots_num_before_compaction.to_string(),
1984 );
1985
1986 source
1988 .as_mut()
1989 .map(|x| x.with_properties.remove(COMPACTION_MAX_SNAPSHOTS_NUM));
1990 }
1991
1992 if let Some(small_files_threshold_mb) = handler_args
1993 .with_options
1994 .get(COMPACTION_SMALL_FILES_THRESHOLD_MB)
1995 {
1996 let small_files_threshold_mb = small_files_threshold_mb.parse::<u64>().map_err(|_| {
1997 ErrorCode::InvalidInputSyntax(format!(
1998 "{} must be greater than 0: {}",
1999 COMPACTION_SMALL_FILES_THRESHOLD_MB, small_files_threshold_mb
2000 ))
2001 })?;
2002 if small_files_threshold_mb == 0 {
2003 bail!(format!(
2004 "{} must be a greater than 0",
2005 COMPACTION_SMALL_FILES_THRESHOLD_MB
2006 ));
2007 }
2008 sink_with.insert(
2009 COMPACTION_SMALL_FILES_THRESHOLD_MB.to_owned(),
2010 small_files_threshold_mb.to_string(),
2011 );
2012
2013 source.as_mut().map(|x| {
2015 x.with_properties
2016 .remove(COMPACTION_SMALL_FILES_THRESHOLD_MB)
2017 });
2018 }
2019
2020 if let Some(delete_files_count_threshold) = handler_args
2021 .with_options
2022 .get(COMPACTION_DELETE_FILES_COUNT_THRESHOLD)
2023 {
2024 let delete_files_count_threshold =
2025 delete_files_count_threshold.parse::<usize>().map_err(|_| {
2026 ErrorCode::InvalidInputSyntax(format!(
2027 "{} must be greater than 0: {}",
2028 COMPACTION_DELETE_FILES_COUNT_THRESHOLD, delete_files_count_threshold
2029 ))
2030 })?;
2031 if delete_files_count_threshold == 0 {
2032 bail!(format!(
2033 "{} must be greater than 0",
2034 COMPACTION_DELETE_FILES_COUNT_THRESHOLD
2035 ));
2036 }
2037 sink_with.insert(
2038 COMPACTION_DELETE_FILES_COUNT_THRESHOLD.to_owned(),
2039 delete_files_count_threshold.to_string(),
2040 );
2041
2042 source.as_mut().map(|x| {
2044 x.with_properties
2045 .remove(COMPACTION_DELETE_FILES_COUNT_THRESHOLD)
2046 });
2047 }
2048
2049 if let Some(trigger_snapshot_count) = handler_args
2050 .with_options
2051 .get(COMPACTION_TRIGGER_SNAPSHOT_COUNT)
2052 {
2053 let trigger_snapshot_count = trigger_snapshot_count.parse::<usize>().map_err(|_| {
2054 ErrorCode::InvalidInputSyntax(format!(
2055 "{} must be greater than 0: {}",
2056 COMPACTION_TRIGGER_SNAPSHOT_COUNT, trigger_snapshot_count
2057 ))
2058 })?;
2059 if trigger_snapshot_count == 0 {
2060 bail!(format!(
2061 "{} must be greater than 0",
2062 COMPACTION_TRIGGER_SNAPSHOT_COUNT
2063 ));
2064 }
2065 sink_with.insert(
2066 COMPACTION_TRIGGER_SNAPSHOT_COUNT.to_owned(),
2067 trigger_snapshot_count.to_string(),
2068 );
2069
2070 source
2072 .as_mut()
2073 .map(|x| x.with_properties.remove(COMPACTION_TRIGGER_SNAPSHOT_COUNT));
2074 }
2075
2076 if let Some(target_file_size_mb) = handler_args
2077 .with_options
2078 .get(COMPACTION_TARGET_FILE_SIZE_MB)
2079 {
2080 let target_file_size_mb = target_file_size_mb.parse::<u64>().map_err(|_| {
2081 ErrorCode::InvalidInputSyntax(format!(
2082 "{} must be greater than 0: {}",
2083 COMPACTION_TARGET_FILE_SIZE_MB, target_file_size_mb
2084 ))
2085 })?;
2086 if target_file_size_mb == 0 {
2087 bail!(format!(
2088 "{} must be greater than 0",
2089 COMPACTION_TARGET_FILE_SIZE_MB
2090 ));
2091 }
2092 sink_with.insert(
2093 COMPACTION_TARGET_FILE_SIZE_MB.to_owned(),
2094 target_file_size_mb.to_string(),
2095 );
2096 source
2098 .as_mut()
2099 .map(|x| x.with_properties.remove(COMPACTION_TARGET_FILE_SIZE_MB));
2100 }
2101
2102 if let Some(compaction_type) = handler_args.with_options.get(COMPACTION_TYPE) {
2103 let compaction_type = CompactionType::try_from(compaction_type.as_str()).map_err(|_| {
2104 ErrorCode::InvalidInputSyntax(format!(
2105 "invalid compaction_type: {}, must be one of {:?}",
2106 compaction_type,
2107 &[
2108 CompactionType::Full,
2109 CompactionType::SmallFiles,
2110 CompactionType::FilesWithDelete
2111 ]
2112 ))
2113 })?;
2114
2115 sink_with.insert(
2116 COMPACTION_TYPE.to_owned(),
2117 compaction_type.as_str().to_owned(),
2118 );
2119
2120 source
2122 .as_mut()
2123 .map(|x| x.with_properties.remove(COMPACTION_TYPE));
2124 }
2125
2126 let partition_by = handler_args
2127 .with_options
2128 .get("partition_by")
2129 .map(|v| v.to_owned());
2130
2131 if let Some(partition_by) = &partition_by {
2132 let mut partition_columns = vec![];
2133 for (column, _) in parse_partition_by_exprs(partition_by.clone())? {
2134 table
2135 .columns()
2136 .iter()
2137 .find(|col| col.name().eq_ignore_ascii_case(&column))
2138 .ok_or_else(|| {
2139 ErrorCode::InvalidInputSyntax(format!(
2140 "Partition source column does not exist in schema: {}",
2141 column
2142 ))
2143 })?;
2144
2145 partition_columns.push(column);
2146 }
2147
2148 ensure_partition_columns_are_prefix_of_primary_key(&partition_columns, &pks).map_err(
2149 |_| {
2150 ErrorCode::InvalidInputSyntax(
2151 "The partition columns should be the prefix of the primary key".to_owned(),
2152 )
2153 },
2154 )?;
2155
2156 sink_with.insert("partition_by".to_owned(), partition_by.to_owned());
2157
2158 source
2160 .as_mut()
2161 .map(|x| x.with_properties.remove("partition_by"));
2162 }
2163
2164 sink_handler_args.with_options =
2165 WithOptions::new(sink_with, Default::default(), connection_ref.clone());
2166 let SinkPlanContext {
2167 sink_plan,
2168 sink_catalog,
2169 ..
2170 } = gen_sink_plan(sink_handler_args, create_sink_stmt, None, true).await?;
2171 let sink_graph = build_graph(sink_plan, Some(GraphJobType::Sink))?;
2172
2173 let mut source_name = table_name.clone();
2174 *source_name.0.last_mut().unwrap() = Ident::from(
2175 (ICEBERG_SOURCE_PREFIX.to_owned() + &source_name.0.last().unwrap().real_value()).as_str(),
2176 );
2177 let create_source_stmt = CreateSourceStatement {
2178 temporary: false,
2179 if_not_exists: false,
2180 columns: vec![],
2181 source_name,
2182 wildcard_idx: Some(0),
2183 constraints: vec![],
2184 with_properties: WithProperties(vec![]),
2185 format_encode: CompatibleFormatEncode::V2(FormatEncodeOptions::none()),
2186 source_watermarks: vec![],
2187 include_column_options: vec![],
2188 };
2189
2190 let mut source_handler_args = handler_args.clone();
2191 let source_with = with_common;
2192 source_handler_args.with_options =
2193 WithOptions::new(source_with, Default::default(), connection_ref);
2194
2195 let overwrite_options = OverwriteOptions::new(&mut source_handler_args);
2196 let format_encode = create_source_stmt.format_encode.into_v2_with_warning();
2197 let (with_properties, refresh_mode) =
2198 bind_connector_props(&source_handler_args, &format_encode, true)?;
2199
2200 let (iceberg_catalog, table_identifier) = {
2203 let sink_param = SinkParam::try_from_sink_catalog(sink_catalog.clone())?;
2204 let iceberg_sink = IcebergSink::try_from(sink_param)?;
2205 iceberg_sink.create_table_if_not_exists().await?;
2206
2207 let iceberg_catalog = iceberg_sink.config.create_catalog().await?;
2208 let table_identifier = iceberg_sink.config.full_table_name()?;
2209 (iceberg_catalog, table_identifier)
2210 };
2211
2212 let create_source_type = CreateSourceType::for_newly_created(&session, &*with_properties);
2213 let (columns_from_resolve_source, source_info) = bind_columns_from_source(
2214 &session,
2215 &format_encode,
2216 Either::Left(&with_properties),
2217 create_source_type,
2218 )
2219 .await?;
2220 let mut col_id_gen = ColumnIdGenerator::new_initial();
2221
2222 let iceberg_source_catalog = bind_create_source_or_table_with_connector(
2223 source_handler_args,
2224 create_source_stmt.source_name,
2225 format_encode,
2226 with_properties,
2227 &create_source_stmt.columns,
2228 create_source_stmt.constraints,
2229 create_source_stmt.wildcard_idx,
2230 create_source_stmt.source_watermarks,
2231 columns_from_resolve_source,
2232 source_info,
2233 create_source_stmt.include_column_options,
2234 &mut col_id_gen,
2235 create_source_type,
2236 overwrite_options.source_rate_limit,
2237 SqlColumnStrategy::FollowChecked,
2238 refresh_mode,
2239 )
2240 .await?;
2241
2242 let _ = Jvm::get_or_init()?;
2245
2246 let catalog_writer = session.catalog_writer()?;
2247 let action = match job_type {
2248 TableJobType::SharedCdcSource => LongRunningNotificationAction::MonitorBackfillJob,
2249 _ => LongRunningNotificationAction::DiagnoseBarrierLatency,
2250 };
2251 let res = execute_with_long_running_notification(
2252 catalog_writer.create_iceberg_table(
2253 PbTableJobInfo {
2254 source,
2255 table: Some(table.to_prost()),
2256 fragment_graph: Some(graph),
2257 job_type: job_type as _,
2258 },
2259 PbSinkJobInfo {
2260 sink: Some(sink_catalog.to_proto()),
2261 fragment_graph: Some(sink_graph),
2262 },
2263 iceberg_source_catalog.to_prost(),
2264 if_not_exists,
2265 ),
2266 &session,
2267 "CREATE TABLE",
2268 action,
2269 )
2270 .await;
2271
2272 if res.is_err() {
2273 let _ = iceberg_catalog
2274 .drop_table(&table_identifier)
2275 .await
2276 .inspect_err(|err| {
2277 tracing::error!(
2278 "failed to drop iceberg table {} after create iceberg engine table failed: {}",
2279 table_identifier,
2280 err.as_report()
2281 );
2282 });
2283 res?
2284 }
2285
2286 Ok(())
2287}
2288
2289pub fn check_create_table_with_source(
2290 with_options: &WithOptions,
2291 format_encode: Option<FormatEncodeOptions>,
2292 include_column_options: &IncludeOption,
2293 cdc_table_info: &Option<CdcTableInfo>,
2294) -> Result<Option<FormatEncodeOptions>> {
2295 if cdc_table_info.is_some() {
2297 return Ok(format_encode);
2298 }
2299 let defined_source = with_options.is_source_connector();
2300
2301 if !include_column_options.is_empty() && !defined_source {
2302 return Err(ErrorCode::InvalidInputSyntax(
2303 "INCLUDE should be used with a connector".to_owned(),
2304 )
2305 .into());
2306 }
2307 if defined_source {
2308 format_encode.as_ref().ok_or_else(|| {
2309 ErrorCode::InvalidInputSyntax("Please specify a source schema using FORMAT".to_owned())
2310 })?;
2311 }
2312 Ok(format_encode)
2313}
2314
2315fn ensure_partition_columns_are_prefix_of_primary_key(
2316 partition_columns: &[String],
2317 primary_key_columns: &[String],
2318) -> std::result::Result<(), String> {
2319 if partition_columns.len() > primary_key_columns.len() {
2320 return Err("Partition columns cannot be longer than primary key columns.".to_owned());
2321 }
2322
2323 for (i, partition_col) in partition_columns.iter().enumerate() {
2324 if primary_key_columns.get(i) != Some(partition_col) {
2325 return Err(format!(
2326 "Partition column '{}' is not a prefix of the primary key.",
2327 partition_col
2328 ));
2329 }
2330 }
2331
2332 Ok(())
2333}
2334
2335#[allow(clippy::too_many_arguments)]
2336pub async fn generate_stream_graph_for_replace_table(
2337 _session: &Arc<SessionImpl>,
2338 table_name: ObjectName,
2339 original_catalog: &Arc<TableCatalog>,
2340 handler_args: HandlerArgs,
2341 statement: Statement,
2342 col_id_gen: ColumnIdGenerator,
2343 sql_column_strategy: SqlColumnStrategy,
2344) -> Result<(
2345 StreamFragmentGraph,
2346 TableCatalog,
2347 Option<SourceCatalog>,
2348 TableJobType,
2349)> {
2350 let Statement::CreateTable {
2351 columns,
2352 constraints,
2353 source_watermarks,
2354 append_only,
2355 on_conflict,
2356 with_version_columns,
2357 wildcard_idx,
2358 cdc_table_info,
2359 format_encode,
2360 include_column_options,
2361 engine,
2362 with_options,
2363 ..
2364 } = statement
2365 else {
2366 panic!("unexpected statement type: {:?}", statement);
2367 };
2368
2369 let format_encode = format_encode
2370 .clone()
2371 .map(|format_encode| format_encode.into_v2_with_warning());
2372
2373 let engine = match engine {
2374 risingwave_sqlparser::ast::Engine::Hummock => Engine::Hummock,
2375 risingwave_sqlparser::ast::Engine::Iceberg => Engine::Iceberg,
2376 };
2377
2378 let is_drop_connector =
2379 original_catalog.associated_source_id().is_some() && format_encode.is_none();
2380 if is_drop_connector {
2381 debug_assert!(
2382 source_watermarks.is_empty()
2383 && include_column_options.is_empty()
2384 && with_options
2385 .iter()
2386 .all(|opt| opt.name.real_value().to_lowercase() != "connector")
2387 );
2388 }
2389
2390 let props = CreateTableProps {
2391 definition: handler_args.normalized_sql.clone(),
2392 append_only,
2393 on_conflict: on_conflict.into(),
2394 with_version_columns: with_version_columns
2395 .iter()
2396 .map(|col| col.real_value())
2397 .collect(),
2398 webhook_info: original_catalog.webhook_info.clone(),
2399 engine,
2400 };
2401
2402 let ((plan, mut source, mut table), job_type) = match (format_encode, cdc_table_info.as_ref()) {
2403 (Some(format_encode), None) => (
2404 gen_create_table_plan_with_source(
2405 handler_args,
2406 ExplainOptions::default(),
2407 table_name,
2408 columns,
2409 wildcard_idx,
2410 constraints,
2411 format_encode,
2412 source_watermarks,
2413 col_id_gen,
2414 include_column_options,
2415 props,
2416 sql_column_strategy,
2417 )
2418 .await?,
2419 TableJobType::General,
2420 ),
2421 (None, None) => {
2422 let context = OptimizerContext::from_handler_args(handler_args);
2423 let (plan, table) = gen_create_table_plan(
2424 context,
2425 table_name,
2426 columns,
2427 constraints,
2428 col_id_gen,
2429 source_watermarks,
2430 props,
2431 true,
2432 )?;
2433 ((plan, None, table), TableJobType::General)
2434 }
2435 (None, Some(cdc_table)) => {
2436 let session = &handler_args.session;
2437 let (source, resolved_table_name) =
2438 get_source_and_resolved_table_name(session, cdc_table.clone(), table_name.clone())?;
2439
2440 let cdc_with_options = derive_with_options_for_cdc_table(
2441 &source.with_properties,
2442 cdc_table.external_table_name.clone(),
2443 )?;
2444
2445 let (column_catalogs, pk_names) = bind_cdc_table_schema(&columns, &constraints, true)?;
2446
2447 let context: OptimizerContextRef =
2448 OptimizerContext::new(handler_args, ExplainOptions::default()).into();
2449 let (plan, table) = gen_create_table_plan_for_cdc_table(
2450 context,
2451 source,
2452 cdc_table.external_table_name.clone(),
2453 columns,
2454 column_catalogs,
2455 pk_names,
2456 cdc_with_options,
2457 col_id_gen,
2458 on_conflict,
2459 with_version_columns
2460 .iter()
2461 .map(|col| col.real_value())
2462 .collect(),
2463 include_column_options,
2464 table_name,
2465 resolved_table_name,
2466 original_catalog.database_id,
2467 original_catalog.schema_id,
2468 original_catalog.id(),
2469 engine,
2470 )?;
2471
2472 ((plan, None, table), TableJobType::SharedCdcSource)
2473 }
2474 (Some(_), Some(_)) => {
2475 return Err(ErrorCode::NotSupported(
2476 "Data format and encoding format doesn't apply to table created from a CDC source"
2477 .into(),
2478 "Remove the FORMAT and ENCODE specification".into(),
2479 )
2480 .into());
2481 }
2482 };
2483
2484 if table.pk_column_ids() != original_catalog.pk_column_ids() {
2485 Err(ErrorCode::InvalidInputSyntax(
2486 "alter primary key of table is not supported".to_owned(),
2487 ))?
2488 }
2489
2490 let graph = build_graph(plan, Some(GraphJobType::Table))?;
2491
2492 table.id = original_catalog.id();
2494 if !is_drop_connector && let Some(source_id) = original_catalog.associated_source_id() {
2495 table.associated_source_id = Some(source_id);
2496
2497 let source = source.as_mut().unwrap();
2498 source.id = source_id;
2499 source.associated_table_id = Some(table.id());
2500 }
2501
2502 Ok((graph, table, source, job_type))
2503}
2504
2505fn get_source_and_resolved_table_name(
2506 session: &Arc<SessionImpl>,
2507 cdc_table: CdcTableInfo,
2508 table_name: ObjectName,
2509) -> Result<(Arc<SourceCatalog>, String)> {
2510 let db_name = &session.database();
2511 let (_, resolved_table_name) = Binder::resolve_schema_qualified_name(db_name, &table_name)?;
2512
2513 let (source_schema, source_name) =
2514 Binder::resolve_schema_qualified_name(db_name, &cdc_table.source_name)?;
2515
2516 let source = {
2517 let catalog_reader = session.env().catalog_reader().read_guard();
2518 let schema_name = source_schema.unwrap_or(DEFAULT_SCHEMA_NAME.to_owned());
2519 let (source, _) = catalog_reader.get_source_by_name(
2520 db_name,
2521 SchemaPath::Name(schema_name.as_str()),
2522 source_name.as_str(),
2523 )?;
2524 source.clone()
2525 };
2526
2527 Ok((source, resolved_table_name))
2528}
2529
2530fn bind_webhook_info(
2532 session: &Arc<SessionImpl>,
2533 columns_defs: &[ColumnDef],
2534 webhook_info: WebhookSourceInfo,
2535) -> Result<PbWebhookSourceInfo> {
2536 if columns_defs.len() != 1 || columns_defs[0].data_type.as_ref().unwrap() != &AstDataType::Jsonb
2538 {
2539 return Err(ErrorCode::InvalidInputSyntax(
2540 "Table with webhook source should have exactly one JSONB column".to_owned(),
2541 )
2542 .into());
2543 }
2544
2545 let WebhookSourceInfo {
2546 secret_ref,
2547 signature_expr,
2548 wait_for_persistence,
2549 is_batched,
2550 } = webhook_info;
2551
2552 let (pb_secret_ref, secret_name) = if let Some(secret_ref) = secret_ref {
2554 let db_name = &session.database();
2555 let (schema_name, secret_name) =
2556 Binder::resolve_schema_qualified_name(db_name, &secret_ref.secret_name)?;
2557 let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?;
2558 (
2559 Some(PbSecretRef {
2560 secret_id: secret_catalog.id,
2561 ref_as: match secret_ref.ref_as {
2562 SecretRefAsType::Text => PbRefAsType::Text,
2563 SecretRefAsType::File => PbRefAsType::File,
2564 }
2565 .into(),
2566 }),
2567 Some(secret_name),
2568 )
2569 } else {
2570 (None, None)
2571 };
2572
2573 let secure_compare_context = SecureCompareContext {
2574 column_name: columns_defs[0].name.real_value(),
2575 secret_name,
2576 };
2577 let mut binder = Binder::new_for_ddl(session).with_secure_compare(secure_compare_context);
2578 let expr = binder.bind_expr(&signature_expr)?;
2579
2580 if expr.as_function_call().is_none()
2582 || expr.as_function_call().unwrap().func_type()
2583 != crate::optimizer::plan_node::generic::ExprType::SecureCompare
2584 {
2585 return Err(ErrorCode::InvalidInputSyntax(
2586 "The signature verification function must be SECURE_COMPARE()".to_owned(),
2587 )
2588 .into());
2589 }
2590
2591 let pb_webhook_info = PbWebhookSourceInfo {
2592 secret_ref: pb_secret_ref,
2593 signature_expr: Some(expr.to_expr_proto()),
2594 wait_for_persistence,
2595 is_batched,
2596 };
2597
2598 Ok(pb_webhook_info)
2599}
2600
2601#[cfg(test)]
2602mod tests {
2603 use risingwave_common::catalog::{
2604 DEFAULT_DATABASE_NAME, ROW_ID_COLUMN_NAME, RW_TIMESTAMP_COLUMN_NAME,
2605 };
2606 use risingwave_common::types::{DataType, StructType};
2607
2608 use super::*;
2609 use crate::test_utils::{LocalFrontend, PROTO_FILE_DATA, create_proto_file};
2610
2611 #[tokio::test]
2612 async fn test_create_table_handler() {
2613 let sql =
2614 "create table t (v1 smallint, v2 struct<v3 bigint, v4 float, v5 double>) append only;";
2615 let frontend = LocalFrontend::new(Default::default()).await;
2616 frontend.run_sql(sql).await.unwrap();
2617
2618 let session = frontend.session_ref();
2619 let catalog_reader = session.env().catalog_reader().read_guard();
2620 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
2621
2622 let (table, _) = catalog_reader
2624 .get_created_table_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
2625 .unwrap();
2626 assert_eq!(table.name(), "t");
2627
2628 let columns = table
2629 .columns
2630 .iter()
2631 .map(|col| (col.name(), col.data_type().clone()))
2632 .collect::<HashMap<&str, DataType>>();
2633
2634 let expected_columns = maplit::hashmap! {
2635 ROW_ID_COLUMN_NAME => DataType::Serial,
2636 "v1" => DataType::Int16,
2637 "v2" => StructType::new(
2638 vec![("v3", DataType::Int64),("v4", DataType::Float64),("v5", DataType::Float64)],
2639 )
2640 .with_ids([3, 4, 5].map(ColumnId::new))
2641 .into(),
2642 RW_TIMESTAMP_COLUMN_NAME => DataType::Timestamptz,
2643 };
2644
2645 assert_eq!(columns, expected_columns, "{columns:#?}");
2646 }
2647
2648 #[test]
2649 fn test_bind_primary_key() {
2650 for (sql, expected) in [
2653 ("create table t (v1 int, v2 int)", Ok(&[0] as &[_])),
2654 ("create table t (v1 int primary key, v2 int)", Ok(&[1])),
2655 ("create table t (v1 int, v2 int primary key)", Ok(&[2])),
2656 (
2657 "create table t (v1 int primary key, v2 int primary key)",
2658 Err("multiple primary keys are not allowed"),
2659 ),
2660 (
2661 "create table t (v1 int primary key primary key, v2 int)",
2662 Err("multiple primary keys are not allowed"),
2663 ),
2664 (
2665 "create table t (v1 int, v2 int, primary key (v1))",
2666 Ok(&[1]),
2667 ),
2668 (
2669 "create table t (v1 int, primary key (v2), v2 int)",
2670 Ok(&[2]),
2671 ),
2672 (
2673 "create table t (primary key (v2, v1), v1 int, v2 int)",
2674 Ok(&[2, 1]),
2675 ),
2676 (
2677 "create table t (v1 int, primary key (v1), v2 int, primary key (v1))",
2678 Err("multiple primary keys are not allowed"),
2679 ),
2680 (
2681 "create table t (v1 int primary key, primary key (v1), v2 int)",
2682 Err("multiple primary keys are not allowed"),
2683 ),
2684 (
2685 "create table t (v1 int, primary key (V3), v2 int)",
2686 Err("column \"v3\" named in key does not exist"),
2687 ),
2688 ] {
2689 let mut ast = risingwave_sqlparser::parser::Parser::parse_sql(sql).unwrap();
2690 let risingwave_sqlparser::ast::Statement::CreateTable {
2691 columns: column_defs,
2692 constraints,
2693 ..
2694 } = ast.remove(0)
2695 else {
2696 panic!("test case should be create table")
2697 };
2698 let actual: Result<_> = (|| {
2699 let mut columns = bind_sql_columns(&column_defs, false)?;
2700 let mut col_id_gen = ColumnIdGenerator::new_initial();
2701 for c in &mut columns {
2702 col_id_gen.generate(c)?;
2703 }
2704
2705 let pk_names =
2706 bind_sql_pk_names(&column_defs, bind_table_constraints(&constraints)?)?;
2707 let (_, pk_column_ids, _) =
2708 bind_pk_and_row_id_on_relation(columns, pk_names, true)?;
2709 Ok(pk_column_ids)
2710 })();
2711 match (expected, actual) {
2712 (Ok(expected), Ok(actual)) => assert_eq!(
2713 expected.iter().copied().map(ColumnId::new).collect_vec(),
2714 actual,
2715 "sql: {sql}"
2716 ),
2717 (Ok(_), Err(actual)) => panic!("sql: {sql}\nunexpected error: {actual:?}"),
2718 (Err(_), Ok(actual)) => panic!("sql: {sql}\nexpects error but got: {actual:?}"),
2719 (Err(expected), Err(actual)) => assert!(
2720 actual.to_string().contains(expected),
2721 "sql: {sql}\nexpected: {expected:?}\nactual: {actual:?}"
2722 ),
2723 }
2724 }
2725 }
2726
2727 #[tokio::test]
2728 async fn test_duplicate_props_options() {
2729 let proto_file = create_proto_file(PROTO_FILE_DATA);
2730 let sql = format!(
2731 r#"CREATE TABLE t
2732 WITH (
2733 connector = 'kinesis',
2734 aws.region='user_test_topic',
2735 endpoint='172.10.1.1:9090,172.10.1.2:9090',
2736 aws.credentials.access_key_id = 'your_access_key_1',
2737 aws.credentials.secret_access_key = 'your_secret_key_1'
2738 )
2739 FORMAT PLAIN ENCODE PROTOBUF (
2740 message = '.test.TestRecord',
2741 aws.credentials.access_key_id = 'your_access_key_2',
2742 aws.credentials.secret_access_key = 'your_secret_key_2',
2743 schema.location = 'file://{}',
2744 )"#,
2745 proto_file.path().to_str().unwrap()
2746 );
2747 let frontend = LocalFrontend::new(Default::default()).await;
2748 frontend.run_sql(sql).await.unwrap();
2749
2750 let session = frontend.session_ref();
2751 let catalog_reader = session.env().catalog_reader().read_guard();
2752 let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);
2753
2754 let (source, _) = catalog_reader
2756 .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
2757 .unwrap();
2758 assert_eq!(source.name, "t");
2759
2760 assert_eq!(
2762 source
2763 .info
2764 .format_encode_options
2765 .get("aws.credentials.access_key_id")
2766 .unwrap(),
2767 "your_access_key_2"
2768 );
2769 assert_eq!(
2770 source
2771 .info
2772 .format_encode_options
2773 .get("aws.credentials.secret_access_key")
2774 .unwrap(),
2775 "your_secret_key_2"
2776 );
2777
2778 assert_eq!(
2780 source
2781 .with_properties
2782 .get("aws.credentials.access_key_id")
2783 .unwrap(),
2784 "your_access_key_1"
2785 );
2786 assert_eq!(
2787 source
2788 .with_properties
2789 .get("aws.credentials.secret_access_key")
2790 .unwrap(),
2791 "your_secret_key_1"
2792 );
2793
2794 assert!(!source.with_properties.contains_key("schema.location"));
2796 }
2797}