From 0c9d6afd6bb8098ae83685346089aca660981810 Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Wed, 17 Jul 2024 09:54:30 -0400 Subject: [PATCH 1/3] storage/sql: Refactor subsource purification to split out statement generation and sequence after source planning --- src/adapter/src/coord/message_handler.rs | 12 +- src/adapter/src/coord/sequencer/inner.rs | 48 ++-- src/sql/src/pure.rs | 233 +++++++++++------- src/sql/src/pure/mysql.rs | 77 +++--- src/sql/src/pure/postgres.rs | 154 ++++++------ src/storage-controller/src/lib.rs | 2 +- src/storage-types/src/sources.rs | 12 +- src/storage-types/src/sources/kafka.rs | 2 +- .../src/sources/load_generator.rs | 2 +- src/storage-types/src/sources/mysql.rs | 2 +- src/storage-types/src/sources/postgres.rs | 2 +- 11 files changed, 314 insertions(+), 232 deletions(-) diff --git a/src/adapter/src/coord/message_handler.rs b/src/adapter/src/coord/message_handler.rs index e9d787212fcbd..6475c8a7fdc0b 100644 --- a/src/adapter/src/coord/message_handler.rs +++ b/src/adapter/src/coord/message_handler.rs @@ -521,28 +521,28 @@ impl Coordinator { PurifiedStatement::PurifiedCreateSource { create_progress_subsource_stmt, create_source_stmt, - create_subsource_stmts, + subsources, } => { self.plan_purified_create_source( &ctx, params, create_progress_subsource_stmt, create_source_stmt, - create_subsource_stmts, + subsources, ) .await } PurifiedStatement::PurifiedAlterSourceAddSubsources { - altered_id, + source_name, options, - create_subsource_stmts, + subsources, } => { self.plan_purified_alter_source_add_subsource( ctx.session(), params, - altered_id, + source_name, options, - create_subsource_stmts, + subsources, ) .await } diff --git a/src/adapter/src/coord/sequencer/inner.rs b/src/adapter/src/coord/sequencer/inner.rs index 274bcff3c38bc..e23c23f295f1e 100644 --- a/src/adapter/src/coord/sequencer/inner.rs +++ b/src/adapter/src/coord/sequencer/inner.rs @@ -49,6 +49,8 @@ use mz_sql::names::{ Aug, ObjectId, QualifiedItemName, ResolvedDatabaseSpecifier, ResolvedIds, ResolvedItemName, SchemaSpecifier, SystemObjectId, }; +use mz_sql::plan::StatementContext; +use mz_sql::pure::{generate_subsource_statements, PurifiedSourceExport}; use mz_storage_types::sinks::StorageSinkDesc; use mz_storage_types::sources::mysql::{MySqlSourceDetails, ProtoMySqlSourceDetails}; use mz_storage_types::sources::postgres::{ @@ -382,25 +384,37 @@ impl Coordinator { &mut self, session: &Session, params: Params, - id: GlobalId, + source_name: ResolvedItemName, options: Vec>, - create_subsource_stmts: Vec>, + subsources: BTreeMap, ) -> Result<(Plan, ResolvedIds), AdapterError> { - let mut subsources = Vec::with_capacity(create_subsource_stmts.len()); - for subsource_stmt in create_subsource_stmts { + let mut subsource_plans = Vec::with_capacity(subsources.len()); + + // Generate subsource statements + let conn_catalog = self.catalog().for_system_session(); + let pcx = plan::PlanContext::zero(); + let scx = StatementContext::new(Some(&pcx), &conn_catalog); + + let source_id = *source_name.item_id(); + let subsource_stmts = generate_subsource_statements(&scx, source_name, subsources)?; + + for subsource_stmt in subsource_stmts { let s = self .plan_subsource(session, ¶ms, subsource_stmt) .await?; - subsources.push(s); + subsource_plans.push(s); } let action = mz_sql::plan::AlterSourceAction::AddSubsourceExports { - subsources, + subsources: subsource_plans, options, }; Ok(( - Plan::AlterSource(mz_sql::plan::AlterSourcePlan { id, action }), + Plan::AlterSource(mz_sql::plan::AlterSourcePlan { + id: source_id, + action, + }), ResolvedIds(BTreeSet::new()), )) } @@ -414,9 +428,9 @@ impl Coordinator { params: Params, progress_stmt: CreateSubsourceStatement, mut source_stmt: mz_sql::ast::CreateSourceStatement, - subsource_stmts: Vec>, + subsources: BTreeMap, ) -> Result<(Plan, ResolvedIds), AdapterError> { - let mut create_source_plans = Vec::with_capacity(subsource_stmts.len() + 2); + let mut create_source_plans = Vec::with_capacity(subsources.len() + 2); // 1. First plan the progress subsource. // @@ -440,15 +454,11 @@ impl Coordinator { create_source_plans.push(progress_plan); - // 2. Then plan the main source. - // - // The subsources need this to exist in order to set their `OF SOURCE` - // correctly. source_stmt.progress_subsource = Some(DeferredItemName::Named(progress_subsource)); let resolved_ids = mz_sql::names::visit_dependencies(&source_stmt); - // Plan primary source. + // 2. Then plan the main source. let source_plan = match self.plan_statement( ctx.session(), Statement::CreateSource(source_stmt), @@ -468,6 +478,13 @@ impl Coordinator { print_id: true, }; + // Generate subsource statements + let conn_catalog = self.catalog().for_system_session(); + let pcx = plan::PlanContext::zero(); + let scx = StatementContext::new(Some(&pcx), &conn_catalog); + + let subsource_stmts = generate_subsource_statements(&scx, of_source, subsources)?; + create_source_plans.push(CreateSourcePlanBundle { source_id, plan: source_plan, @@ -475,8 +492,7 @@ impl Coordinator { }); // 3. Finally, plan all the subsources - for mut stmt in subsource_stmts { - stmt.of_source = Some(of_source.clone()); + for stmt in subsource_stmts { let plan = self.plan_subsource(ctx.session(), ¶ms, stmt).await?; create_source_plans.push(plan); } diff --git a/src/sql/src/pure.rs b/src/sql/src/pure.rs index 70dfda22c9efe..2904e72516bcc 100644 --- a/src/sql/src/pure.rs +++ b/src/sql/src/pure.rs @@ -22,14 +22,16 @@ use itertools::Itertools; use mz_ccsr::{Client, GetByIdError, GetBySubjectError, Schema as CcsrSchema}; use mz_controller_types::ClusterId; use mz_kafka_util::client::MzClientContext; +use mz_mysql_util::MySqlTableDesc; use mz_ore::assert_none; use mz_ore::error::ErrorExt; use mz_ore::future::InTask; use mz_ore::iter::IteratorExt; use mz_ore::str::StrExt; +use mz_postgres_util::desc::PostgresTableDesc; use mz_postgres_util::replication::WalLevel; use mz_proto::RustType; -use mz_repr::{strconv, Timestamp}; +use mz_repr::{strconv, RelationDesc, Timestamp}; use mz_sql_parser::ast::display::AstDisplay; use mz_sql_parser::ast::visit::{visit_function, Visit}; use mz_sql_parser::ast::visit_mut::{visit_expr_mut, VisitMut}; @@ -81,11 +83,11 @@ use self::error::{ }; pub(crate) mod error; -mod mysql; -mod postgres; +pub mod mysql; +pub mod postgres; pub(crate) struct RequestedSubsource<'a, T> { - upstream_name: UnresolvedItemName, + external_reference: UnresolvedItemName, subsource_name: UnresolvedItemName, table: &'a T, } @@ -122,13 +124,13 @@ fn subsource_gen<'a, T: SubsourceCatalogReference>( } }; - let (qualified_upstream_name, idx) = + let (external_reference, idx) = resolver.resolve(&subsource.reference.0, canonical_width)?; let table = &references[idx]; validated_requested_subsources.push(RequestedSubsource { - upstream_name: qualified_upstream_name, + external_reference, subsource_name, table, }); @@ -169,7 +171,7 @@ fn validate_subsource_names( .into_iter() .filter_map(|subsource| { if &subsource.subsource_name == &name { - Some(subsource.upstream_name.clone()) + Some(subsource.external_reference.clone()) } else { None } @@ -188,7 +190,7 @@ fn validate_subsource_names( // it is almost certainly an error on the user's end. if let Some(name) = requested_subsources .iter() - .map(|subsource| &subsource.upstream_name) + .map(|subsource| &subsource.external_reference) .duplicates() .next() .cloned() @@ -196,7 +198,7 @@ fn validate_subsource_names( let mut target_names: Vec<_> = requested_subsources .into_iter() .filter_map(|subsource| { - if &subsource.upstream_name == &name { + if &subsource.external_reference == &name { Some(subsource.subsource_name.clone()) } else { None @@ -212,27 +214,53 @@ fn validate_subsource_names( Ok(()) } -#[derive(Debug, Clone, PartialEq, Eq, Hash)] +#[derive(Debug, Clone, PartialEq, Eq)] pub enum PurifiedStatement { PurifiedCreateSource { create_progress_subsource_stmt: CreateSubsourceStatement, create_source_stmt: CreateSourceStatement, - create_subsource_stmts: Vec>, + // Map of subsource names to external details + subsources: BTreeMap, }, PurifiedAlterSource { alter_source_stmt: AlterSourceStatement, }, PurifiedAlterSourceAddSubsources { // This just saves us an annoying catalog lookup - altered_id: mz_repr::GlobalId, + source_name: ResolvedItemName, /// Options that we will need the values of to update the source's /// definition. options: Vec>, - create_subsource_stmts: Vec>, + // Map of subsource names to external details + subsources: BTreeMap, }, PurifiedCreateSink(CreateSinkStatement), } +#[derive(Debug, Clone, PartialEq, Eq)] +pub struct PurifiedSourceExport { + pub external_reference: UnresolvedItemName, + pub details: PurifiedExportDetails, +} + +#[derive(Debug, Clone, PartialEq, Eq)] +pub enum PurifiedExportDetails { + MySql { + table: MySqlTableDesc, + }, + Postgres { + table: PostgresTableDesc, + // Text Columns are needed in Postgres to generate cast expressions + // when (re)-planning the primary source + // TODO: Refactor this when we move casts to the source exports themselves + text_columns: Option>, + }, + Kafka {}, + LoadGenerator { + table: RelationDesc, + }, +} + /// Purifies a statement, removing any dependencies on external state. /// /// See the section on [purification](crate#purification) in the crate @@ -551,7 +579,7 @@ async fn purify_create_source( sql_bail!("Cannot manually ID qualify progress subsource") } - let mut create_subsource_stmts = vec![]; + let mut requested_subsource_map = BTreeMap::new(); let progress_desc = match &connection { CreateSourceConnection::Kafka { .. } => { @@ -746,7 +774,7 @@ async fn purify_create_source( } let postgres::PurifiedSubsources { - new_subsources, + subsources, referenced_tables, normalized_text_columns, } = postgres::purify_subsources( @@ -756,9 +784,7 @@ async fn purify_create_source( &connection, referenced_subsources, text_columns, - None, source_name, - &catalog, ) .await?; @@ -769,11 +795,7 @@ async fn purify_create_source( text_cols_option.value = Some(WithOptionValue::Sequence(normalized_text_columns)); } - // Now that we know which subsources to create alongside this - // statement, remove the references so it is not canonicalized as - // part of the `CREATE SOURCE` statement in the catalog. - *referenced_subsources = None; - create_subsource_stmts.extend(new_subsources); + requested_subsource_map.extend(subsources); // Record the active replication timeline_id to allow detection of a future upstream // point-in-time-recovery that will put the source into an error state. @@ -878,7 +900,7 @@ async fn purify_create_source( mz_mysql_util::query_sys_var(&mut conn, "global.gtid_executed").await?; let mysql::PurifiedSubsources { - new_subsources, + subsources, tables, normalized_text_columns, normalized_ignore_columns, @@ -887,11 +909,10 @@ async fn purify_create_source( referenced_subsources, text_columns, ignore_columns, - None, source_name, - &catalog, ) .await?; + requested_subsource_map.extend(subsources); // Create/Update the details for the source to include the new tables let details = MySqlSourceDetails { @@ -925,13 +946,6 @@ async fn purify_create_source( ignore_cols_option.value = Some(WithOptionValue::Sequence(normalized_ignore_columns)); } - - // Now that we know which subsources to create alongside this - // statement, remove the references so it is not canonicalized as - // part of the `CREATE SOURCE` statement in the catalog. - *referenced_subsources = None; - - create_subsource_stmts.extend(new_subsources); } CreateSourceConnection::LoadGenerator { generator, options } => { let scx = StatementContext::new(None, &catalog); @@ -939,7 +953,6 @@ async fn purify_create_source( let (_load_generator, available_subsources) = load_generator_ast_to_generator(&scx, generator, options, include_metadata)?; - let mut validated_requested_subsources = vec![]; match referenced_subsources { Some(ReferencedSubsources::All) => { let available_subsources = match &available_subsources { @@ -947,9 +960,17 @@ async fn purify_create_source( None => Err(LoadGeneratorSourcePurificationError::ForAllTables)?, }; for (name, (_, desc)) in available_subsources { - let upstream_name = UnresolvedItemName::from(name.clone()); + let external_reference = UnresolvedItemName::from(name.clone()); let subsource_name = subsource_name_gen(source_name, &name.item)?; - validated_requested_subsources.push((upstream_name, subsource_name, desc)); + requested_subsource_map.insert( + subsource_name, + PurifiedSourceExport { + external_reference, + details: PurifiedExportDetails::LoadGenerator { + table: desc.clone(), + }, + }, + ); } } Some(ReferencedSubsources::SubsetSchemas(..)) => { @@ -964,40 +985,14 @@ async fn purify_create_source( } } }; - - // Now that we have an explicit list of validated requested subsources we can create them - for (upstream_name, subsource_name, desc) in validated_requested_subsources.into_iter() - { - let (columns, table_constraints) = scx.relation_desc_into_table_defs(desc)?; - - // Create the subsource statement - let subsource = CreateSubsourceStatement { - name: subsource_name, - columns, - // We don't know the primary source's `GlobalId` yet; fill - // it in once we generate it. - of_source: None, - // unlike sources that come from an external upstream, we - // have more leniency to introduce different constraints - // every time the load generator is run; i.e. we are not as - // worried about introducing junk data. - constraints: table_constraints, - if_not_exists: false, - with_options: vec![CreateSubsourceOption { - name: CreateSubsourceOptionName::ExternalReference, - value: Some(WithOptionValue::UnresolvedItemName(upstream_name)), - }], - }; - create_subsource_stmts.push(subsource); - } - - // Now that we know which subsources to create alongside this - // statement, remove the references so it is not canonicalized as - // part of the `CREATE SOURCE` statement in the catalog. - *referenced_subsources = None; } } + // Now that we know which subsources to create alongside this + // statement, remove the references so it is not canonicalized as + // part of the `CREATE SOURCE` statement in the catalog. + *referenced_subsources = None; + // Generate progress subsource // Create the targeted AST node for the original CREATE SOURCE statement @@ -1062,14 +1057,14 @@ async fn purify_create_source( Ok(PurifiedStatement::PurifiedCreateSource { create_progress_subsource_stmt, create_source_stmt, - create_subsource_stmts, + subsources: requested_subsource_map, }) } /// Equivalent to `purify_create_source` but for `AlterSourceStatement`. /// -/// On success, returns the a set of statements that sequencing expects for -/// handling `ALTER SOURCE` statements. +/// On success, returns the details on new subsources and updated +/// 'options' that sequencing expects for handling `ALTER SOURCE` statements. async fn purify_alter_source( catalog: impl SessionCatalog, stmt: AlterSourceStatement, @@ -1097,8 +1092,6 @@ async fn purify_alter_source( Err(e) => return Err(e), }; - let altered_id = item.id(); - // Ensure it's an ingestion-based and alterable source. let desc = match item.source_desc()? { Some(desc) => desc.clone().into_inline_connection(scx.catalog), @@ -1107,12 +1100,12 @@ async fn purify_alter_source( } }; - let name = item.name(); - let full_name = scx.catalog.resolve_full_name(name); + let source_name = item.name(); + let resolved_source_name = ResolvedItemName::Item { id: item.id(), qualifiers: item.name().qualifiers.clone(), - full_name, + full_name: scx.catalog.resolve_full_name(source_name), print_id: true, }; let connection_name = desc.connection.name(); @@ -1123,7 +1116,7 @@ async fn purify_alter_source( GenericSourceConnection::MySql(_) => {} _ => sql_bail!( "{} is a {} source, which does not support ALTER SOURCE.", - scx.catalog.minimal_qualification(name), + scx.catalog.minimal_qualification(source_name), connection_name, ), }; @@ -1151,7 +1144,7 @@ async fn purify_alter_source( } = options.clone().try_into()?; assert_none!(details, "details cannot be explicitly set"); - let mut create_subsource_stmts = vec![]; + let mut requested_subsource_map = BTreeMap::new(); match desc.connection { GenericSourceConnection::Postgres(pg_source_connection) => { @@ -1184,7 +1177,7 @@ async fn purify_alter_source( if !ignore_columns.is_empty() { sql_bail!( "{} is a {} source, which does not support IGNORE COLUMNS.", - scx.catalog.minimal_qualification(name), + scx.catalog.minimal_qualification(source_name), connection_name ) } @@ -1193,7 +1186,7 @@ async fn purify_alter_source( Some(ReferencedSubsources::SubsetTables(targeted_subsources)); let postgres::PurifiedSubsources { - new_subsources, + subsources, referenced_tables, normalized_text_columns, } = postgres::purify_subsources( @@ -1203,9 +1196,7 @@ async fn purify_alter_source( pg_connection, &mut referenced_subsources, text_columns, - Some(resolved_source_name), &unresolved_source_name, - &catalog, ) .await?; @@ -1216,7 +1207,7 @@ async fn purify_alter_source( text_cols_option.value = Some(WithOptionValue::Sequence(normalized_text_columns)); } - create_subsource_stmts.extend(new_subsources); + requested_subsource_map.extend(subsources); let timeline_id = match pg_source_connection.publication_details.timeline_id { None => { @@ -1280,7 +1271,7 @@ async fn purify_alter_source( Some(ReferencedSubsources::SubsetTables(targeted_subsources)); let mysql::PurifiedSubsources { - new_subsources, + subsources, tables, normalized_text_columns, normalized_ignore_columns, @@ -1289,11 +1280,10 @@ async fn purify_alter_source( &mut referenced_subsources, text_columns, ignore_columns, - Some(resolved_source_name.clone()), &unresolved_source_name, - &catalog, ) .await?; + requested_subsource_map.extend(subsources); // Retrieve the current @gtid_executed value of the server to mark as the effective // initial snapshot point for these subsources. @@ -1338,16 +1328,14 @@ async fn purify_alter_source( ignore_cols_option.value = Some(WithOptionValue::Sequence(normalized_ignore_columns)); } - - create_subsource_stmts.extend(new_subsources); } _ => unreachable!(), }; Ok(PurifiedStatement::PurifiedAlterSourceAddSubsources { - altered_id, + source_name: resolved_source_name, options, - create_subsource_stmts, + subsources: requested_subsource_map, }) } @@ -1430,6 +1418,75 @@ async fn purify_source_format_single( Ok(()) } +pub fn generate_subsource_statements( + scx: &StatementContext, + source_name: ResolvedItemName, + subsources: BTreeMap, +) -> Result>, PlanError> { + // get the first subsource to determine the connection type + if subsources.is_empty() { + return Ok(vec![]); + } + let (_, purified_export) = subsources.iter().next().unwrap(); + + let statements = match &purified_export.details { + PurifiedExportDetails::Postgres { .. } => { + crate::pure::postgres::generate_create_subsource_statements( + scx, + source_name, + subsources, + )? + } + PurifiedExportDetails::MySql { .. } => { + crate::pure::mysql::generate_create_subsource_statements(scx, source_name, subsources)? + } + PurifiedExportDetails::LoadGenerator { .. } => { + let mut subsource_stmts = Vec::with_capacity(subsources.len()); + for (subsource_name, purified_export) in subsources { + let desc = match &purified_export.details { + PurifiedExportDetails::LoadGenerator { table } => table, + _ => unreachable!("purified export details must be load generator"), + }; + + let (columns, table_constraints) = scx.relation_desc_into_table_defs(desc)?; + + // Create the subsource statement + let subsource = CreateSubsourceStatement { + name: subsource_name, + columns, + of_source: Some(source_name.clone()), + // unlike sources that come from an external upstream, we + // have more leniency to introduce different constraints + // every time the load generator is run; i.e. we are not as + // worried about introducing junk data. + constraints: table_constraints, + if_not_exists: false, + with_options: vec![CreateSubsourceOption { + name: CreateSubsourceOptionName::ExternalReference, + value: Some(WithOptionValue::UnresolvedItemName( + purified_export.external_reference, + )), + }], + }; + subsource_stmts.push(subsource); + } + + subsource_stmts + } + PurifiedExportDetails::Kafka { .. } => { + // TODO: as part of #20208, Kafka sources will begin + // producing data––we'll need to understand the schema + // of the output here. + assert!( + subsources.is_empty(), + "Kafka sources do not produce data-bearing subsources" + ); + vec![] + } + }; + Ok(statements) +} + async fn purify_csr_connection_proto( catalog: &dyn SessionCatalog, connection: &mut CreateSourceConnection, diff --git a/src/sql/src/pure/mysql.rs b/src/sql/src/pure/mysql.rs index 3e824a2e3b5a5..cb5f0918765d2 100644 --- a/src/sql/src/pure/mysql.rs +++ b/src/sql/src/pure/mysql.rs @@ -21,19 +21,18 @@ use mz_sql_parser::ast::{ }; use mz_storage_types::sources::SubsourceResolver; -use crate::catalog::SessionCatalog; use crate::names::Aug; use crate::plan::{PlanError, StatementContext}; use crate::pure::{MySqlSourcePurificationError, ResolvedItemName}; -use super::RequestedSubsource; +use super::{PurifiedExportDetails, PurifiedSourceExport, RequestedSubsource}; /// The name of the fake database that we use for MySQL sources /// to fit our model of a 3-layer catalog. MySQL doesn't have a concept /// of databases AND schemas, it treats both as the same thing. pub(crate) static MYSQL_DATABASE_FAKE_NAME: &str = "mysql"; -pub(super) fn mysql_upstream_name( +pub(super) fn mysql_table_to_external_reference( table: &MySqlTableDesc, ) -> Result { Ok(UnresolvedItemName::qualified(&[ @@ -42,8 +41,8 @@ pub(super) fn mysql_upstream_name( ])) } -/// Reverses the `mysql_upstream_name` function. -pub(super) fn upstream_name_to_table( +/// Reverses the `mysql_table_external_reference` function. +pub(super) fn external_reference_to_table( name: &UnresolvedItemName, ) -> Result { if name.0.len() != 2 { @@ -55,20 +54,19 @@ pub(super) fn upstream_name_to_table( }) } -pub(super) fn generate_targeted_subsources( +pub fn generate_create_subsource_statements( scx: &StatementContext, - source_name: Option, - validated_requested_subsources: Vec>, + source_name: ResolvedItemName, + requested_subsources: BTreeMap, ) -> Result>, PlanError> { - let mut subsources = vec![]; - - // Now that we have an explicit list of validated requested subsources we can create them - for RequestedSubsource { - upstream_name, - subsource_name, - table, - } in validated_requested_subsources.into_iter() - { + let mut subsources = Vec::with_capacity(requested_subsources.len()); + + for (subsource_name, purified_export) in requested_subsources { + let table = match &purified_export.details { + PurifiedExportDetails::MySql { table } => table, + _ => unreachable!("purified export details must be mysql"), + }; + // Figure out the schema of the subsource let mut columns = vec![]; for c in table.columns.iter() { @@ -123,14 +121,14 @@ pub(super) fn generate_targeted_subsources( let subsource = CreateSubsourceStatement { name: subsource_name, columns, - // We might not know the primary source's `GlobalId` yet; if not, - // we'll fill it in once we generate it. - of_source: source_name.clone(), + of_source: Some(source_name.clone()), constraints, if_not_exists: false, with_options: vec![CreateSubsourceOption { name: CreateSubsourceOptionName::ExternalReference, - value: Some(WithOptionValue::UnresolvedItemName(upstream_name)), + value: Some(WithOptionValue::UnresolvedItemName( + purified_export.external_reference, + )), }], }; subsources.push(subsource); @@ -213,7 +211,11 @@ pub(super) async fn validate_requested_subsources_privileges( // snapshotting, we break the entire source. let tables_to_check_permissions = requested_subsources .iter() - .map(|RequestedSubsource { upstream_name, .. }| upstream_name_to_table(upstream_name)) + .map( + |RequestedSubsource { + external_reference, .. + }| external_reference_to_table(external_reference), + ) .collect::, _>>()?; validate_source_privileges(&mut *conn, &tables_to_check_permissions) @@ -235,7 +237,7 @@ pub(super) async fn validate_requested_subsources_privileges( } pub(super) struct PurifiedSubsources { - pub(super) new_subsources: Vec>, + pub(super) subsources: BTreeMap, pub(super) tables: Vec, pub(super) normalized_text_columns: Vec>, pub(super) normalized_ignore_columns: Vec>, @@ -249,9 +251,7 @@ pub(super) async fn purify_subsources( referenced_subsources: &mut Option, text_columns: Vec, ignore_columns: Vec, - resolved_source_name: Option, unresolved_source_name: &UnresolvedItemName, - catalog: &impl SessionCatalog, ) -> Result { // Determine which table schemas to request from mysql. Note that in mysql // a 'schema' is the same as a 'database', and a fully qualified table @@ -327,11 +327,11 @@ pub(super) async fn purify_subsources( { ReferencedSubsources::All => { for table in &tables { - let upstream_name = mysql_upstream_name(table)?; + let external_reference = mysql_table_to_external_reference(table)?; let subsource_name = super::subsource_name_gen(unresolved_source_name, &table.name)?; validated_requested_subsources.push(RequestedSubsource { - upstream_name, + external_reference, subsource_name, table, }); @@ -356,11 +356,11 @@ pub(super) async fn purify_subsources( continue; } - let upstream_name = mysql_upstream_name(table)?; + let external_reference = mysql_table_to_external_reference(table)?; let subsource_name = super::subsource_name_gen(unresolved_source_name, &table.name)?; validated_requested_subsources.push(RequestedSubsource { - upstream_name, + external_reference, subsource_name, table, }); @@ -390,12 +390,23 @@ pub(super) async fn purify_subsources( validate_requested_subsources_privileges(&validated_requested_subsources, conn).await?; - let scx = StatementContext::new(None, catalog); - let new_subsources = - generate_targeted_subsources(&scx, resolved_source_name, validated_requested_subsources)?; + let requested_subsources = validated_requested_subsources + .into_iter() + .map(|r| { + ( + r.subsource_name, + PurifiedSourceExport { + external_reference: r.external_reference, + details: PurifiedExportDetails::MySql { + table: r.table.clone(), + }, + }, + ) + }) + .collect(); Ok(PurifiedSubsources { - new_subsources, + subsources: requested_subsources, // Normalize column options and remove unused column references. normalized_text_columns: normalize_column_refs(text_columns, &subsource_resolver, &tables)?, normalized_ignore_columns: normalize_column_refs( diff --git a/src/sql/src/pure/postgres.rs b/src/sql/src/pure/postgres.rs index 21808fd33a2ec..1f1783f657ee9 100644 --- a/src/sql/src/pure/postgres.rs +++ b/src/sql/src/pure/postgres.rs @@ -24,13 +24,12 @@ use mz_storage_types::sources::SubsourceResolver; use tokio_postgres::types::Oid; use tokio_postgres::Client; -use crate::catalog::SessionCatalog; -use crate::names::{Aug, PartialItemName, ResolvedItemName}; +use crate::names::{Aug, ResolvedItemName}; use crate::normalize; use crate::plan::{PlanError, StatementContext}; use super::error::PgSourcePurificationError; -use super::RequestedSubsource; +use super::{PartialItemName, PurifiedExportDetails, PurifiedSourceExport, RequestedSubsource}; /// Ensure that we have select permissions on all tables; we have to do this before we /// start snapshotting because if we discover we cannot `COPY` from a table while @@ -127,47 +126,37 @@ pub(super) fn generate_text_columns( Ok(text_cols_dict) } -pub(crate) fn generate_targeted_subsources( +pub fn generate_create_subsource_statements( scx: &StatementContext, - source_name: Option, - validated_requested_subsources: Vec>, - mut text_cols_dict: BTreeMap>, - publication_tables: &[PostgresTableDesc], -) -> Result< - ( - Vec>, - // These are the tables referenced by the subsources. We want this set - // of tables separately so we can retain only table definitions that are - // referenced by subsources. This helps avoid issues when generating PG - // source table casts. - Vec, - ), - PlanError, -> { - let mut subsources = vec![]; - let mut referenced_tables = vec![]; - + source_name: ResolvedItemName, + requested_subsources: BTreeMap, +) -> Result>, PlanError> { // Aggregate all unrecognized types. let mut unsupported_cols = vec![]; // Now that we have an explicit list of validated requested subsources we can create them - for RequestedSubsource { - upstream_name, - subsource_name, - table, - } in validated_requested_subsources.into_iter() - { + let mut subsources = Vec::with_capacity(requested_subsources.len()); + + for (subsource_name, purified_export) in requested_subsources { + let (text_columns, table) = match &purified_export.details { + PurifiedExportDetails::Postgres { + text_columns, + table, + } => (text_columns, table), + _ => unreachable!("purified export details must be postgres"), + }; + // Figure out the schema of the subsource let mut columns = vec![]; - let text_cols_dict = text_cols_dict.remove(&table.oid); for c in table.columns.iter() { let name = Ident::new(c.name.clone())?; - let ty = match &text_cols_dict { + + let ty = match text_columns { Some(names) if names.contains(&c.name) => mz_pgrepr::Type::Text, _ => match mz_pgrepr::Type::from_oid_and_typmod(c.type_oid, c.type_mod) { Ok(t) => t, Err(_) => { - let mut full_name = upstream_name.0.clone(); + let mut full_name = purified_export.external_reference.0.clone(); full_name.push(name); unsupported_cols.push(( UnresolvedItemName(full_name).to_ast_string(), @@ -234,7 +223,7 @@ pub(crate) fn generate_targeted_subsources( columns, // We might not know the primary source's `GlobalId` yet; if not, // we'll fill it in once we generate it. - of_source: source_name.clone(), + of_source: Some(source_name.clone()), // TODO(petrosagg): nothing stops us from getting the constraints of the // upstream tables and mirroring them here which will lead to more optimization // opportunities if for example there is a primary key or an index. @@ -247,11 +236,12 @@ pub(crate) fn generate_targeted_subsources( if_not_exists: false, with_options: vec![CreateSubsourceOption { name: CreateSubsourceOptionName::ExternalReference, - value: Some(WithOptionValue::UnresolvedItemName(upstream_name)), + value: Some(WithOptionValue::UnresolvedItemName( + purified_export.external_reference, + )), }], }; subsources.push(subsource); - referenced_tables.push(table.clone()) } if !unsupported_cols.is_empty() { @@ -261,35 +251,11 @@ pub(crate) fn generate_targeted_subsources( })?; } - // If any any item was not removed from the text_cols dict, it wasn't being - // added. - let mut dangling_text_column_refs = vec![]; - - for id in text_cols_dict.keys() { - let desc = publication_tables - .iter() - .find(|t| t.oid == *id) - .expect("validated when generating text columns"); - - dangling_text_column_refs.push(PartialItemName { - database: None, - schema: Some(desc.namespace.clone()), - item: desc.name.clone(), - }); - } - - if !dangling_text_column_refs.is_empty() { - dangling_text_column_refs.sort(); - Err(PgSourcePurificationError::DanglingTextColumns { - items: dangling_text_column_refs, - })?; - } - - Ok((subsources, referenced_tables)) + Ok(subsources) } pub(super) struct PurifiedSubsources { - pub(super) new_subsources: Vec>, + pub(super) subsources: BTreeMap, pub(super) referenced_tables: Vec, pub(super) normalized_text_columns: Vec>, } @@ -304,11 +270,9 @@ pub(super) async fn purify_subsources( connection: &PostgresConnection, referenced_subsources: &mut Option, mut text_columns: Vec, - resolved_source_name: Option, unresolved_source_name: &UnresolvedItemName, - catalog: &impl SessionCatalog, ) -> Result { - let publication_tables = mz_postgres_util::publication_info(client, publication).await?; + let mut publication_tables = mz_postgres_util::publication_info(client, publication).await?; if publication_tables.is_empty() { Err(PgSourcePurificationError::EmptyPublication( @@ -325,7 +289,7 @@ pub(super) async fn purify_subsources( { ReferencedSubsources::All => { for table in &publication_tables { - let upstream_name = UnresolvedItemName::qualified(&[ + let external_reference = UnresolvedItemName::qualified(&[ Ident::new(&connection.database)?, Ident::new(&table.namespace)?, Ident::new(&table.name)?, @@ -333,7 +297,7 @@ pub(super) async fn purify_subsources( let subsource_name = super::subsource_name_gen(unresolved_source_name, &table.name)?; validated_requested_subsources.push(RequestedSubsource { - upstream_name, + external_reference, subsource_name, table, }); @@ -366,7 +330,7 @@ pub(super) async fn purify_subsources( continue; } - let upstream_name = UnresolvedItemName::qualified(&[ + let external_reference = UnresolvedItemName::qualified(&[ Ident::new(&connection.database)?, Ident::new(&table.namespace)?, Ident::new(&table.name)?, @@ -374,7 +338,7 @@ pub(super) async fn purify_subsources( let subsource_name = super::subsource_name_gen(unresolved_source_name, &table.name)?; validated_requested_subsources.push(RequestedSubsource { - upstream_name, + external_reference, subsource_name, table, }); @@ -409,7 +373,7 @@ pub(super) async fn purify_subsources( validate_requested_subsources_privileges(config, client, &table_oids).await?; - let text_cols_dict = + let mut text_column_map = generate_text_columns(&subsource_resolver, &publication_tables, &mut text_columns)?; // Normalize options to contain full qualified values. @@ -420,18 +384,52 @@ pub(super) async fn purify_subsources( .map(WithOptionValue::UnresolvedItemName) .collect(); - let scx = StatementContext::new(None, catalog); - let (new_subsources, referenced_tables) = generate_targeted_subsources( - &scx, - resolved_source_name, - validated_requested_subsources, - text_cols_dict, - &publication_tables, - )?; + let requested_subsources = validated_requested_subsources + .into_iter() + .map(|r| { + ( + r.subsource_name, + PurifiedSourceExport { + external_reference: r.external_reference, + details: PurifiedExportDetails::Postgres { + table: r.table.clone(), + text_columns: text_column_map.remove(&r.table.oid), + }, + }, + ) + }) + .collect(); + + // If any any item was not removed from the text_column_map, it wasn't being + // added. + let mut dangling_text_column_refs = vec![]; + + for id in text_column_map.keys() { + let desc = publication_tables + .iter() + .find(|t| t.oid == *id) + .expect("validated when generating text columns"); + + dangling_text_column_refs.push(PartialItemName { + database: None, + schema: Some(desc.namespace.clone()), + item: desc.name.clone(), + }); + } + + if !dangling_text_column_refs.is_empty() { + dangling_text_column_refs.sort(); + Err(PgSourcePurificationError::DanglingTextColumns { + items: dangling_text_column_refs, + })?; + } + + // Trim any un-referred-to tables + publication_tables.retain(|t| table_oids.contains(&t.oid)); Ok(PurifiedSubsources { - new_subsources, - referenced_tables, + subsources: requested_subsources, + referenced_tables: publication_tables, normalized_text_columns, }) } diff --git a/src/storage-controller/src/lib.rs b/src/storage-controller/src/lib.rs index adb47b8923041..b5d960f38c40a 100644 --- a/src/storage-controller/src/lib.rs +++ b/src/storage-controller/src/lib.rs @@ -2235,7 +2235,7 @@ where .await .map_err(|_| StorageError::ReadBeforeSince(remap_id))?; - tracing::debug!(?id, type_ = source_conn.name(), upstream = ?source_conn.upstream_name(), "fetching real time recency"); + tracing::debug!(?id, type_ = source_conn.name(), upstream = ?source_conn.external_reference(), "fetching real time recency"); let result = rtr::real_time_recency_ts(source_conn, id, config, as_of, remap_subscribe) .await.map_err(|e| { diff --git a/src/storage-types/src/sources.rs b/src/storage-types/src/sources.rs index c7271d6b3ea17..e24bf2406c358 100644 --- a/src/storage-types/src/sources.rs +++ b/src/storage-types/src/sources.rs @@ -713,7 +713,7 @@ pub trait SourceConnection: Debug + Clone + PartialEq + AlterCompatible { fn name(&self) -> &'static str; /// The name of the resource in the external system (e.g kafka topic) if any - fn upstream_name(&self) -> Option<&str>; + fn external_reference(&self) -> Option<&str>; /// The schema of this connection's key rows. // This is mostly setting the stage for the subsequent PRs that will attempt to compute and @@ -986,12 +986,12 @@ impl SourceConnection for GenericSourceConnection { } } - fn upstream_name(&self) -> Option<&str> { + fn external_reference(&self) -> Option<&str> { match self { - Self::Kafka(conn) => conn.upstream_name(), - Self::Postgres(conn) => conn.upstream_name(), - Self::MySql(conn) => conn.upstream_name(), - Self::LoadGenerator(conn) => conn.upstream_name(), + Self::Kafka(conn) => conn.external_reference(), + Self::Postgres(conn) => conn.external_reference(), + Self::MySql(conn) => conn.external_reference(), + Self::LoadGenerator(conn) => conn.external_reference(), } } diff --git a/src/storage-types/src/sources/kafka.rs b/src/storage-types/src/sources/kafka.rs index 054f919a35edc..9b5c5c72a09c3 100644 --- a/src/storage-types/src/sources/kafka.rs +++ b/src/storage-types/src/sources/kafka.rs @@ -189,7 +189,7 @@ impl SourceConnection for KafkaSourceConnection { "kafka" } - fn upstream_name(&self) -> Option<&str> { + fn external_reference(&self) -> Option<&str> { Some(self.topic.as_str()) } diff --git a/src/storage-types/src/sources/load_generator.rs b/src/storage-types/src/sources/load_generator.rs index f5b61073a34fa..1470ef8192351 100644 --- a/src/storage-types/src/sources/load_generator.rs +++ b/src/storage-types/src/sources/load_generator.rs @@ -54,7 +54,7 @@ impl SourceConnection for LoadGeneratorSourceConnection { "load-generator" } - fn upstream_name(&self) -> Option<&str> { + fn external_reference(&self) -> Option<&str> { None } diff --git a/src/storage-types/src/sources/mysql.rs b/src/storage-types/src/sources/mysql.rs index 96df6d0fefe94..f98ca683339d6 100644 --- a/src/storage-types/src/sources/mysql.rs +++ b/src/storage-types/src/sources/mysql.rs @@ -170,7 +170,7 @@ impl SourceConnection for MySqlSourceConnection { "mysql" } - fn upstream_name(&self) -> Option<&str> { + fn external_reference(&self) -> Option<&str> { None } diff --git a/src/storage-types/src/sources/postgres.rs b/src/storage-types/src/sources/postgres.rs index 8f939ea7de5f8..75231a3c14abb 100644 --- a/src/storage-types/src/sources/postgres.rs +++ b/src/storage-types/src/sources/postgres.rs @@ -107,7 +107,7 @@ impl SourceConnection for PostgresSourceConnection { "postgres" } - fn upstream_name(&self) -> Option<&str> { + fn external_reference(&self) -> Option<&str> { None } From 4f8ca419100b0e25da89b20b6fbccac94d02d4ad Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Thu, 18 Jul 2024 11:46:16 -0400 Subject: [PATCH 2/3] Rename all overloaded 'subsources' names involved in purification --- src/sql-parser/src/ast/defs/statement.rs | 36 ++--- src/sql-parser/src/parser.rs | 16 +-- src/sql/src/normalize.rs | 2 +- src/sql/src/plan/error.rs | 8 +- src/sql/src/plan/statement/ddl.rs | 10 +- src/sql/src/plan/statement/show.rs | 10 +- src/sql/src/pure.rs | 128 +++++++++--------- src/sql/src/pure/error.rs | 12 +- src/sql/src/pure/mysql.rs | 101 +++++++------- src/sql/src/pure/postgres.rs | 86 ++++++------ src/storage-controller/src/lib.rs | 4 +- src/storage-types/src/sources.rs | 89 ++++++------ src/storage-types/src/sources/kafka.rs | 4 +- .../src/sources/load_generator.rs | 6 +- src/storage-types/src/sources/mysql.rs | 6 +- src/storage-types/src/sources/postgres.rs | 6 +- 16 files changed, 265 insertions(+), 259 deletions(-) diff --git a/src/sql-parser/src/ast/defs/statement.rs b/src/sql-parser/src/ast/defs/statement.rs index f0784037c0be3..fbbe3b76d5c3a 100644 --- a/src/sql-parser/src/ast/defs/statement.rs +++ b/src/sql-parser/src/ast/defs/statement.rs @@ -979,7 +979,7 @@ pub struct CreateSourceStatement { pub if_not_exists: bool, pub key_constraint: Option, pub with_options: Vec>, - pub referenced_subsources: Option, + pub external_references: Option, pub progress_subsource: Option>, } @@ -1022,7 +1022,7 @@ impl AstDisplay for CreateSourceStatement { f.write_node(envelope); } - if let Some(subsources) = &self.referenced_subsources { + if let Some(subsources) = &self.external_references { f.write_str(" "); f.write_node(subsources); } @@ -1041,40 +1041,42 @@ impl AstDisplay for CreateSourceStatement { } impl_display_t!(CreateSourceStatement); -/// A selected subsource in a FOR TABLES (..) statement +/// A selected external reference in a FOR TABLES (..) statement #[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Ord)] -pub struct CreateSourceSubsource { +pub struct ExternalReferenceExport { pub reference: UnresolvedItemName, - pub subsource: Option, + pub alias: Option, } -impl AstDisplay for CreateSourceSubsource { +impl AstDisplay for ExternalReferenceExport { fn fmt(&self, f: &mut AstFormatter) { f.write_node(&self.reference); - if let Some(subsource) = &self.subsource { + if let Some(alias) = &self.alias { f.write_str(" AS "); - f.write_node(subsource); + f.write_node(alias); } } } -impl_display!(CreateSourceSubsource); +impl_display!(ExternalReferenceExport); +/// Specifies which set of external references to generate a source export +/// for in a `CREATE SOURCE` statement. #[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub enum ReferencedSubsources { +pub enum ExternalReferences { /// A subset defined with FOR TABLES (...) - SubsetTables(Vec), + SubsetTables(Vec), /// A subset defined with FOR SCHEMAS (...) SubsetSchemas(Vec), /// FOR ALL TABLES All, } -impl AstDisplay for ReferencedSubsources { +impl AstDisplay for ExternalReferences { fn fmt(&self, f: &mut AstFormatter) { match self { - Self::SubsetTables(subsources) => { + Self::SubsetTables(tables) => { f.write_str("FOR TABLES ("); - f.write_node(&display::comma_separated(subsources)); + f.write_node(&display::comma_separated(tables)); f.write_str(")"); } Self::SubsetSchemas(schemas) => { @@ -1086,7 +1088,7 @@ impl AstDisplay for ReferencedSubsources { } } } -impl_display!(ReferencedSubsources); +impl_display!(ExternalReferences); /// An option in a `CREATE SUBSOURCE` statement. #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] @@ -2462,7 +2464,7 @@ pub enum AlterSourceAction { SetOptions(Vec>), ResetOptions(Vec), AddSubsources { - subsources: Vec, + external_references: Vec, options: Vec>, }, DropSubsources { @@ -2505,7 +2507,7 @@ impl AstDisplay for AlterSourceAction { } } AlterSourceAction::AddSubsources { - subsources, + external_references: subsources, options, } => { f.write_str("ADD SUBSOURCE "); diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index 647f12d1e3269..9dda429d0d25e 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -2764,14 +2764,14 @@ impl<'a> Parser<'a> { self.expect_token(&Token::LParen)?; let subsources = self.parse_comma_separated(Parser::parse_subsource_references)?; self.expect_token(&Token::RParen)?; - Some(ReferencedSubsources::SubsetTables(subsources)) + Some(ExternalReferences::SubsetTables(subsources)) } else if self.parse_keywords(&[FOR, SCHEMAS]) { self.expect_token(&Token::LParen)?; let schemas = self.parse_comma_separated(Parser::parse_identifier)?; self.expect_token(&Token::RParen)?; - Some(ReferencedSubsources::SubsetSchemas(schemas)) + Some(ExternalReferences::SubsetSchemas(schemas)) } else if self.parse_keywords(&[FOR, ALL, TABLES]) { - Some(ReferencedSubsources::All) + Some(ExternalReferences::All) } else { None }; @@ -2802,13 +2802,13 @@ impl<'a> Parser<'a> { envelope, if_not_exists, key_constraint, - referenced_subsources, + external_references: referenced_subsources, progress_subsource, with_options, })) } - fn parse_subsource_references(&mut self) -> Result { + fn parse_subsource_references(&mut self) -> Result { let reference = self.parse_item_name()?; let subsource = if self.parse_one_of_keywords(&[AS, INTO]).is_some() { Some(self.parse_item_name()?) @@ -2816,9 +2816,9 @@ impl<'a> Parser<'a> { None }; - Ok(CreateSourceSubsource { + Ok(ExternalReferenceExport { reference, - subsource, + alias: subsource, }) } @@ -4843,7 +4843,7 @@ impl<'a> Parser<'a> { source_name, if_exists, action: AlterSourceAction::AddSubsources { - subsources, + external_references: subsources, options, }, }) diff --git a/src/sql/src/normalize.rs b/src/sql/src/normalize.rs index 1e998521b8457..d12cf6a6772f6 100644 --- a/src/sql/src/normalize.rs +++ b/src/sql/src/normalize.rs @@ -270,7 +270,7 @@ pub fn create_statement( if_not_exists, key_constraint: _, with_options: _, - referenced_subsources: _, + external_references: _, progress_subsource: _, }) => { *name = allocate_name(name)?; diff --git a/src/sql/src/plan/error.rs b/src/sql/src/plan/error.rs index a2ff798c868b5..1f47bcd599dd8 100644 --- a/src/sql/src/plan/error.rs +++ b/src/sql/src/plan/error.rs @@ -30,7 +30,7 @@ use mz_repr::{strconv, ColumnName, GlobalId}; use mz_sql_parser::ast::display::AstDisplay; use mz_sql_parser::ast::{IdentError, UnresolvedItemName}; use mz_sql_parser::parser::{ParserError, ParserStatementError}; -use mz_storage_types::sources::SubsourceResolutionError; +use mz_storage_types::sources::ExternalReferenceResolutionError; use crate::catalog::{ CatalogError, CatalogItemType, ErrorMessageObjectDescription, SystemObjectType, @@ -262,7 +262,7 @@ pub enum PlanError { limit: Duration, }, RetainHistoryRequired, - SubsourceResolutionError(SubsourceResolutionError), + SubsourceResolutionError(ExternalReferenceResolutionError), Replan(String), // TODO(benesch): eventually all errors should be structured. Unstructured(String), @@ -884,8 +884,8 @@ impl From for PlanError { } } -impl From for PlanError { - fn from(e: SubsourceResolutionError) -> Self { +impl From for PlanError { + fn from(e: ExternalReferenceResolutionError) -> Self { PlanError::SubsourceResolutionError(e) } } diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index 1f3cf50decbd4..748b3e8445547 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -102,7 +102,7 @@ use mz_storage_types::sources::postgres::{ ProtoPostgresSourcePublicationDetails, }; use mz_storage_types::sources::{ - GenericSourceConnection, SourceConnection, SourceDesc, SubsourceResolver, Timeline, + GenericSourceConnection, SourceConnection, SourceDesc, SourceReferenceResolver, Timeline, }; use prost::Message; @@ -634,7 +634,7 @@ pub fn plan_create_source( key_constraint, include_metadata, with_options, - referenced_subsources, + external_references: referenced_subsources, progress_subsource, } = &stmt; @@ -825,8 +825,8 @@ pub fn plan_create_source( let publication_details = PostgresSourcePublicationDetails::from_proto(details) .map_err(|e| sql_err!("{}", e))?; - let subsource_resolver = - SubsourceResolver::new(&connection.database, &publication_details.tables) + let reference_resolver = + SourceReferenceResolver::new(&connection.database, &publication_details.tables) .expect("references validated during purification"); let mut text_cols: BTreeMap> = BTreeMap::new(); @@ -835,7 +835,7 @@ pub fn plan_create_source( for name in text_columns { let (col, qual) = name.0.split_last().expect("must have at least one element"); - let idx = subsource_resolver + let idx = reference_resolver .resolve_idx(qual) .expect("known to exist from purification"); diff --git a/src/sql/src/plan/statement/show.rs b/src/sql/src/plan/statement/show.rs index 6bef57c70eba3..54b5a20d2b9b6 100644 --- a/src/sql/src/plan/statement/show.rs +++ b/src/sql/src/plan/statement/show.rs @@ -21,7 +21,7 @@ use mz_ore::collections::CollectionExt; use mz_repr::{Datum, GlobalId, RelationDesc, Row, ScalarType}; use mz_sql_parser::ast::display::AstDisplay; use mz_sql_parser::ast::{ - CreateSourceSubsource, ObjectType, ReferencedSubsources, ShowCreateClusterStatement, + ExternalReferenceExport, ExternalReferences, ObjectType, ShowCreateClusterStatement, ShowCreateConnectionStatement, ShowCreateMaterializedViewStatement, ShowObjectType, SystemObjectType, UnresolvedItemName, WithOptionValue, }; @@ -1041,7 +1041,7 @@ fn humanize_sql_for_show_create( // `FOR ALL TABLES`. However, this would change if #26765 // landed. curr_references.clear(); - stmt.referenced_subsources = Some(ReferencedSubsources::All); + stmt.external_references = Some(ExternalReferences::All); } CreateSourceConnection::Kafka { .. } | CreateSourceConnection::LoadGenerator { .. } => {} @@ -1051,13 +1051,13 @@ fn humanize_sql_for_show_create( if !curr_references.is_empty() { let mut subsources: Vec<_> = curr_references .into_iter() - .map(|(reference, name)| CreateSourceSubsource { + .map(|(reference, name)| ExternalReferenceExport { reference, - subsource: Some(name), + alias: Some(name), }) .collect(); subsources.sort(); - stmt.referenced_subsources = Some(ReferencedSubsources::SubsetTables(subsources)); + stmt.external_references = Some(ExternalReferences::SubsetTables(subsources)); } } _ => (), diff --git a/src/sql/src/pure.rs b/src/sql/src/pure.rs index 2904e72516bcc..4c0071d8c5be3 100644 --- a/src/sql/src/pure.rs +++ b/src/sql/src/pure.rs @@ -53,7 +53,7 @@ use mz_storage_types::errors::ContextCreationError; use mz_storage_types::sources::mysql::MySqlSourceDetails; use mz_storage_types::sources::postgres::PostgresSourcePublicationDetails; use mz_storage_types::sources::{ - GenericSourceConnection, SourceConnection, SubsourceCatalogReference, SubsourceResolver, + ExternalCatalogReference, GenericSourceConnection, SourceConnection, SourceReferenceResolver, }; use prost::Message; use protobuf_native::compiler::{SourceTreeDescriptorDatabase, VirtualSourceTree}; @@ -62,9 +62,9 @@ use rdkafka::admin::AdminClient; use uuid::Uuid; use crate::ast::{ - AvroSchema, CreateSourceConnection, CreateSourceStatement, CreateSourceSubsource, - CreateSubsourceStatement, CsrConnectionAvro, CsrConnectionProtobuf, Format, FormatSpecifier, - ProtobufSchema, ReferencedSubsources, Value, WithOptionValue, + AvroSchema, CreateSourceConnection, CreateSourceStatement, CreateSubsourceStatement, + CsrConnectionAvro, CsrConnectionProtobuf, ExternalReferenceExport, ExternalReferences, Format, + FormatSpecifier, ProtobufSchema, Value, WithOptionValue, }; use crate::catalog::{CatalogItemType, SessionCatalog}; use crate::kafka_util::{KafkaSinkConfigOptionExtracted, KafkaSourceConfigOptionExtracted}; @@ -86,30 +86,31 @@ pub(crate) mod error; pub mod mysql; pub mod postgres; -pub(crate) struct RequestedSubsource<'a, T> { +pub(crate) struct RequestedSourceExport<'a, T> { external_reference: UnresolvedItemName, - subsource_name: UnresolvedItemName, + name: UnresolvedItemName, table: &'a T, } -fn subsource_gen<'a, T: SubsourceCatalogReference>( - selected_subsources: &mut Vec, - resolver: &SubsourceResolver, +/// Generates appropriate source exports for a set of external references to export from a source. +fn source_export_gen<'a, T: ExternalCatalogReference>( + selected_references: &mut Vec, + resolver: &SourceReferenceResolver, references: &'a [T], canonical_width: usize, source_name: &UnresolvedItemName, -) -> Result>, PlanError> { - let mut validated_requested_subsources = vec![]; +) -> Result>, PlanError> { + let mut validated_exports = vec![]; - for subsource in selected_subsources { - let subsource_name = match &subsource.subsource { + for reference in selected_references { + let name = match &reference.alias { Some(name) => { let partial = normalize::unresolved_item_name(name.clone())?; match partial.schema { Some(_) => name.clone(), // In cases when a prefix is not provided for the deferred name // fallback to using the schema of the source with the given name - None => subsource_name_gen(source_name, &partial.item)?, + None => source_export_name_gen(source_name, &partial.item)?, } } None => { @@ -117,33 +118,33 @@ fn subsource_gen<'a, T: SubsourceCatalogReference>( // the item as the subsource name to ensure it's created in the // current schema or the source's schema if provided, not mirroring // the schema of the reference. - subsource_name_gen( + source_export_name_gen( source_name, - &normalize::unresolved_item_name(subsource.reference.clone())?.item, + &normalize::unresolved_item_name(reference.reference.clone())?.item, )? } }; let (external_reference, idx) = - resolver.resolve(&subsource.reference.0, canonical_width)?; + resolver.resolve(&reference.reference.0, canonical_width)?; let table = &references[idx]; - validated_requested_subsources.push(RequestedSubsource { + validated_exports.push(RequestedSourceExport { external_reference, - subsource_name, + name, table, }); } - Ok(validated_requested_subsources) + Ok(validated_exports) } /// Generates a subsource name by prepending source schema name if present /// /// For eg. if source is `a.b`, then `a` will be prepended to the subsource name /// so that it's generated in the same schema as source -fn subsource_name_gen( +fn source_export_name_gen( source_name: &UnresolvedItemName, subsource_name: &String, ) -> Result { @@ -154,23 +155,23 @@ fn subsource_name_gen( /// Validates the requested subsources do not have name conflicts with each other /// and that the same upstream table is not referenced multiple times. -fn validate_subsource_names( - requested_subsources: &[RequestedSubsource], +fn validate_source_export_names( + requested_source_exports: &[RequestedSourceExport], ) -> Result<(), PlanError> { // This condition would get caught during the catalog transaction, but produces a // vague, non-contextual error. Instead, error here so we can suggest to the user // how to fix the problem. - if let Some(name) = requested_subsources + if let Some(name) = requested_source_exports .iter() - .map(|subsource| &subsource.subsource_name) + .map(|subsource| &subsource.name) .duplicates() .next() .cloned() { - let mut upstream_references: Vec<_> = requested_subsources + let mut upstream_references: Vec<_> = requested_source_exports .into_iter() .filter_map(|subsource| { - if &subsource.subsource_name == &name { + if &subsource.name == &name { Some(subsource.external_reference.clone()) } else { None @@ -186,20 +187,23 @@ fn validate_subsource_names( })?; } + // TODO: Remove this when supporting source-fed tables which can ingest the same external + // reference. + // We technically could allow multiple subsources to ingest the same upstream table, but // it is almost certainly an error on the user's end. - if let Some(name) = requested_subsources + if let Some(name) = requested_source_exports .iter() - .map(|subsource| &subsource.external_reference) + .map(|export| &export.external_reference) .duplicates() .next() .cloned() { - let mut target_names: Vec<_> = requested_subsources + let mut target_names: Vec<_> = requested_source_exports .into_iter() - .filter_map(|subsource| { - if &subsource.external_reference == &name { - Some(subsource.subsource_name.clone()) + .filter_map(|export| { + if &export.external_reference == &name { + Some(export.name.clone()) } else { None } @@ -570,7 +574,7 @@ async fn purify_create_source( format, envelope, include_metadata, - referenced_subsources, + external_references, progress_subsource, .. } = &mut create_source_stmt; @@ -602,9 +606,9 @@ async fn purify_create_source( options: base_with_options, .. } => { - if let Some(referenced_subsources) = referenced_subsources { + if let Some(external_references) = external_references { Err(KafkaSourcePurificationError::ReferencedSubsources( - referenced_subsources.clone(), + external_references.clone(), ))?; } @@ -773,16 +777,16 @@ async fn purify_create_source( Err(PgSourcePurificationError::InsufficientReplicationSlotsAvailable { count: 2 })?; } - let postgres::PurifiedSubsources { - subsources, + let postgres::PurifiedSourceExports { + source_exports: subsources, referenced_tables, normalized_text_columns, - } = postgres::purify_subsources( + } = postgres::purify_source_exports( &client, &config, &publication, &connection, - referenced_subsources, + external_references, text_columns, source_name, ) @@ -899,14 +903,14 @@ async fn purify_create_source( let initial_gtid_set = mz_mysql_util::query_sys_var(&mut conn, "global.gtid_executed").await?; - let mysql::PurifiedSubsources { - subsources, + let mysql::PurifiedSourceExports { + source_exports: subsources, tables, normalized_text_columns, normalized_ignore_columns, - } = mysql::purify_subsources( + } = mysql::purify_source_exports( &mut conn, - referenced_subsources, + external_references, text_columns, ignore_columns, source_name, @@ -953,15 +957,15 @@ async fn purify_create_source( let (_load_generator, available_subsources) = load_generator_ast_to_generator(&scx, generator, options, include_metadata)?; - match referenced_subsources { - Some(ReferencedSubsources::All) => { + match external_references { + Some(ExternalReferences::All) => { let available_subsources = match &available_subsources { Some(available_subsources) => available_subsources, None => Err(LoadGeneratorSourcePurificationError::ForAllTables)?, }; for (name, (_, desc)) in available_subsources { let external_reference = UnresolvedItemName::from(name.clone()); - let subsource_name = subsource_name_gen(source_name, &name.item)?; + let subsource_name = source_export_name_gen(source_name, &name.item)?; requested_subsource_map.insert( subsource_name, PurifiedSourceExport { @@ -973,10 +977,10 @@ async fn purify_create_source( ); } } - Some(ReferencedSubsources::SubsetSchemas(..)) => { + Some(ExternalReferences::SubsetSchemas(..)) => { Err(LoadGeneratorSourcePurificationError::ForSchemas)? } - Some(ReferencedSubsources::SubsetTables(_)) => { + Some(ExternalReferences::SubsetTables(_)) => { Err(LoadGeneratorSourcePurificationError::ForTables)? } None => { @@ -991,7 +995,7 @@ async fn purify_create_source( // Now that we know which subsources to create alongside this // statement, remove the references so it is not canonicalized as // part of the `CREATE SOURCE` statement in the catalog. - *referenced_subsources = None; + *external_references = None; // Generate progress subsource @@ -1123,7 +1127,7 @@ async fn purify_alter_source( // If we don't need to handle added subsources, early return. let AlterSourceAction::AddSubsources { - subsources: targeted_subsources, + external_references, mut options, } = action else { @@ -1182,19 +1186,18 @@ async fn purify_alter_source( ) } - let mut referenced_subsources = - Some(ReferencedSubsources::SubsetTables(targeted_subsources)); + let mut references = Some(ExternalReferences::SubsetTables(external_references)); - let postgres::PurifiedSubsources { - subsources, + let postgres::PurifiedSourceExports { + source_exports: subsources, referenced_tables, normalized_text_columns, - } = postgres::purify_subsources( + } = postgres::purify_source_exports( &client, &config, &pg_source_connection.publication, pg_connection, - &mut referenced_subsources, + &mut references, text_columns, &unresolved_source_name, ) @@ -1267,17 +1270,16 @@ async fn purify_alter_source( ) .await?; - let mut referenced_subsources = - Some(ReferencedSubsources::SubsetTables(targeted_subsources)); + let mut references = Some(ExternalReferences::SubsetTables(external_references)); - let mysql::PurifiedSubsources { - subsources, + let mysql::PurifiedSourceExports { + source_exports: subsources, tables, normalized_text_columns, normalized_ignore_columns, - } = mysql::purify_subsources( + } = mysql::purify_source_exports( &mut conn, - &mut referenced_subsources, + &mut references, text_columns, ignore_columns, &unresolved_source_name, diff --git a/src/sql/src/pure/error.rs b/src/sql/src/pure/error.rs index 82980172cc3f3..f8b0838f205ee 100644 --- a/src/sql/src/pure/error.rs +++ b/src/sql/src/pure/error.rs @@ -12,7 +12,7 @@ use std::sync::Arc; use mz_ccsr::ListError; use mz_repr::adt::system::Oid; use mz_sql_parser::ast::display::AstDisplay; -use mz_sql_parser::ast::{ReferencedSubsources, UnresolvedItemName}; +use mz_sql_parser::ast::{ExternalReferences, UnresolvedItemName}; use mz_storage_types::errors::{ContextCreationError, CsrConnectError}; use crate::names::{FullItemName, PartialItemName}; @@ -30,7 +30,7 @@ pub enum PgSourcePurificationError { schemas: Vec, }, #[error("missing TABLES specification")] - RequiresReferencedSubsources, + RequiresExternalReferences, #[error("insufficient privileges")] UserLacksUsageOnSchemas { user: String, schemas: Vec }, #[error("insufficient privileges")] @@ -105,7 +105,7 @@ impl PgSourcePurificationError { "If trying to use the output of SHOW CREATE SOURCE, remove the DETAILS option." .into(), ), - Self::RequiresReferencedSubsources => { + Self::RequiresExternalReferences => { Some("provide a FOR TABLES (..), FOR SCHEMAS (..), or FOR ALL TABLES clause".into()) } Self::UnrecognizedTypes { @@ -128,7 +128,7 @@ impl PgSourcePurificationError { #[derive(Debug, Clone, thiserror::Error)] pub enum KafkaSourcePurificationError { #[error("{} is only valid for multi-output sources", .0.to_ast_string())] - ReferencedSubsources(ReferencedSubsources), + ReferencedSubsources(ExternalReferences), #[error("KAFKA CONNECTION without TOPIC")] ConnectionMissingTopic, #[error("{0} is not a KAFKA CONNECTION")] @@ -250,7 +250,7 @@ pub enum MySqlSourcePurificationError { #[error("No tables found")] EmptyDatabase, #[error("missing TABLES specification")] - RequiresReferencedSubsources, + RequiresExternalReferences, #[error("No tables found in referenced schemas")] NoTablesFoundForSchemas(Vec), } @@ -308,7 +308,7 @@ impl MySqlSourcePurificationError { Self::ReplicationSettingsError(_) => { Some("Set the necessary MySQL database system settings.".into()) } - Self::RequiresReferencedSubsources => { + Self::RequiresExternalReferences => { Some("provide a FOR TABLES (..), FOR SCHEMAS (..), or FOR ALL TABLES clause".into()) } Self::InvalidTableReference(_) => Some( diff --git a/src/sql/src/pure/mysql.rs b/src/sql/src/pure/mysql.rs index cb5f0918765d2..f2e2aae5847e0 100644 --- a/src/sql/src/pure/mysql.rs +++ b/src/sql/src/pure/mysql.rs @@ -16,16 +16,16 @@ use mz_mysql_util::{validate_source_privileges, MySqlError, MySqlTableDesc, Qual use mz_sql_parser::ast::display::AstDisplay; use mz_sql_parser::ast::UnresolvedItemName; use mz_sql_parser::ast::{ - ColumnDef, CreateSubsourceOption, CreateSubsourceOptionName, CreateSubsourceStatement, Ident, - IdentError, MySqlConfigOptionName, ReferencedSubsources, WithOptionValue, + ColumnDef, CreateSubsourceOption, CreateSubsourceOptionName, CreateSubsourceStatement, + ExternalReferences, Ident, IdentError, MySqlConfigOptionName, WithOptionValue, }; -use mz_storage_types::sources::SubsourceResolver; +use mz_storage_types::sources::SourceReferenceResolver; use crate::names::Aug; use crate::plan::{PlanError, StatementContext}; use crate::pure::{MySqlSourcePurificationError, ResolvedItemName}; -use super::{PurifiedExportDetails, PurifiedSourceExport, RequestedSubsource}; +use super::{PurifiedExportDetails, PurifiedSourceExport, RequestedSourceExport}; /// The name of the fake database that we use for MySQL sources /// to fit our model of a 3-layer catalog. MySQL doesn't have a concept @@ -173,12 +173,12 @@ pub(super) fn map_column_refs<'a>( /// Normalize column references to a sorted, deduplicated options list of column names. pub(super) fn normalize_column_refs<'a>( cols: Vec, - subsource_resolver: &SubsourceResolver, + reference_resolver: &SourceReferenceResolver, tables: &[MySqlTableDesc], ) -> Result>, MySqlSourcePurificationError> { let (seq, unknown): (Vec<_>, Vec<_>) = cols.into_iter().partition(|name| { let (column_name, qual) = name.0.split_last().expect("non-empty"); - match subsource_resolver.resolve_idx(qual) { + match reference_resolver.resolve_idx(qual) { // TODO: this needs to also introduce the maximum qualification on // the columns, i.e. ensure they have the schema name. Ok(idx) => tables[idx] @@ -202,17 +202,17 @@ pub(super) fn normalize_column_refs<'a>( Ok(seq) } -pub(super) async fn validate_requested_subsources_privileges( - requested_subsources: &[RequestedSubsource<'_, MySqlTableDesc>], +pub(super) async fn validate_requested_references_privileges( + requested_references: &[RequestedSourceExport<'_, MySqlTableDesc>], conn: &mut mz_mysql_util::MySqlConn, ) -> Result<(), PlanError> { // Ensure that we have correct privileges on all tables; we have to do this before we // start snapshotting because if we discover we cannot `SELECT` from a table while // snapshotting, we break the entire source. - let tables_to_check_permissions = requested_subsources + let tables_to_check_permissions = requested_references .iter() .map( - |RequestedSubsource { + |RequestedSourceExport { external_reference, .. }| external_reference_to_table(external_reference), ) @@ -236,35 +236,36 @@ pub(super) async fn validate_requested_subsources_privileges( Ok(()) } -pub(super) struct PurifiedSubsources { - pub(super) subsources: BTreeMap, +pub(super) struct PurifiedSourceExports { + /// map of source export names to the details of the export + pub(super) source_exports: BTreeMap, pub(super) tables: Vec, pub(super) normalized_text_columns: Vec>, pub(super) normalized_ignore_columns: Vec>, } -// Purify the referenced subsources, return the list of subsource statements, -// corresponding tables, and and additional fields necessary to update the source -// options -pub(super) async fn purify_subsources( +// Purify the requested external references, returning a set of purified +// source exports corresponding to external tables, and and additional +// fields necessary to generate relevant statements and update statement options +pub(super) async fn purify_source_exports( conn: &mut mz_mysql_util::MySqlConn, - referenced_subsources: &mut Option, + external_references: &mut Option, text_columns: Vec, ignore_columns: Vec, unresolved_source_name: &UnresolvedItemName, -) -> Result { +) -> Result { // Determine which table schemas to request from mysql. Note that in mysql // a 'schema' is the same as a 'database', and a fully qualified table // name is 'schema_name.table_name' (there is no db_name) - let table_schema_request = match referenced_subsources + let table_schema_request = match external_references .as_mut() - .ok_or(MySqlSourcePurificationError::RequiresReferencedSubsources)? + .ok_or(MySqlSourcePurificationError::RequiresExternalReferences)? { - ReferencedSubsources::All => mz_mysql_util::SchemaRequest::All, - ReferencedSubsources::SubsetSchemas(schemas) => mz_mysql_util::SchemaRequest::Schemas( + ExternalReferences::All => mz_mysql_util::SchemaRequest::All, + ExternalReferences::SubsetSchemas(schemas) => mz_mysql_util::SchemaRequest::Schemas( schemas.iter().map(|s| s.as_str()).collect::>(), ), - ReferencedSubsources::SubsetTables(tables) => mz_mysql_util::SchemaRequest::Tables( + ExternalReferences::SubsetTables(tables) => mz_mysql_util::SchemaRequest::Tables( tables .iter() .map(|t| { @@ -318,26 +319,26 @@ pub(super) async fn purify_subsources( Err(MySqlSourcePurificationError::EmptyDatabase)?; } - let subsource_resolver = SubsourceResolver::new(MYSQL_DATABASE_FAKE_NAME, &tables)?; + let reference_resolver = SourceReferenceResolver::new(MYSQL_DATABASE_FAKE_NAME, &tables)?; - let mut validated_requested_subsources = vec![]; - match referenced_subsources + let mut validated_source_exports = vec![]; + match external_references .as_mut() - .ok_or(MySqlSourcePurificationError::RequiresReferencedSubsources)? + .ok_or(MySqlSourcePurificationError::RequiresExternalReferences)? { - ReferencedSubsources::All => { + ExternalReferences::All => { for table in &tables { let external_reference = mysql_table_to_external_reference(table)?; let subsource_name = - super::subsource_name_gen(unresolved_source_name, &table.name)?; - validated_requested_subsources.push(RequestedSubsource { + super::source_export_name_gen(unresolved_source_name, &table.name)?; + validated_source_exports.push(RequestedSourceExport { external_reference, - subsource_name, + name: subsource_name, table, }); } } - ReferencedSubsources::SubsetSchemas(schemas) => { + ExternalReferences::SubsetSchemas(schemas) => { let available_schemas: BTreeSet<_> = tables.iter().map(|t| t.schema_name.as_str()).collect(); let requested_schemas: BTreeSet<_> = schemas.iter().map(|s| s.as_str()).collect(); @@ -358,20 +359,20 @@ pub(super) async fn purify_subsources( let external_reference = mysql_table_to_external_reference(table)?; let subsource_name = - super::subsource_name_gen(unresolved_source_name, &table.name)?; - validated_requested_subsources.push(RequestedSubsource { + super::source_export_name_gen(unresolved_source_name, &table.name)?; + validated_source_exports.push(RequestedSourceExport { external_reference, - subsource_name, + name: subsource_name, table, }); } } - ReferencedSubsources::SubsetTables(subsources) => { + ExternalReferences::SubsetTables(references) => { // The user manually selected a subset of upstream tables so we need to // validate that the names actually exist and are not ambiguous - validated_requested_subsources = super::subsource_gen( - subsources, - &subsource_resolver, + validated_source_exports = super::source_export_gen( + references, + &reference_resolver, &tables, 2, unresolved_source_name, @@ -379,22 +380,24 @@ pub(super) async fn purify_subsources( } } - if validated_requested_subsources.is_empty() { + if validated_source_exports.is_empty() { sql_bail!( "[internal error]: MySQL source must ingest at least one table, but {} matched none", - referenced_subsources.as_ref().unwrap().to_ast_string() + external_references.as_ref().unwrap().to_ast_string() ); } - super::validate_subsource_names(&validated_requested_subsources)?; + // TODO: Move this to be used only when working with subsources, since we will allow source-fed + // tables to reference the same external reference. + super::validate_source_export_names(&validated_source_exports)?; - validate_requested_subsources_privileges(&validated_requested_subsources, conn).await?; + validate_requested_references_privileges(&validated_source_exports, conn).await?; - let requested_subsources = validated_requested_subsources + let source_exports = validated_source_exports .into_iter() .map(|r| { ( - r.subsource_name, + r.name, PurifiedSourceExport { external_reference: r.external_reference, details: PurifiedExportDetails::MySql { @@ -405,13 +408,13 @@ pub(super) async fn purify_subsources( }) .collect(); - Ok(PurifiedSubsources { - subsources: requested_subsources, + Ok(PurifiedSourceExports { + source_exports, // Normalize column options and remove unused column references. - normalized_text_columns: normalize_column_refs(text_columns, &subsource_resolver, &tables)?, + normalized_text_columns: normalize_column_refs(text_columns, &reference_resolver, &tables)?, normalized_ignore_columns: normalize_column_refs( ignore_columns, - &subsource_resolver, + &reference_resolver, &tables, )?, tables, diff --git a/src/sql/src/pure/postgres.rs b/src/sql/src/pure/postgres.rs index 1f1783f657ee9..d8eb40bf43d8b 100644 --- a/src/sql/src/pure/postgres.rs +++ b/src/sql/src/pure/postgres.rs @@ -18,9 +18,9 @@ use mz_sql_parser::ast::{ ColumnDef, CreateSubsourceOption, CreateSubsourceOptionName, CreateSubsourceStatement, Ident, WithOptionValue, }; -use mz_sql_parser::ast::{ReferencedSubsources, UnresolvedItemName}; +use mz_sql_parser::ast::{ExternalReferences, UnresolvedItemName}; use mz_storage_types::connections::PostgresConnection; -use mz_storage_types::sources::SubsourceResolver; +use mz_storage_types::sources::SourceReferenceResolver; use tokio_postgres::types::Oid; use tokio_postgres::Client; @@ -29,12 +29,12 @@ use crate::normalize; use crate::plan::{PlanError, StatementContext}; use super::error::PgSourcePurificationError; -use super::{PartialItemName, PurifiedExportDetails, PurifiedSourceExport, RequestedSubsource}; +use super::{PartialItemName, PurifiedExportDetails, PurifiedSourceExport, RequestedSourceExport}; /// Ensure that we have select permissions on all tables; we have to do this before we /// start snapshotting because if we discover we cannot `COPY` from a table while /// snapshotting, we break the entire source. -pub(super) async fn validate_requested_subsources_privileges( +pub(super) async fn validate_requested_references_privileges( config: &Config, client: &Client, table_oids: &[Oid], @@ -51,7 +51,7 @@ pub(super) async fn validate_requested_subsources_privileges( /// Additionally, modify `text_columns` so that they contain database-qualified /// references to the columns. pub(super) fn generate_text_columns( - subsource_resolver: &SubsourceResolver, + reference_resolver: &SourceReferenceResolver, references: &[PostgresTableDesc], text_columns: &mut [UnresolvedItemName], ) -> Result>, PlanError> { @@ -73,7 +73,7 @@ pub(super) fn generate_text_columns( let qual_name = UnresolvedItemName(qual); let (mut fully_qualified_name, idx) = - subsource_resolver.resolve(&qual_name.0, 3).map_err(|e| { + reference_resolver.resolve(&qual_name.0, 3).map_err(|e| { PlanError::InvalidOptionValue { option_name: "TEXT COLUMNS".to_string(), err: Box::new(e.into()), @@ -254,24 +254,24 @@ pub fn generate_create_subsource_statements( Ok(subsources) } -pub(super) struct PurifiedSubsources { - pub(super) subsources: BTreeMap, +pub(super) struct PurifiedSourceExports { + pub(super) source_exports: BTreeMap, pub(super) referenced_tables: Vec, pub(super) normalized_text_columns: Vec>, } -// Purify the referenced subsources, return the list of subsource statements, -// corresponding tables, and and additional fields necessary to update the source -// options -pub(super) async fn purify_subsources( +// Purify the requested external references, returning a set of purified +// source exports corresponding to external tables, and and additional +// fields necessary to generate relevant statements and update statement options +pub(super) async fn purify_source_exports( client: &Client, config: &mz_postgres_util::Config, publication: &str, connection: &PostgresConnection, - referenced_subsources: &mut Option, + external_references: &mut Option, mut text_columns: Vec, unresolved_source_name: &UnresolvedItemName, -) -> Result { +) -> Result { let mut publication_tables = mz_postgres_util::publication_info(client, publication).await?; if publication_tables.is_empty() { @@ -280,14 +280,15 @@ pub(super) async fn purify_subsources( ))?; } - let subsource_resolver = SubsourceResolver::new(&connection.database, &publication_tables)?; + let reference_resolver = + SourceReferenceResolver::new(&connection.database, &publication_tables)?; - let mut validated_requested_subsources = vec![]; - match referenced_subsources + let mut validated_references = vec![]; + match external_references .as_mut() - .ok_or(PgSourcePurificationError::RequiresReferencedSubsources)? + .ok_or(PgSourcePurificationError::RequiresExternalReferences)? { - ReferencedSubsources::All => { + ExternalReferences::All => { for table in &publication_tables { let external_reference = UnresolvedItemName::qualified(&[ Ident::new(&connection.database)?, @@ -295,15 +296,15 @@ pub(super) async fn purify_subsources( Ident::new(&table.name)?, ]); let subsource_name = - super::subsource_name_gen(unresolved_source_name, &table.name)?; - validated_requested_subsources.push(RequestedSubsource { + super::source_export_name_gen(unresolved_source_name, &table.name)?; + validated_references.push(RequestedSourceExport { external_reference, - subsource_name, + name: subsource_name, table, }); } } - ReferencedSubsources::SubsetSchemas(schemas) => { + ExternalReferences::SubsetSchemas(schemas) => { let available_schemas: BTreeSet<_> = mz_postgres_util::get_schemas(client) .await? .into_iter() @@ -336,20 +337,20 @@ pub(super) async fn purify_subsources( Ident::new(&table.name)?, ]); let subsource_name = - super::subsource_name_gen(unresolved_source_name, &table.name)?; - validated_requested_subsources.push(RequestedSubsource { + super::source_export_name_gen(unresolved_source_name, &table.name)?; + validated_references.push(RequestedSourceExport { external_reference, - subsource_name, + name: subsource_name, table, }); } } - ReferencedSubsources::SubsetTables(subsources) => { + ExternalReferences::SubsetTables(references) => { // The user manually selected a subset of upstream tables so we need to // validate that the names actually exist and are not ambiguous - validated_requested_subsources.extend(super::subsource_gen( - subsources, - &subsource_resolver, + validated_references.extend(super::source_export_gen( + references, + &reference_resolver, &publication_tables, 3, unresolved_source_name, @@ -357,24 +358,23 @@ pub(super) async fn purify_subsources( } }; - if validated_requested_subsources.is_empty() { + // TODO: Remove this check once we allow creating a source with no exports and adding + // source-fed tables to that source later. + if validated_references.is_empty() { sql_bail!( "[internal error]: Postgres source must ingest at least one table, but {} matched none", - referenced_subsources.as_ref().unwrap().to_ast_string() + external_references.as_ref().unwrap().to_ast_string() ); } - super::validate_subsource_names(&validated_requested_subsources)?; + super::validate_source_export_names(&validated_references)?; - let table_oids: Vec<_> = validated_requested_subsources - .iter() - .map(|r| r.table.oid) - .collect(); + let table_oids: Vec<_> = validated_references.iter().map(|r| r.table.oid).collect(); - validate_requested_subsources_privileges(config, client, &table_oids).await?; + validate_requested_references_privileges(config, client, &table_oids).await?; let mut text_column_map = - generate_text_columns(&subsource_resolver, &publication_tables, &mut text_columns)?; + generate_text_columns(&reference_resolver, &publication_tables, &mut text_columns)?; // Normalize options to contain full qualified values. text_columns.sort(); @@ -384,11 +384,11 @@ pub(super) async fn purify_subsources( .map(WithOptionValue::UnresolvedItemName) .collect(); - let requested_subsources = validated_requested_subsources + let requested_subsources = validated_references .into_iter() .map(|r| { ( - r.subsource_name, + r.name, PurifiedSourceExport { external_reference: r.external_reference, details: PurifiedExportDetails::Postgres { @@ -427,8 +427,8 @@ pub(super) async fn purify_subsources( // Trim any un-referred-to tables publication_tables.retain(|t| table_oids.contains(&t.oid)); - Ok(PurifiedSubsources { - subsources: requested_subsources, + Ok(PurifiedSourceExports { + source_exports: requested_subsources, referenced_tables: publication_tables, normalized_text_columns, }) diff --git a/src/storage-controller/src/lib.rs b/src/storage-controller/src/lib.rs index b5d960f38c40a..8ba2a51e6a7c9 100644 --- a/src/storage-controller/src/lib.rs +++ b/src/storage-controller/src/lib.rs @@ -887,7 +887,7 @@ where .desc .alter_compatible(ingestion_id, source_desc)?; - let subsource_resolver = cur_ingestion.desc.connection.get_subsource_resolver(); + let reference_resolver = cur_ingestion.desc.connection.get_reference_resolver(); // Ensure updated `SourceDesc` contains reference to all // current external references. @@ -913,7 +913,7 @@ where } }; - if subsource_resolver + if reference_resolver .resolve_idx(&external_reference.0) .is_err() { diff --git a/src/storage-types/src/sources.rs b/src/storage-types/src/sources.rs index e24bf2406c358..1d622d404108b 100644 --- a/src/storage-types/src/sources.rs +++ b/src/storage-types/src/sources.rs @@ -182,7 +182,7 @@ impl IngestionDescription { impl IngestionDescription { pub fn source_exports_with_output_indices(&self) -> BTreeMap> { - let subsource_resolver = self.desc.connection.get_subsource_resolver(); + let reference_resolver = self.desc.connection.get_reference_resolver(); let mut source_exports = BTreeMap::new(); @@ -196,7 +196,7 @@ impl IngestionDescription { { let ingestion_output = match ingestion_output { Some(ingestion_output) => { - subsource_resolver + reference_resolver .resolve_idx(&ingestion_output.0) .expect("must have all subsource references") // output indices are the native details idx + 1 to @@ -739,8 +739,9 @@ pub trait SourceConnection: Debug + Clone + PartialEq + AlterCompatible { /// columns are returned in the order specified by the user. fn metadata_columns(&self) -> Vec<(&str, ColumnType)>; - /// Returns a [`SubsourceResolver`] for this source connection's subsources. - fn get_subsource_resolver(&self) -> SubsourceResolver; + /// Returns a [`SourceReferenceResolver`] for this source connection's source exports, keyed + /// by their external reference. + fn get_reference_resolver(&self) -> SourceReferenceResolver; } #[derive(Arbitrary, Clone, Copy, Debug, Eq, PartialEq, Serialize, Deserialize)] @@ -1040,12 +1041,12 @@ impl SourceConnection for GenericSourceConnection { } } - fn get_subsource_resolver(&self) -> SubsourceResolver { + fn get_reference_resolver(&self) -> SourceReferenceResolver { match self { - Self::Kafka(conn) => conn.get_subsource_resolver(), - Self::Postgres(conn) => conn.get_subsource_resolver(), - Self::MySql(conn) => conn.get_subsource_resolver(), - Self::LoadGenerator(conn) => conn.get_subsource_resolver(), + Self::Kafka(conn) => conn.get_reference_resolver(), + Self::Postgres(conn) => conn.get_reference_resolver(), + Self::MySql(conn) => conn.get_reference_resolver(), + Self::LoadGenerator(conn) => conn.get_reference_resolver(), } } } @@ -1378,21 +1379,21 @@ impl Schema for RelationDesc { } } -/// Describes how subsource references should be organized in a multi-level +/// Describes how external references should be organized in a multi-level /// hierarchy. /// /// For both PostgreSQL and MySQL sources, these levels of reference are /// intrinsic to the items which we're referencing. If there are other naming /// schemas for other types of sources we discover, we might need to revisit /// this. -pub trait SubsourceCatalogReference { +pub trait ExternalCatalogReference { /// The "second" level of namespacing for the reference. fn schema_name(&self) -> &str; /// The lowest level of namespacing for the reference. fn item_name(&self) -> &str; } -impl SubsourceCatalogReference for mz_mysql_util::MySqlTableDesc { +impl ExternalCatalogReference for mz_mysql_util::MySqlTableDesc { fn schema_name(&self) -> &str { &self.schema_name } @@ -1402,7 +1403,7 @@ impl SubsourceCatalogReference for mz_mysql_util::MySqlTableDesc { } } -impl SubsourceCatalogReference for mz_postgres_util::desc::PostgresTableDesc { +impl ExternalCatalogReference for mz_postgres_util::desc::PostgresTableDesc { fn schema_name(&self) -> &str { &self.namespace } @@ -1414,7 +1415,7 @@ impl SubsourceCatalogReference for mz_postgres_util::desc::PostgresTableDesc { // This implementation provides a means of converting arbitrary objects into a // `SubsourceCatalogReference`, e.g. load generator view names. -impl<'a> SubsourceCatalogReference for (&'a str, &'a str) { +impl<'a> ExternalCatalogReference for (&'a str, &'a str) { fn schema_name(&self) -> &str { self.0 } @@ -1424,7 +1425,7 @@ impl<'a> SubsourceCatalogReference for (&'a str, &'a str) { } } -/// Stores and resolves references to a `&[T: SubsourceCatalogTable]`. +/// Stores and resolves references to a `&[T: ExternalCatalogReference]`. /// /// This is meant to provide an API to quickly look up a source's subsources. /// @@ -1432,12 +1433,12 @@ impl<'a> SubsourceCatalogReference for (&'a str, &'a str) { /// implementation, which is empty and will not be able to resolve any /// references. #[derive(Debug, Clone, Default, PartialEq, Eq, Serialize, Deserialize)] -pub struct SubsourceResolver { +pub struct SourceReferenceResolver { inner: BTreeMap>>, } #[derive(Debug, Clone, thiserror::Error)] -pub enum SubsourceResolutionError { +pub enum ExternalReferenceResolutionError { #[error("reference to {name} not found in source")] DoesNotExist { name: String }, #[error( @@ -1449,16 +1450,16 @@ pub enum SubsourceResolutionError { Ident(#[from] IdentError), } -impl<'a> SubsourceResolver { - /// Constructs a new `SubsourceResolver` from a slice of `T: +impl<'a> SourceReferenceResolver { + /// Constructs a new `SourceReferenceResolver` from a slice of `T: /// SubsourceCatalogReference`. /// /// # Errors /// - If any `&str` provided cannot be taken to an [`Ident`]. - pub fn new( + pub fn new( database: &str, referenceable_items: &'a [T], - ) -> Result { + ) -> Result { // An index from table name -> schema name -> database name -> index in // `referenceable_items`. let mut inner = BTreeMap::new(); @@ -1478,7 +1479,7 @@ impl<'a> SubsourceResolver { .or_insert(reference_idx); } - Ok(SubsourceResolver { inner }) + Ok(SourceReferenceResolver { inner }) } /// Returns the canonical reference and index from which it originated in @@ -1501,7 +1502,7 @@ impl<'a> SubsourceResolver { &self, name: &[Ident], canonicalize_to_width: usize, - ) -> Result<(UnresolvedItemName, usize), SubsourceResolutionError> { + ) -> Result<(UnresolvedItemName, usize), ExternalReferenceResolutionError> { let (db, schema, idx) = self.resolve_inner(name)?; let item = name.last().expect("must have provided at least 1 element"); @@ -1525,7 +1526,7 @@ impl<'a> SubsourceResolver { /// /// # Errors /// - If `name` does not resolve to an item in `self.inner`. - pub fn resolve_idx(&self, name: &[Ident]) -> Result { + pub fn resolve_idx(&self, name: &[Ident]) -> Result { let (_db, _schema, idx) = self.resolve_inner(name)?; Ok(idx) } @@ -1542,19 +1543,19 @@ impl<'a> SubsourceResolver { /// 1. The "database"- or top-level namespace of the reference. /// 2. The "schema"- or second-level namespace of the reference. /// 3. The index to find the item in `referenceable_items` argument provided - /// to `SubsourceResolver::new`. + /// to `SourceReferenceResolver::new`. /// /// # Errors /// - If `name` does not resolve to an item in `self.inner`. fn resolve_inner<'name: 'a>( &'a self, name: &'name [Ident], - ) -> Result<(&'a Ident, &'a Ident, usize), SubsourceResolutionError> { + ) -> Result<(&'a Ident, &'a Ident, usize), ExternalReferenceResolutionError> { let get_provided_name = || UnresolvedItemName(name.to_vec()).to_string(); // Names must be composed of 1..=3 elements. if !(1..=3).contains(&name.len()) { - Err(SubsourceResolutionError::DoesNotExist { + Err(ExternalReferenceResolutionError::DoesNotExist { name: get_provided_name(), })?; } @@ -1576,42 +1577,40 @@ impl<'a> SubsourceResolver { let schemas = self.inner .get(item) - .ok_or_else(|| SubsourceResolutionError::DoesNotExist { + .ok_or_else(|| ExternalReferenceResolutionError::DoesNotExist { name: get_provided_name(), })?; - let schema = - match schema { - Some(schema) => schema, - None => schemas.keys().exactly_one().map_err(|_e| { - SubsourceResolutionError::Ambiguous { - name: get_provided_name(), - } - })?, - }; + let schema = match schema { + Some(schema) => schema, + None => schemas.keys().exactly_one().map_err(|_e| { + ExternalReferenceResolutionError::Ambiguous { + name: get_provided_name(), + } + })?, + }; let databases = schemas .get(schema) - .ok_or_else(|| SubsourceResolutionError::DoesNotExist { + .ok_or_else(|| ExternalReferenceResolutionError::DoesNotExist { name: get_provided_name(), })?; let database = match database { Some(database) => database, None => databases.keys().exactly_one().map_err(|_e| { - SubsourceResolutionError::Ambiguous { + ExternalReferenceResolutionError::Ambiguous { name: get_provided_name(), } })?, }; - let reference_idx = - databases - .get(database) - .ok_or_else(|| SubsourceResolutionError::DoesNotExist { - name: get_provided_name(), - })?; + let reference_idx = databases.get(database).ok_or_else(|| { + ExternalReferenceResolutionError::DoesNotExist { + name: get_provided_name(), + } + })?; Ok((database, schema, *reference_idx)) } diff --git a/src/storage-types/src/sources/kafka.rs b/src/storage-types/src/sources/kafka.rs index 9b5c5c72a09c3..2a3130d94ab97 100644 --- a/src/storage-types/src/sources/kafka.rs +++ b/src/storage-types/src/sources/kafka.rs @@ -254,8 +254,8 @@ impl SourceConnection for KafkaSourceConnection { .collect() } - fn get_subsource_resolver(&self) -> super::SubsourceResolver { - super::SubsourceResolver::default() + fn get_reference_resolver(&self) -> super::SourceReferenceResolver { + super::SourceReferenceResolver::default() } } diff --git a/src/storage-types/src/sources/load_generator.rs b/src/storage-types/src/sources/load_generator.rs index 1470ef8192351..efb7c1fca5a9a 100644 --- a/src/storage-types/src/sources/load_generator.rs +++ b/src/storage-types/src/sources/load_generator.rs @@ -123,15 +123,15 @@ impl SourceConnection for LoadGeneratorSourceConnection { vec![] } - fn get_subsource_resolver(&self) -> super::SubsourceResolver { + fn get_reference_resolver(&self) -> super::SourceReferenceResolver { let views: Vec<_> = self .load_generator .views() .into_iter() .map(|(name, _)| (self.load_generator.schema_name(), name)) .collect(); - super::SubsourceResolver::new(LOAD_GENERATOR_DATABASE_NAME, &views) - .expect("already validated that SubsourceResolver elements are valid") + super::SourceReferenceResolver::new(LOAD_GENERATOR_DATABASE_NAME, &views) + .expect("already validated that SourceReferenceResolver elements are valid") } } diff --git a/src/storage-types/src/sources/mysql.rs b/src/storage-types/src/sources/mysql.rs index f98ca683339d6..b319166379d0d 100644 --- a/src/storage-types/src/sources/mysql.rs +++ b/src/storage-types/src/sources/mysql.rs @@ -196,9 +196,9 @@ impl SourceConnection for MySqlSourceConnection { vec![] } - fn get_subsource_resolver(&self) -> super::SubsourceResolver { - super::SubsourceResolver::new("mysql", &self.details.tables) - .expect("already validated that SubsourceResolver elements are valid") + fn get_reference_resolver(&self) -> super::SourceReferenceResolver { + super::SourceReferenceResolver::new("mysql", &self.details.tables) + .expect("already validated that SourceReferenceResolver elements are valid") } } diff --git a/src/storage-types/src/sources/postgres.rs b/src/storage-types/src/sources/postgres.rs index 75231a3c14abb..75f4cacca72ba 100644 --- a/src/storage-types/src/sources/postgres.rs +++ b/src/storage-types/src/sources/postgres.rs @@ -133,12 +133,12 @@ impl SourceConnection for PostgresSourceConnection { vec![] } - fn get_subsource_resolver(&self) -> super::SubsourceResolver { - super::SubsourceResolver::new( + fn get_reference_resolver(&self) -> super::SourceReferenceResolver { + super::SourceReferenceResolver::new( &self.publication_details.database, &self.publication_details.tables, ) - .expect("already validated that SubsourceResolver elements are valid") + .expect("already validated that SourceReferenceResolver elements are valid") } } From e48c44d2b126536747e84decfc1b901ec9ae127c Mon Sep 17 00:00:00 2001 From: Roshan Jobanputra Date: Thu, 18 Jul 2024 12:33:18 -0400 Subject: [PATCH 3/3] Fix ddl changed by rename --- src/sql-parser/tests/testdata/ddl | 132 +++++++++++++++--------------- 1 file changed, 66 insertions(+), 66 deletions(-) diff --git a/src/sql-parser/tests/testdata/ddl b/src/sql-parser/tests/testdata/ddl index f35ec840753f3..e0a24365f0d3b 100644 --- a/src/sql-parser/tests/testdata/ddl +++ b/src/sql-parser/tests/testdata/ddl @@ -524,7 +524,7 @@ CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn FOR TABLES (foo, bar as ---- CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn FOR TABLES (foo, bar AS qux, baz AS zop) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(SubsetTables([CreateSourceSubsource { reference: UnresolvedItemName([Ident("foo")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("bar")]), subsource: Some(UnresolvedItemName([Ident("qux")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("baz")]), subsource: Some(UnresolvedItemName([Ident("zop")])) }])), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(SubsetTables([ExternalReferenceExport { reference: UnresolvedItemName([Ident("foo")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("bar")]), alias: Some(UnresolvedItemName([Ident("qux")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("baz")]), alias: Some(UnresolvedItemName([Ident("zop")])) }])), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn FOR TABLES ([s1 AS foo.bar]); @@ -538,28 +538,28 @@ CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn (TEXT COLUMNS (public.fo ---- CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn (TEXT COLUMNS = (public.foo.bar)) FOR ALL TABLES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [MySqlConfigOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("bar")]))])) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [MySqlConfigOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("bar")]))])) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn (IGNORE COLUMNS (public.foo.bar)) FOR ALL TABLES; ---- CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn (IGNORE COLUMNS = (public.foo.bar)) FOR ALL TABLES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [MySqlConfigOption { name: IgnoreColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("bar")]))])) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [MySqlConfigOption { name: IgnoreColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("bar")]))])) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn (IGNORE COLUMNS (public.foo.bar), TEXT COLUMNS (public.foo.baz)) FOR ALL TABLES; ---- CREATE SOURCE mz_source FROM MYSQL CONNECTION mysqlconn (IGNORE COLUMNS = (public.foo.bar), TEXT COLUMNS = (public.foo.baz)) FOR ALL TABLES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [MySqlConfigOption { name: IgnoreColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("bar")]))])) }, MySqlConfigOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("baz")]))])) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: MySql { connection: Name(UnresolvedItemName([Ident("mysqlconn")])), options: [MySqlConfigOption { name: IgnoreColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("bar")]))])) }, MySqlConfigOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("public"), Ident("foo"), Ident("baz")]))])) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: None }) parse-statement CREATE SOURCE psychic FROM POSTGRES CONNECTION pgconn (PUBLICATION 'red'); ---- CREATE SOURCE psychic FROM POSTGRES CONNECTION pgconn (PUBLICATION = 'red') => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("psychic")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pgconn")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("red"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("psychic")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pgconn")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("red"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic', PROGRESS GROUP ID PREFIX 'prefix', COMPRESSION TYPE = gzip) FORMAT BYTES @@ -608,7 +608,7 @@ CREATE SOURCE psychic IN CLUSTER c FROM POSTGRES CONNECTION pgconn (PUBLICATION ---- CREATE SOURCE psychic IN CLUSTER c FROM POSTGRES CONNECTION pgconn (PUBLICATION = 'red') INCLUDE TIMESTAMP => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("psychic")]), in_cluster: Some(Unresolved(Ident("c"))), col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pgconn")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("red"))) }] }, include_metadata: [Timestamp { alias: None }], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("psychic")]), in_cluster: Some(Unresolved(Ident("c"))), col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pgconn")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("red"))) }] }, include_metadata: [Timestamp { alias: None }], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SINK IF NOT EXISTS foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') KEY (a, b) FORMAT BYTES ENVELOPE NONE @@ -1389,49 +1389,49 @@ ALTER SOURCE n ADD SUBSOURCE a.b.c.d ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: None }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: None }], options: [] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }], options: [] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: None }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: None }], options: [] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g, a.b.c ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g, a.b.c => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: None }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: None }], options: [] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c, a.b.c.d AS e.f.g ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c, a.b.c.d AS e.f.g => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a, b.c, c, d.e ---- ALTER SOURCE n ADD SUBSOURCE a, b.c, c, d.e => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("b"), Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("d"), Ident("e")]), subsource: None }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("b"), Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("d"), Ident("e")]), alias: None }], options: [] } }) parse-statement @@ -1439,56 +1439,56 @@ ALTER SOURCE n ADD SUBSOURCE a.b.c.d WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g, a.b.c WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d, a.b.c AS d.e.f.g, a.b.c WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: Some(UnresolvedItemName([Ident("d"), Ident("e"), Ident("f"), Ident("g")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c, a.b.c.d AS e.f.g WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE n ADD SUBSOURCE a.b.c.d AS e.f.g, a.b.c, a.b.c.d AS e.f.g WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), subsource: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("a"), Ident("b"), Ident("c"), Ident("d")]), alias: Some(UnresolvedItemName([Ident("e"), Ident("f"), Ident("g")])) }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE n ADD SUBSOURCE a, b.c, c, d.e WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE n ADD SUBSOURCE a, b.c, c, d.e WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("b"), Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("d"), Ident("e")]), subsource: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: false, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("b"), Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("d"), Ident("e")]), alias: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE IF EXISTS n ADD SUBSOURCE a, b.c, c, d.e WITH (TEXT COLUMNS [a.b, c.d]) ---- ALTER SOURCE IF EXISTS n ADD SUBSOURCE a, b.c, c, d.e WITH (TEXT COLUMNS = (a.b, c.d)) => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: true, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("b"), Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("c")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("d"), Ident("e")]), subsource: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: true, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("b"), Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("c")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("d"), Ident("e")]), alias: None }], options: [AlterSourceAddSubsourceOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("a"), Ident("b")])), UnresolvedItemName(UnresolvedItemName([Ident("c"), Ident("d")]))])) }] } }) parse-statement ALTER SOURCE IF EXISTS n ADD SUBSOURCE a, b.c, c, d.e WITH () @@ -1502,7 +1502,7 @@ ALTER SOURCE IF EXISTS n ADD TABLE a, b.c AS d ---- ALTER SOURCE IF EXISTS n ADD SUBSOURCE a, b.c AS d => -AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: true, action: AddSubsources { subsources: [CreateSourceSubsource { reference: UnresolvedItemName([Ident("a")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("b"), Ident("c")]), subsource: Some(UnresolvedItemName([Ident("d")])) }], options: [] } }) +AlterSource(AlterSourceStatement { source_name: UnresolvedItemName([Ident("n")]), if_exists: true, action: AddSubsources { external_references: [ExternalReferenceExport { reference: UnresolvedItemName([Ident("a")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("b"), Ident("c")]), alias: Some(UnresolvedItemName([Ident("d")])) }], options: [] } }) parse-statement ALTER SOURCE IF EXISTS n ADD SOURCE a, b.c AS d @@ -2228,7 +2228,7 @@ CREATE SOURCE IF NOT EXISTS src1 (a, b, PRIMARY KEY (a) NOT ENFORCED) FROM KAFKA ---- CREATE SOURCE IF NOT EXISTS src1 (a, b, PRIMARY KEY (a) NOT ENFORCED) FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT BYTES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [Ident("a"), Ident("b")], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Bytes)), envelope: None, if_not_exists: true, key_constraint: Some(PrimaryKeyNotEnforced { columns: [Ident("a")] }), with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [Ident("a"), Ident("b")], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Bytes)), envelope: None, if_not_exists: true, key_constraint: Some(PrimaryKeyNotEnforced { columns: [Ident("a")] }), with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY (a, b) FORMAT BYTES @@ -2242,7 +2242,7 @@ CREATE SOURCE src1 (PRIMARY KEY (key1, key2) NOT ENFORCED) FROM KAFKA CONNECTION ---- CREATE SOURCE src1 (PRIMARY KEY (key1, key2) NOT ENFORCED) FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT BYTES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Bytes)), envelope: None, if_not_exists: false, key_constraint: Some(PrimaryKeyNotEnforced { columns: [Ident("key1"), Ident("key2")] }), with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Bytes)), envelope: None, if_not_exists: false, key_constraint: Some(PrimaryKeyNotEnforced { columns: [Ident("key1"), Ident("key2")] }), with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE CONNECTION conn1 FOR CONFLUENT SCHEMA REGISTRY URL 'http://localhost:8081', USERNAME 'user', PASSWORD 'word', PORT 1234, AWS PRIVATELINK apl @@ -2285,7 +2285,7 @@ CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT AVRO USING C ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 ENVELOPE DEBEZIUM => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement @@ -2293,7 +2293,7 @@ CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT PROTOBUF USI ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 ENVELOPE DEBEZIUM => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, seed: None } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, seed: None } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement @@ -2331,35 +2331,35 @@ CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT AVRO USING S ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT AVRO USING SCHEMA 'schema' => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(InlineSchema { schema: Schema { schema: "schema" }, with_options: [] }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(InlineSchema { schema: Schema { schema: "schema" }, with_options: [] }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT AVRO USING SCHEMA 'schema' (CONFLUENT WIRE FORMAT = false) ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT AVRO USING SCHEMA 'schema' (CONFLUENT WIRE FORMAT = false) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(InlineSchema { schema: Schema { schema: "schema" }, with_options: [AvroSchemaOption { name: ConfluentWireFormat, value: Some(Value(Boolean(false))) }] }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(InlineSchema { schema: Schema { schema: "schema" }, with_options: [AvroSchemaOption { name: ConfluentWireFormat, value: Some(Value(Boolean(false))) }] }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT AVRO USING SCHEMA 'schema' (CONFLUENT WIRE FORMAT = true) ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT AVRO USING SCHEMA 'schema' (CONFLUENT WIRE FORMAT = true) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(InlineSchema { schema: Schema { schema: "schema" }, with_options: [AvroSchemaOption { name: ConfluentWireFormat, value: Some(Value(Boolean(true))) }] }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(InlineSchema { schema: Schema { schema: "schema" }, with_options: [AvroSchemaOption { name: ConfluentWireFormat, value: Some(Value(Boolean(true))) }] }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT PROTOBUF MESSAGE 'Batch' USING SCHEMA '\x0a300a0d62696' ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT PROTOBUF MESSAGE 'Batch' USING SCHEMA '\x0a300a0d62696' => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(InlineSchema { message_name: "Batch", schema: Schema { schema: "\\x0a300a0d62696" } }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(InlineSchema { message_name: "Batch", schema: Schema { schema: "\\x0a300a0d62696" } }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED VALUE SCHEMA '{"some": "seed"}' MESSAGE 'Batch' ENVELOPE DEBEZIUM ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED VALUE SCHEMA '{"some": "seed"}' MESSAGE 'Batch' ENVELOPE DEBEZIUM => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: None, value: CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" } }) } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: None, value: CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" } }) } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement @@ -2367,77 +2367,77 @@ CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT PROTOBUF USI ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "seed"}' MESSAGE 'Batch' VALUE SCHEMA '123' MESSAGE 'M' ENVELOPE DEBEZIUM => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(Debezium), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "seed"}' MESSAGE 'Batch' VALUE SCHEMA '123' MESSAGE 'M' ENVELOPE UPSERT ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "seed"}' MESSAGE 'Batch' VALUE SCHEMA '123' MESSAGE 'M' ENVELOPE UPSERT => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(Upsert { value_decode_err_policy: [] }), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(Upsert { value_decode_err_policy: [] }), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "seed"}' MESSAGE 'Batch' VALUE SCHEMA '123' MESSAGE 'M' ENVELOPE MATERIALIZE ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "seed"}' MESSAGE 'Batch' VALUE SCHEMA '123' MESSAGE 'M' ENVELOPE MATERIALIZE => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(CdcV2), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(CdcV2), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "seed"}' MESSAGE 'Batch' VALUE SCHEMA '123' MESSAGE 'M' ENVELOPE NONE ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT PROTOBUF USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "seed"}' MESSAGE 'Batch' VALUE SCHEMA '123' MESSAGE 'M' ENVELOPE NONE => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, seed: Some(CsrSeedProtobuf { key: Some(CsrSeedProtobufSchema { schema: "{\"some\": \"seed\"}", message_name: "Batch" }), value: CsrSeedProtobufSchema { schema: "123", message_name: "M" } }) } }))), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY FORMAT TEXT VALUE FORMAT REGEX '(?P[^,]+),(?P\w+)' INCLUDE KEY ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') KEY FORMAT TEXT VALUE FORMAT REGEX '(?P[^,]+),(?P\w+)' INCLUDE KEY => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [Key { alias: None }], format: Some(KeyValue { key: Text, value: Regex("(?P[^,]+),(?P\\w+)") }), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [Key { alias: None }], format: Some(KeyValue { key: Text, value: Regex("(?P[^,]+),(?P\\w+)") }), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY FORMAT TEXT VALUE FORMAT CSV WITH 2 COLUMNS DELIMITED BY ',' ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') KEY FORMAT TEXT VALUE FORMAT CSV WITH 2 COLUMNS => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Csv { columns: Count(2), delimiter: ',' } }), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Csv { columns: Count(2), delimiter: ',' } }), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY FORMAT TEXT VALUE FORMAT CSV WITH 2 COLUMNS DELIMITED BY ';' ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') KEY FORMAT TEXT VALUE FORMAT CSV WITH 2 COLUMNS DELIMITED BY ';' => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Csv { columns: Count(2), delimiter: ';' } }), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Csv { columns: Count(2), delimiter: ';' } }), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn (AVRO KEY FULLNAME = 'some.neat.class.foo', AVRO VALUE FULLNAME = 'some.neat.class.bar') ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn (AVRO KEY FULLNAME = 'some.neat.class.foo', AVRO VALUE FULLNAME = 'some.neat.class.bar') => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [CsrConfigOption { name: AvroKeyFullname, value: Some(Value(String("some.neat.class.foo"))) }, CsrConfigOption { name: AvroValueFullname, value: Some(Value(String("some.neat.class.bar"))) }] }, key_strategy: None, value_strategy: None, seed: None } }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [CsrConfigOption { name: AvroKeyFullname, value: Some(Value(String("some.neat.class.foo"))) }, CsrConfigOption { name: AvroValueFullname, value: Some(Value(String("some.neat.class.bar"))) }] }, key_strategy: None, value_strategy: None, seed: None } }))), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED VALUE SCHEMA '{"some": "schema"}' ENVELOPE NONE ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED VALUE SCHEMA '{"some": "schema"}' ENVELOPE NONE => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: None, value_schema: "{\"some\": \"schema\"}" }) } }))), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: None, value_schema: "{\"some\": \"schema\"}" }) } }))), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "schema"}' VALUE SCHEMA '123' ENVELOPE NONE ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "schema"}' VALUE SCHEMA '123' ENVELOPE NONE => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: Some("{\"some\": \"schema\"}"), value_schema: "123" }) } }))), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: Some("{\"some\": \"schema\"}"), value_schema: "123" }) } }))), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn VALUE FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "schema"}' VALUE SCHEMA '123' ENVELOPE NONE ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') KEY FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn VALUE FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "schema"}' VALUE SCHEMA '123' ENVELOPE NONE => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(KeyValue { key: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } }), value: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: Some("{\"some\": \"schema\"}"), value_schema: "123" }) } }) }), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(KeyValue { key: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } }), value: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: Some("{\"some\": \"schema\"}"), value_schema: "123" }) } }) }), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement @@ -2445,7 +2445,7 @@ CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY FORMAT AVRO USI ---- CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') KEY FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn VALUE FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn SEED KEY SCHEMA '{"some": "schema"}' VALUE SCHEMA '123' INCLUDE KEY, TIMESTAMP, PARTITION AS "PART2", OFFSET, HEADERS ENVELOPE NONE => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [Key { alias: None }, Timestamp { alias: None }, Partition { alias: Some(Ident("PART2")) }, Offset { alias: None }, Headers { alias: None }], format: Some(KeyValue { key: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } }), value: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: Some("{\"some\": \"schema\"}"), value_schema: "123" }) } }) }), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [Key { alias: None }, Timestamp { alias: None }, Partition { alias: Some(Ident("PART2")) }, Offset { alias: None }, Headers { alias: None }], format: Some(KeyValue { key: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } }), value: Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("csr_conn")])), options: [] }, key_strategy: None, value_strategy: None, seed: Some(CsrSeedAvro { key_schema: Some("{\"some\": \"schema\"}"), value_schema: "123" }) } }) }), envelope: Some(None), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE golbat FROM KAFKA BROKER 'zubat' TOPIC 'hoothoot' KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE KEY ENVELOPE NONE @@ -2459,14 +2459,14 @@ CREATE SOURCE example FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT CSV WIT ---- CREATE SOURCE example FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT CSV WITH HEADER (id, value) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("example")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Csv { columns: Header { names: [Ident("id"), Ident("value")] }, delimiter: ',' })), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("example")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Csv { columns: Header { names: [Ident("id"), Ident("value")] }, delimiter: ',' })), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE example FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT CSV WITH 5 COLUMNS ---- CREATE SOURCE example FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') FORMAT CSV WITH 5 COLUMNS => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("example")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Csv { columns: Count(5), delimiter: ',' })), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("example")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, include_metadata: [], format: Some(Bare(Csv { columns: Count(5), delimiter: ',' })), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE CONNECTION my_ssh_tunnel FOR SSH TUNNEL HOST 'ssh-bastion', PORT 1234, USER 'blah' @@ -2485,56 +2485,56 @@ CREATE SOURCE lg FROM LOAD GENERATOR COUNTER ---- CREATE SOURCE lg FROM LOAD GENERATOR COUNTER => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Counter, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Counter, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE lg FROM LOAD GENERATOR KEY VALUE (KEYS 1, PARTITIONS 2, TICK INTERVAL '1m', BATCH SIZE 100, SEED 200, VALUE SIZE 150, SNAPSHOT ROUNDS 3, TRANSACTIONAL SNAPSHOT = false) ---- CREATE SOURCE lg FROM LOAD GENERATOR KEY VALUE (KEYS = 1, PARTITIONS = 2, TICK INTERVAL = '1m', BATCH SIZE = 100, SEED = 200, VALUE SIZE = 150, SNAPSHOT ROUNDS = 3, TRANSACTIONAL SNAPSHOT = false) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: KeyValue, options: [LoadGeneratorOption { name: Keys, value: Some(Value(Number("1"))) }, LoadGeneratorOption { name: Partitions, value: Some(Value(Number("2"))) }, LoadGeneratorOption { name: TickInterval, value: Some(Value(String("1m"))) }, LoadGeneratorOption { name: BatchSize, value: Some(Value(Number("100"))) }, LoadGeneratorOption { name: Seed, value: Some(Value(Number("200"))) }, LoadGeneratorOption { name: ValueSize, value: Some(Value(Number("150"))) }, LoadGeneratorOption { name: SnapshotRounds, value: Some(Value(Number("3"))) }, LoadGeneratorOption { name: TransactionalSnapshot, value: Some(Value(Boolean(false))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: KeyValue, options: [LoadGeneratorOption { name: Keys, value: Some(Value(Number("1"))) }, LoadGeneratorOption { name: Partitions, value: Some(Value(Number("2"))) }, LoadGeneratorOption { name: TickInterval, value: Some(Value(String("1m"))) }, LoadGeneratorOption { name: BatchSize, value: Some(Value(Number("100"))) }, LoadGeneratorOption { name: Seed, value: Some(Value(Number("200"))) }, LoadGeneratorOption { name: ValueSize, value: Some(Value(Number("150"))) }, LoadGeneratorOption { name: SnapshotRounds, value: Some(Value(Number("3"))) }, LoadGeneratorOption { name: TransactionalSnapshot, value: Some(Value(Boolean(false))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE lg FROM LOAD GENERATOR COUNTER (TICK INTERVAL '1s', SCALE FACTOR 1, MAX CARDINALITY 100, UP TO 5, AS OF 5) ---- CREATE SOURCE lg FROM LOAD GENERATOR COUNTER (TICK INTERVAL = '1s', SCALE FACTOR = 1, MAX CARDINALITY = 100, UP TO = 5, AS OF = 5) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Counter, options: [LoadGeneratorOption { name: TickInterval, value: Some(Value(String("1s"))) }, LoadGeneratorOption { name: ScaleFactor, value: Some(Value(Number("1"))) }, LoadGeneratorOption { name: MaxCardinality, value: Some(Value(Number("100"))) }, LoadGeneratorOption { name: UpTo, value: Some(Value(Number("5"))) }, LoadGeneratorOption { name: AsOf, value: Some(Value(Number("5"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Counter, options: [LoadGeneratorOption { name: TickInterval, value: Some(Value(String("1s"))) }, LoadGeneratorOption { name: ScaleFactor, value: Some(Value(Number("1"))) }, LoadGeneratorOption { name: MaxCardinality, value: Some(Value(Number("100"))) }, LoadGeneratorOption { name: UpTo, value: Some(Value(Number("5"))) }, LoadGeneratorOption { name: AsOf, value: Some(Value(Number("5"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE lg FROM LOAD GENERATOR MARKETING ---- CREATE SOURCE lg FROM LOAD GENERATOR MARKETING => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Marketing, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Marketing, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE lg FROM LOAD GENERATOR AUCTION ---- CREATE SOURCE lg FROM LOAD GENERATOR AUCTION => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Auction, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Auction, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE lg FROM LOAD GENERATOR DATUMS ---- CREATE SOURCE lg FROM LOAD GENERATOR DATUMS => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Datums, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Datums, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE lg FROM LOAD GENERATOR TPCH ---- CREATE SOURCE lg FROM LOAD GENERATOR TPCH => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Tpch, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("lg")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Tpch, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE psychic FROM POSTGRES CONNECTION pgconn (PUBLICATION 'red') with (IGNORE KEYS 'true', TIMELINE 'timeline', TIMESTAMP INTERVAL 'interval') ---- CREATE SOURCE psychic FROM POSTGRES CONNECTION pgconn (PUBLICATION = 'red') WITH (IGNORE KEYS = 'true', TIMELINE = 'timeline', TIMESTAMP INTERVAL = 'interval') => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("psychic")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pgconn")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("red"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [CreateSourceOption { name: IgnoreKeys, value: Some(Value(String("true"))) }, CreateSourceOption { name: Timeline, value: Some(Value(String("timeline"))) }, CreateSourceOption { name: TimestampInterval, value: Some(Value(String("interval"))) }], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("psychic")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pgconn")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("red"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [CreateSourceOption { name: IgnoreKeys, value: Some(Value(String("true"))) }, CreateSourceOption { name: Timeline, value: Some(Value(String("timeline"))) }, CreateSourceOption { name: TimestampInterval, value: Some(Value(String("interval"))) }], external_references: None, progress_subsource: None }) parse-statement ALTER SYSTEM SET wal_level TO logical @@ -2660,35 +2660,35 @@ CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FO ---- CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION = 'mz_source') FOR ALL TABLES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FOR SCHEMAS (one, two); ---- CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION = 'mz_source') FOR SCHEMAS (one, two) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(SubsetSchemas([Ident("one"), Ident("two")])), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(SubsetSchemas([Ident("one"), Ident("two")])), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FOR ALL TABLES; ---- CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION = 'mz_source') FOR ALL TABLES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (TEXT COLUMNS = [foo, foo.bar, foo.bar.qux, foo.bar.qux.qax, foo.bar.qux.qax.baz], DETAILS = 'details') FOR ALL TABLES; ---- CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (TEXT COLUMNS = (foo, foo.bar, foo.bar.qux, foo.bar.qux.qax, foo.bar.qux.qax.baz), DETAILS = 'details') FOR ALL TABLES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("foo")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar"), Ident("qux")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar"), Ident("qux"), Ident("qax")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar"), Ident("qux"), Ident("qax"), Ident("baz")]))])) }, PgConfigOption { name: Details, value: Some(Value(String("details"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: TextColumns, value: Some(Sequence([UnresolvedItemName(UnresolvedItemName([Ident("foo")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar"), Ident("qux")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar"), Ident("qux"), Ident("qax")])), UnresolvedItemName(UnresolvedItemName([Ident("foo"), Ident("bar"), Ident("qux"), Ident("qax"), Ident("baz")]))])) }, PgConfigOption { name: Details, value: Some(Value(String("details"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FOR TABLES (foo, bar as qux, baz into zop); ---- CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION = 'mz_source') FOR TABLES (foo, bar AS qux, baz AS zop) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(SubsetTables([CreateSourceSubsource { reference: UnresolvedItemName([Ident("foo")]), subsource: None }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("bar")]), subsource: Some(UnresolvedItemName([Ident("qux")])) }, CreateSourceSubsource { reference: UnresolvedItemName([Ident("baz")]), subsource: Some(UnresolvedItemName([Ident("zop")])) }])), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(SubsetTables([ExternalReferenceExport { reference: UnresolvedItemName([Ident("foo")]), alias: None }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("bar")]), alias: Some(UnresolvedItemName([Ident("qux")])) }, ExternalReferenceExport { reference: UnresolvedItemName([Ident("baz")]), alias: Some(UnresolvedItemName([Ident("zop")])) }])), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FOR TABLES ([s1 AS foo.bar]); @@ -2716,14 +2716,14 @@ CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FO ---- CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION = 'mz_source') FOR ALL TABLES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: None }) parse-statement CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION 'mz_source') FOR ALL TABLES EXPOSE PROGRESS AS foo.bar; ---- CREATE SOURCE mz_source FROM POSTGRES CONNECTION pg (PUBLICATION = 'mz_source') FOR ALL TABLES EXPOSE PROGRESS AS foo.bar => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: Some(All), progress_subsource: Some(Deferred(UnresolvedItemName([Ident("foo"), Ident("bar")]))) }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("mz_source")]), in_cluster: None, col_names: [], connection: Postgres { connection: Name(UnresolvedItemName([Ident("pg")])), options: [PgConfigOption { name: Publication, value: Some(Value(String("mz_source"))) }] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: Some(All), progress_subsource: Some(Deferred(UnresolvedItemName([Ident("foo"), Ident("bar")]))) }) parse-statement ALTER CLUSTER foo RENAME TO joe @@ -3031,35 +3031,35 @@ CREATE SOURCE header1 FROM KAFKA CONNECTION conn (TOPIC 'test') FORMAT JSON INCL ---- CREATE SOURCE header1 FROM KAFKA CONNECTION conn (TOPIC = 'test') FORMAT JSON INCLUDE HEADERS, HEADER 'header3' AS h3, HEADER 'header5' AS h5 BYTES => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [Headers { alias: None }, Header { key: "header3", alias: Ident("h3"), use_bytes: false }, Header { key: "header5", alias: Ident("h5"), use_bytes: true }], format: Some(Bare(Json { array: false })), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header1")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [Headers { alias: None }, Header { key: "header3", alias: Ident("h3"), use_bytes: false }, Header { key: "header5", alias: Ident("h5"), use_bytes: true }], format: Some(Bare(Json { array: false })), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE header2 FROM KAFKA CONNECTION conn (TOPIC 'test') KEY FORMAT TEXT VALUE FORMAT JSON INCLUDE HEADER 'header1' AS h1, HEADER 'header2' AS h2 BYTES ENVELOPE UPSERT ---- CREATE SOURCE header2 FROM KAFKA CONNECTION conn (TOPIC = 'test') KEY FORMAT TEXT VALUE FORMAT JSON INCLUDE HEADER 'header1' AS h1, HEADER 'header2' AS h2 BYTES ENVELOPE UPSERT => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header2")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [Header { key: "header1", alias: Ident("h1"), use_bytes: false }, Header { key: "header2", alias: Ident("h2"), use_bytes: true }], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [] }), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header2")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [Header { key: "header1", alias: Ident("h1"), use_bytes: false }, Header { key: "header2", alias: Ident("h2"), use_bytes: true }], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [] }), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE header3 FROM KAFKA CONNECTION conn (TOPIC 'test') KEY FORMAT TEXT VALUE FORMAT JSON ENVELOPE UPSERT (VALUE DECODING ERRORS = (INLINE)) ---- CREATE SOURCE header3 FROM KAFKA CONNECTION conn (TOPIC = 'test') KEY FORMAT TEXT VALUE FORMAT JSON ENVELOPE UPSERT (VALUE DECODING ERRORS = (INLINE)) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header3")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [Inline { alias: None }] }), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header3")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [Inline { alias: None }] }), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE header3 FROM KAFKA CONNECTION conn (TOPIC 'test') KEY FORMAT TEXT VALUE FORMAT JSON ENVELOPE UPSERT (VALUE DECODING ERRORS = INLINE) ---- CREATE SOURCE header3 FROM KAFKA CONNECTION conn (TOPIC = 'test') KEY FORMAT TEXT VALUE FORMAT JSON ENVELOPE UPSERT (VALUE DECODING ERRORS = (INLINE)) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header3")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [Inline { alias: None }] }), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header3")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [Inline { alias: None }] }), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE header3 FROM KAFKA CONNECTION conn (TOPIC 'test') KEY FORMAT TEXT VALUE FORMAT JSON ENVELOPE UPSERT (VALUE DECODING ERRORS = INLINE AS my_col) ---- CREATE SOURCE header3 FROM KAFKA CONNECTION conn (TOPIC = 'test') KEY FORMAT TEXT VALUE FORMAT JSON ENVELOPE UPSERT (VALUE DECODING ERRORS = (INLINE AS my_col)) => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header3")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [Inline { alias: Some(Ident("my_col")) }] }), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("header3")]), in_cluster: None, col_names: [], connection: Kafka { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaSourceConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, include_metadata: [], format: Some(KeyValue { key: Text, value: Json { array: false } }), envelope: Some(Upsert { value_decode_err_policy: [Inline { alias: Some(Ident("my_col")) }] }), if_not_exists: false, key_constraint: None, with_options: [], external_references: None, progress_subsource: None }) parse-statement CREATE SOURCE header3 FROM KAFKA CONNECTION conn (TOPIC 'test') KEY FORMAT TEXT VALUE FORMAT JSON ENVELOPE UPSERT (VALUE DECODING ERRORS) @@ -3073,7 +3073,7 @@ CREATE SOURCE s FROM LOAD GENERATOR COUNTER WITH (RETAIN HISTORY FOR '1s'); ---- CREATE SOURCE s FROM LOAD GENERATOR COUNTER WITH (RETAIN HISTORY = FOR '1s') => -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("s")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Counter, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [CreateSourceOption { name: RetainHistory, value: Some(RetainHistoryFor(String("1s"))) }], referenced_subsources: None, progress_subsource: None }) +CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("s")]), in_cluster: None, col_names: [], connection: LoadGenerator { generator: Counter, options: [] }, include_metadata: [], format: None, envelope: None, if_not_exists: false, key_constraint: None, with_options: [CreateSourceOption { name: RetainHistory, value: Some(RetainHistoryFor(String("1s"))) }], external_references: None, progress_subsource: None }) parse-statement ALTER MATERIALIZED VIEW mv SET (RETAIN HISTORY FOR '3m')