From cd908770fe51c6801f394fdfe52fd31e3c537965 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Sat, 9 Dec 2023 13:22:26 -0500 Subject: [PATCH 1/5] sql,storage: improve topic metadata refresh interval Kafka sink option Drop the `MS` from `TOPIC METADATA REFRESH INTERVAL MS`. The new option is spelled `TOPIC METADATA REFRESH INTERVAL` and takes a SQL interval rather than a raw number of milliseconds. This option is unsafe, so this is not a backwards incompatible change. This commit also adjusts the implementation so that `topic_metadata_refresh_interval` is a dedicated field of a `KafkaSourceConnection`. Part of #14656. --- .../checks/all_checks/multiple_partitions.py | 2 +- src/kafka-util/src/client.rs | 6 ++ src/sql-parser/src/ast/defs/ddl.rs | 6 +- src/sql-parser/src/parser.rs | 4 +- src/sql-parser/tests/testdata/ddl | 6 +- src/sql/src/kafka_util.rs | 56 +----------------- src/sql/src/plan/statement/ddl.rs | 11 +++- src/sql/src/pure.rs | 16 +---- src/storage-types/src/sources.proto | 4 +- src/storage-types/src/sources.rs | 44 ++++++-------- src/storage/src/source/kafka.rs | 59 +++++++------------ .../kafka-sources/partition-change-before.td | 2 +- test/ssh-connection/kafka-sink.td | 2 +- test/ssh-connection/kafka-source.td | 4 +- test/testdrive/kafka-avro-sources.td | 4 +- test/testdrive/kafka-source-errors.td | 24 ++++++++ test/testdrive/kafka-time-offset.td | 22 +++---- test/testdrive/timestamps-kafka-avro-multi.td | 2 +- 18 files changed, 112 insertions(+), 162 deletions(-) diff --git a/misc/python/materialize/checks/all_checks/multiple_partitions.py b/misc/python/materialize/checks/all_checks/multiple_partitions.py index 5a31f1092f622..1cfd35dcadb90 100644 --- a/misc/python/materialize/checks/all_checks/multiple_partitions.py +++ b/misc/python/materialize/checks/all_checks/multiple_partitions.py @@ -37,7 +37,7 @@ def initialize(self) -> Testdrive: {"key1": "A${kafka-ingest.iteration}"} {"f1": "A${kafka-ingest.iteration}"} > CREATE SOURCE multiple_partitions_source - FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-multiple-partitions-topic-${testdrive.seed}', TOPIC METADATA REFRESH INTERVAL MS 500) + FROM KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-multiple-partitions-topic-${testdrive.seed}', TOPIC METADATA REFRESH INTERVAL '500ms') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE UPSERT diff --git a/src/kafka-util/src/client.rs b/src/kafka-util/src/client.rs index 01d8f79778ca7..0b414c554c060 100644 --- a/src/kafka-util/src/client.rs +++ b/src/kafka-util/src/client.rs @@ -34,6 +34,12 @@ use rdkafka::{ClientContext, Statistics, TopicPartitionList}; use tokio::runtime::Handle; use tracing::{debug, error, info, warn, Level}; +/// A reasonable default timeout when refreshing topic metadata. +// 30s may seem infrequent, but the default is 5m. More frequent metadata +// refresh rates are surprising to Kafka users, as topic partition counts hardly +// ever change in production. +pub const DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL: Duration = Duration::from_secs(30); + /// A reasonable default timeout when fetching metadata or partitions. pub const DEFAULT_FETCH_METADATA_TIMEOUT: Duration = Duration::from_secs(10); diff --git a/src/sql-parser/src/ast/defs/ddl.rs b/src/sql-parser/src/ast/defs/ddl.rs index a7f74436ea6fa..361909bbe0556 100644 --- a/src/sql-parser/src/ast/defs/ddl.rs +++ b/src/sql-parser/src/ast/defs/ddl.rs @@ -823,7 +823,7 @@ pub enum KafkaConfigOptionName { CompressionType, GroupIdPrefix, Topic, - TopicMetadataRefreshIntervalMs, + TopicMetadataRefreshInterval, StartTimestamp, StartOffset, PartitionCount, @@ -838,8 +838,8 @@ impl AstDisplay for KafkaConfigOptionName { KafkaConfigOptionName::CompressionType => "COMPRESSION TYPE", KafkaConfigOptionName::GroupIdPrefix => "GROUP ID PREFIX", KafkaConfigOptionName::Topic => "TOPIC", - KafkaConfigOptionName::TopicMetadataRefreshIntervalMs => { - "TOPIC METADATA REFRESH INTERVAL MS" + KafkaConfigOptionName::TopicMetadataRefreshInterval => { + "TOPIC METADATA REFRESH INTERVAL" } KafkaConfigOptionName::StartOffset => "START OFFSET", KafkaConfigOptionName::StartTimestamp => "START TIMESTAMP", diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index 96bac7ef4d94a..19f211ee8c7f4 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -2387,8 +2387,8 @@ impl<'a> Parser<'a> { }, TOPIC => { if self.parse_keyword(METADATA) { - self.expect_keywords(&[REFRESH, INTERVAL, MS])?; - KafkaConfigOptionName::TopicMetadataRefreshIntervalMs + self.expect_keywords(&[REFRESH, INTERVAL])?; + KafkaConfigOptionName::TopicMetadataRefreshInterval } else { KafkaConfigOptionName::Topic } diff --git a/src/sql-parser/tests/testdata/ddl b/src/sql-parser/tests/testdata/ddl index 79458599c6b65..30e077b7be443 100644 --- a/src/sql-parser/tests/testdata/ddl +++ b/src/sql-parser/tests/testdata/ddl @@ -451,11 +451,11 @@ 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 }) parse-statement -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (REPLICATION FACTOR = 7, RETENTION MS = 10000, RETENTION BYTES = 10000000000, TOPIC 'topic', GROUP ID PREFIX 'prefix', TOPIC METADATA REFRESH INTERVAL MS = 100, START OFFSET = 1, START TIMESTAMP = 1234, PARTITION COUNT = 2, COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (REPLICATION FACTOR = 7, RETENTION MS = 10000, RETENTION BYTES = 10000000000, TOPIC 'topic', GROUP ID PREFIX 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, PARTITION COUNT = 2, COMPRESSION TYPE = gzip) FORMAT BYTES ---- -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (REPLICATION FACTOR = 7, RETENTION MS = 10000, RETENTION BYTES = 10000000000, TOPIC = 'topic', GROUP ID PREFIX = 'prefix', TOPIC METADATA REFRESH INTERVAL MS = 100, START OFFSET = 1, START TIMESTAMP = 1234, PARTITION COUNT = 2, COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (REPLICATION FACTOR = 7, RETENTION MS = 10000, RETENTION BYTES = 10000000000, TOPIC = 'topic', GROUP ID PREFIX = 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, PARTITION COUNT = 2, COMPRESSION TYPE = gzip) FORMAT BYTES => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: ReplicationFactor, value: Some(Value(Number("7"))) }, KafkaConfigOption { name: RetentionMs, value: Some(Value(Number("10000"))) }, KafkaConfigOption { name: RetentionBytes, value: Some(Value(Number("10000000000"))) }, KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaConfigOption { name: GroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaConfigOption { name: TopicMetadataRefreshIntervalMs, value: Some(Value(Number("100"))) }, KafkaConfigOption { name: StartOffset, value: Some(Value(Number("1"))) }, KafkaConfigOption { name: StartTimestamp, value: Some(Value(Number("1234"))) }, KafkaConfigOption { name: PartitionCount, value: Some(Value(Number("2"))) }, KafkaConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: ReplicationFactor, value: Some(Value(Number("7"))) }, KafkaConfigOption { name: RetentionMs, value: Some(Value(Number("10000"))) }, KafkaConfigOption { name: RetentionBytes, value: Some(Value(Number("10000000000"))) }, KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaConfigOption { name: GroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaConfigOption { name: TopicMetadataRefreshInterval, value: Some(Value(Number("100"))) }, KafkaConfigOption { name: StartOffset, value: Some(Value(Number("1"))) }, KafkaConfigOption { name: StartTimestamp, value: Some(Value(Number("1234"))) }, KafkaConfigOption { name: PartitionCount, value: Some(Value(Number("2"))) }, KafkaConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [] }) parse-statement CREATE SINK FROM bar INTO KAFKA CONNECTION baz diff --git a/src/sql/src/kafka_util.rs b/src/sql/src/kafka_util.rs index 36d0f27b6fe31..1878b914351c4 100644 --- a/src/sql/src/kafka_util.rs +++ b/src/sql/src/kafka_util.rs @@ -9,15 +9,13 @@ //! Provides parsing and convenience functions for working with Kafka from the `sql` package. -use std::collections::BTreeMap; use std::sync::Arc; use anyhow::bail; -use mz_kafka_util::client::DEFAULT_FETCH_METADATA_TIMEOUT; +use mz_kafka_util::client::{DEFAULT_FETCH_METADATA_TIMEOUT, DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL}; use mz_ore::task; use mz_sql_parser::ast::display::AstDisplay; use mz_sql_parser::ast::{AstInfo, KafkaConfigOption, KafkaConfigOptionName}; -use mz_storage_types::connections::StringOrSecret; use mz_storage_types::sinks::KafkaSinkCompressionType; use rdkafka::consumer::{BaseConsumer, Consumer, ConsumerContext}; use rdkafka::{Offset, TopicPartitionList}; @@ -48,7 +46,7 @@ pub fn validate_options_for_context( CompressionType => Some(Sink), GroupIdPrefix => None, Topic => None, - TopicMetadataRefreshIntervalMs => None, + TopicMetadataRefreshInterval => None, StartTimestamp => Some(Source), StartOffset => Some(Source), PartitionCount => Some(Sink), @@ -80,7 +78,7 @@ generate_extracted_config!( ), (GroupIdPrefix, String), (Topic, String), - (TopicMetadataRefreshIntervalMs, i32), + (TopicMetadataRefreshInterval, Duration, Default(DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL)), (StartTimestamp, i64), (StartOffset, Vec), (PartitionCount, i32, Default(-1)), @@ -117,54 +115,6 @@ impl ImpliedValue for KafkaSinkCompressionType { } } -/// The config options we expect to pass along when connecting to librdkafka. -/// -/// Note that these are meant to be disjoint from the options we permit being -/// set on Kafka connections (CREATE CONNECTION), i.e. these are meant to be -/// per-client options (CREATE SOURCE, CREATE SINK). -#[derive(Debug)] -pub struct LibRdKafkaConfig(pub BTreeMap); - -impl TryFrom<&KafkaConfigOptionExtracted> for LibRdKafkaConfig { - type Error = PlanError; - // We are in a macro, so allow calling a closure immediately. - #[allow(clippy::redundant_closure_call)] - fn try_from( - KafkaConfigOptionExtracted { - topic_metadata_refresh_interval_ms, - .. - }: &KafkaConfigOptionExtracted, - ) -> Result { - let mut o = BTreeMap::new(); - - macro_rules! fill_options { - // Values that are not option can just be wrapped in some before being passed to the macro - ($v:expr, $s:expr) => { - if let Some(v) = $v { - o.insert($s.to_string(), StringOrSecret::String(v.to_string())); - } - }; - ($v:expr, $s:expr, $check:expr, $err:expr) => { - if let Some(v) = $v { - if !$check(v) { - sql_bail!($err); - } - o.insert($s.to_string(), StringOrSecret::String(v.to_string())); - } - }; - } - - fill_options!( - topic_metadata_refresh_interval_ms, - "topic.metadata.refresh.interval.ms", - |i: &i32| { 0 <= *i && *i <= 3_600_000 }, - "TOPIC METADATA REFRESH INTERVAL MS must be within [0, 3,600,000]" - ); - - Ok(LibRdKafkaConfig(o)) - } -} - /// An enum that represents start offsets for a kafka consumer. #[derive(Debug)] pub enum KafkaStartOffsetType { diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index a06e33a262c81..7b18c91743dac 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -648,8 +648,6 @@ pub fn plan_create_source( let optional_start_offset = Option::::try_from(&extracted_options)?; - let connection_options = kafka_util::LibRdKafkaConfig::try_from(&extracted_options)?.0; - let topic = extracted_options .topic .expect("validated exists during purification"); @@ -675,6 +673,13 @@ pub fn plan_create_source( sql_bail!("START OFFSET is not supported with ENVELOPE {}", envelope) } + let topic_metadata_refresh_interval = extracted_options.topic_metadata_refresh_interval; + if topic_metadata_refresh_interval > Duration::from_secs(60 * 60) { + // This is a librdkafka-enforced restriction that, if violated, + // would result in a runtime error for the source. + sql_bail!("TOPIC METADATA REFRESH INTERVAL cannot be greater than 1 hour"); + } + let encoding = get_encoding(scx, format, &envelope, Some(connection))?; if !include_metadata.is_empty() @@ -742,8 +747,8 @@ pub fn plan_create_source( topic, start_offsets, group_id_prefix, + topic_metadata_refresh_interval, metadata_columns, - connection_options, }; let connection = GenericSourceConnection::Kafka(connection); diff --git a/src/sql/src/pure.rs b/src/sql/src/pure.rs index 0299b6e2b8afc..335d093f17eee 100644 --- a/src/sql/src/pure.rs +++ b/src/sql/src/pure.rs @@ -290,12 +290,12 @@ async fn purify_create_sink( connection: KafkaConnection { connection, - options, + options: _, }, key: _, } => { let scx = StatementContext::new(None, &catalog); - let mut connection = { + let connection = { let item = scx.get_item_by_resolved_name(connection)?; // Get Kafka connection match item.connection()? { @@ -309,12 +309,6 @@ async fn purify_create_sink( } }; - let extracted_options: KafkaConfigOptionExtracted = options.clone().try_into()?; - - for (k, v) in kafka_util::LibRdKafkaConfig::try_from(&extracted_options)?.0 { - connection.options.insert(k, v); - } - let client: AdminClient<_> = connection .create_with_context( storage_configuration, @@ -458,7 +452,7 @@ async fn purify_create_source( } let scx = StatementContext::new(None, &catalog); - let mut connection = { + let connection = { let item = scx.get_item_by_resolved_name(connection)?; // Get Kafka connection match item.connection()? { @@ -477,10 +471,6 @@ async fn purify_create_source( let offset_type = Option::::try_from(&extracted_options)?; - for (k, v) in kafka_util::LibRdKafkaConfig::try_from(&extracted_options)?.0 { - connection.options.insert(k, v); - } - let topic = extracted_options .topic .ok_or(KafkaSourcePurificationError::ConnectionMissingTopic)?; diff --git a/src/storage-types/src/sources.proto b/src/storage-types/src/sources.proto index 2b41ee2e3835b..6132dee04fd13 100644 --- a/src/storage-types/src/sources.proto +++ b/src/storage-types/src/sources.proto @@ -150,14 +150,14 @@ message ProtoKafkaMetadataColumn { } message ProtoKafkaSourceConnection { - reserved 5, 6, 7, 8, 9, 10, 12; + reserved 5, 6, 7, 8, 9, 10, 12, 14; mz_storage_types.connections.ProtoKafkaConnection connection = 1; mz_repr.global_id.ProtoGlobalId connection_id = 13; string topic = 2; map start_offsets = 3; optional string group_id_prefix = 4; repeated ProtoKafkaMetadataColumn metadata_columns = 11; - map connection_options = 14; + mz_proto.ProtoDuration topic_metadata_refresh_interval = 15; } message ProtoSourceDesc { diff --git a/src/storage-types/src/sources.rs b/src/storage-types/src/sources.rs index e5ed5a160acee..99ee1c526fea1 100644 --- a/src/storage-types/src/sources.rs +++ b/src/storage-types/src/sources.rs @@ -50,7 +50,7 @@ use crate::connections::inline::{ ConnectionAccess, ConnectionResolver, InlinedConnection, IntoInlineConnection, ReferencedConnection, }; -use crate::connections::{ConnectionContext, StringOrSecret}; +use crate::connections::ConnectionContext; use crate::controller::{CollectionMetadata, StorageError}; use crate::errors::{DataflowError, ProtoDataflowError}; use crate::instances::StorageInstanceId; @@ -1397,9 +1397,7 @@ pub struct KafkaSourceConnection { pub start_offsets: BTreeMap, pub group_id_prefix: Option, pub metadata_columns: Vec<(String, KafkaMetadataKind)>, - /// Additional options that need to be set on the connection whenever it's - /// inlined. - pub connection_options: BTreeMap, + pub topic_metadata_refresh_interval: Duration, } impl IntoInlineConnection @@ -1413,20 +1411,16 @@ impl IntoInlineConnection start_offsets, group_id_prefix, metadata_columns, - connection_options, + topic_metadata_refresh_interval, } = self; - - let mut connection = r.resolve_connection(connection).unwrap_kafka(); - connection.options.extend(connection_options); - KafkaSourceConnection { - connection, + connection: r.resolve_connection(connection).unwrap_kafka(), connection_id, topic, start_offsets, group_id_prefix, metadata_columns, - connection_options: BTreeMap::default(), + topic_metadata_refresh_interval, } } } @@ -1536,7 +1530,7 @@ impl crate::AlterCompatible for KafkaSourceConnection { start_offsets, group_id_prefix, metadata_columns, - connection_options, + topic_metadata_refresh_interval, } = self; let compatibility_checks = [ @@ -1549,8 +1543,8 @@ impl crate::AlterCompatible for KafkaSourceConnection { "metadata_columns", ), ( - connection_options == &other.connection_options, - "connection_options", + topic_metadata_refresh_interval == &other.topic_metadata_refresh_interval, + "topic_metadata_refresh_interval", ), ]; @@ -1585,7 +1579,7 @@ where proptest::collection::btree_map(any::(), any::(), 1..4), any::>(), proptest::collection::vec(any::<(String, KafkaMetadataKind)>(), 0..4), - proptest::collection::btree_map(any::(), any::(), 0..4), + any::(), ) .prop_map( |( @@ -1595,7 +1589,7 @@ where start_offsets, group_id_prefix, metadata_columns, - connection_options, + topic_metadata_refresh_interval, )| KafkaSourceConnection { connection, connection_id, @@ -1603,7 +1597,7 @@ where start_offsets, group_id_prefix, metadata_columns, - connection_options, + topic_metadata_refresh_interval, }, ) .boxed() @@ -1626,11 +1620,9 @@ impl RustType for KafkaSourceConnection for KafkaSourceConnection>()?, + topic_metadata_refresh_interval: proto + .topic_metadata_refresh_interval + .into_rust_if_some("ProtoKafkaSourceConnection::topic_metadata_refresh_interval")?, }) } } diff --git a/src/storage/src/source/kafka.rs b/src/storage/src/source/kafka.rs index 674451a2ae4d4..71961b2023a0c 100644 --- a/src/storage/src/source/kafka.rs +++ b/src/storage/src/source/kafka.rs @@ -29,7 +29,6 @@ use mz_ore::thread::{JoinHandleExt, UnparkOnDropHandle}; use mz_repr::adt::timestamp::CheckedTimestamp; use mz_repr::{adt::jsonb::Jsonb, Datum, Diff, GlobalId, Row}; use mz_ssh_util::tunnel::SshTunnelStatus; -use mz_storage_types::connections::StringOrSecret; use mz_storage_types::errors::ContextCreationError; use mz_storage_types::sources::{ KafkaMetadataKind, KafkaSourceConnection, MzOffset, SourceTimestamp, @@ -164,9 +163,21 @@ impl SourceRender for KafkaSourceConnection { let button = builder.build(move |caps| async move { let [mut data_cap, mut progress_cap, health_cap]: [_; 3] = caps.try_into().unwrap(); + let group_id = self.group_id(&config.config.connection_context, config.id); + let KafkaSourceConnection { + connection, + topic, + topic_metadata_refresh_interval, + start_offsets, + metadata_columns, + // Exhaustive match protects against forgetting to apply an + // option. Ignored fields are justified below. + connection_id: _, // not needed here + group_id_prefix: _, // used above via `self.group_id` + } = self; + // Start offsets is a map from partition to the next offset to read from. - let mut start_offsets: BTreeMap<_, i64> = self - .start_offsets + let mut start_offsets: BTreeMap<_, i64> = start_offsets .clone() .into_iter() .filter(|(pid, _offset)| config.responsible_for(pid)) @@ -212,10 +223,6 @@ impl SourceRender for KafkaSourceConnection { "instantiating Kafka source reader at offsets {start_offsets:?}" ); - let group_id = self.group_id(&config.config.connection_context, config.id); - let KafkaSourceConnection { - connection, topic, .. - } = self; let (stats_tx, stats_rx) = crossbeam_channel::unbounded(); let health_status = Arc::new(Mutex::new(Default::default())); let notificator = Arc::new(Notify::new()); @@ -235,18 +242,9 @@ impl SourceRender for KafkaSourceConnection { // Always begin ingest at 0 when restarted, even if Kafka // contains committed consumer read offsets "auto.offset.reset" => "earliest".into(), - // How often to refresh metadata from the Kafka broker. This - // can have a minor impact on startup latency and latency - // after adding a new partition, as the metadata for a - // partition must be fetched before we can retrieve data - // from it. We try to manually trigger metadata fetches when - // it makes sense, but if those manual fetches fail, this is - // the interval at which we retry. - // - // 30s may seem low, but the default is 5m. More frequent - // metadata refresh rates are surprising to Kafka users, as - // topic partition counts hardly ever change in production. - "topic.metadata.refresh.interval.ms" => "30000".into(), // 30s + // Use the user-configured topic metadata refresh + // interval. + "topic.metadata.refresh.interval.ms" => topic_metadata_refresh_interval.as_millis().to_string(), // TODO: document the rationale for this. "fetch.message.max.bytes" => "134217728".into(), // Consumer group ID. librdkafka requires this, and we use @@ -318,23 +316,11 @@ impl SourceRender for KafkaSourceConnection { let partition_info = Arc::downgrade(&partition_info); let topic = topic.clone(); let consumer = Arc::clone(&consumer); - let metadata_refresh_interval = connection - .options - .get("topic.metadata.refresh.interval.ms") - // Safe conversion: statement::extract_config enforces that option is a value - // between 0 and 3600000 - .map(|s| match s { - StringOrSecret::String(s) => Duration::from_millis(s.parse().unwrap()), - StringOrSecret::Secret(_) => unreachable!(), - }) - // By default, rdkafka will check for updated metadata every five minutes: - // https://github.com/edenhill/librdkafka/blob/master/CONFIGURATION.md - .unwrap_or_else(|| Duration::from_secs(15)); // We want a fairly low ceiling on our polling frequency, since we rely // on this heartbeat to determine the health of our Kafka connection. - let metadata_refresh_frequency = - metadata_refresh_interval.min(Duration::from_secs(60)); + let topic_metadata_refresh_interval = + topic_metadata_refresh_interval.min(Duration::from_secs(60)); let status_report = Arc::clone(&health_status); @@ -345,7 +331,7 @@ impl SourceRender for KafkaSourceConnection { source_id = config.id.to_string(), worker_id = config.worker_id, num_workers = config.worker_count, - refresh_frequency =? metadata_refresh_frequency, + poll_interval =? poll_interval, "kafka metadata thread: starting..." ); while let Some(partition_info) = partition_info.upgrade() { @@ -397,7 +383,7 @@ impl SourceRender for KafkaSourceConnection { } } } - thread::park_timeout(metadata_refresh_frequency); + thread::park_timeout(topic_metadata_refresh_interval); } info!( source_id = config.id.to_string(), @@ -425,8 +411,7 @@ impl SourceRender for KafkaSourceConnection { start_offsets, stats_rx, partition_info, - metadata_columns: self - .metadata_columns + metadata_columns: metadata_columns .into_iter() .map(|(_name, kind)| kind) .collect(), diff --git a/test/persistence/kafka-sources/partition-change-before.td b/test/persistence/kafka-sources/partition-change-before.td index ff2eec24e2c81..f83607bf20d45 100644 --- a/test/persistence/kafka-sources/partition-change-before.td +++ b/test/persistence/kafka-sources/partition-change-before.td @@ -45,7 +45,7 @@ $ kafka-ingest format=avro topic=partition-change key-format=avro key-schema=${k > CREATE SOURCE partition_change FROM KAFKA CONNECTION kafka_conn ( TOPIC 'testdrive-partition-change-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS = 100 + TOPIC METADATA REFRESH INTERVAL = '100ms' ) FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE UPSERT diff --git a/test/ssh-connection/kafka-sink.td b/test/ssh-connection/kafka-sink.td index a485efe5c0b50..ad29969f2212a 100644 --- a/test/ssh-connection/kafka-sink.td +++ b/test/ssh-connection/kafka-sink.td @@ -32,7 +32,7 @@ one > CREATE SOURCE non_ssh IN CLUSTER sc FROM KAFKA CONNECTION kafka_conn_non_ssh ( TOPIC 'testdrive-thetopic-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 1000 + TOPIC METADATA REFRESH INTERVAL '1s' ) FORMAT TEXT ENVELOPE NONE diff --git a/test/ssh-connection/kafka-source.td b/test/ssh-connection/kafka-source.td index 8f5bf9837afc6..88704ed43b3b2 100644 --- a/test/ssh-connection/kafka-source.td +++ b/test/ssh-connection/kafka-source.td @@ -40,7 +40,7 @@ one > CREATE SOURCE fixed_text IN CLUSTER sc FROM KAFKA CONNECTION kafka_conn_using ( TOPIC 'testdrive-thetopic-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 1000 + TOPIC METADATA REFRESH INTERVAL '1s' ) FORMAT TEXT ENVELOPE NONE @@ -48,7 +48,7 @@ one > CREATE SOURCE dynamic_text IN CLUSTER sc FROM KAFKA CONNECTION kafka_conn_dynamic ( TOPIC 'testdrive-thetopic-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 1000 + TOPIC METADATA REFRESH INTERVAL '1s' ) FORMAT TEXT ENVELOPE NONE diff --git a/test/testdrive/kafka-avro-sources.td b/test/testdrive/kafka-avro-sources.td index bdb6cb47f26cd..d2ceda62184d0 100644 --- a/test/testdrive/kafka-avro-sources.td +++ b/test/testdrive/kafka-avro-sources.td @@ -247,7 +247,7 @@ $ kafka-ingest format=avro topic=non-dbz-data-varying-partition schema=${non-dbz > CREATE SOURCE non_dbz_data_varying_partition FROM KAFKA CONNECTION kafka_conn ( TOPIC 'testdrive-non-dbz-data-varying-partition-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS=10, + TOPIC METADATA REFRESH INTERVAL = '10ms', START OFFSET=[1] ) FORMAT AVRO USING SCHEMA '${non-dbz-schema}' @@ -275,7 +275,7 @@ a b > CREATE SOURCE non_dbz_data_varying_partition_2 FROM KAFKA CONNECTION kafka_conn ( TOPIC 'testdrive-non-dbz-data-varying-partition-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS=10, + TOPIC METADATA REFRESH INTERVAL = '10ms', START OFFSET=[1,1] ) FORMAT AVRO USING SCHEMA '${non-dbz-schema}' diff --git a/test/testdrive/kafka-source-errors.td b/test/testdrive/kafka-source-errors.td index dd3c6d2d792af..2bf7401b4b69e 100644 --- a/test/testdrive/kafka-source-errors.td +++ b/test/testdrive/kafka-source-errors.td @@ -9,6 +9,10 @@ # Test that Kafka sources with no format are disallowed. +$ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} +ALTER SYSTEM SET enable_create_source_denylist_with_options = true +ALTER SYSTEM SET enable_kafka_config_denylist_options = true + > CREATE CONNECTION kafka_conn TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT); ! CREATE SOURCE s @@ -44,3 +48,23 @@ contains: FOR ALL TABLES is only valid for multi-output sources FROM KAFKA CONNECTION kafka_conn (TOPIC 'any_topic') FORMAT BYTES FOR TABLES (t1); contains: FOR TABLES (t1) is only valid for multi-output sources + +# Ensure the `TOPIC METADATA REFRESH INTERVAL` rejects too large and too +# small values. +! CREATE SOURCE bad_topic_metadata_refresh_interval + FROM KAFKA CONNECTION kafka_conn ( + TOPIC 'testdrive-thetopic-${testdrive.seed}', + TOPIC METADATA REFRESH INTERVAL '-30s' + ) + FORMAT TEXT + ENVELOPE NONE +contains:cannot convert negative interval to duration + +! CREATE SOURCE bad_topic_metadata_refresh_interval + FROM KAFKA CONNECTION kafka_conn ( + TOPIC 'testdrive-thetopic-${testdrive.seed}', + TOPIC METADATA REFRESH INTERVAL '1hr 1ms' + ) + FORMAT TEXT + ENVELOPE NONE +contains:TOPIC METADATA REFRESH INTERVAL cannot be greater than 1 hour diff --git a/test/testdrive/kafka-time-offset.td b/test/testdrive/kafka-time-offset.td index 4abf36cce1971..9c0b6d9a3c981 100644 --- a/test/testdrive/kafka-time-offset.td +++ b/test/testdrive/kafka-time-offset.td @@ -74,7 +74,7 @@ grape:grape > CREATE SOURCE append_time_offset_1 FROM KAFKA CONNECTION kafka_conn ( - TOPIC METADATA REFRESH INTERVAL MS=10, + TOPIC METADATA REFRESH INTERVAL = '10ms', START TIMESTAMP=1, TOPIC 'testdrive-t1-${testdrive.seed}' ) @@ -83,7 +83,7 @@ grape:grape > CREATE SOURCE append_time_offset_2 FROM KAFKA CONNECTION kafka_conn ( - TOPIC METADATA REFRESH INTERVAL MS=10, + TOPIC METADATA REFRESH INTERVAL = '10ms', START TIMESTAMP=2, TOPIC 'testdrive-t1-${testdrive.seed}' ) @@ -92,7 +92,7 @@ grape:grape > CREATE SOURCE append_time_offset_3 FROM KAFKA CONNECTION kafka_conn ( - TOPIC METADATA REFRESH INTERVAL MS=10, + TOPIC METADATA REFRESH INTERVAL = '10ms', START TIMESTAMP=3, TOPIC 'testdrive-t1-${testdrive.seed}' ) @@ -101,7 +101,7 @@ grape:grape > CREATE SOURCE append_time_offset_4 FROM KAFKA CONNECTION kafka_conn ( - TOPIC METADATA REFRESH INTERVAL MS=10, + TOPIC METADATA REFRESH INTERVAL = '10ms', START TIMESTAMP=4, TOPIC 'testdrive-t1-${testdrive.seed}' ) @@ -110,7 +110,7 @@ grape:grape > CREATE SOURCE append_time_offset_5 FROM KAFKA CONNECTION kafka_conn ( - TOPIC METADATA REFRESH INTERVAL MS=10, + TOPIC METADATA REFRESH INTERVAL = '10ms', START TIMESTAMP=5, TOPIC 'testdrive-t1-${testdrive.seed}' ) @@ -204,7 +204,7 @@ grape:grape FROM KAFKA CONNECTION kafka_conn ( START TIMESTAMP=0, TOPIC 'testdrive-t2-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 10 + TOPIC METADATA REFRESH INTERVAL '10ms' ) KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE OFFSET @@ -214,7 +214,7 @@ grape:grape FROM KAFKA CONNECTION kafka_conn ( START TIMESTAMP 1, TOPIC 'testdrive-t2-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 10 + TOPIC METADATA REFRESH INTERVAL '10ms' ) KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE OFFSET @@ -224,7 +224,7 @@ grape:grape FROM KAFKA CONNECTION kafka_conn ( START TIMESTAMP 2, TOPIC 'testdrive-t2-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 10 + TOPIC METADATA REFRESH INTERVAL '10ms' ) KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE OFFSET @@ -234,7 +234,7 @@ grape:grape FROM KAFKA CONNECTION kafka_conn ( START TIMESTAMP 3, TOPIC 'testdrive-t2-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 10 + TOPIC METADATA REFRESH INTERVAL '10ms' ) KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE OFFSET @@ -244,7 +244,7 @@ grape:grape FROM KAFKA CONNECTION kafka_conn ( START TIMESTAMP 4, TOPIC 'testdrive-t2-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 10 + TOPIC METADATA REFRESH INTERVAL '10ms' ) KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE OFFSET @@ -254,7 +254,7 @@ grape:grape FROM KAFKA CONNECTION kafka_conn ( START TIMESTAMP 5, TOPIC 'testdrive-t2-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS 10 + TOPIC METADATA REFRESH INTERVAL '10ms' ) KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE OFFSET diff --git a/test/testdrive/timestamps-kafka-avro-multi.td b/test/testdrive/timestamps-kafka-avro-multi.td index 6e844bd9d50d9..5c3f0c709f775 100644 --- a/test/testdrive/timestamps-kafka-avro-multi.td +++ b/test/testdrive/timestamps-kafka-avro-multi.td @@ -36,7 +36,7 @@ $ kafka-create-topic topic=data2 partitions=2 > CREATE SOURCE data_rt FROM KAFKA CONNECTION kafka_conn ( TOPIC 'testdrive-data-${testdrive.seed}', - TOPIC METADATA REFRESH INTERVAL MS=50 + TOPIC METADATA REFRESH INTERVAL = '50ms' ) FORMAT AVRO USING SCHEMA '${schema}' From c1b962cf242071f36c5723a7535ead87347b5d94 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Sat, 9 Dec 2023 18:52:06 -0500 Subject: [PATCH 2/5] sql,storage: remove code to set Kafka sink topic options This commit removes the following options that configure the topic created for the Kafka sink: * `PARTITION COUNT` * `REPLICATION FACTOR` * `RETENTION MS` * `RETENTION BYTES` These options are all behind unsafe mode right now, so this is a backwards compatible change. The rationale: there are semantic questions about how these options should behave if the Kafka topic already exists. They also should be named `TOPIC PARTITION COUNT`, `TOPIC REPLICATION FACTOR`, etc., and equally available for the progress topic on the Kafka connection. Rather than spending the time trying to fix them now, just remove them, to keep the code simpler. Fix #14656. --- src/sql-lexer/src/keywords.txt | 2 -- src/sql-parser/src/ast/defs/ddl.rs | 8 ----- src/sql-parser/src/parser.rs | 30 +++------------- src/sql-parser/tests/testdata/ddl | 6 ++-- src/sql/src/kafka_util.rs | 20 +++++------ src/sql/src/plan/statement/ddl.rs | 35 +----------------- src/storage-client/src/sink.rs | 56 +++++++++++++++++++---------- src/storage-types/src/sinks.proto | 10 +----- src/storage-types/src/sinks.rs | 48 ------------------------- test/kafka-multi-broker/01-init.td | 2 +- test/testdrive/kafka-sink-errors.td | 56 ----------------------------- test/testdrive/kafka-sinks.td | 50 ++------------------------ 12 files changed, 60 insertions(+), 263 deletions(-) diff --git a/src/sql-lexer/src/keywords.txt b/src/sql-lexer/src/keywords.txt index 16c7e31891873..bc1f6e8771e69 100644 --- a/src/sql-lexer/src/keywords.txt +++ b/src/sql-lexer/src/keywords.txt @@ -229,7 +229,6 @@ Minutes Mode Month Months -Ms Mutually Name Names @@ -307,7 +306,6 @@ Replication Reset Respect Restrict -Retention Return Returning Revoke diff --git a/src/sql-parser/src/ast/defs/ddl.rs b/src/sql-parser/src/ast/defs/ddl.rs index 361909bbe0556..a901cdcd01825 100644 --- a/src/sql-parser/src/ast/defs/ddl.rs +++ b/src/sql-parser/src/ast/defs/ddl.rs @@ -826,10 +826,6 @@ pub enum KafkaConfigOptionName { TopicMetadataRefreshInterval, StartTimestamp, StartOffset, - PartitionCount, - ReplicationFactor, - RetentionMs, - RetentionBytes, } impl AstDisplay for KafkaConfigOptionName { @@ -843,10 +839,6 @@ impl AstDisplay for KafkaConfigOptionName { } KafkaConfigOptionName::StartOffset => "START OFFSET", KafkaConfigOptionName::StartTimestamp => "START TIMESTAMP", - KafkaConfigOptionName::PartitionCount => "PARTITION COUNT", - KafkaConfigOptionName::ReplicationFactor => "REPLICATION FACTOR", - KafkaConfigOptionName::RetentionBytes => "RETENTION BYTES", - KafkaConfigOptionName::RetentionMs => "RETENTION MS", }) } } diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index 19f211ee8c7f4..ad22a3e3acaf3 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -2354,16 +2354,7 @@ impl<'a> Parser<'a> { } fn parse_kafka_config_option(&mut self) -> Result, ParserError> { - let name = match self.expect_one_of_keywords(&[ - COMPRESSION, - GROUP, - PARTITION, - REPLICATION, - RETENTION, - SNAPSHOT, - START, - TOPIC, - ])? { + let name = match self.expect_one_of_keywords(&[COMPRESSION, GROUP, START, TOPIC])? { COMPRESSION => { self.expect_keyword(TYPE)?; KafkaConfigOptionName::CompressionType @@ -2372,17 +2363,9 @@ impl<'a> Parser<'a> { self.expect_keywords(&[ID, PREFIX])?; KafkaConfigOptionName::GroupIdPrefix } - PARTITION => { - self.expect_keyword(COUNT)?; - KafkaConfigOptionName::PartitionCount - } - REPLICATION => { - self.expect_keyword(FACTOR)?; - KafkaConfigOptionName::ReplicationFactor - } - RETENTION => match self.expect_one_of_keywords(&[BYTES, MS])? { - BYTES => KafkaConfigOptionName::RetentionBytes, - MS => KafkaConfigOptionName::RetentionMs, + START => match self.expect_one_of_keywords(&[OFFSET, TIMESTAMP])? { + OFFSET => KafkaConfigOptionName::StartOffset, + TIMESTAMP => KafkaConfigOptionName::StartTimestamp, _ => unreachable!(), }, TOPIC => { @@ -2393,11 +2376,6 @@ impl<'a> Parser<'a> { KafkaConfigOptionName::Topic } } - START => match self.expect_one_of_keywords(&[OFFSET, TIMESTAMP])? { - OFFSET => KafkaConfigOptionName::StartOffset, - TIMESTAMP => KafkaConfigOptionName::StartTimestamp, - _ => unreachable!(), - }, _ => unreachable!(), }; Ok(KafkaConfigOption { diff --git a/src/sql-parser/tests/testdata/ddl b/src/sql-parser/tests/testdata/ddl index 30e077b7be443..3b4078ad04603 100644 --- a/src/sql-parser/tests/testdata/ddl +++ b/src/sql-parser/tests/testdata/ddl @@ -451,11 +451,11 @@ 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 }) parse-statement -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (REPLICATION FACTOR = 7, RETENTION MS = 10000, RETENTION BYTES = 10000000000, TOPIC 'topic', GROUP ID PREFIX 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, PARTITION COUNT = 2, COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic', GROUP ID PREFIX 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, COMPRESSION TYPE = gzip) FORMAT BYTES ---- -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (REPLICATION FACTOR = 7, RETENTION MS = 10000, RETENTION BYTES = 10000000000, TOPIC = 'topic', GROUP ID PREFIX = 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, PARTITION COUNT = 2, COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic', GROUP ID PREFIX = 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, COMPRESSION TYPE = gzip) FORMAT BYTES => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: ReplicationFactor, value: Some(Value(Number("7"))) }, KafkaConfigOption { name: RetentionMs, value: Some(Value(Number("10000"))) }, KafkaConfigOption { name: RetentionBytes, value: Some(Value(Number("10000000000"))) }, KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaConfigOption { name: GroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaConfigOption { name: TopicMetadataRefreshInterval, value: Some(Value(Number("100"))) }, KafkaConfigOption { name: StartOffset, value: Some(Value(Number("1"))) }, KafkaConfigOption { name: StartTimestamp, value: Some(Value(Number("1234"))) }, KafkaConfigOption { name: PartitionCount, value: Some(Value(Number("2"))) }, KafkaConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaConfigOption { name: GroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaConfigOption { name: TopicMetadataRefreshInterval, value: Some(Value(Number("100"))) }, KafkaConfigOption { name: StartOffset, value: Some(Value(Number("1"))) }, KafkaConfigOption { name: StartTimestamp, value: Some(Value(Number("1234"))) }, KafkaConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [] }) parse-statement CREATE SINK FROM bar INTO KAFKA CONNECTION baz diff --git a/src/sql/src/kafka_util.rs b/src/sql/src/kafka_util.rs index 1878b914351c4..04848659e29b5 100644 --- a/src/sql/src/kafka_util.rs +++ b/src/sql/src/kafka_util.rs @@ -12,7 +12,9 @@ use std::sync::Arc; use anyhow::bail; -use mz_kafka_util::client::{DEFAULT_FETCH_METADATA_TIMEOUT, DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL}; +use mz_kafka_util::client::{ + DEFAULT_FETCH_METADATA_TIMEOUT, DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL, +}; use mz_ore::task; use mz_sql_parser::ast::display::AstDisplay; use mz_sql_parser::ast::{AstInfo, KafkaConfigOption, KafkaConfigOptionName}; @@ -49,10 +51,6 @@ pub fn validate_options_for_context( TopicMetadataRefreshInterval => None, StartTimestamp => Some(Source), StartOffset => Some(Source), - PartitionCount => Some(Sink), - ReplicationFactor => Some(Sink), - RetentionBytes => Some(Sink), - RetentionMs => Some(Sink), }; if limited_to_context.is_some() && limited_to_context != Some(context) { bail!( @@ -78,13 +76,13 @@ generate_extracted_config!( ), (GroupIdPrefix, String), (Topic, String), - (TopicMetadataRefreshInterval, Duration, Default(DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL)), + ( + TopicMetadataRefreshInterval, + Duration, + Default(DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL) + ), (StartTimestamp, i64), - (StartOffset, Vec), - (PartitionCount, i32, Default(-1)), - (ReplicationFactor, i32, Default(-1)), - (RetentionBytes, i64), - (RetentionMs, i64) + (StartOffset, Vec) ); impl TryFromValue for KafkaSinkCompressionType { diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index 7b18c91743dac..65647bb591e49 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -46,8 +46,7 @@ use mz_sql_parser::ident; use mz_storage_types::connections::inline::{ConnectionAccess, ReferencedConnection}; use mz_storage_types::connections::Connection; use mz_storage_types::sinks::{ - KafkaSinkConnection, KafkaSinkConnectionRetention, KafkaSinkFormat, SinkEnvelope, - StorageSinkConnection, + KafkaSinkConnection, KafkaSinkFormat, SinkEnvelope, StorageSinkConnection, }; use mz_storage_types::sources::encoding::{ included_column_desc, AvroEncoding, ColumnSpec, CsvEncoding, DataEncoding, DataEncodingInner, @@ -2546,11 +2545,7 @@ fn kafka_sink_builder( let KafkaConfigOptionExtracted { topic, - partition_count, - replication_factor, compression_type, - retention_ms, - retention_bytes, .. } = extracted_options; @@ -2649,43 +2644,15 @@ fn kafka_sink_builder( None => bail_unsupported!("sink without format"), }; - if partition_count == 0 || partition_count < -1 { - sql_bail!( - "PARTION COUNT for sink topics must be a positive integer or -1 for broker default" - ); - } - - if replication_factor == 0 || replication_factor < -1 { - sql_bail!( - "REPLICATION FACTOR for sink topics must be a positive integer or -1 for broker default" - ); - } - - if retention_ms.unwrap_or(0) < -1 { - sql_bail!("RETENTION MS for sink topics must be greater than or equal to -1"); - } - - if retention_bytes.unwrap_or(0) < -1 { - sql_bail!("RETENTION BYTES for sink topics must be greater than or equal to -1"); - } - - let retention = KafkaSinkConnectionRetention { - duration: retention_ms, - bytes: retention_bytes, - }; - Ok(StorageSinkConnection::Kafka(KafkaSinkConnection { connection_id, connection: connection_id, format, topic: topic_name, - partition_count, - replication_factor, fuel: 10000, relation_key_indices, key_desc_and_indices, value_desc, - retention, compression_type, })) } diff --git a/src/storage-client/src/sink.rs b/src/storage-client/src/sink.rs index c85b23cdc8026..f123f4c155552 100644 --- a/src/storage-client/src/sink.rs +++ b/src/storage-client/src/sink.rs @@ -21,7 +21,7 @@ use mz_ore::task; use mz_repr::{GlobalId, Timestamp}; use mz_storage_types::configuration::StorageConfiguration; use mz_storage_types::errors::{ContextCreationError, ContextCreationErrorExt}; -use mz_storage_types::sinks::{KafkaSinkConnection, KafkaSinkConnectionRetention}; +use mz_storage_types::sinks::KafkaSinkConnection; use rdkafka::admin::{AdminClient, AdminOptions, NewTopic, ResourceSpecifier, TopicReplication}; use rdkafka::consumer::{BaseConsumer, Consumer, ConsumerContext}; use rdkafka::error::KafkaError; @@ -161,6 +161,15 @@ async fn discover_topic_configs( } /// Configuration of a topic created by `ensure_kafka_topic`. +// TODO(benesch): some fields here use `-1` to indicate broker default, while +// others use `None` to indicate broker default and `-1` has a different special +// meaning. This is not very Rusty and very easy to get wrong. We should adjust +// the API for this method to use types that are safer and have more obvious +// meaning. For example, `partition_count` could have type +// `Option>`, where `-1` is prohibited as a value and the Rustier +// `None` value represents the broker default (n.b.: `u32` is not a good choice +// because the maximum number of Kafka partitions is `i32::MAX`, not +// `u32::MAX`). #[derive(Debug, Clone)] pub struct TopicConfig { /// The number of partitions to create. @@ -179,9 +188,18 @@ pub struct TopicConfig { #[derive(Debug, Clone)] pub enum TopicCleanupPolicy { /// Clean up the topic using a time and/or size based retention policies. - /// - /// Use `-1` to indicate infinite retention. - Retention(KafkaSinkConnectionRetention), + Retention { + /// A time-based retention policy. + /// + /// `None` indicates broker default. `Some(-1)` indicates infinite + /// retention. + ms: Option, + /// A size based retention policy. + /// + /// `None` indicates broker default. `Some(-1)` indicates infinite + /// retention. + bytes: Option, + }, /// Clean up the topic using key-based compaction. Compaction, } @@ -231,18 +249,18 @@ where TopicReplication::Fixed(replication_factor), ); - let retention_ms_slot; - let retention_bytes_slot; + let retention_ms; + let retention_bytes; match cleanup_policy { - TopicCleanupPolicy::Retention(retention) => { + TopicCleanupPolicy::Retention { ms, bytes } => { kafka_topic = kafka_topic.set("cleanup.policy", "delete"); - if let Some(retention_ms) = &retention.duration { - retention_ms_slot = retention_ms.to_string(); - kafka_topic = kafka_topic.set("retention.ms", &retention_ms_slot); + if let Some(ms) = ms { + retention_ms = ms.to_string(); + kafka_topic = kafka_topic.set("retention.ms", &retention_ms); } - if let Some(retention_bytes) = &retention.bytes { - retention_bytes_slot = retention_bytes.to_string(); - kafka_topic = kafka_topic.set("retention.bytes", &retention_bytes_slot); + if let Some(bytes) = &bytes { + retention_bytes = bytes.to_string(); + kafka_topic = kafka_topic.set("retention.bytes", &retention_bytes); } } TopicCleanupPolicy::Compaction => { @@ -392,12 +410,14 @@ pub async fn build_kafka( ensure_kafka_topic( &admin_client, &connection.topic, + // TODO: allow users to configure these parameters. TopicConfig { - partition_count: connection.partition_count, - replication_factor: connection.replication_factor, - // TODO: allow users to request compaction cleanup policy instead of - // retention. - cleanup_policy: TopicCleanupPolicy::Retention(connection.retention), + partition_count: -1, + replication_factor: -1, + cleanup_policy: TopicCleanupPolicy::Retention { + ms: None, + bytes: None, + }, }, ) .await diff --git a/src/storage-types/src/sinks.proto b/src/storage-types/src/sinks.proto index 76387dc7a8ad6..cbec04664168a 100644 --- a/src/storage-types/src/sinks.proto +++ b/src/storage-types/src/sinks.proto @@ -65,11 +65,6 @@ message ProtoKafkaSinkFormat { } } -message ProtoKafkaSinkConnectionRetention { - optional int64 duration = 1; - optional int64 bytes = 2; -} - message ProtoKafkaSinkConnectionV2 { message ProtoKeyDescAndIndices { mz_repr.relation_and_scalar.ProtoRelationDesc desc = 1; @@ -80,7 +75,7 @@ message ProtoKafkaSinkConnectionV2 { repeated uint64 relation_key_indices = 1; } - reserved 10, 13; + reserved 8 to 10, 12, 13; mz_repr.global_id.ProtoGlobalId connection_id = 1; mz_storage_types.connections.ProtoKafkaConnection connection = 2; @@ -89,10 +84,7 @@ message ProtoKafkaSinkConnectionV2 { optional ProtoRelationKeyIndicesVec relation_key_indices = 5; mz_repr.relation_and_scalar.ProtoRelationDesc value_desc = 6; uint64 fuel = 7; - ProtoKafkaSinkConnectionRetention retention = 8; - int32 replication_factor = 9; ProtoKafkaSinkFormat format = 11; - int32 partition_count = 12; oneof compression_type { google.protobuf.Empty none = 14; google.protobuf.Empty gzip = 15; diff --git a/src/storage-types/src/sinks.rs b/src/storage-types/src/sinks.rs index 1d4f96ae7f882..2f147d53663fe 100644 --- a/src/storage-types/src/sinks.rs +++ b/src/storage-types/src/sinks.rs @@ -430,10 +430,7 @@ pub struct KafkaSinkConnection { pub key_desc_and_indices: Option<(RelationDesc, Vec)>, pub value_desc: RelationDesc, pub topic: String, - pub partition_count: i32, - pub replication_factor: i32, pub fuel: usize, - pub retention: KafkaSinkConnectionRetention, pub compression_type: KafkaSinkCompressionType, } @@ -467,10 +464,7 @@ impl KafkaSinkConnection { key_desc_and_indices, value_desc, topic, - partition_count, - replication_factor, fuel, - retention, compression_type, } = self; @@ -487,13 +481,7 @@ impl KafkaSinkConnection { ), (value_desc == &other.value_desc, "value_desc"), (topic == &other.topic, "topic"), - (partition_count == &other.partition_count, "partition_count"), - ( - replication_factor == &other.replication_factor, - "replication_factor", - ), (fuel == &other.fuel, "fuel"), - (retention == &other.retention, "retention"), ( compression_type == &other.compression_type, "compression_type", @@ -527,10 +515,7 @@ impl IntoInlineConnection key_desc_and_indices, value_desc, topic, - partition_count, - replication_factor, fuel, - retention, compression_type, } = self; KafkaSinkConnection { @@ -541,10 +526,7 @@ impl IntoInlineConnection key_desc_and_indices, value_desc, topic, - partition_count, - replication_factor, fuel, - retention, compression_type, } } @@ -561,10 +543,7 @@ impl RustType for KafkaSinkConnection { relation_key_indices: self.relation_key_indices.into_proto(), value_desc: Some(self.value_desc.into_proto()), topic: self.topic.clone(), - partition_count: self.partition_count, - replication_factor: self.replication_factor, fuel: u64::cast_from(self.fuel), - retention: Some(self.retention.into_proto()), compression_type: Some(match self.compression_type { KafkaSinkCompressionType::None => CompressionType::None(()), KafkaSinkCompressionType::Gzip => CompressionType::Gzip(()), @@ -593,12 +572,7 @@ impl RustType for KafkaSinkConnection { .value_desc .into_rust_if_some("ProtoKafkaSinkConnectionV2::value_desc")?, topic: proto.topic, - partition_count: proto.partition_count, - replication_factor: proto.replication_factor, fuel: proto.fuel.into_rust()?, - retention: proto - .retention - .into_rust_if_some("ProtoKafkaSinkConnectionV2::retention")?, compression_type: match proto.compression_type { Some(CompressionType::None(())) => KafkaSinkCompressionType::None, Some(CompressionType::Gzip(())) => KafkaSinkCompressionType::Gzip, @@ -615,28 +589,6 @@ impl RustType for KafkaSinkConnection { } } -#[derive(Arbitrary, Copy, Clone, Debug, Default, Eq, PartialEq, Serialize, Deserialize)] -pub struct KafkaSinkConnectionRetention { - pub duration: Option, - pub bytes: Option, -} - -impl RustType for KafkaSinkConnectionRetention { - fn into_proto(&self) -> ProtoKafkaSinkConnectionRetention { - ProtoKafkaSinkConnectionRetention { - duration: self.duration, - bytes: self.bytes, - } - } - - fn from_proto(proto: ProtoKafkaSinkConnectionRetention) -> Result { - Ok(KafkaSinkConnectionRetention { - duration: proto.duration, - bytes: proto.bytes, - }) - } -} - #[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] pub enum KafkaSinkFormat { Avro { diff --git a/test/kafka-multi-broker/01-init.td b/test/kafka-multi-broker/01-init.td index 315d2e1094066..eed08e26c074a 100644 --- a/test/kafka-multi-broker/01-init.td +++ b/test/kafka-multi-broker/01-init.td @@ -49,6 +49,6 @@ ALTER SYSTEM SET enable_kafka_config_denylist_options = true > CREATE SINK multi_broker_sink FROM kafka_multi_broker - INTO KAFKA CONNECTION kafka_conn (REPLICATION FACTOR = 2, PARTITION COUNT = 2, TOPIC 'testdrive-kafka-multi-broker-sink-${testdrive.seed}') + INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-kafka-multi-broker-sink-${testdrive.seed}') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM diff --git a/test/testdrive/kafka-sink-errors.td b/test/testdrive/kafka-sink-errors.td index 4eea868f6b4ba..ba5ecd3790ed9 100644 --- a/test/testdrive/kafka-sink-errors.td +++ b/test/testdrive/kafka-sink-errors.td @@ -23,35 +23,6 @@ ALTER SYSTEM SET enable_kafka_config_denylist_options = true URL '${testdrive.schema-registry-url}' ); -! CREATE SINK invalid_partition_count FROM v1 - INTO KAFKA CONNECTION kafka_conn (PARTITION COUNT = a, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:invalid PARTITION COUNT: cannot use value as number - -! CREATE SINK invalid_partition_count FROM v1 - INTO KAFKA CONNECTION kafka_conn (PARTITION COUNT = -2, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:PARTION COUNT for sink topics must be a positive integer or -1 for broker default - -! CREATE SINK invalid_replication_factor FROM v1 - INTO KAFKA CONNECTION kafka_conn (REPLICATION FACTOR = a, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:invalid REPLICATION FACTOR: cannot use value as number - -! CREATE SINK invalid_replication_factor FROM v1 - INTO KAFKA CONNECTION kafka_conn (REPLICATION FACTOR = -2, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:REPLICATION FACTOR for sink topics must be a positive integer or -1 for broker default - -# Ensure that a sink whose topic fails to create does not result in an -# orphaned linked cluster. See #17061. -> SELECT count(*) FROM mz_clusters WHERE name = 'materialize_public_invalid_acks' -0 - ! CREATE SINK invalid_key FROM v1 INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') KEY(f2) @@ -59,33 +30,6 @@ contains:REPLICATION FACTOR for sink topics must be a positive integer or -1 for ENVELOPE DEBEZIUM contains:No such column: f2 -# -# Retention options -# -! CREATE SINK invalid_retention_ms FROM v1 - INTO KAFKA CONNECTION kafka_conn (RETENTION MS = a, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:invalid RETENTION MS: cannot use value as number - -! CREATE SINK invalid_retention_ms FROM v1 - INTO KAFKA CONNECTION kafka_conn (RETENTION MS = -2, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:RETENTION MS for sink topics must be greater than or equal to -1 - -! CREATE SINK invalid_retention_bytes FROM v1 - INTO KAFKA CONNECTION kafka_conn (RETENTION BYTES = a, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:invalid RETENTION BYTES: cannot use value as number - -! CREATE SINK invalid_retention_bytes FROM v1 - INTO KAFKA CONNECTION kafka_conn (RETENTION BYTES = -2, TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM -contains:RETENTION BYTES for sink topics must be greater than or equal to -1 - # # Sink dependencies # diff --git a/test/testdrive/kafka-sinks.td b/test/testdrive/kafka-sinks.td index 2bac26864da52..c17bbf1c1211b 100644 --- a/test/testdrive/kafka-sinks.td +++ b/test/testdrive/kafka-sinks.td @@ -103,7 +103,7 @@ name type size cluster ENVELOPE DEBEZIUM > CREATE SINK snk3 FROM v3 - INTO KAFKA CONNECTION kafka_conn (RETENTION BYTES = 1000000000000, TOPIC 'testdrive-snk3-${testdrive.seed}') + INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-snk3-${testdrive.seed}') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM @@ -214,52 +214,14 @@ snk7 kafka ${arg.default-storage-size} materialize_public_snk7 snk8 kafka ${arg.default-storage-size} materialize_public_snk8 snk_unsigned kafka ${arg.default-storage-size} materialize_public_snk_unsigned -# test explicit partition count -> CREATE SINK snk9 FROM foo - INTO KAFKA CONNECTION kafka_conn (PARTITION COUNT=1, TOPIC 'testdrive-snk9-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM - -# test explicit replication factor -> CREATE SINK snk10 FROM foo - INTO KAFKA CONNECTION kafka_conn (REPLICATION FACTOR=1, TOPIC 'testdrive-snk10-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM - -# test explicit partition count and replication factor -> CREATE SINK snk11 FROM foo - INTO KAFKA CONNECTION kafka_conn (PARTITION COUNT=1, REPLICATION FACTOR=1, TOPIC 'testdrive-snk11-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM - -# test broker defaulted partition count and replication factor -> CREATE SINK snk12 FROM foo - INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-snk12-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM - -# test explicit request for broker defaulted partition count and replication factor -> CREATE SINK snk13 FROM foo - INTO KAFKA CONNECTION kafka_conn (PARTITION COUNT=-1, REPLICATION FACTOR=-1, TOPIC 'testdrive-snk13-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM - # test already existing topic with non-default partition count -$ kafka-create-topic topic=snk14 partitions=4 +$ kafka-create-topic topic=snk9 partitions=4 -> CREATE SINK snk14 FROM foo +> CREATE SINK snk9 FROM foo INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-snk14-${testdrive.seed}') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn ENVELOPE DEBEZIUM -# test already existing topic with non-default partition count -- even if arg specified -$ kafka-create-topic topic=snk15 partitions=4 - -> CREATE SINK snk15 FROM foo - INTO KAFKA CONNECTION kafka_conn (PARTITION COUNT=1, TOPIC 'testdrive-snk15-${testdrive.seed}') - FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn - ENVELOPE DEBEZIUM - # create sink with SIZE set > CREATE SINK sink_with_size FROM src INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-snk1-${testdrive.seed}') @@ -295,12 +257,6 @@ snk6 kafka ${arg.default-storage-size} materialize_public_snk6 snk7 kafka ${arg.default-storage-size} materialize_public_snk7 snk8 kafka ${arg.default-storage-size} materialize_public_snk8 snk9 kafka ${arg.default-storage-size} materialize_public_snk9 -snk10 kafka ${arg.default-storage-size} materialize_public_snk10 -snk11 kafka ${arg.default-storage-size} materialize_public_snk11 -snk12 kafka ${arg.default-storage-size} materialize_public_snk12 -snk13 kafka ${arg.default-storage-size} materialize_public_snk13 -snk14 kafka ${arg.default-storage-size} materialize_public_snk14 -snk15 kafka ${arg.default-storage-size} materialize_public_snk15 sink_with_size kafka 2 materialize_public_sink_with_size sink_with_options kafka 2 materialize_public_sink_with_options snk_unsigned kafka ${arg.default-storage-size} materialize_public_snk_unsigned From f7bed267f4b5fd96c98b62d6d6e4cdde91dc8326 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Sat, 9 Dec 2023 19:46:59 -0500 Subject: [PATCH 3/5] sql: split Kafka source and sink options Splitting the parsing and planning of Kafka sources and sinks is a net simplification for the code. Even though a few options are shared between sources and sinks (TOPIC, GROUP ID PREFIX), most are not. Having separate types for sources and sinks avoids a separate validation step to determine whether each option is valid for the context. This is a pure refactoring commit. --- src/sql-parser/src/ast/defs/ddl.rs | 105 ++++++----- src/sql-parser/src/parser.rs | 101 ++++++----- src/sql-parser/tests/testdata/ddl | 98 +++++----- src/sql-parser/tests/testdata/explain | 6 +- src/sql/src/kafka_util.rs | 252 ++++++++++---------------- src/sql/src/plan/statement/ddl.rs | 102 +++++------ src/sql/src/pure.rs | 123 ++++++------- 7 files changed, 374 insertions(+), 413 deletions(-) diff --git a/src/sql-parser/src/ast/defs/ddl.rs b/src/sql-parser/src/ast/defs/ddl.rs index a901cdcd01825..ecfa5d4da245a 100644 --- a/src/sql-parser/src/ast/defs/ddl.rs +++ b/src/sql-parser/src/ast/defs/ddl.rs @@ -819,8 +819,7 @@ impl AstDisplay for CreateConnectionOption { impl_display_t!(CreateConnectionOption); #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] -pub enum KafkaConfigOptionName { - CompressionType, +pub enum KafkaSourceConfigOptionName { GroupIdPrefix, Topic, TopicMetadataRefreshInterval, @@ -828,30 +827,28 @@ pub enum KafkaConfigOptionName { StartOffset, } -impl AstDisplay for KafkaConfigOptionName { +impl AstDisplay for KafkaSourceConfigOptionName { fn fmt(&self, f: &mut AstFormatter) { f.write_str(match self { - KafkaConfigOptionName::CompressionType => "COMPRESSION TYPE", - KafkaConfigOptionName::GroupIdPrefix => "GROUP ID PREFIX", - KafkaConfigOptionName::Topic => "TOPIC", - KafkaConfigOptionName::TopicMetadataRefreshInterval => { + KafkaSourceConfigOptionName::GroupIdPrefix => "GROUP ID PREFIX", + KafkaSourceConfigOptionName::Topic => "TOPIC", + KafkaSourceConfigOptionName::TopicMetadataRefreshInterval => { "TOPIC METADATA REFRESH INTERVAL" } - KafkaConfigOptionName::StartOffset => "START OFFSET", - KafkaConfigOptionName::StartTimestamp => "START TIMESTAMP", + KafkaSourceConfigOptionName::StartOffset => "START OFFSET", + KafkaSourceConfigOptionName::StartTimestamp => "START TIMESTAMP", }) } } -impl_display!(KafkaConfigOptionName); +impl_display!(KafkaSourceConfigOptionName); #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] -/// An option in a `{FROM|INTO} CONNECTION ...` statement. -pub struct KafkaConfigOption { - pub name: KafkaConfigOptionName, +pub struct KafkaSourceConfigOption { + pub name: KafkaSourceConfigOptionName, pub value: Option>, } -impl AstDisplay for KafkaConfigOption { +impl AstDisplay for KafkaSourceConfigOption { fn fmt(&self, f: &mut AstFormatter) { f.write_node(&self.name); if let Some(v) = &self.value { @@ -860,33 +857,43 @@ impl AstDisplay for KafkaConfigOption { } } } -impl_display_t!(KafkaConfigOption); +impl_display_t!(KafkaSourceConfigOption); -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct KafkaConnection { - pub connection: T::ItemName, - pub options: Vec>, +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub enum KafkaSinkConfigOptionName { + CompressionType, + GroupIdPrefix, + Topic, } -impl AstDisplay for KafkaConnection { +impl AstDisplay for KafkaSinkConfigOptionName { fn fmt(&self, f: &mut AstFormatter) { - f.write_str("CONNECTION "); - f.write_node(&self.connection); - if !self.options.is_empty() { - f.write_str(" ("); - f.write_node(&display::comma_separated(&self.options)); - f.write_str(")"); - } + f.write_str(match self { + KafkaSinkConfigOptionName::CompressionType => "COMPRESSION TYPE", + KafkaSinkConfigOptionName::GroupIdPrefix => "GROUP ID PREFIX", + KafkaSinkConfigOptionName::Topic => "TOPIC", + }) } } -impl_display_t!(KafkaConnection); +impl_display!(KafkaSinkConfigOptionName); -#[derive(Debug, Clone, PartialEq, Eq, Hash)] -pub struct KafkaSourceConnection { - pub connection: KafkaConnection, - pub key: Option>, +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] +pub struct KafkaSinkConfigOption { + pub name: KafkaSinkConfigOptionName, + pub value: Option>, } +impl AstDisplay for KafkaSinkConfigOption { + fn fmt(&self, f: &mut AstFormatter) { + f.write_node(&self.name); + if let Some(v) = &self.value { + f.write_str(" = "); + f.write_node(v); + } + } +} +impl_display_t!(KafkaSinkConfigOption); + #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] pub enum PgConfigOptionName { /// Hex encoded string of binary serialization of `dataflow_types::PostgresSourceDetails` @@ -928,7 +935,10 @@ impl_display_t!(PgConfigOption); #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub enum CreateSourceConnection { - Kafka(KafkaSourceConnection), + Kafka { + connection: T::ItemName, + options: Vec>, + }, Postgres { /// The postgres connection. connection: T::ItemName, @@ -946,12 +956,15 @@ pub enum CreateSourceConnection { impl AstDisplay for CreateSourceConnection { fn fmt(&self, f: &mut AstFormatter) { match self { - CreateSourceConnection::Kafka(KafkaSourceConnection { connection, key }) => { - f.write_str("KAFKA "); + CreateSourceConnection::Kafka { + connection, + options, + } => { + f.write_str("KAFKA CONNECTION "); f.write_node(connection); - if let Some(key) = key.as_ref() { - f.write_str(" KEY ("); - f.write_node(&display::comma_separated(key)); + if !options.is_empty() { + f.write_str(" ("); + f.write_node(&display::comma_separated(options)); f.write_str(")"); } } @@ -1048,7 +1061,8 @@ impl_display_t!(LoadGeneratorOption); #[derive(Debug, Clone, PartialEq, Eq, Hash)] pub enum CreateSinkConnection { Kafka { - connection: KafkaConnection, + connection: T::ItemName, + options: Vec>, key: Option, }, } @@ -1056,9 +1070,18 @@ pub enum CreateSinkConnection { impl AstDisplay for CreateSinkConnection { fn fmt(&self, f: &mut AstFormatter) { match self { - CreateSinkConnection::Kafka { connection, key } => { - f.write_str("KAFKA "); + CreateSinkConnection::Kafka { + connection, + options, + key, + } => { + f.write_str("KAFKA CONNECTION "); f.write_node(connection); + if !options.is_empty() { + f.write_str(" ("); + f.write_node(&display::comma_separated(options)); + f.write_str(")"); + } if let Some(key) = key.as_ref() { f.write_node(key); } diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index ad22a3e3acaf3..a5f7b55df70e8 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -2337,48 +2337,51 @@ impl<'a> Parser<'a> { Ok(KafkaBrokerAwsPrivatelinkOption { name, value }) } - fn parse_kafka_connection_reference(&mut self) -> Result, ParserError> { - let connection = self.parse_raw_name()?; - let options = if self.consume_token(&Token::LParen) { - let options = self.parse_comma_separated(Parser::parse_kafka_config_option)?; - self.expect_token(&Token::RParen)?; - options - } else { - vec![] - }; - - Ok(KafkaConnection { - connection, - options, - }) - } - - fn parse_kafka_config_option(&mut self) -> Result, ParserError> { - let name = match self.expect_one_of_keywords(&[COMPRESSION, GROUP, START, TOPIC])? { - COMPRESSION => { - self.expect_keyword(TYPE)?; - KafkaConfigOptionName::CompressionType - } + fn parse_kafka_source_config_option( + &mut self, + ) -> Result, ParserError> { + let name = match self.expect_one_of_keywords(&[GROUP, START, TOPIC])? { GROUP => { self.expect_keywords(&[ID, PREFIX])?; - KafkaConfigOptionName::GroupIdPrefix + KafkaSourceConfigOptionName::GroupIdPrefix } START => match self.expect_one_of_keywords(&[OFFSET, TIMESTAMP])? { - OFFSET => KafkaConfigOptionName::StartOffset, - TIMESTAMP => KafkaConfigOptionName::StartTimestamp, + OFFSET => KafkaSourceConfigOptionName::StartOffset, + TIMESTAMP => KafkaSourceConfigOptionName::StartTimestamp, _ => unreachable!(), }, TOPIC => { if self.parse_keyword(METADATA) { self.expect_keywords(&[REFRESH, INTERVAL])?; - KafkaConfigOptionName::TopicMetadataRefreshInterval + KafkaSourceConfigOptionName::TopicMetadataRefreshInterval } else { - KafkaConfigOptionName::Topic + KafkaSourceConfigOptionName::Topic } } _ => unreachable!(), }; - Ok(KafkaConfigOption { + Ok(KafkaSourceConfigOption { + name, + value: self.parse_optional_option_value()?, + }) + } + + fn parse_kafka_sink_config_option( + &mut self, + ) -> Result, ParserError> { + let name = match self.expect_one_of_keywords(&[COMPRESSION, GROUP, TOPIC])? { + COMPRESSION => { + self.expect_keyword(TYPE)?; + KafkaSinkConfigOptionName::CompressionType + } + GROUP => { + self.expect_keywords(&[ID, PREFIX])?; + KafkaSinkConfigOptionName::GroupIdPrefix + } + TOPIC => KafkaSinkConfigOptionName::Topic, + _ => unreachable!(), + }; + Ok(KafkaSinkConfigOption { name, value: self.parse_optional_option_value()?, }) @@ -2981,22 +2984,21 @@ impl<'a> Parser<'a> { } KAFKA => { self.expect_keyword(CONNECTION)?; - let connection = self.parse_kafka_connection_reference()?; - // one token of lookahead: - // * `KEY (` means we're parsing a list of columns for the key - // * `KEY FORMAT` means there is no key, we'll parse a KeyValueFormat later - let key = if self.peek_keyword(KEY) - && self.peek_nth_token(1) != Some(Token::Keyword(FORMAT)) - { - let _ = self.expect_keyword(KEY); - Some(self.parse_parenthesized_column_list(Mandatory)?) + let connection = self.parse_raw_name()?; + + let options = if self.consume_token(&Token::LParen) { + let options = + self.parse_comma_separated(Parser::parse_kafka_source_config_option)?; + self.expect_token(&Token::RParen)?; + options } else { - None + vec![] }; - Ok(CreateSourceConnection::Kafka(KafkaSourceConnection { + + Ok(CreateSourceConnection::Kafka { connection, - key, - })) + options, + }) } LOAD => { self.expect_keyword(GENERATOR)?; @@ -3091,7 +3093,15 @@ impl<'a> Parser<'a> { self.expect_keyword(KAFKA)?; self.expect_keyword(CONNECTION)?; - let connection = self.parse_kafka_connection_reference()?; + let connection = self.parse_raw_name()?; + + let options = if self.consume_token(&Token::LParen) { + let options = self.parse_comma_separated(Parser::parse_kafka_sink_config_option)?; + self.expect_token(&Token::RParen)?; + options + } else { + vec![] + }; // one token of lookahead: // * `KEY (` means we're parsing a list of columns for the key @@ -3114,7 +3124,12 @@ impl<'a> Parser<'a> { } else { None }; - Ok(CreateSinkConnection::Kafka { connection, key }) + + Ok(CreateSinkConnection::Kafka { + connection, + options, + key, + }) } fn parse_create_view(&mut self) -> Result, ParserError> { diff --git a/src/sql-parser/tests/testdata/ddl b/src/sql-parser/tests/testdata/ddl index 3b4078ad04603..6dd6b82a56bf8 100644 --- a/src/sql-parser/tests/testdata/ddl +++ b/src/sql-parser/tests/testdata/ddl @@ -451,46 +451,46 @@ 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 }) parse-statement -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic', GROUP ID PREFIX 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic', GROUP ID PREFIX 'prefix', COMPRESSION TYPE = gzip) FORMAT BYTES ---- -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic', GROUP ID PREFIX = 'prefix', TOPIC METADATA REFRESH INTERVAL = 100, START OFFSET = 1, START TIMESTAMP = 1234, COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic', GROUP ID PREFIX = 'prefix', COMPRESSION TYPE = gzip) FORMAT BYTES => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaConfigOption { name: GroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaConfigOption { name: TopicMetadataRefreshInterval, value: Some(Value(Number("100"))) }, KafkaConfigOption { name: StartOffset, value: Some(Value(Number("1"))) }, KafkaConfigOption { name: StartTimestamp, value: Some(Value(Number("1234"))) }, KafkaConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaSinkConfigOption { name: GroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaSinkConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [] }) parse-statement CREATE SINK FROM bar INTO KAFKA CONNECTION baz ---- CREATE SINK FROM bar INTO KAFKA CONNECTION baz => -CreateSink(CreateSinkStatement { name: None, in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [] }, key: None }, format: None, envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: None, in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [], key: None }, format: None, envelope: None, with_options: [] }) parse-statement CREATE SINK from FROM bar INTO KAFKA CONNECTION baz ---- CREATE SINK "from" FROM bar INTO KAFKA CONNECTION baz => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("from")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [] }, key: None }, format: None, envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("from")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [], key: None }, format: None, envelope: None, with_options: [] }) parse-statement CREATE SINK IN CLUSTER c FROM bar INTO KAFKA CONNECTION baz ---- CREATE SINK IN CLUSTER c FROM bar INTO KAFKA CONNECTION baz => -CreateSink(CreateSinkStatement { name: None, in_cluster: Some(Unresolved(Ident("c"))), if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [] }, key: None }, format: None, envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: None, in_cluster: Some(Unresolved(Ident("c"))), if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [], key: None }, format: None, envelope: None, with_options: [] }) parse-statement CREATE SINK in IN CLUSTER c FROM bar INTO KAFKA CONNECTION baz ---- CREATE SINK in IN CLUSTER c FROM bar INTO KAFKA CONNECTION baz => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("in")])), in_cluster: Some(Unresolved(Ident("c"))), if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [] }, key: None }, format: None, envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("in")])), in_cluster: Some(Unresolved(Ident("c"))), if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [], key: None }, format: None, envelope: None, with_options: [] }) parse-statement CREATE SINK in FROM bar INTO KAFKA CONNECTION baz ---- CREATE SINK in FROM bar INTO KAFKA CONNECTION baz => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("in")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [] }, key: None }, format: None, envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("in")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [], key: None }, format: None, envelope: None, with_options: [] }) parse-statement CREATE SOURCE psychic IN CLUSTER c FROM POSTGRES CONNECTION pgconn (PUBLICATION 'red') INCLUDE TIMESTAMP; @@ -504,21 +504,21 @@ CREATE SINK IF NOT EXISTS foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') ---- CREATE SINK IF NOT EXISTS foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') KEY (a, b) FORMAT BYTES ENVELOPE NONE => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: true, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: Some(KafkaSinkKey { key_columns: [Ident("a"), Ident("b")], not_enforced: false }) }, format: Some(Bytes), envelope: Some(None), with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: true, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: Some(KafkaSinkKey { key_columns: [Ident("a"), Ident("b")], not_enforced: false }) }, format: Some(Bytes), envelope: Some(None), with_options: [] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') KEY (a, b) NOT ENFORCED FORMAT BYTES ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') KEY (a, b) NOT ENFORCED FORMAT BYTES => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: Some(KafkaSinkKey { key_columns: [Ident("a"), Ident("b")], not_enforced: true }) }, format: Some(Bytes), envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: Some(KafkaSinkKey { key_columns: [Ident("a"), Ident("b")], not_enforced: true }) }, format: Some(Bytes), envelope: None, with_options: [] }) parse-statement CREATE SINK foo IN CLUSTER c FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') KEY (a, b) NOT ENFORCED FORMAT BYTES ---- CREATE SINK foo IN CLUSTER c FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') KEY (a, b) NOT ENFORCED FORMAT BYTES => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: Some(Unresolved(Ident("c"))), if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: Some(KafkaSinkKey { key_columns: [Ident("a"), Ident("b")], not_enforced: true }) }, format: Some(Bytes), envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: Some(Unresolved(Ident("c"))), if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: Some(KafkaSinkKey { key_columns: [Ident("a"), Ident("b")], not_enforced: true }) }, format: Some(Bytes), envelope: None, with_options: [] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') KEY (a, b) CONSISTENCY (TOPIC 'consistency' FORMAT BYTES) FORMAT BYTES @@ -546,56 +546,56 @@ CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT BYTES ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT BYTES WITH (SNAPSHOT = true) => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(true))) }] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(true))) }] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT BYTES WITH (SNAPSHOT = false) ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT BYTES WITH (SNAPSHOT = false) => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(false))) }] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(false))) }] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT BYTES WITH (SIZE = 'xlarge') ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT BYTES WITH (SIZE = 'xlarge') => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Size, value: Some(Value(String("xlarge"))) }] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Size, value: Some(Value(String("xlarge"))) }] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT BYTES WITH (SIZE = 'xlarge', SNAPSHOT = true) ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT BYTES WITH (SIZE = 'xlarge', SNAPSHOT = true) => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Size, value: Some(Value(String("xlarge"))) }, CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(true))) }] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Size, value: Some(Value(String("xlarge"))) }, CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(true))) }] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT BYTES WITH (SIZE = 'xlarge', SNAPSHOT = true) ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT BYTES WITH (SIZE = 'xlarge', SNAPSHOT = true) => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Size, value: Some(Value(String("xlarge"))) }, CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(true))) }] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [CreateSinkOption { name: Size, value: Some(Value(String("xlarge"))) }, CreateSinkOption { name: Snapshot, value: Some(Value(Boolean(true))) }] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL DEFAULTS = TRUE) ENVELOPE UPSERT ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL DEFAULTS = true) ENVELOPE UPSERT => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: Some(Value(Boolean(true))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: Some(Value(Boolean(true))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL DEFAULTS TRUE) ENVELOPE UPSERT ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL DEFAULTS = true) ENVELOPE UPSERT => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: Some(Value(Boolean(true))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: Some(Value(Boolean(true))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL DEFAULTS) ENVELOPE UPSERT ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL DEFAULTS) ENVELOPE UPSERT => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: None }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: None }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL VALUES) ENVELOPE UPSERT @@ -609,7 +609,7 @@ CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO U ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (NULL DEFAULTS = false) ENVELOPE UPSERT => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: Some(Value(Boolean(false))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: NullDefaults, value: Some(Value(Boolean(false))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) parse-statement CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (DOC ON bar) ENVELOPE UPSERT @@ -631,7 +631,7 @@ CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO U ---- CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 (DOC ON TYPE bar = 'bar comment', KEY DOC ON COLUMN bar.a = 'a comment', VALUE DOC ON COLUMN bar.b = 'b comment') ENVELOPE UPSERT => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: AvroDocOn(AvroDocOn { identifier: Type(Name(UnresolvedItemName([Ident("bar")]))), for_schema: All }), value: Some(Value(String("bar comment"))) }, CsrConfigOption { name: AvroDocOn(AvroDocOn { identifier: Column(RawColumnName { relation: Name(UnresolvedItemName([Ident("bar")])), column: Ident("a") }), for_schema: KeyOnly }), value: Some(Value(String("a comment"))) }, CsrConfigOption { name: AvroDocOn(AvroDocOn { identifier: Column(RawColumnName { relation: Name(UnresolvedItemName([Ident("bar")])), column: Ident("b") }), for_schema: ValueOnly }), value: Some(Value(String("b comment"))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [CsrConfigOption { name: AvroDocOn(AvroDocOn { identifier: Type(Name(UnresolvedItemName([Ident("bar")]))), for_schema: All }), value: Some(Value(String("bar comment"))) }, CsrConfigOption { name: AvroDocOn(AvroDocOn { identifier: Column(RawColumnName { relation: Name(UnresolvedItemName([Ident("bar")])), column: Ident("a") }), for_schema: KeyOnly }), value: Some(Value(String("a comment"))) }, CsrConfigOption { name: AvroDocOn(AvroDocOn { identifier: Column(RawColumnName { relation: Name(UnresolvedItemName([Ident("bar")])), column: Ident("b") }), for_schema: ValueOnly }), value: Some(Value(String("b comment"))) }] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] }) parse-statement CREATE INDEX foo ON myschema.bar (a, b) @@ -2012,21 +2012,21 @@ 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: Bare(Bytes), envelope: None, if_not_exists: true, key_constraint: Some(PrimaryKeyNotEnforced { columns: [Ident("a")] }), with_options: [], referenced_subsources: None, progress_subsource: None }) parse-statement CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY (a, b) FORMAT BYTES ---- -CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC = 'baz') KEY (a, b) FORMAT BYTES -=> -CreateSource(CreateSourceStatement { name: UnresolvedItemName([Ident("src1")]), in_cluster: None, col_names: [], connection: Kafka(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: Some([Ident("a"), Ident("b")]) }), include_metadata: [], format: Bare(Bytes), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) +error: Expected FORMAT, found left parenthesis +CREATE SOURCE src1 FROM KAFKA CONNECTION conn1 (TOPIC 'baz') KEY (a, b) FORMAT BYTES + ^ parse-statement CREATE SOURCE src1 (PRIMARY KEY (key1, key2) NOT ENFORCED) FROM KAFKA CONNECTION conn1 (TOPIC 'baz') FORMAT BYTES ---- 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) parse-statement CREATE CONNECTION conn1 FOR CONFLUENT SCHEMA REGISTRY URL 'http://localhost:8081', USERNAME 'user', PASSWORD 'word', PORT 1234, AWS PRIVATELINK apl @@ -2069,7 +2069,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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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(Plain)), 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: 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(Plain)), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) parse-statement @@ -2077,7 +2077,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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, seed: None } })), envelope: Some(Debezium(Plain)), 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: Bare(Protobuf(Csr { csr_connection: CsrConnectionProtobuf { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, seed: None } })), envelope: Some(Debezium(Plain)), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) parse-statement @@ -2115,35 +2115,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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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(Plain)), 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: 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(Plain)), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) parse-statement @@ -2151,77 +2151,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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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(Plain)), 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: 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(Plain)), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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), 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: 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), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [Key { alias: None }], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) parse-statement @@ -2229,7 +2229,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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [Key { alias: None }, Timestamp { alias: None }, Partition { alias: Some(Ident("PART2")) }, Offset { alias: None }, Headers { alias: None }], format: 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: 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 }) parse-statement CREATE SOURCE golbat FROM KAFKA BROKER 'zubat' TOPIC 'hoothoot' KEY FORMAT TEXT VALUE FORMAT TEXT INCLUDE KEY ENVELOPE NONE WITH (SIZE large) @@ -2243,14 +2243,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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: 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 }) 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn1")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("baz"))) }] }, key: None }), include_metadata: [], format: 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: Bare(Csv { columns: Count(5), delimiter: ',' }), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) parse-statement CREATE CONNECTION my_ssh_tunnel FOR SSH TUNNEL HOST 'ssh-bastion', PORT 1234, USER 'blah' @@ -2815,11 +2815,11 @@ 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, key: None }), include_metadata: [Headers { alias: None }, Header { key: "header3", alias: Ident("h3"), use_bytes: false }, Header { key: "header5", alias: Ident("h5"), use_bytes: true }], format: Bare(Json), 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: Bare(Json), envelope: None, if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: 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(KafkaSourceConnection { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("conn")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("test"))) }] }, key: None }), include_metadata: [Header { key: "header1", alias: Ident("h1"), use_bytes: false }, Header { key: "header2", alias: Ident("h2"), use_bytes: true }], format: KeyValue { key: Text, value: Json }, envelope: Some(Upsert), 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: KeyValue { key: Text, value: Json }, envelope: Some(Upsert), if_not_exists: false, key_constraint: None, with_options: [], referenced_subsources: None, progress_subsource: None }) diff --git a/src/sql-parser/tests/testdata/explain b/src/sql-parser/tests/testdata/explain index 7775194e3fe73..dd50f86228952 100644 --- a/src/sql-parser/tests/testdata/explain +++ b/src/sql-parser/tests/testdata/explain @@ -180,18 +180,18 @@ EXPLAIN VALUE SCHEMA FOR CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOP ---- EXPLAIN VALUE SCHEMA AS JSON FOR CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 ENVELOPE UPSERT => -ExplainSinkSchema(ExplainSinkSchemaStatement { schema_for: Value, statement: CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] } }) +ExplainSinkSchema(ExplainSinkSchemaStatement { schema_for: Value, statement: CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] } }) parse-statement EXPLAIN KEY SCHEMA FOR CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 ENVELOPE UPSERT ---- EXPLAIN KEY SCHEMA AS JSON FOR CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 ENVELOPE UPSERT => -ExplainSinkSchema(ExplainSinkSchemaStatement { schema_for: Key, statement: CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] } }) +ExplainSinkSchema(ExplainSinkSchemaStatement { schema_for: Key, statement: CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] } }) parse-statement EXPLAIN KEY SCHEMA FOR CREATE SINK FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 ENVELOPE UPSERT ---- EXPLAIN KEY SCHEMA AS JSON FOR CREATE SINK FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic') FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION conn2 ENVELOPE UPSERT => -ExplainSinkSchema(ExplainSinkSchemaStatement { schema_for: Key, statement: CreateSinkStatement { name: None, in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: KafkaConnection { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaConfigOption { name: Topic, value: Some(Value(String("topic"))) }] }, key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] } }) +ExplainSinkSchema(ExplainSinkSchemaStatement { schema_for: Key, statement: CreateSinkStatement { name: None, in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }], key: None }, format: Some(Avro(Csr { csr_connection: CsrConnectionAvro { connection: CsrConnection { connection: Name(UnresolvedItemName([Ident("conn2")])), options: [] }, key_strategy: None, value_strategy: None, seed: None } })), envelope: Some(Upsert), with_options: [] } }) diff --git a/src/sql/src/kafka_util.rs b/src/sql/src/kafka_util.rs index 04848659e29b5..b50bea7d4f70f 100644 --- a/src/sql/src/kafka_util.rs +++ b/src/sql/src/kafka_util.rs @@ -11,13 +11,15 @@ use std::sync::Arc; -use anyhow::bail; use mz_kafka_util::client::{ DEFAULT_FETCH_METADATA_TIMEOUT, DEFAULT_TOPIC_METADATA_REFRESH_INTERVAL, }; use mz_ore::task; use mz_sql_parser::ast::display::AstDisplay; -use mz_sql_parser::ast::{AstInfo, KafkaConfigOption, KafkaConfigOptionName}; +use mz_sql_parser::ast::{ + KafkaSinkConfigOption, KafkaSinkConfigOptionName, KafkaSourceConfigOption, + KafkaSourceConfigOptionName, +}; use mz_storage_types::sinks::KafkaSinkCompressionType; use rdkafka::consumer::{BaseConsumer, Consumer, ConsumerContext}; use rdkafka::{Offset, TopicPartitionList}; @@ -29,51 +31,8 @@ use crate::normalize::generate_extracted_config; use crate::plan::with_options::{ImpliedValue, TryFromValue}; use crate::plan::PlanError; -#[derive(Debug, Copy, Clone, PartialEq, Eq)] -pub enum KafkaOptionCheckContext { - Source, - Sink, -} - -/// Verifies that [`KafkaConfigOption`]s are only used in the appropriate contexts -pub fn validate_options_for_context( - options: &[KafkaConfigOption], - context: KafkaOptionCheckContext, -) -> Result<(), anyhow::Error> { - use KafkaConfigOptionName::*; - use KafkaOptionCheckContext::*; - - for KafkaConfigOption { name, .. } in options { - let limited_to_context = match name { - CompressionType => Some(Sink), - GroupIdPrefix => None, - Topic => None, - TopicMetadataRefreshInterval => None, - StartTimestamp => Some(Source), - StartOffset => Some(Source), - }; - if limited_to_context.is_some() && limited_to_context != Some(context) { - bail!( - "cannot set {} for {}", - name.to_ast_string(), - match context { - Source => "SOURCE", - Sink => "SINK", - } - ); - } - } - - Ok(()) -} - generate_extracted_config!( - KafkaConfigOption, - ( - CompressionType, - KafkaSinkCompressionType, - Default(KafkaSinkCompressionType::None) - ), + KafkaSourceConfigOption, (GroupIdPrefix, String), (Topic, String), ( @@ -85,6 +44,17 @@ generate_extracted_config!( (StartOffset, Vec) ); +generate_extracted_config!( + KafkaSinkConfigOption, + ( + CompressionType, + KafkaSinkCompressionType, + Default(KafkaSinkCompressionType::None) + ), + (GroupIdPrefix, String), + (Topic, String) +); + impl TryFromValue for KafkaSinkCompressionType { fn try_from_value(v: Value) -> Result { match v { @@ -113,35 +83,6 @@ impl ImpliedValue for KafkaSinkCompressionType { } } -/// An enum that represents start offsets for a kafka consumer. -#[derive(Debug)] -pub enum KafkaStartOffsetType { - /// Fully specified, either by the user or generated. - StartOffset(Vec), - /// Specified by the user. - StartTimestamp(i64), -} - -impl TryFrom<&KafkaConfigOptionExtracted> for Option { - type Error = PlanError; - fn try_from( - KafkaConfigOptionExtracted { - start_offset, - start_timestamp, - .. - }: &KafkaConfigOptionExtracted, - ) -> Result, Self::Error> { - Ok(match (start_offset, start_timestamp) { - (Some(_), Some(_)) => { - sql_bail!("cannot specify START TIMESTAMP and START OFFSET at same time") - } - (Some(so), _) => Some(KafkaStartOffsetType::StartOffset(so.clone())), - (_, Some(sto)) => Some(KafkaStartOffsetType::StartTimestamp(*sto)), - _ => None, - }) - } -} - /// Returns start offsets for the partitions of `topic` and the provided /// `START TIMESTAMP` option. /// @@ -151,40 +92,28 @@ impl TryFrom<&KafkaConfigOptionExtracted> for Option { /// 0.10.0), the current end offset is returned for the partition. /// /// The provided `START TIMESTAMP` option must be a non-zero number: -/// * Non-Negative numbers will used as is (e.g. `1622659034343`) +/// * Non-negative numbers will used as is (e.g. `1622659034343`) /// * Negative numbers will be translated to a timestamp in millis /// before now (e.g. `-10` means 10 millis ago) -/// -/// If `START TIMESTAMP` has not been configured, an empty Option is -/// returned, but we also validate that partitions for offsets in -/// `offsets` do actually exist. pub async fn lookup_start_offsets( consumer: Arc>, topic: &str, - offsets: KafkaStartOffsetType, + time_offset: i64, now: u64, -) -> Result>, PlanError> +) -> Result, PlanError> where C: ConsumerContext + 'static, { - let (requested_offsets, time_offset) = match offsets { - KafkaStartOffsetType::StartTimestamp(time_offset) => { - ( - None, - Some(if time_offset < 0 { - let now: i64 = now.try_into()?; - let ts = now - time_offset.abs(); + let time_offset = if time_offset < 0 { + let now: i64 = now.try_into()?; + let ts = now - time_offset.abs(); - if ts <= 0 { - sql_bail!("Relative START TIMESTAMP must be smaller than current system timestamp") - } - ts - } else { - time_offset - }), - ) + if ts <= 0 { + sql_bail!("Relative START TIMESTAMP must be smaller than current system timestamp") } - KafkaStartOffsetType::StartOffset(offsets) => (Some(offsets.len()), None), + ts + } else { + time_offset }; // Lookup offsets @@ -192,6 +121,7 @@ where task::spawn_blocking(|| format!("kafka_lookup_start_offsets:{topic}"), { let topic = topic.to_string(); move || { + // There cannot be more than i32 partitions let num_partitions = mz_kafka_util::client::get_partitions( consumer.as_ref().client(), &topic, @@ -200,61 +130,42 @@ where .map_err(|e| sql_err!("{}", e))? .len(); - match (requested_offsets, time_offset) { - (Some(requested_offsets), _) => { - if requested_offsets > num_partitions { - return Err(sql_err!( - "START OFFSET specified more partitions ({}) than topic ({}) contains ({})", - requested_offsets, - topic, - num_partitions - )); - } - - Ok(None) - } - (_, Some(time_offset)) => { - // There cannot be more than i32 partitions - let num_partitions_i32 = i32::try_from(num_partitions).map_err(|_| { - sql_err!("kafka topic had more than {} partitions", i32::MAX) - })?; - - let mut tpl = TopicPartitionList::with_capacity(1); - tpl.add_partition_range(&topic, 0, num_partitions_i32 - 1); - tpl.set_all_offsets(Offset::Offset(time_offset)) - .map_err(|e| sql_err!("{}", e))?; - - let offsets_for_times = consumer - .offsets_for_times(tpl, Duration::from_secs(10)) - .map_err(|e| sql_err!("{}", e))?; - - // Translate to `start_offsets` - let start_offsets = offsets_for_times - .elements() - .iter() - .map(|elem| match elem.offset() { - Offset::Offset(offset) => Ok(offset), - Offset::End => fetch_end_offset(&consumer, &topic, elem.partition()), - _ => sql_bail!( - "Unexpected offset {:?} for partition {}", - elem.offset(), - elem.partition() - ), - }) - .collect::, _>>()?; - - if start_offsets.len() != num_partitions { - sql_bail!( - "Expected offsets for {} partitions, but received {}", - num_partitions, - start_offsets.len(), - ); - } - - Ok(Some(start_offsets)) - } - _ => unreachable!(), + let num_partitions_i32 = i32::try_from(num_partitions) + .map_err(|_| sql_err!("kafka topic had more than {} partitions", i32::MAX))?; + + let mut tpl = TopicPartitionList::with_capacity(1); + tpl.add_partition_range(&topic, 0, num_partitions_i32 - 1); + tpl.set_all_offsets(Offset::Offset(time_offset)) + .map_err(|e| sql_err!("{}", e))?; + + let offsets_for_times = consumer + .offsets_for_times(tpl, Duration::from_secs(10)) + .map_err(|e| sql_err!("{}", e))?; + + // Translate to `start_offsets` + let start_offsets = offsets_for_times + .elements() + .iter() + .map(|elem| match elem.offset() { + Offset::Offset(offset) => Ok(offset), + Offset::End => fetch_end_offset(&consumer, &topic, elem.partition()), + _ => sql_bail!( + "Unexpected offset {:?} for partition {}", + elem.offset(), + elem.partition() + ), + }) + .collect::, _>>()?; + + if start_offsets.len() != num_partitions { + sql_bail!( + "Expected offsets for {} partitions, but received {}", + num_partitions, + start_offsets.len(), + ); } + + Ok(start_offsets) } }) .await @@ -274,3 +185,40 @@ where .map_err(|e| sql_err!("{}", e))?; Ok(high) } + +/// Validates that the provided start offsets are valid for the specified topic. +/// At present, the validation is merely that there are not more start offsets +/// than parts in the topic. +pub async fn validate_start_offsets( + consumer: Arc>, + topic: &str, + start_offsets: Vec, +) -> Result<(), PlanError> +where + C: ConsumerContext + 'static, +{ + // TODO(guswynn): see if we can add broker to this name + task::spawn_blocking(|| format!("kafka_validate_start_offsets:{topic}"), { + let topic = topic.to_string(); + move || { + let num_partitions = mz_kafka_util::client::get_partitions( + consumer.as_ref().client(), + &topic, + DEFAULT_FETCH_METADATA_TIMEOUT, + ) + .map_err(|e| sql_err!("{}", e))? + .len(); + if start_offsets.len() > num_partitions { + sql_bail!( + "START OFFSET specified more partitions ({}) than topic ({}) contains ({})", + start_offsets.len(), + topic, + num_partitions + ) + } + Ok(()) + } + }) + .await + .map_err(|e| sql_err!("{}", e))? +} diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index 65647bb591e49..426ed7cdd5a3e 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -38,9 +38,9 @@ use mz_sql_parser::ast::{ AlterSystemResetStatement, AlterSystemSetStatement, CommentObjectType, CommentStatement, CreateConnectionOption, CreateConnectionOptionName, CreateConnectionType, CreateTypeListOption, CreateTypeListOptionName, CreateTypeMapOption, CreateTypeMapOptionName, DeferredItemName, - DocOnIdentifier, DocOnSchema, DropOwnedStatement, MaterializedViewOption, - MaterializedViewOptionName, SetRoleVar, UnresolvedItemName, UnresolvedObjectName, - UnresolvedSchemaName, Value, + DocOnIdentifier, DocOnSchema, DropOwnedStatement, KafkaSinkConfigOption, + KafkaSinkConfigOptionName, MaterializedViewOption, MaterializedViewOptionName, SetRoleVar, + UnresolvedItemName, UnresolvedObjectName, UnresolvedSchemaName, Value, }; use mz_sql_parser::ident; use mz_storage_types::connections::inline::{ConnectionAccess, ReferencedConnection}; @@ -75,7 +75,7 @@ use crate::ast::{ CreateWebhookSourceStatement, CsrConfigOption, CsrConfigOptionName, CsrConnection, CsrConnectionAvro, CsrConnectionProtobuf, CsrSeedProtobuf, CsvColumns, DbzMode, DropObjectsStatement, Envelope, Expr, Format, Ident, IfExistsBehavior, IndexOption, - IndexOptionName, KafkaConfigOptionName, KeyConstraint, LoadGeneratorOption, + IndexOptionName, KafkaSourceConfigOptionName, KeyConstraint, LoadGeneratorOption, LoadGeneratorOptionName, PgConfigOption, PgConfigOptionName, ProtobufSchema, QualifiedReplica, ReferencedSubsources, ReplicaDefinition, ReplicaOption, ReplicaOptionName, RoleAttribute, SourceIncludeMetadata, Statement, TableConstraint, UnresolvedDatabaseName, ViewDefinition, @@ -84,7 +84,7 @@ use crate::catalog::{ CatalogCluster, CatalogDatabase, CatalogError, CatalogItem, CatalogItemType, CatalogRecordField, CatalogType, CatalogTypeDetails, ObjectType, SystemObjectType, }; -use crate::kafka_util::{self, KafkaConfigOptionExtracted, KafkaStartOffsetType}; +use crate::kafka_util::{KafkaSinkConfigOptionExtracted, KafkaSourceConfigOptionExtracted}; use crate::names::{ Aug, CommentObjectId, DatabaseId, ObjectId, PartialItemName, QualifiedItemName, RawDatabaseSpecifier, ResolvedClusterName, ResolvedColumnName, ResolvedDataType, @@ -602,14 +602,10 @@ pub fn plan_create_source( } let (mut external_connection, encoding, available_subsources) = match connection { - CreateSourceConnection::Kafka(mz_sql_parser::ast::KafkaSourceConnection { - connection: - mz_sql_parser::ast::KafkaConnection { - connection: connection_name, - options, - }, - key: _, - }) => { + CreateSourceConnection::Kafka { + connection: connection_name, + options, + } => { let connection_item = scx.get_item_by_resolved_name(connection_name)?; if !matches!(connection_item.connection()?, Connection::Kafka(_)) { sql_bail!( @@ -620,10 +616,10 @@ pub fn plan_create_source( // Starting offsets are allowed out with feature flags mode, as they are a simple, // useful way to specify where to start reading a topic. - const ALLOWED_OPTIONS: &[KafkaConfigOptionName] = &[ - KafkaConfigOptionName::StartOffset, - KafkaConfigOptionName::StartTimestamp, - KafkaConfigOptionName::Topic, + const ALLOWED_OPTIONS: &[KafkaSourceConfigOptionName] = &[ + KafkaSourceConfigOptionName::StartOffset, + KafkaSourceConfigOptionName::StartTimestamp, + KafkaSourceConfigOptionName::Topic, ]; if let Some(op) = options @@ -637,34 +633,24 @@ pub fn plan_create_source( )?; } - kafka_util::validate_options_for_context( - options, - kafka_util::KafkaOptionCheckContext::Source, - )?; - - let extracted_options: KafkaConfigOptionExtracted = options.clone().try_into()?; - - let optional_start_offset = - Option::::try_from(&extracted_options)?; + let KafkaSourceConfigOptionExtracted { + group_id_prefix, + topic, + topic_metadata_refresh_interval, + start_timestamp: _, // purified into `start_offset` + start_offset, + seen: _, + }: KafkaSourceConfigOptionExtracted = options.clone().try_into()?; - let topic = extracted_options - .topic - .expect("validated exists during purification"); - let group_id_prefix = extracted_options.group_id_prefix; + let topic = topic.expect("validated exists during purification"); let mut start_offsets = BTreeMap::new(); - match optional_start_offset { - None => (), - Some(KafkaStartOffsetType::StartOffset(offsets)) => { - for (part, offset) in offsets.iter().enumerate() { - if *offset < 0 { - sql_bail!("START OFFSET must be a nonnegative integer"); - } - start_offsets.insert(i32::try_from(part)?, *offset); + if let Some(offsets) = start_offset { + for (part, offset) in offsets.iter().enumerate() { + if *offset < 0 { + sql_bail!("START OFFSET must be a nonnegative integer"); } - } - Some(KafkaStartOffsetType::StartTimestamp(_)) => { - unreachable!("time offsets should be converted in purification") + start_offsets.insert(i32::try_from(part)?, *offset); } } @@ -672,7 +658,6 @@ pub fn plan_create_source( sql_bail!("START OFFSET is not supported with ENVELOPE {}", envelope) } - let topic_metadata_refresh_interval = extracted_options.topic_metadata_refresh_interval; if topic_metadata_refresh_interval > Duration::from_secs(60 * 60) { // This is a librdkafka-enforced restriction that, if violated, // would result in a runtime error for the source. @@ -1607,7 +1592,7 @@ fn get_encoding( } }; - let force_nullable_keys = matches!(connection, Some(CreateSourceConnection::Kafka(_))) + let force_nullable_keys = matches!(connection, Some(CreateSourceConnection::Kafka { .. })) && matches!(envelope, Envelope::None); let encoding = encoding.into_source_data_encoding(force_nullable_keys); @@ -2347,9 +2332,14 @@ pub fn plan_create_sink( } let connection_builder = match connection { - CreateSinkConnection::Kafka { connection, .. } => kafka_sink_builder( + CreateSinkConnection::Kafka { + connection, + options, + .. + } => kafka_sink_builder( scx, connection, + options, format, relation_key_indices, key_desc_and_indices, @@ -2499,10 +2489,8 @@ impl std::convert::TryFrom>> for CsrConfigOptionExtract fn kafka_sink_builder( scx: &StatementContext, - mz_sql_parser::ast::KafkaConnection { - connection, - options, - }: mz_sql_parser::ast::KafkaConnection, + connection: ResolvedItemName, + options: Vec>, format: Option>, relation_key_indices: Option>, key_desc_and_indices: Option<(RelationDesc, Vec)>, @@ -2523,9 +2511,9 @@ fn kafka_sink_builder( // Starting offsets are allowed with feature flags mode, as they are a simple, // useful way to specify where to start reading a topic. - const ALLOWED_OPTIONS: &[KafkaConfigOptionName] = &[ - KafkaConfigOptionName::Topic, - KafkaConfigOptionName::CompressionType, + const ALLOWED_OPTIONS: &[KafkaSinkConfigOptionName] = &[ + KafkaSinkConfigOptionName::Topic, + KafkaSinkConfigOptionName::CompressionType, ]; if let Some(op) = options @@ -2539,15 +2527,13 @@ fn kafka_sink_builder( )?; } - kafka_util::validate_options_for_context(&options, kafka_util::KafkaOptionCheckContext::Sink)?; - - let extracted_options: KafkaConfigOptionExtracted = options.try_into()?; - - let KafkaConfigOptionExtracted { + let KafkaSinkConfigOptionExtracted { topic, compression_type, - .. - } = extracted_options; + // TODO: plumb group ID through to sink. + group_id_prefix: _, + seen: _, + }: KafkaSinkConfigOptionExtracted = options.try_into()?; let topic_name = topic.ok_or_else(|| sql_err!("KAFKA CONNECTION must specify TOPIC"))?; diff --git a/src/sql/src/pure.rs b/src/sql/src/pure.rs index 335d093f17eee..5c7d8b1452881 100644 --- a/src/sql/src/pure.rs +++ b/src/sql/src/pure.rs @@ -32,9 +32,8 @@ use mz_sql_parser::ast::{ CreateSinkConnection, CreateSinkStatement, CreateSubsourceOption, CreateSubsourceOptionName, CsrConfigOption, CsrConfigOptionName, CsrConnection, CsrSeedAvro, CsrSeedProtobuf, CsrSeedProtobufSchema, DbzMode, DeferredItemName, DocOnIdentifier, DocOnSchema, Envelope, - Ident, KafkaConfigOption, KafkaConfigOptionName, KafkaConnection, KafkaSourceConnection, - PgConfigOption, PgConfigOptionName, RawItemName, ReaderSchemaSelectionStrategy, Statement, - UnresolvedItemName, + Ident, KafkaSourceConfigOption, KafkaSourceConfigOptionName, PgConfigOption, + PgConfigOptionName, RawItemName, ReaderSchemaSelectionStrategy, Statement, UnresolvedItemName, }; use mz_storage_types::configuration::StorageConfiguration; use mz_storage_types::connections::inline::IntoInlineConnection; @@ -47,7 +46,6 @@ use prost::Message; use protobuf_native::compiler::{SourceTreeDescriptorDatabase, VirtualSourceTree}; use protobuf_native::MessageLite; use rdkafka::admin::AdminClient; -use tracing::info; use uuid::Uuid; use crate::ast::{ @@ -56,7 +54,7 @@ use crate::ast::{ Format, ProtobufSchema, ReferencedSubsources, Value, WithOptionValue, }; use crate::catalog::{CatalogItemType, ErsatzCatalog, SessionCatalog}; -use crate::kafka_util::KafkaConfigOptionExtracted; +use crate::kafka_util::KafkaSourceConfigOptionExtracted; use crate::names::{Aug, ResolvedColumnName, ResolvedItemName}; use crate::plan::error::PlanError; use crate::plan::statement::ddl::load_generator_ast_to_generator; @@ -287,11 +285,8 @@ async fn purify_create_sink( match &connection { CreateSinkConnection::Kafka { - connection: - KafkaConnection { - connection, - options: _, - }, + connection, + options: _, key: _, } => { let scx = StatementContext::new(None, &catalog); @@ -426,7 +421,7 @@ async fn purify_create_source( let mut subsources = vec![]; let progress_desc = match &connection { - CreateSourceConnection::Kafka(_) => &mz_storage_types::sources::KAFKA_PROGRESS_DESC, + CreateSourceConnection::Kafka { .. } => &mz_storage_types::sources::KAFKA_PROGRESS_DESC, CreateSourceConnection::Postgres { .. } => &mz_storage_types::sources::PG_PROGRESS_DESC, CreateSourceConnection::LoadGenerator { .. } => { &mz_storage_types::sources::LOAD_GEN_PROGRESS_DESC @@ -437,14 +432,11 @@ async fn purify_create_source( }; match connection { - CreateSourceConnection::Kafka(KafkaSourceConnection { - connection: - KafkaConnection { - connection, - options: base_with_options, - }, + CreateSourceConnection::Kafka { + connection, + options: base_with_options, .. - }) => { + } => { if let Some(referenced_subsources) = referenced_subsources { Err(KafkaSourcePurificationError::ReferencedSubsources( referenced_subsources.clone(), @@ -465,12 +457,9 @@ async fn purify_create_source( } }; - let extracted_options: KafkaConfigOptionExtracted = + let extracted_options: KafkaSourceConfigOptionExtracted = base_with_options.clone().try_into()?; - let offset_type = - Option::::try_from(&extracted_options)?; - let topic = extracted_options .topic .ok_or(KafkaSourcePurificationError::ConnectionMissingTopic)?; @@ -490,39 +479,47 @@ async fn purify_create_source( })?; let consumer = Arc::new(consumer); - if let Some(offset_type) = offset_type { - // Translate `START TIMESTAMP` to a start offset - match kafka_util::lookup_start_offsets( - Arc::clone(&consumer), - &topic, - offset_type, - now, - ) - .await? - { - Some(start_offsets) => { - // Drop the value we are purifying - base_with_options.retain(|val| match val { - KafkaConfigOption { - name: KafkaConfigOptionName::StartTimestamp, - .. - } => false, - _ => true, - }); - info!("add start_offset {:?}", start_offsets); - base_with_options.push(KafkaConfigOption { - name: KafkaConfigOptionName::StartOffset, - value: Some(WithOptionValue::Sequence( - start_offsets - .iter() - .map(|offset| { - WithOptionValue::Value(Value::Number(offset.to_string())) - }) - .collect(), - )), - }); - } - None => {} + match ( + extracted_options.start_offset, + extracted_options.start_timestamp, + ) { + (None, None) => (), + (Some(_), Some(_)) => { + sql_bail!("cannot specify START TIMESTAMP and START OFFSET at same time") + } + (Some(start_offsets), None) => { + // Validate the start offsets. + kafka_util::validate_start_offsets( + Arc::clone(&consumer), + &topic, + start_offsets, + ) + .await?; + } + (None, Some(time_offset)) => { + // Translate `START TIMESTAMP` to a start offset. + let start_offsets = kafka_util::lookup_start_offsets( + Arc::clone(&consumer), + &topic, + time_offset, + now, + ) + .await?; + + base_with_options.retain(|val| { + !matches!(val.name, KafkaSourceConfigOptionName::StartTimestamp) + }); + base_with_options.push(KafkaSourceConfigOption { + name: KafkaSourceConfigOptionName::StartOffset, + value: Some(WithOptionValue::Sequence( + start_offsets + .iter() + .map(|offset| { + WithOptionValue::Value(Value::Number(offset.to_string())) + }) + .collect(), + )), + }); } } } @@ -1253,12 +1250,8 @@ async fn purify_csr_connection_proto( envelope: &Option, storage_configuration: &StorageConfiguration, ) -> Result<(), PlanError> { - let topic = if let CreateSourceConnection::Kafka(KafkaSourceConnection { - connection: KafkaConnection { options, .. }, - .. - }) = connection - { - let KafkaConfigOptionExtracted { topic, .. } = options + let topic = if let CreateSourceConnection::Kafka { options, .. } = connection { + let KafkaSourceConfigOptionExtracted { topic, .. } = options .clone() .try_into() .expect("already verified options valid provided"); @@ -1312,12 +1305,8 @@ async fn purify_csr_connection_avro( envelope: &Option, storage_configuration: &StorageConfiguration, ) -> Result<(), PlanError> { - let topic = if let CreateSourceConnection::Kafka(KafkaSourceConnection { - connection: KafkaConnection { options, .. }, - .. - }) = connection - { - let KafkaConfigOptionExtracted { topic, .. } = options + let topic = if let CreateSourceConnection::Kafka { options, .. } = connection { + let KafkaSourceConfigOptionExtracted { topic, .. } = options .clone() .try_into() .expect("already verified options valid provided"); From 6fbd56c4827463cafdf7105d13779170a34c7550 Mon Sep 17 00:00:00 2001 From: Nikhil Benesch Date: Sat, 9 Dec 2023 20:33:58 -0500 Subject: [PATCH 4/5] sql,storage: enable configurable group and transactional IDs for Kafka Allow users to configure the consumer group and transactional IDs when creating Kafka sources and sinks. Specifically: * The stabilized `GROUP ID PREFIX` option for Kafka sources allows setting the prefix of the consumer group ID. * The renamed and stabilized `PROGRESS GROUP ID PREFIX` option for Kafka sinks allows setting the prefix of the consumer group ID used when Materialize reads records from the sink's progress topic. * The new and instastabilized `TRANSACTIONAL ID PREFIX` option for Kafka sinks allows setting the transactional ID prefix used when Materialize produces records to the sink's data topic. Also change the default progress group and transactional ID to be materialize-{REGION-ID}-{CONNECTION-ID}-{SINK-ID} to be in line with the default group ID for Kafka sources. Importantly, the new default includes the region ID, which makes it possible to point multiple Materialize environments at the same Kafka cluster without conflicts. This is a backwards incompatible change, so this commit adds a migration to keep the group and transactional IDs for existing Kafka sinks the same. Fix #11357. Fix #23198. --- .../content/sql/system-catalog/mz_internal.md | 2 +- .../checks/all_checks/multiple_partitions.py | 1 - src/adapter/src/catalog/migrate.rs | 59 +++++- src/catalog/src/builtin.rs | 2 +- src/sql-lexer/src/keywords.txt | 1 + src/sql-parser/src/ast/defs/ddl.rs | 6 +- src/sql-parser/src/parser.rs | 29 +-- src/sql-parser/tests/testdata/ddl | 6 +- src/sql/src/kafka_util.rs | 5 +- src/sql/src/plan/statement/ddl.rs | 55 +---- src/sql/src/session/vars.rs | 7 - src/storage-client/src/sink.rs | 20 +- src/storage-types/src/sinks.proto | 2 + src/storage-types/src/sinks.rs | 64 ++++++ src/storage-types/src/sources.rs | 21 +- src/storage/src/sink/kafka.rs | 34 +-- src/storage/src/source/kafka.rs | 36 ++-- src/testdrive/src/action.rs | 4 + test/kafka-auth/kafka.jaas.config | 3 +- test/kafka-auth/mzcompose.py | 74 +++---- test/kafka-auth/test-kafka-acl-lockdown.td | 200 ++++++++++++++++++ test/kafka-multi-broker/01-init.td | 3 - .../kafka-sources/partition-change-before.td | 1 - .../autogenerated/mz_internal.slt | 2 +- test/ssh-connection/kafka-sink.td | 1 - test/ssh-connection/kafka-source.td | 1 - test/testdrive/kafka-avro-sources.td | 1 - test/testdrive/kafka-commit.td | 7 +- test/testdrive/kafka-sink-errors.td | 3 - test/testdrive/kafka-sinks.td | 5 +- test/testdrive/kafka-source-errors.td | 1 - test/testdrive/kafka-start-offset.td | 1 - test/testdrive/kafka-time-offset.td | 1 - test/testdrive/timestamps-kafka-avro-multi.td | 1 - 34 files changed, 451 insertions(+), 208 deletions(-) create mode 100644 test/kafka-auth/test-kafka-acl-lockdown.td diff --git a/doc/user/content/sql/system-catalog/mz_internal.md b/doc/user/content/sql/system-catalog/mz_internal.md index def903ef8e843..69f46ed8b5d9e 100644 --- a/doc/user/content/sql/system-catalog/mz_internal.md +++ b/doc/user/content/sql/system-catalog/mz_internal.md @@ -286,7 +286,7 @@ The `mz_kafka_sources` table contains a row for each Kafka source in the system. | Field | Type | Meaning | |------------------------|----------------|-----------------------------------------------------------------------------------------------------------| | `id` | [`text`] | The ID of the Kafka source. Corresponds to [`mz_catalog.mz_sources.id`](../mz_catalog#mz_sources). | -| `group_id_base` | [`text`] | The prefix of the group ID that Materialize will use when consuming data for the Kafka source. | +| `group_id_prefix` | [`text`] | The prefix of the group ID that Materialize will use when consuming data for the Kafka source. | ### `mz_materialization_lag` diff --git a/misc/python/materialize/checks/all_checks/multiple_partitions.py b/misc/python/materialize/checks/all_checks/multiple_partitions.py index 1cfd35dcadb90..95541e8a9adf4 100644 --- a/misc/python/materialize/checks/all_checks/multiple_partitions.py +++ b/misc/python/materialize/checks/all_checks/multiple_partitions.py @@ -28,7 +28,6 @@ def initialize(self) -> Testdrive: """ $[version>=5500] postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true - ALTER SYSTEM SET enable_kafka_config_denylist_options = true $ kafka-create-topic topic=multiple-partitions-topic diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index b1e1cecd6b5ee..223d1bea6dc58 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -13,8 +13,9 @@ use futures::future::BoxFuture; use mz_catalog::durable::Transaction; use mz_ore::collections::CollectionExt; use mz_ore::now::{EpochMillis, NowFn}; +use mz_repr::GlobalId; use mz_sql::ast::display::AstDisplay; -use mz_sql::ast::Raw; +use mz_sql::ast::{Raw, Statement}; use mz_storage_types::configuration::StorageConfiguration; use mz_storage_types::connections::ConnectionContext; use mz_storage_types::sources::GenericSourceConnection; @@ -33,7 +34,8 @@ where F: for<'a> FnMut( &'a mut Transaction<'_>, &'a &ConnCatalog<'_>, - &'a mut mz_sql::ast::Statement, + GlobalId, + &'a mut Statement, ) -> BoxFuture<'a, Result<(), anyhow::Error>>, { let mut updated_items = BTreeMap::new(); @@ -41,7 +43,7 @@ where for mut item in items { let mut stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - f(tx, &cat, &mut stmt).await?; + f(tx, &cat, item.id, &mut stmt).await?; item.create_sql = stmt.to_ast_string_stable(); @@ -70,8 +72,8 @@ pub(crate) async fn migrate( // Perform per-item AST migrations. let conn_cat = state.for_system_session(); - rewrite_items(tx, &conn_cat, |_tx, _conn_cat, _item| { - let _catalog_version = catalog_version.clone(); + rewrite_items(tx, &conn_cat, |_tx, _conn_cat, id, stmt| { + let catalog_version = catalog_version.clone(); Box::pin(async move { // Add per-item AST migrations below. // @@ -87,6 +89,10 @@ pub(crate) async fn migrate( // Migration functions may also take `tx` as input to stage // arbitrary changes to the catalog. + if catalog_version <= Version::new(0, 79, u64::MAX) { + ast_rewrite_create_sink_into_kafka_options_0_80_0(id, stmt)?; + } + Ok(()) }) }) @@ -244,6 +250,49 @@ async fn ast_rewrite_postgres_source_timeline_id_0_80_0( Ok(()) } +fn ast_rewrite_create_sink_into_kafka_options_0_80_0( + id: GlobalId, + stmt: &mut Statement, +) -> Result<(), anyhow::Error> { + use mz_sql::ast::visit_mut::VisitMut; + use mz_sql::ast::{ + CreateSinkConnection, CreateSinkStatement, KafkaSinkConfigOption, + KafkaSinkConfigOptionName, Value, WithOptionValue, + }; + + struct Rewriter { + id: GlobalId, + } + + impl<'ast> VisitMut<'ast, Raw> for Rewriter { + fn visit_create_sink_statement_mut(&mut self, node: &'ast mut CreateSinkStatement) { + match &mut node.connection { + CreateSinkConnection::Kafka { options, .. } => { + options.push(KafkaSinkConfigOption { + name: KafkaSinkConfigOptionName::ProgressGroupIdPrefix, + value: Some(WithOptionValue::Value(Value::String(format!( + "materialize-bootstrap-sink-{}", + self.id + )))), + }); + options.push(KafkaSinkConfigOption { + name: KafkaSinkConfigOptionName::TransactionalIdPrefix, + value: Some(WithOptionValue::Value(Value::String(format!( + "mz-producer-{}-0", + self.id + )))), + }); + } + } + } + } + + let mut rewriter = Rewriter { id }; + rewriter.visit_statement_mut(stmt); + + Ok(()) +} + fn _add_to_audit_log( tx: &mut Transaction, event_type: mz_audit_log::EventType, diff --git a/src/catalog/src/builtin.rs b/src/catalog/src/builtin.rs index 3d5d4be2a6af8..5850bcffb0043 100644 --- a/src/catalog/src/builtin.rs +++ b/src/catalog/src/builtin.rs @@ -1890,7 +1890,7 @@ pub static MZ_KAFKA_SOURCES: Lazy = Lazy::new(|| BuiltinTable { schema: MZ_INTERNAL_SCHEMA, desc: RelationDesc::empty() .with_column("id", ScalarType::String.nullable(false)) - .with_column("group_id_base", ScalarType::String.nullable(false)), + .with_column("group_id_prefix", ScalarType::String.nullable(false)), is_retained_metrics_object: false, sensitivity: DataSensitivity::Public, }); diff --git a/src/sql-lexer/src/keywords.txt b/src/sql-lexer/src/keywords.txt index bc1f6e8771e69..68897742026f6 100644 --- a/src/sql-lexer/src/keywords.txt +++ b/src/sql-lexer/src/keywords.txt @@ -382,6 +382,7 @@ Tpch Trace Trailing Transaction +Transactional Trim True Tunnel diff --git a/src/sql-parser/src/ast/defs/ddl.rs b/src/sql-parser/src/ast/defs/ddl.rs index ecfa5d4da245a..2a1c50486ac22 100644 --- a/src/sql-parser/src/ast/defs/ddl.rs +++ b/src/sql-parser/src/ast/defs/ddl.rs @@ -862,16 +862,18 @@ impl_display_t!(KafkaSourceConfigOption); #[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Ord, Hash)] pub enum KafkaSinkConfigOptionName { CompressionType, - GroupIdPrefix, + ProgressGroupIdPrefix, Topic, + TransactionalIdPrefix, } impl AstDisplay for KafkaSinkConfigOptionName { fn fmt(&self, f: &mut AstFormatter) { f.write_str(match self { KafkaSinkConfigOptionName::CompressionType => "COMPRESSION TYPE", - KafkaSinkConfigOptionName::GroupIdPrefix => "GROUP ID PREFIX", + KafkaSinkConfigOptionName::ProgressGroupIdPrefix => "PROGRESS GROUP ID PREFIX", KafkaSinkConfigOptionName::Topic => "TOPIC", + KafkaSinkConfigOptionName::TransactionalIdPrefix => "TRANSACTIONAL ID PREFIX", }) } } diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index a5f7b55df70e8..7a54a477d1fe4 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -2369,18 +2369,23 @@ impl<'a> Parser<'a> { fn parse_kafka_sink_config_option( &mut self, ) -> Result, ParserError> { - let name = match self.expect_one_of_keywords(&[COMPRESSION, GROUP, TOPIC])? { - COMPRESSION => { - self.expect_keyword(TYPE)?; - KafkaSinkConfigOptionName::CompressionType - } - GROUP => { - self.expect_keywords(&[ID, PREFIX])?; - KafkaSinkConfigOptionName::GroupIdPrefix - } - TOPIC => KafkaSinkConfigOptionName::Topic, - _ => unreachable!(), - }; + let name = + match self.expect_one_of_keywords(&[COMPRESSION, PROGRESS, TOPIC, TRANSACTIONAL])? { + COMPRESSION => { + self.expect_keyword(TYPE)?; + KafkaSinkConfigOptionName::CompressionType + } + PROGRESS => { + self.expect_keywords(&[GROUP, ID, PREFIX])?; + KafkaSinkConfigOptionName::ProgressGroupIdPrefix + } + TOPIC => KafkaSinkConfigOptionName::Topic, + TRANSACTIONAL => { + self.expect_keywords(&[ID, PREFIX])?; + KafkaSinkConfigOptionName::TransactionalIdPrefix + } + _ => unreachable!(), + }; Ok(KafkaSinkConfigOption { name, value: self.parse_optional_option_value()?, diff --git a/src/sql-parser/tests/testdata/ddl b/src/sql-parser/tests/testdata/ddl index 6dd6b82a56bf8..a2df7d6b3ff98 100644 --- a/src/sql-parser/tests/testdata/ddl +++ b/src/sql-parser/tests/testdata/ddl @@ -451,11 +451,11 @@ 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 }) parse-statement -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic', GROUP ID PREFIX 'prefix', COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC 'topic', PROGRESS GROUP ID PREFIX 'prefix', COMPRESSION TYPE = gzip) FORMAT BYTES ---- -CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic', GROUP ID PREFIX = 'prefix', COMPRESSION TYPE = gzip) FORMAT BYTES +CREATE SINK foo FROM bar INTO KAFKA CONNECTION baz (TOPIC = 'topic', PROGRESS GROUP ID PREFIX = 'prefix', COMPRESSION TYPE = gzip) FORMAT BYTES => -CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaSinkConfigOption { name: GroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaSinkConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [] }) +CreateSink(CreateSinkStatement { name: Some(UnresolvedItemName([Ident("foo")])), in_cluster: None, if_not_exists: false, from: Name(UnresolvedItemName([Ident("bar")])), connection: Kafka { connection: Name(UnresolvedItemName([Ident("baz")])), options: [KafkaSinkConfigOption { name: Topic, value: Some(Value(String("topic"))) }, KafkaSinkConfigOption { name: ProgressGroupIdPrefix, value: Some(Value(String("prefix"))) }, KafkaSinkConfigOption { name: CompressionType, value: Some(Ident(Ident("gzip"))) }], key: None }, format: Some(Bytes), envelope: None, with_options: [] }) parse-statement CREATE SINK FROM bar INTO KAFKA CONNECTION baz diff --git a/src/sql/src/kafka_util.rs b/src/sql/src/kafka_util.rs index b50bea7d4f70f..63f635537d5e9 100644 --- a/src/sql/src/kafka_util.rs +++ b/src/sql/src/kafka_util.rs @@ -51,8 +51,9 @@ generate_extracted_config!( KafkaSinkCompressionType, Default(KafkaSinkCompressionType::None) ), - (GroupIdPrefix, String), - (Topic, String) + (ProgressGroupIdPrefix, String), + (Topic, String), + (TransactionalIdPrefix, String) ); impl TryFromValue for KafkaSinkCompressionType { diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index 426ed7cdd5a3e..6f5395543788e 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -39,8 +39,8 @@ use mz_sql_parser::ast::{ CreateConnectionOption, CreateConnectionOptionName, CreateConnectionType, CreateTypeListOption, CreateTypeListOptionName, CreateTypeMapOption, CreateTypeMapOptionName, DeferredItemName, DocOnIdentifier, DocOnSchema, DropOwnedStatement, KafkaSinkConfigOption, - KafkaSinkConfigOptionName, MaterializedViewOption, MaterializedViewOptionName, SetRoleVar, - UnresolvedItemName, UnresolvedObjectName, UnresolvedSchemaName, Value, + MaterializedViewOption, MaterializedViewOptionName, SetRoleVar, UnresolvedItemName, + UnresolvedObjectName, UnresolvedSchemaName, Value, }; use mz_sql_parser::ident; use mz_storage_types::connections::inline::{ConnectionAccess, ReferencedConnection}; @@ -75,10 +75,10 @@ use crate::ast::{ CreateWebhookSourceStatement, CsrConfigOption, CsrConfigOptionName, CsrConnection, CsrConnectionAvro, CsrConnectionProtobuf, CsrSeedProtobuf, CsvColumns, DbzMode, DropObjectsStatement, Envelope, Expr, Format, Ident, IfExistsBehavior, IndexOption, - IndexOptionName, KafkaSourceConfigOptionName, KeyConstraint, LoadGeneratorOption, - LoadGeneratorOptionName, PgConfigOption, PgConfigOptionName, ProtobufSchema, QualifiedReplica, - ReferencedSubsources, ReplicaDefinition, ReplicaOption, ReplicaOptionName, RoleAttribute, - SourceIncludeMetadata, Statement, TableConstraint, UnresolvedDatabaseName, ViewDefinition, + IndexOptionName, KeyConstraint, LoadGeneratorOption, LoadGeneratorOptionName, PgConfigOption, + PgConfigOptionName, ProtobufSchema, QualifiedReplica, ReferencedSubsources, ReplicaDefinition, + ReplicaOption, ReplicaOptionName, RoleAttribute, SourceIncludeMetadata, Statement, + TableConstraint, UnresolvedDatabaseName, ViewDefinition, }; use crate::catalog::{ CatalogCluster, CatalogDatabase, CatalogError, CatalogItem, CatalogItemType, @@ -614,25 +614,6 @@ pub fn plan_create_source( ) } - // Starting offsets are allowed out with feature flags mode, as they are a simple, - // useful way to specify where to start reading a topic. - const ALLOWED_OPTIONS: &[KafkaSourceConfigOptionName] = &[ - KafkaSourceConfigOptionName::StartOffset, - KafkaSourceConfigOptionName::StartTimestamp, - KafkaSourceConfigOptionName::Topic, - ]; - - if let Some(op) = options - .iter() - .find(|op| !ALLOWED_OPTIONS.contains(&op.name)) - { - scx.require_feature_flag_w_dynamic_desc( - &vars::ENABLE_KAFKA_CONFIG_DENYLIST_OPTIONS, - format!("FROM KAFKA CONNECTION ({}...)", op.name.to_ast_string()), - format!("permitted options are {}", comma_separated(ALLOWED_OPTIONS)), - )?; - } - let KafkaSourceConfigOptionExtracted { group_id_prefix, topic, @@ -2509,29 +2490,11 @@ fn kafka_sink_builder( ), }; - // Starting offsets are allowed with feature flags mode, as they are a simple, - // useful way to specify where to start reading a topic. - const ALLOWED_OPTIONS: &[KafkaSinkConfigOptionName] = &[ - KafkaSinkConfigOptionName::Topic, - KafkaSinkConfigOptionName::CompressionType, - ]; - - if let Some(op) = options - .iter() - .find(|op| !ALLOWED_OPTIONS.contains(&op.name)) - { - scx.require_feature_flag_w_dynamic_desc( - &vars::ENABLE_KAFKA_CONFIG_DENYLIST_OPTIONS, - format!("FROM KAFKA CONNECTION ({}...)", op.name.to_ast_string()), - format!("permitted options are {}", comma_separated(ALLOWED_OPTIONS)), - )?; - } - let KafkaSinkConfigOptionExtracted { topic, compression_type, - // TODO: plumb group ID through to sink. - group_id_prefix: _, + progress_group_id_prefix, + transactional_id_prefix, seen: _, }: KafkaSinkConfigOptionExtracted = options.try_into()?; @@ -2640,6 +2603,8 @@ fn kafka_sink_builder( key_desc_and_indices, value_desc, compression_type, + progress_group_id_prefix, + transactional_id_prefix, })) } diff --git a/src/sql/src/session/vars.rs b/src/sql/src/session/vars.rs index 2a143372757c6..2a762a7a9849d 100644 --- a/src/sql/src/session/vars.rs +++ b/src/sql/src/session/vars.rs @@ -1854,13 +1854,6 @@ feature_flags!( internal: true, enable_for_item_parsing: true, }, - { - name: enable_kafka_config_denylist_options, - desc: "Kafka sources with non-allowlisted options", - default: false, - internal: true, - enable_for_item_parsing: true, - }, { name: enable_list_length_max, desc: "the list_length_max function", diff --git a/src/storage-client/src/sink.rs b/src/storage-client/src/sink.rs index f123f4c155552..e2a6796c664e0 100644 --- a/src/storage-client/src/sink.rs +++ b/src/storage-client/src/sink.rs @@ -18,7 +18,7 @@ use mz_kafka_util::client::{ }; use mz_ore::collections::CollectionExt; use mz_ore::task; -use mz_repr::{GlobalId, Timestamp}; +use mz_repr::Timestamp; use mz_storage_types::configuration::StorageConfiguration; use mz_storage_types::errors::{ContextCreationError, ContextCreationErrorExt}; use mz_storage_types::sinks::KafkaSinkConnection; @@ -32,15 +32,6 @@ use tracing::{info, warn}; use crate::sink::progress_key::ProgressKey; -/// Formatter for Kafka group.id setting -pub struct SinkGroupId; - -impl SinkGroupId { - pub fn new(sink_id: GlobalId) -> String { - format!("materialize-bootstrap-sink-{sink_id}") - } -} - pub mod progress_key { use std::fmt; @@ -333,6 +324,8 @@ pub async fn build_kafka( storage_configuration: &StorageConfiguration, ) -> Result, ContextCreationError> { // Fetch the progress of the last incarnation of the sink, if any. + let client_id = connection.client_id(&storage_configuration.connection_context, sink_id); + let group_id = connection.progress_group_id(&storage_configuration.connection_context, sink_id); let progress_topic = connection .progress_topic(&storage_configuration.connection_context) .into_owned(); @@ -345,7 +338,12 @@ pub async fn build_kafka( storage_configuration, MzClientContext::default(), &btreemap! { - "group.id" => SinkGroupId::new(sink_id), + // Consumer group ID, which may have been overridden by the + // user. librdkafka requires this, even though we'd prefer + // to disable the consumer group protocol entirely. + "group.id" => group_id.clone(), + // Allow Kafka monitoring tools to identify this consumer. + "client.id" => client_id.clone(), "isolation.level" => isolation_level.into(), "enable.auto.commit" => "false".into(), "auto.offset.reset" => "earliest".into(), diff --git a/src/storage-types/src/sinks.proto b/src/storage-types/src/sinks.proto index cbec04664168a..8f2761ddb36c5 100644 --- a/src/storage-types/src/sinks.proto +++ b/src/storage-types/src/sinks.proto @@ -92,6 +92,8 @@ message ProtoKafkaSinkConnectionV2 { google.protobuf.Empty lz4 = 17; google.protobuf.Empty zstd = 18; } + optional string progress_group_id_prefix = 19; + optional string transactional_id_prefix = 20; } message ProtoPersistSinkConnection { diff --git a/src/storage-types/src/sinks.rs b/src/storage-types/src/sinks.rs index 2f147d53663fe..7e69258a442f4 100644 --- a/src/storage-types/src/sinks.rs +++ b/src/storage-types/src/sinks.rs @@ -432,14 +432,60 @@ pub struct KafkaSinkConnection { pub topic: String, pub fuel: usize, pub compression_type: KafkaSinkCompressionType, + pub progress_group_id_prefix: Option, + pub transactional_id_prefix: Option, } impl KafkaSinkConnection { + /// Returns the client ID to register with librdkafka with. + /// + /// The caller is responsible for providing the sink ID as it is not known + /// to `KafkaSinkConnection`. + pub fn client_id(&self, connection_context: &ConnectionContext, sink_id: GlobalId) -> String { + format!( + "materialize-{}-{}-{}", + connection_context.environment_id, self.connection_id, sink_id, + ) + } + /// Returns the name of the progress topic to use for the sink. pub fn progress_topic(&self, connection_context: &ConnectionContext) -> Cow { self.connection .progress_topic(connection_context, self.connection_id) } + + /// Returns the ID for the consumer group the sink will use to read the + /// progress topic on resumption. + /// + /// The caller is responsible for providing the sink ID as it is not known + /// to `KafkaSinkConnection`. + pub fn progress_group_id( + &self, + connection_context: &ConnectionContext, + sink_id: GlobalId, + ) -> String { + format!( + "{}{}", + self.progress_group_id_prefix.as_deref().unwrap_or(""), + self.client_id(connection_context, sink_id) + ) + } + + /// Returns the transactional ID to use for the sink. + /// + /// The caller is responsible for providing the sink ID as it is not known + /// to `KafkaSinkConnection`. + pub fn transactional_id( + &self, + connection_context: &ConnectionContext, + sink_id: GlobalId, + ) -> String { + format!( + "{}{}", + self.transactional_id_prefix.as_deref().unwrap_or(""), + self.client_id(connection_context, sink_id) + ) + } } impl KafkaSinkConnection { @@ -466,6 +512,8 @@ impl KafkaSinkConnection { topic, fuel, compression_type, + progress_group_id_prefix, + transactional_id_prefix: transactional_id, } = self; let compatibility_checks = [ @@ -486,6 +534,14 @@ impl KafkaSinkConnection { compression_type == &other.compression_type, "compression_type", ), + ( + progress_group_id_prefix == &other.progress_group_id_prefix, + "progress_group_id_prefix", + ), + ( + transactional_id == &other.transactional_id_prefix, + "transactional_id", + ), ]; for (compatible, field) in compatibility_checks { if !compatible { @@ -517,6 +573,8 @@ impl IntoInlineConnection topic, fuel, compression_type, + progress_group_id_prefix, + transactional_id_prefix: transactional_id, } = self; KafkaSinkConnection { connection_id, @@ -528,6 +586,8 @@ impl IntoInlineConnection topic, fuel, compression_type, + progress_group_id_prefix, + transactional_id_prefix: transactional_id, } } } @@ -551,6 +611,8 @@ impl RustType for KafkaSinkConnection { KafkaSinkCompressionType::Lz4 => CompressionType::Lz4(()), KafkaSinkCompressionType::Zstd => CompressionType::Zstd(()), }), + progress_group_id_prefix: self.progress_group_id_prefix.clone(), + transactional_id_prefix: self.transactional_id_prefix.clone(), } } @@ -585,6 +647,8 @@ impl RustType for KafkaSinkConnection { )) } }, + progress_group_id_prefix: proto.progress_group_id_prefix, + transactional_id_prefix: proto.transactional_id_prefix, }) } } diff --git a/src/storage-types/src/sources.rs b/src/storage-types/src/sources.rs index 99ee1c526fea1..275e20faa5caa 100644 --- a/src/storage-types/src/sources.rs +++ b/src/storage-types/src/sources.rs @@ -1438,17 +1438,26 @@ pub static KAFKA_PROGRESS_DESC: Lazy = Lazy::new(|| { }); impl KafkaSourceConnection { - /// Returns the id for the consumer group the configured source will use. + /// Returns the client ID to register with librdkafka with. + /// + /// The caller is responsible for providing the source ID as it is not known + /// to `KafkaSourceConnection`. + pub fn client_id(&self, connection_context: &ConnectionContext, source_id: GlobalId) -> String { + format!( + "materialize-{}-{}-{}", + connection_context.environment_id, self.connection_id, source_id, + ) + } + + /// Returns the ID for the consumer group the configured source will use. /// /// The caller is responsible for providing the source ID as it is not known /// to `KafkaSourceConnection`. pub fn group_id(&self, connection_context: &ConnectionContext, source_id: GlobalId) -> String { format!( - "{}materialize-{}-{}-{}", - self.group_id_prefix.clone().unwrap_or_else(String::new), - connection_context.environment_id, - self.connection_id, - source_id, + "{}{}", + self.group_id_prefix.as_deref().unwrap_or(""), + self.client_id(connection_context, source_id) ) } } diff --git a/src/storage/src/sink/kafka.rs b/src/storage/src/sink/kafka.rs index 26fe5392b41f0..cedab5fbcda21 100644 --- a/src/storage/src/sink/kafka.rs +++ b/src/storage/src/sink/kafka.rs @@ -220,7 +220,6 @@ struct KafkaTxProducerConfig<'a> { storage_configuration: &'a StorageConfiguration, producer_context: SinkProducerContext, sink_id: GlobalId, - worker_id: String, } #[derive(Clone)] @@ -238,31 +237,11 @@ impl KafkaTxProducer { storage_configuration, producer_context, sink_id, - worker_id, }: KafkaTxProducerConfig<'a>, ) -> Result { - // Create a producer with the old transactional id to fence out writers from older - // versions, if any. This section should be removed after it has been running in - // production for a week or two. - let fence_producer: ThreadedProducer<_> = connection - .connection - .create_with_context( - storage_configuration, - MzClientContext::default(), - &btreemap! { - "transactional.id" => format!("mz-producer-{sink_id}-{worker_id}"), - }, - ) - .await?; - let fence_producer = Arc::new(fence_producer); - - task::spawn_blocking(|| format!("init_transactions:{name}"), { - let fence_producer = Arc::clone(&fence_producer); - move || fence_producer.init_transactions(Duration::from_secs(5)) - }) - .unwrap_or_else(|_| Err(KafkaError::Canceled)) - .await - .check_ssh_status(fence_producer.context())?; + let client_id = connection.client_id(&storage_configuration.connection_context, sink_id); + let transactional_id = + connection.transactional_id(&storage_configuration.connection_context, sink_id); let producer = connection .connection @@ -270,6 +249,8 @@ impl KafkaTxProducer { storage_configuration, producer_context, &btreemap! { + // Allow Kafka monitoring tools to identify this producer. + "client.id" => client_id, // Ensure that messages are sinked in order and without // duplicates. Note that this only applies to a single // instance of a producer - in the case of restarts, all @@ -296,7 +277,8 @@ impl KafkaTxProducer { // different settings for this value to see if it makes a // big difference. "queue.buffering.max.ms" => format!("{}", 10), - "transactional.id" => format!("mz-producer-{sink_id}-0"), + // Use the transactional ID requested by the user. + "transactional.id" => transactional_id, // Use the default transaction timeout. (At time of writing: 60s.) // The Kafka sink may have long-running transactions, since it expects // to be able to write all data at the same timestamp in a single @@ -478,7 +460,6 @@ impl KafkaSinkState { healthchecker: HealthOutputHandle, ) -> (Self, Option) { let metrics = Arc::new(metrics.get_sink_metrics(&connection.topic, sink_id, worker_id)); - let worker_id = worker_id.to_string(); let retry_manager = Arc::new(Mutex::new(KafkaSinkSendRetryManager::new())); @@ -502,7 +483,6 @@ impl KafkaSinkState { storage_configuration, producer_context, sink_id, - worker_id, }) .await })() diff --git a/src/storage/src/source/kafka.rs b/src/storage/src/source/kafka.rs index 71961b2023a0c..a79f4b64c76b9 100644 --- a/src/storage/src/source/kafka.rs +++ b/src/storage/src/source/kafka.rs @@ -163,6 +163,7 @@ impl SourceRender for KafkaSourceConnection { let button = builder.build(move |caps| async move { let [mut data_cap, mut progress_cap, health_cap]: [_; 3] = caps.try_into().unwrap(); + let client_id = self.client_id(&config.config.connection_context, config.id); let group_id = self.group_id(&config.config.connection_context, config.id); let KafkaSourceConnection { connection, @@ -235,9 +236,10 @@ impl SourceRender for KafkaSourceConnection { inner: MzClientContext::default(), }, &btreemap! { - // Default to disabling Kafka auto commit. This can be - // explicitly enabled by the user if they want to use it for - // progress tracking. + // Disable Kafka auto commit. We manually commit offsets + // to Kafka once we have reclocked those offsets, so + // that users can use standard Kafka tools for progress + // tracking. "enable.auto.commit" => "false".into(), // Always begin ingest at 0 when restarted, even if Kafka // contains committed consumer read offsets @@ -247,23 +249,15 @@ impl SourceRender for KafkaSourceConnection { "topic.metadata.refresh.interval.ms" => topic_metadata_refresh_interval.as_millis().to_string(), // TODO: document the rationale for this. "fetch.message.max.bytes" => "134217728".into(), - // Consumer group ID. librdkafka requires this, and we use - // offset committing to provide a way for users to monitor - // ingest progress, though we do not rely on the committed - // offsets for any functionality. - // - // The group ID is partially dictated by the user and - // partially dictated by us. Users can set a prefix so they - // can see which consumers belong to which Materialize - // deployment, but we set a suffix to ensure uniqueness. A - // unique consumer group ID is the most surefire way to - // ensure that librdkafka does not try to perform its own - // consumer group balancing, which would wreak havoc with - // our careful partition assignment strategy. + // Consumer group ID, which may have been overridden by + // the user. librdkafka requires this, and we use offset + // committing to provide a way for users to monitor + // ingest progress, though we do not rely on the + // committed offsets for any functionality. "group.id" => group_id.clone(), - // We just use the `group.id` as the `client.id`, for simplicity, - // as we present to kafka as a single consumer. - "client.id" => group_id, + // Allow Kafka monitoring tools to identify this + // consumer. + "client.id" => client_id.clone(), }, ) .await; @@ -319,7 +313,7 @@ impl SourceRender for KafkaSourceConnection { // We want a fairly low ceiling on our polling frequency, since we rely // on this heartbeat to determine the health of our Kafka connection. - let topic_metadata_refresh_interval = + let poll_interval = topic_metadata_refresh_interval.min(Duration::from_secs(60)); let status_report = Arc::clone(&health_status); @@ -383,7 +377,7 @@ impl SourceRender for KafkaSourceConnection { } } } - thread::park_timeout(topic_metadata_refresh_interval); + thread::park_timeout(poll_interval); } info!( source_id = config.id.to_string(), diff --git a/src/testdrive/src/action.rs b/src/testdrive/src/action.rs index 42da78491392d..09f1f11fe63c0 100644 --- a/src/testdrive/src/action.rs +++ b/src/testdrive/src/action.rs @@ -272,6 +272,10 @@ impl State { .unwrap_or_else(String::new), ); } + self.cmd_vars.insert( + "testdrive.materialize-environment-id".into(), + self.environment_id.to_string(), + ); self.cmd_vars.insert( "testdrive.materialize-sql-addr".into(), self.materialize_sql_addr.clone(), diff --git a/test/kafka-auth/kafka.jaas.config b/test/kafka-auth/kafka.jaas.config index f5abe36937570..c6f5657e77e4b 100644 --- a/test/kafka-auth/kafka.jaas.config +++ b/test/kafka-auth/kafka.jaas.config @@ -10,6 +10,7 @@ KafkaServer { org.apache.kafka.common.security.plain.PlainLoginModule required user_materialize=sekurity - user_materialize_no_describe_configs=sekurity; + user_materialize_no_describe_configs=sekurity + user_materialize_lockdown=sekurity; org.apache.kafka.common.security.scram.ScramLoginModule required; }; diff --git a/test/kafka-auth/mzcompose.py b/test/kafka-auth/mzcompose.py index 1fbc2dc418aba..14f6855e4e84a 100644 --- a/test/kafka-auth/mzcompose.py +++ b/test/kafka-auth/mzcompose.py @@ -189,46 +189,22 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: # Restrict the `materialize_no_describe_configs` user from running the # `DescribeConfigs` cluster operation, but allow it to idempotently read and # write to all topics. - c.exec( - "kafka", - "kafka-acls", - "--bootstrap-server", - "localhost:9092", - "--add", - "--cluster", - "--deny-principal=User:materialize_no_describe_configs", - "--operation=DescribeConfigs", - ) - c.exec( - "kafka", - "kafka-acls", - "--bootstrap-server", - "localhost:9092", - "--add", - "--allow-principal=User:materialize_no_describe_configs", - "--operation=ALL", - "--transactional-id=*", - ) - c.exec( - "kafka", - "kafka-acls", - "--bootstrap-server", - "localhost:9092", - "--add", - "--allow-principal=User:materialize_no_describe_configs", - "--operation=ALL", - "--topic=*", - ) - c.exec( - "kafka", - "kafka-acls", - "--bootstrap-server", - "localhost:9092", - "--add", - "--allow-principal=User:materialize_no_describe_configs", - "--operation=ALL", - "--group=*", + user = "materialize_no_describe_configs" + add_acl(c, user, "deny", "DescribeConfigs") + add_acl(c, user, "allow", "ALL", "transactional-id=*") + add_acl(c, user, "allow", "ALL", "topic=*") + add_acl(c, user, "allow", "ALL", "group=*") + + # Only allow the `materialize_lockdown` user access to specific + # transactional IDs, topics, and group IDs. + user = "materialize_lockdown" + add_acl( + c, user, "allow", "ALL", "transactional-id=lockdown", pattern_type="prefixed" ) + add_acl(c, user, "allow", "ALL", "topic=lockdown-progress") + add_acl(c, user, "allow", "ALL", "topic=lockdown-data", pattern_type="prefixed") + add_acl(c, user, "allow", "ALL", "group=lockdown", pattern_type="prefixed") + add_acl(c, user, "allow", "ALL", "topic=testdrive-data", pattern_type="prefixed") # Now that the Kafka topic has been bootstrapped, it's safe to bring up all # the other schema registries in parallel. @@ -283,3 +259,23 @@ def workflow_default(c: Composition, parser: WorkflowArgumentParser) -> None: ) c.run("testdrive", f"test-{args.filter}.td") + + +def add_acl( + c: Composition, + user: str, + action: str, + operation: str, + resource: str | None = None, + pattern_type: str = "literal", +): + c.exec( + "kafka", + "kafka-acls", + "--bootstrap-server=localhost:9092", + "--add", + f"--{action}-principal=User:{user}", + f"--operation={operation}", + f"--{resource}" if resource else "--cluster", + f"--resource-pattern-type={pattern_type}", + ) diff --git a/test/kafka-auth/test-kafka-acl-lockdown.td b/test/kafka-auth/test-kafka-acl-lockdown.td new file mode 100644 index 0000000000000..2a63114c83abd --- /dev/null +++ b/test/kafka-auth/test-kafka-acl-lockdown.td @@ -0,0 +1,200 @@ +# Copyright Materialize, Inc. and contributors. All rights reserved. +# +# Use of this software is governed by the Business Source License +# included in the LICENSE file at the root of this repository. +# +# As of the Change Date specified in that file, in accordance with +# the Business Source License, use of this software will be governed +# by the Apache License, Version 2.0. + +# Test that Materialize can create a sink using a Kafka user with restricted +# access to consumer groups and transactional IDs. + +# ==> Set up. <== + +$ kafka-create-topic topic=data +$ kafka-ingest topic=data format=bytes +banana + +> CREATE SECRET kafka_password AS 'sekurity' + +> CREATE CONNECTION kafka_bad_progress_topic TO KAFKA ( + BROKER 'kafka:9095', + SASL MECHANISMS = 'PLAIN', + SASL USERNAME = 'materialize_lockdown', + SASL PASSWORD = SECRET kafka_password, + SECURITY PROTOCOL SASL_PLAINTEXT + ); + +> CREATE CONNECTION kafka_good_progress_topic TO KAFKA ( + BROKER 'kafka:9095', + SASL MECHANISMS = 'PLAIN', + SASL USERNAME = 'materialize_lockdown', + SASL PASSWORD = SECRET kafka_password, + SECURITY PROTOCOL SASL_PLAINTEXT, + PROGRESS TOPIC = 'lockdown-progress' + ); + +> CREATE TABLE t (column1 integer) +> INSERT INTO t VALUES (1), (2) +> CREATE MATERIALIZED VIEW mv AS SELECT DISTINCT column1 FROM t + +# ==> Test. <== + +# The default group ID prefix is not usable by the `materialize_lockdown` +# user. + +> CREATE SOURCE broken + FROM KAFKA CONNECTION kafka_good_progress_topic ( + TOPIC 'testdrive-data-${testdrive.seed}' + ) + FORMAT TEXT + +> SELECT EXISTS ( + SELECT 1 + FROM mz_sources + JOIN mz_internal.mz_source_status_history ON mz_sources.id = mz_source_status_history.source_id + WHERE name = 'broken' + AND error ILIKE '%error when polling consumer for source%Group authorization failed%' + ) +true + +> DROP SOURCE broken + +# The default group ID prefix *is* writeable by the `materialize_lockdown` user. +# Ensure that offsets are committed. + +> CREATE SOURCE working_source + FROM KAFKA CONNECTION kafka_good_progress_topic ( + TOPIC 'testdrive-data-${testdrive.seed}', + GROUP ID PREFIX 'lockdown-' + ) + FORMAT TEXT + +> SELECT * FROM working_source +banana + +$ set-from-sql var=conn-id +SELECT id FROM mz_connections WHERE name = 'kafka_good_progress_topic' + +$ set-from-sql var=source-id +SELECT id FROM mz_sources WHERE name = 'working_source' + +$ kafka-verify-commit topic=data partition=0 consumer-group-id=lockdown-materialize-${testdrive.materialize-environment-id}-${conn-id}-${source-id} +1 + +# A sink which uses a bad transactional ID should fail. +> CREATE SINK broken1 FROM mv + INTO KAFKA CONNECTION kafka_bad_progress_topic ( + TOPIC 'testdrive-broken-${testdrive.seed}' + ) + KEY (column1) FORMAT JSON ENVELOPE UPSERT +> SELECT EXISTS ( + SELECT 1 + FROM mz_sinks + JOIN mz_internal.mz_sink_status_history ON mz_sinks.id = mz_sink_status_history.sink_id + WHERE name = 'broken1' + AND error ILIKE '%Transactional Id authorization failed%' + ) +true +> DROP SINK broken1 + +# A sink which uses a good transactional ID but a bad progress topic should +# fail. +> CREATE SINK broken2 FROM mv + INTO KAFKA CONNECTION kafka_bad_progress_topic ( + TOPIC 'testdrive-broken-${testdrive.seed}', + TRANSACTIONAL ID PREFIX 'lockdown' + ) + KEY (column1) FORMAT JSON ENVELOPE UPSERT +> SELECT EXISTS ( + SELECT 1 + FROM mz_sinks + JOIN mz_internal.mz_sink_status_history ON mz_sinks.id = mz_sink_status_history.sink_id + WHERE name = 'broken2' + AND error ILIKE '%Unable to fetch metadata about progress topic%Topic authorization failed%' + ) +true +> DROP SINK broken2 + +# A sink which uses a good transactional ID and progress topic but a bad data +# topic should fail. +> CREATE SINK broken3 FROM mv + INTO KAFKA CONNECTION kafka_good_progress_topic ( + TOPIC 'testdrive-broken-${testdrive.seed}', + TRANSACTIONAL ID PREFIX 'lockdown' + ) + KEY (column1) FORMAT JSON ENVELOPE UPSERT +> SELECT EXISTS ( + SELECT 1 + FROM mz_sinks + JOIN mz_internal.mz_sink_status_history ON mz_sinks.id = mz_sink_status_history.sink_id + WHERE name = 'broken3' + AND error ILIKE '%Error creating topic testdrive-broken-${testdrive.seed}%Topic authorization failed%' + ) +true +> DROP SINK broken3 + +# A sink which uses a good transactional ID, progress topic, and data topic +# but a bad group ID prefix will fail, but only after restart when the progress +# topic contains entries. + +> CREATE CLUSTER c (SIZE = '1') + +> CREATE SINK broken4 IN CLUSTER c FROM mv + INTO KAFKA CONNECTION kafka_good_progress_topic ( + TOPIC 'lockdown-data1', + TRANSACTIONAL ID PREFIX 'lockdown' + ) + KEY (column1) FORMAT JSON ENVELOPE UPSERT + +$ kafka-verify-data format=json key=false sink=materialize.public.broken4 sort-messages=true +{"column1": 1} +{"column1": 2} + +# Resize the cluster on which the sink is running to force the sink to restart. +> ALTER CLUSTER c SET (SIZE = '2') + +> SELECT EXISTS ( + SELECT 1 + FROM mz_sinks + JOIN mz_internal.mz_sink_status_history ON mz_sinks.id = mz_sink_status_history.sink_id + WHERE name = 'broken4' + AND error ILIKE '%failed to fetch progress message%Group authorization failed%' + ) +true + +> DROP SINK broken4 + +# A sink which uses a good transactional ID, progress topic, data topic, and +# group ID prefix should work. + +> CREATE SINK working IN CLUSTER c FROM mv + INTO KAFKA CONNECTION kafka_good_progress_topic ( + TOPIC 'lockdown-data2', + TRANSACTIONAL ID PREFIX 'lockdown', + PROGRESS GROUP ID PREFIX 'lockdown' + ) + KEY (column1) FORMAT JSON ENVELOPE UPSERT + +# Validate that the sink is actually emitting data. Success of the `CREATE SINK` +# command itself is not sufficient validation. +$ kafka-verify-data format=json key=false sink=materialize.public.working sort-messages=true +{"column1": 1} +{"column1": 2} + +# Resize the cluster on which the sink is running to force the sink to restart. +> ALTER CLUSTER c SET (SIZE = '1') + +# Ensure that the sink is emitting new messages. +> INSERT INTO t VALUES (3) +$ kafka-verify-data format=json key=false sink=materialize.public.working sort-messages=true +{"column1": 3} + +# Ensure that the sink never entered the `stalled` status. +> SELECT DISTINCT status FROM mz_sinks + JOIN mz_internal.mz_sink_status_history ON mz_sinks.id = mz_sink_status_history.sink_id + WHERE mz_sinks.name = 'working' +starting +running +paused diff --git a/test/kafka-multi-broker/01-init.td b/test/kafka-multi-broker/01-init.td index eed08e26c074a..6e84896df260e 100644 --- a/test/kafka-multi-broker/01-init.td +++ b/test/kafka-multi-broker/01-init.td @@ -44,9 +44,6 @@ $ kafka-ingest format=avro topic=kafka-multi-broker schema=${schema} timestamp=6 URL '${testdrive.schema-registry-url}' ); -$ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} -ALTER SYSTEM SET enable_kafka_config_denylist_options = true - > CREATE SINK multi_broker_sink FROM kafka_multi_broker INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-kafka-multi-broker-sink-${testdrive.seed}') diff --git a/test/persistence/kafka-sources/partition-change-before.td b/test/persistence/kafka-sources/partition-change-before.td index f83607bf20d45..23cdf4f08fa77 100644 --- a/test/persistence/kafka-sources/partition-change-before.td +++ b/test/persistence/kafka-sources/partition-change-before.td @@ -9,7 +9,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true # # Change the number of partitions in the topic just prior to restart diff --git a/test/sqllogictest/autogenerated/mz_internal.slt b/test/sqllogictest/autogenerated/mz_internal.slt index 7e752273b999c..86bb2e6ee27f1 100644 --- a/test/sqllogictest/autogenerated/mz_internal.slt +++ b/test/sqllogictest/autogenerated/mz_internal.slt @@ -161,7 +161,7 @@ query ITT SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_kafka_sources' ORDER BY position ---- 1 id text -2 group_id_base text +2 group_id_prefix text query ITT SELECT position, name, type FROM objects WHERE schema = 'mz_internal' AND object = 'mz_materialization_lag' ORDER BY position diff --git a/test/ssh-connection/kafka-sink.td b/test/ssh-connection/kafka-sink.td index ad29969f2212a..2f600295a0ae8 100644 --- a/test/ssh-connection/kafka-sink.td +++ b/test/ssh-connection/kafka-sink.td @@ -9,7 +9,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true # Test creating a Kafka sink using ssh. diff --git a/test/ssh-connection/kafka-source.td b/test/ssh-connection/kafka-source.td index 88704ed43b3b2..af54cbcc63a27 100644 --- a/test/ssh-connection/kafka-source.td +++ b/test/ssh-connection/kafka-source.td @@ -9,7 +9,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true # Test creating a Kafka source using SSH, with and without the CSR diff --git a/test/testdrive/kafka-avro-sources.td b/test/testdrive/kafka-avro-sources.td index d2ceda62184d0..a752edeed2fe0 100644 --- a/test/testdrive/kafka-avro-sources.td +++ b/test/testdrive/kafka-avro-sources.td @@ -9,7 +9,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true # Test support for Avro sources without using the Confluent Schema Registry. diff --git a/test/testdrive/kafka-commit.td b/test/testdrive/kafka-commit.td index 8ab658d1d9c28..04d8b83a0f156 100644 --- a/test/testdrive/kafka-commit.td +++ b/test/testdrive/kafka-commit.td @@ -7,9 +7,6 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -$ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} -ALTER SYSTEM SET enable_kafka_config_denylist_options = true - # Test that the source ingestion pipeline commits offsets back to Kafka with # the expected group ID. @@ -39,7 +36,7 @@ three $ set-from-sql var=consumer-group-id SELECT - ks.group_id_base + ks.group_id_prefix FROM mz_sources s JOIN mz_internal.mz_kafka_sources ks ON s.id = ks.id WHERE s.name = 'topic' @@ -65,7 +62,7 @@ three $ set-from-sql var=consumer-group-id SELECT - ks.group_id_base + ks.group_id_prefix FROM mz_sources s JOIN mz_internal.mz_kafka_sources ks ON s.id = ks.id WHERE s.name = 'topic' diff --git a/test/testdrive/kafka-sink-errors.td b/test/testdrive/kafka-sink-errors.td index ba5ecd3790ed9..6f999e91bca04 100644 --- a/test/testdrive/kafka-sink-errors.td +++ b/test/testdrive/kafka-sink-errors.td @@ -7,9 +7,6 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -$ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} -ALTER SYSTEM SET enable_kafka_config_denylist_options = true - # # Test various errors that can happen at CREATE SINK time and check how they are reported to the user # diff --git a/test/testdrive/kafka-sinks.td b/test/testdrive/kafka-sinks.td index c17bbf1c1211b..487ab977bec3b 100644 --- a/test/testdrive/kafka-sinks.td +++ b/test/testdrive/kafka-sinks.td @@ -7,9 +7,6 @@ # the Business Source License, use of this software will be governed # by the Apache License, Version 2.0. -$ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} -ALTER SYSTEM SET enable_kafka_config_denylist_options = true - $ kafka-create-topic topic=test partitions=1 $ kafka-ingest topic=test format=bytes jack,jill @@ -241,7 +238,7 @@ $ kafka-create-topic topic=snk9 partitions=4 2 # Including for sinks with default size -> SELECT size FROM mz_sinks WHERE name = 'snk13' +> SELECT size FROM mz_sinks WHERE name = 'snk9' "${arg.default-storage-size}" # Check that SHOW SINKS shows the size correctly diff --git a/test/testdrive/kafka-source-errors.td b/test/testdrive/kafka-source-errors.td index 2bf7401b4b69e..509c51b08b01a 100644 --- a/test/testdrive/kafka-source-errors.td +++ b/test/testdrive/kafka-source-errors.td @@ -11,7 +11,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true > CREATE CONNECTION kafka_conn TO KAFKA (BROKER '${testdrive.kafka-addr}', SECURITY PROTOCOL PLAINTEXT); diff --git a/test/testdrive/kafka-start-offset.td b/test/testdrive/kafka-start-offset.td index b78f79359a022..55cb833b03ac5 100644 --- a/test/testdrive/kafka-start-offset.td +++ b/test/testdrive/kafka-start-offset.td @@ -9,7 +9,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true $ kafka-create-topic topic=t0 partitions=1 diff --git a/test/testdrive/kafka-time-offset.td b/test/testdrive/kafka-time-offset.td index 9c0b6d9a3c981..5b0f14130a108 100644 --- a/test/testdrive/kafka-time-offset.td +++ b/test/testdrive/kafka-time-offset.td @@ -9,7 +9,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true # Tests for `START TIMESTAMP` configuration which resolves a start offset # during creation of the source. diff --git a/test/testdrive/timestamps-kafka-avro-multi.td b/test/testdrive/timestamps-kafka-avro-multi.td index 5c3f0c709f775..d363f35e9a5ac 100644 --- a/test/testdrive/timestamps-kafka-avro-multi.td +++ b/test/testdrive/timestamps-kafka-avro-multi.td @@ -9,7 +9,6 @@ $ postgres-execute connection=postgres://mz_system:materialize@${testdrive.materialize-internal-sql-addr} ALTER SYSTEM SET enable_create_source_denylist_with_options = true -ALTER SYSTEM SET enable_kafka_config_denylist_options = true $ set-sql-timeout duration=60s From b61fc78df0894c74c32bc295702f99f7203094f3 Mon Sep 17 00:00:00 2001 From: Petros Angelatos Date: Wed, 13 Dec 2023 18:20:37 +0200 Subject: [PATCH 5/5] support legacy ids with additional syntax Signed-off-by: Petros Angelatos --- src/adapter/src/catalog/migrate.rs | 30 +++------ src/sql-lexer/src/keywords.txt | 2 + src/sql-parser/src/ast/defs/ddl.rs | 2 + src/sql-parser/src/parser.rs | 43 ++++++++----- src/sql/src/kafka_util.rs | 3 +- src/sql/src/plan/statement/ddl.rs | 23 ++++++- src/sql/src/pure.rs | 10 ++- src/storage-types/src/sinks.proto | 14 ++++- src/storage-types/src/sinks.rs | 96 +++++++++++++++++++++-------- test/testdrive/kafka-sink-errors.td | 6 ++ 10 files changed, 156 insertions(+), 73 deletions(-) diff --git a/src/adapter/src/catalog/migrate.rs b/src/adapter/src/catalog/migrate.rs index 223d1bea6dc58..b31e0f64e7c5d 100644 --- a/src/adapter/src/catalog/migrate.rs +++ b/src/adapter/src/catalog/migrate.rs @@ -13,7 +13,6 @@ use futures::future::BoxFuture; use mz_catalog::durable::Transaction; use mz_ore::collections::CollectionExt; use mz_ore::now::{EpochMillis, NowFn}; -use mz_repr::GlobalId; use mz_sql::ast::display::AstDisplay; use mz_sql::ast::{Raw, Statement}; use mz_storage_types::configuration::StorageConfiguration; @@ -34,7 +33,6 @@ where F: for<'a> FnMut( &'a mut Transaction<'_>, &'a &ConnCatalog<'_>, - GlobalId, &'a mut Statement, ) -> BoxFuture<'a, Result<(), anyhow::Error>>, { @@ -43,7 +41,7 @@ where for mut item in items { let mut stmt = mz_sql::parse::parse(&item.create_sql)?.into_element().ast; - f(tx, &cat, item.id, &mut stmt).await?; + f(tx, &cat, &mut stmt).await?; item.create_sql = stmt.to_ast_string_stable(); @@ -72,7 +70,7 @@ pub(crate) async fn migrate( // Perform per-item AST migrations. let conn_cat = state.for_system_session(); - rewrite_items(tx, &conn_cat, |_tx, _conn_cat, id, stmt| { + rewrite_items(tx, &conn_cat, |_tx, _conn_cat, stmt| { let catalog_version = catalog_version.clone(); Box::pin(async move { // Add per-item AST migrations below. @@ -90,7 +88,7 @@ pub(crate) async fn migrate( // arbitrary changes to the catalog. if catalog_version <= Version::new(0, 79, u64::MAX) { - ast_rewrite_create_sink_into_kafka_options_0_80_0(id, stmt)?; + ast_rewrite_create_sink_into_kafka_options_0_80_0(stmt)?; } Ok(()) @@ -251,7 +249,6 @@ async fn ast_rewrite_postgres_source_timeline_id_0_80_0( } fn ast_rewrite_create_sink_into_kafka_options_0_80_0( - id: GlobalId, stmt: &mut Statement, ) -> Result<(), anyhow::Error> { use mz_sql::ast::visit_mut::VisitMut; @@ -260,35 +257,22 @@ fn ast_rewrite_create_sink_into_kafka_options_0_80_0( KafkaSinkConfigOptionName, Value, WithOptionValue, }; - struct Rewriter { - id: GlobalId, - } + struct Rewriter; impl<'ast> VisitMut<'ast, Raw> for Rewriter { fn visit_create_sink_statement_mut(&mut self, node: &'ast mut CreateSinkStatement) { match &mut node.connection { CreateSinkConnection::Kafka { options, .. } => { options.push(KafkaSinkConfigOption { - name: KafkaSinkConfigOptionName::ProgressGroupIdPrefix, - value: Some(WithOptionValue::Value(Value::String(format!( - "materialize-bootstrap-sink-{}", - self.id - )))), - }); - options.push(KafkaSinkConfigOption { - name: KafkaSinkConfigOptionName::TransactionalIdPrefix, - value: Some(WithOptionValue::Value(Value::String(format!( - "mz-producer-{}-0", - self.id - )))), + name: KafkaSinkConfigOptionName::LegacyIds, + value: Some(WithOptionValue::Value(Value::Boolean(true))), }); } } } } - let mut rewriter = Rewriter { id }; - rewriter.visit_statement_mut(stmt); + Rewriter.visit_statement_mut(stmt); Ok(()) } diff --git a/src/sql-lexer/src/keywords.txt b/src/sql-lexer/src/keywords.txt index 68897742026f6..ce5859ead13d4 100644 --- a/src/sql-lexer/src/keywords.txt +++ b/src/sql-lexer/src/keywords.txt @@ -169,6 +169,7 @@ Hour Hours Id Idle +Ids If Ignore Ilike @@ -204,6 +205,7 @@ Latest Leading Least Left +Legacy Level Like Limit diff --git a/src/sql-parser/src/ast/defs/ddl.rs b/src/sql-parser/src/ast/defs/ddl.rs index 2a1c50486ac22..a27954e8059b0 100644 --- a/src/sql-parser/src/ast/defs/ddl.rs +++ b/src/sql-parser/src/ast/defs/ddl.rs @@ -865,6 +865,7 @@ pub enum KafkaSinkConfigOptionName { ProgressGroupIdPrefix, Topic, TransactionalIdPrefix, + LegacyIds, } impl AstDisplay for KafkaSinkConfigOptionName { @@ -874,6 +875,7 @@ impl AstDisplay for KafkaSinkConfigOptionName { KafkaSinkConfigOptionName::ProgressGroupIdPrefix => "PROGRESS GROUP ID PREFIX", KafkaSinkConfigOptionName::Topic => "TOPIC", KafkaSinkConfigOptionName::TransactionalIdPrefix => "TRANSACTIONAL ID PREFIX", + KafkaSinkConfigOptionName::LegacyIds => "LEGACY IDS", }) } } diff --git a/src/sql-parser/src/parser.rs b/src/sql-parser/src/parser.rs index 7a54a477d1fe4..a7f79500f2639 100644 --- a/src/sql-parser/src/parser.rs +++ b/src/sql-parser/src/parser.rs @@ -2369,23 +2369,32 @@ impl<'a> Parser<'a> { fn parse_kafka_sink_config_option( &mut self, ) -> Result, ParserError> { - let name = - match self.expect_one_of_keywords(&[COMPRESSION, PROGRESS, TOPIC, TRANSACTIONAL])? { - COMPRESSION => { - self.expect_keyword(TYPE)?; - KafkaSinkConfigOptionName::CompressionType - } - PROGRESS => { - self.expect_keywords(&[GROUP, ID, PREFIX])?; - KafkaSinkConfigOptionName::ProgressGroupIdPrefix - } - TOPIC => KafkaSinkConfigOptionName::Topic, - TRANSACTIONAL => { - self.expect_keywords(&[ID, PREFIX])?; - KafkaSinkConfigOptionName::TransactionalIdPrefix - } - _ => unreachable!(), - }; + let name = match self.expect_one_of_keywords(&[ + COMPRESSION, + PROGRESS, + TOPIC, + LEGACY, + TRANSACTIONAL, + ])? { + COMPRESSION => { + self.expect_keyword(TYPE)?; + KafkaSinkConfigOptionName::CompressionType + } + PROGRESS => { + self.expect_keywords(&[GROUP, ID, PREFIX])?; + KafkaSinkConfigOptionName::ProgressGroupIdPrefix + } + TOPIC => KafkaSinkConfigOptionName::Topic, + TRANSACTIONAL => { + self.expect_keywords(&[ID, PREFIX])?; + KafkaSinkConfigOptionName::TransactionalIdPrefix + } + LEGACY => { + self.expect_keywords(&[IDS])?; + KafkaSinkConfigOptionName::LegacyIds + } + _ => unreachable!(), + }; Ok(KafkaSinkConfigOption { name, value: self.parse_optional_option_value()?, diff --git a/src/sql/src/kafka_util.rs b/src/sql/src/kafka_util.rs index 63f635537d5e9..fa16c6e143a8a 100644 --- a/src/sql/src/kafka_util.rs +++ b/src/sql/src/kafka_util.rs @@ -53,7 +53,8 @@ generate_extracted_config!( ), (ProgressGroupIdPrefix, String), (Topic, String), - (TransactionalIdPrefix, String) + (TransactionalIdPrefix, String), + (LegacyIds, bool) ); impl TryFromValue for KafkaSinkCompressionType { diff --git a/src/sql/src/plan/statement/ddl.rs b/src/sql/src/plan/statement/ddl.rs index 6f5395543788e..28bdbab3a2d8b 100644 --- a/src/sql/src/plan/statement/ddl.rs +++ b/src/sql/src/plan/statement/ddl.rs @@ -46,7 +46,7 @@ use mz_sql_parser::ident; use mz_storage_types::connections::inline::{ConnectionAccess, ReferencedConnection}; use mz_storage_types::connections::Connection; use mz_storage_types::sinks::{ - KafkaSinkConnection, KafkaSinkFormat, SinkEnvelope, StorageSinkConnection, + KafkaIdStyle, KafkaSinkConnection, KafkaSinkFormat, SinkEnvelope, StorageSinkConnection, }; use mz_storage_types::sources::encoding::{ included_column_desc, AvroEncoding, ColumnSpec, CsvEncoding, DataEncoding, DataEncodingInner, @@ -2495,9 +2495,26 @@ fn kafka_sink_builder( compression_type, progress_group_id_prefix, transactional_id_prefix, + legacy_ids, seen: _, }: KafkaSinkConfigOptionExtracted = options.try_into()?; + let transactional_id = match (transactional_id_prefix, legacy_ids) { + (Some(_), Some(true)) => { + sql_bail!("LEGACY IDS cannot be used at the same time as TRANSACTIONAL ID PREFIX") + } + (None, Some(true)) => KafkaIdStyle::Legacy, + (prefix, _) => KafkaIdStyle::Prefix(prefix), + }; + + let progress_group_id = match (progress_group_id_prefix, legacy_ids) { + (Some(_), Some(true)) => { + sql_bail!("LEGACY IDS cannot be used at the same time as PROGRESS GROUP ID PREFIX") + } + (None, Some(true)) => KafkaIdStyle::Legacy, + (prefix, _) => KafkaIdStyle::Prefix(prefix), + }; + let topic_name = topic.ok_or_else(|| sql_err!("KAFKA CONNECTION must specify TOPIC"))?; let format = match format { @@ -2603,8 +2620,8 @@ fn kafka_sink_builder( key_desc_and_indices, value_desc, compression_type, - progress_group_id_prefix, - transactional_id_prefix, + progress_group_id, + transactional_id, })) } diff --git a/src/sql/src/pure.rs b/src/sql/src/pure.rs index 5c7d8b1452881..d478bd9e9e4ef 100644 --- a/src/sql/src/pure.rs +++ b/src/sql/src/pure.rs @@ -54,7 +54,7 @@ use crate::ast::{ Format, ProtobufSchema, ReferencedSubsources, Value, WithOptionValue, }; use crate::catalog::{CatalogItemType, ErsatzCatalog, SessionCatalog}; -use crate::kafka_util::KafkaSourceConfigOptionExtracted; +use crate::kafka_util::{KafkaSinkConfigOptionExtracted, KafkaSourceConfigOptionExtracted}; use crate::names::{Aug, ResolvedColumnName, ResolvedItemName}; use crate::plan::error::PlanError; use crate::plan::statement::ddl::load_generator_ast_to_generator; @@ -286,7 +286,7 @@ async fn purify_create_sink( match &connection { CreateSinkConnection::Kafka { connection, - options: _, + options, key: _, } => { let scx = StatementContext::new(None, &catalog); @@ -304,6 +304,12 @@ async fn purify_create_sink( } }; + let extracted_options: KafkaSinkConfigOptionExtracted = options.clone().try_into()?; + + if extracted_options.legacy_ids == Some(true) { + sql_bail!("LEGACY IDs option is not supported"); + } + let client: AdminClient<_> = connection .create_with_context( storage_configuration, diff --git a/src/storage-types/src/sinks.proto b/src/storage-types/src/sinks.proto index 8f2761ddb36c5..28d0f338b3cc7 100644 --- a/src/storage-types/src/sinks.proto +++ b/src/storage-types/src/sinks.proto @@ -92,8 +92,18 @@ message ProtoKafkaSinkConnectionV2 { google.protobuf.Empty lz4 = 17; google.protobuf.Empty zstd = 18; } - optional string progress_group_id_prefix = 19; - optional string transactional_id_prefix = 20; + ProtoKafkaIdStyle progress_group_id = 19; + ProtoKafkaIdStyle transactional_id = 20; +} + +message ProtoKafkaIdStyle { + message ProtoKafkaIdStylePrefix { + optional string prefix = 1; + } + oneof kind { + ProtoKafkaIdStylePrefix prefix = 1; + google.protobuf.Empty legacy = 2; + } } message ProtoPersistSinkConnection { diff --git a/src/storage-types/src/sinks.rs b/src/storage-types/src/sinks.rs index 7e69258a442f4..9e8b4920c01fd 100644 --- a/src/storage-types/src/sinks.rs +++ b/src/storage-types/src/sinks.rs @@ -432,8 +432,8 @@ pub struct KafkaSinkConnection { pub topic: String, pub fuel: usize, pub compression_type: KafkaSinkCompressionType, - pub progress_group_id_prefix: Option, - pub transactional_id_prefix: Option, + pub progress_group_id: KafkaIdStyle, + pub transactional_id: KafkaIdStyle, } impl KafkaSinkConnection { @@ -464,11 +464,14 @@ impl KafkaSinkConnection { connection_context: &ConnectionContext, sink_id: GlobalId, ) -> String { - format!( - "{}{}", - self.progress_group_id_prefix.as_deref().unwrap_or(""), - self.client_id(connection_context, sink_id) - ) + match self.progress_group_id { + KafkaIdStyle::Prefix(ref prefix) => format!( + "{}{}", + prefix.as_deref().unwrap_or(""), + self.client_id(connection_context, sink_id) + ), + KafkaIdStyle::Legacy => format!("materialize-bootstrap-sink-{sink_id}"), + } } /// Returns the transactional ID to use for the sink. @@ -480,11 +483,14 @@ impl KafkaSinkConnection { connection_context: &ConnectionContext, sink_id: GlobalId, ) -> String { - format!( - "{}{}", - self.transactional_id_prefix.as_deref().unwrap_or(""), - self.client_id(connection_context, sink_id) - ) + match self.transactional_id { + KafkaIdStyle::Prefix(ref prefix) => format!( + "{}{}", + prefix.as_deref().unwrap_or(""), + self.client_id(connection_context, sink_id) + ), + KafkaIdStyle::Legacy => format!("mz-producer-{sink_id}-0"), + } } } @@ -512,8 +518,8 @@ impl KafkaSinkConnection { topic, fuel, compression_type, - progress_group_id_prefix, - transactional_id_prefix: transactional_id, + progress_group_id, + transactional_id, } = self; let compatibility_checks = [ @@ -535,11 +541,11 @@ impl KafkaSinkConnection { "compression_type", ), ( - progress_group_id_prefix == &other.progress_group_id_prefix, - "progress_group_id_prefix", + progress_group_id == &other.progress_group_id, + "progress_group_id", ), ( - transactional_id == &other.transactional_id_prefix, + transactional_id == &other.transactional_id, "transactional_id", ), ]; @@ -573,8 +579,8 @@ impl IntoInlineConnection topic, fuel, compression_type, - progress_group_id_prefix, - transactional_id_prefix: transactional_id, + progress_group_id, + transactional_id, } = self; KafkaSinkConnection { connection_id, @@ -586,10 +592,46 @@ impl IntoInlineConnection topic, fuel, compression_type, - progress_group_id_prefix, - transactional_id_prefix: transactional_id, + progress_group_id, + transactional_id, + } + } +} + +#[derive(Arbitrary, Clone, Debug, Eq, PartialEq, Serialize, Deserialize)] +pub enum KafkaIdStyle { + /// A new-style id that is optionally prefixed. + Prefix(Option), + /// A legacy style id. + Legacy, +} + +impl RustType for KafkaIdStyle { + fn into_proto(&self) -> ProtoKafkaIdStyle { + use crate::sinks::proto_kafka_id_style::Kind::*; + use crate::sinks::proto_kafka_id_style::ProtoKafkaIdStylePrefix; + + ProtoKafkaIdStyle { + kind: Some(match self { + Self::Prefix(prefix) => Prefix(ProtoKafkaIdStylePrefix { + prefix: prefix.into_proto(), + }), + Self::Legacy => Legacy(()), + }), } } + fn from_proto(proto: ProtoKafkaIdStyle) -> Result { + use crate::sinks::proto_kafka_id_style::Kind::*; + + let kind = proto + .kind + .ok_or_else(|| TryFromProtoError::missing_field("ProtoKafkaIdStyle::kind"))?; + + Ok(match kind { + Prefix(prefix) => Self::Prefix(prefix.prefix.into_rust()?), + Legacy(()) => Self::Legacy, + }) + } } impl RustType for KafkaSinkConnection { @@ -611,8 +653,8 @@ impl RustType for KafkaSinkConnection { KafkaSinkCompressionType::Lz4 => CompressionType::Lz4(()), KafkaSinkCompressionType::Zstd => CompressionType::Zstd(()), }), - progress_group_id_prefix: self.progress_group_id_prefix.clone(), - transactional_id_prefix: self.transactional_id_prefix.clone(), + progress_group_id: Some(self.progress_group_id.into_proto()), + transactional_id: Some(self.transactional_id.into_proto()), } } @@ -647,8 +689,12 @@ impl RustType for KafkaSinkConnection { )) } }, - progress_group_id_prefix: proto.progress_group_id_prefix, - transactional_id_prefix: proto.transactional_id_prefix, + progress_group_id: proto + .progress_group_id + .into_rust_if_some("ProtoKafkaSinkConnectionV2::progress_group_id")?, + transactional_id: proto + .transactional_id + .into_rust_if_some("ProtoKafkaSinkConnectionV2::transactional_id")?, }) } } diff --git a/test/testdrive/kafka-sink-errors.td b/test/testdrive/kafka-sink-errors.td index 6f999e91bca04..049a812cfb3e5 100644 --- a/test/testdrive/kafka-sink-errors.td +++ b/test/testdrive/kafka-sink-errors.td @@ -27,6 +27,12 @@ ENVELOPE DEBEZIUM contains:No such column: f2 +! CREATE SINK invalid_legacy_ids FROM v1 + INTO KAFKA CONNECTION kafka_conn (TOPIC 'testdrive-kafka-sink-errors-${testdrive.seed}', LEGACY IDS) + FORMAT AVRO USING CONFLUENT SCHEMA REGISTRY CONNECTION csr_conn + ENVELOPE DEBEZIUM +contains:LEGACY IDs option is not supported + # # Sink dependencies #