Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

fix(sqlparser): Display of ConnectionRef shall be valid SQL #20656

Merged
merged 4 commits into from
Feb 28, 2025
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
16 changes: 2 additions & 14 deletions src/sqlparser/src/ast/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -2822,33 +2822,21 @@ impl fmt::Display for SqlOption {
}
}

#[derive(Clone, PartialEq, Eq, Hash)]
#[derive(Debug, Clone, PartialEq, Eq, Hash)]
#[cfg_attr(feature = "serde", derive(Serialize, Deserialize))]
pub enum SqlOptionValue {
Value(Value),
SecretRef(SecretRefValue),
ConnectionRef(ConnectionRefValue),
}

impl fmt::Debug for SqlOptionValue {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
SqlOptionValue::Value(value) => write!(f, "{:?}", value),
SqlOptionValue::SecretRef(secret_ref) => write!(f, "secret {:?}", secret_ref),
SqlOptionValue::ConnectionRef(connection_ref) => {
write!(f, "connection {:?}", connection_ref)
}
}
}
}

impl fmt::Display for SqlOptionValue {
fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result {
match self {
SqlOptionValue::Value(value) => write!(f, "{}", value),
SqlOptionValue::SecretRef(secret_ref) => write!(f, "secret {}", secret_ref),
SqlOptionValue::ConnectionRef(connection_ref) => {
write!(f, "connection {}", connection_ref)
write!(f, "{}", connection_ref)
}
}
}
Expand Down
7 changes: 7 additions & 0 deletions src/sqlparser/src/parser.rs
Original file line number Diff line number Diff line change
Expand Up @@ -3017,6 +3017,13 @@ impl Parser<'_> {
const CONNECTION_REF_KEY: &str = "connection";
if name.real_value().eq_ignore_ascii_case(CONNECTION_REF_KEY) {
let connection_name = self.parse_object_name()?;
// tolerate previous buggy Display that outputs `connection = connection foo`
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

sorry for that

let connection_name = match connection_name.0.as_slice() {
[ident] if ident.real_value() == CONNECTION_REF_KEY => {
self.parse_object_name()?
}
_ => connection_name,
};
SqlOptionValue::ConnectionRef(ConnectionRefValue { connection_name })
} else {
self.parse_value_and_obj_ref::<false>()?
Expand Down
14 changes: 9 additions & 5 deletions src/sqlparser/tests/testdata/create.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -43,19 +43,19 @@
^
- input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://')
formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.location = 'file://')
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: None, constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: SingleQuotedString("file://") }], key_encode: None }), source_watermarks: [], include_column_options: [] } }'
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: None, constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: Value(SingleQuotedString("abc")) }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: Value(SingleQuotedString("localhost:1001")) }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: Value(SingleQuotedString("Foo")) }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "location", quote_style: None }]), value: Value(SingleQuotedString("file://")) }], key_encode: None }), source_watermarks: [], include_column_options: [] } }'
- input: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://')
formatted_sql: CREATE SOURCE IF NOT EXISTS src WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://')
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: None, constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }], key_encode: None }), source_watermarks: [], include_column_options: [] } }'
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: None, constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: Value(SingleQuotedString("abc")) }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: Value(SingleQuotedString("localhost:1001")) }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: Value(SingleQuotedString("Foo")) }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: Value(SingleQuotedString("http://")) }], key_encode: None }), source_watermarks: [], include_column_options: [] } }'
- input: CREATE SOURCE IF NOT EXISTS src (*, WATERMARK FOR event_time AS event_time - INTERVAL '60' SECOND) WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://')
formatted_sql: CREATE SOURCE IF NOT EXISTS src (*, WATERMARK FOR event_time AS event_time - INTERVAL '60' SECOND) WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://')
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: Some(0), constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }], key_encode: None }), source_watermarks: [SourceWatermark { column: Ident { value: "event_time", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "event_time", quote_style: None }), op: Minus, right: Value(Interval { value: "60", leading_field: Some(Second), leading_precision: None, last_field: None, fractional_seconds_precision: None }) } }], include_column_options: [] } }'
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: Some(0), constraints: [], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: Value(SingleQuotedString("abc")) }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: Value(SingleQuotedString("localhost:1001")) }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: Value(SingleQuotedString("Foo")) }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: Value(SingleQuotedString("http://")) }], key_encode: None }), source_watermarks: [SourceWatermark { column: Ident { value: "event_time", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "event_time", quote_style: None }), op: Minus, right: Value(Interval { value: "60", leading_field: Some(Second), leading_precision: None, last_field: None, fractional_seconds_precision: None }) } }], include_column_options: [] } }'
- input: CREATE SOURCE IF NOT EXISTS src (PRIMARY KEY (event_id), WATERMARK FOR event_time AS event_time - INTERVAL '60' SECOND) WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://')
formatted_sql: CREATE SOURCE IF NOT EXISTS src (PRIMARY KEY (event_id), WATERMARK FOR event_time AS event_time - INTERVAL '60' SECOND) WITH (kafka.topic = 'abc', kafka.brokers = 'localhost:1001') FORMAT PLAIN ENCODE PROTOBUF (message = 'Foo', schema.registry = 'http://')
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: None, constraints: [Unique { name: None, columns: [Ident { value: "event_id", quote_style: None }], is_primary: true }], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: SingleQuotedString("abc") }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: SingleQuotedString("localhost:1001") }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: SingleQuotedString("Foo") }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: SingleQuotedString("http://") }], key_encode: None }), source_watermarks: [SourceWatermark { column: Ident { value: "event_time", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "event_time", quote_style: None }), op: Minus, right: Value(Interval { value: "60", leading_field: Some(Second), leading_precision: None, last_field: None, fractional_seconds_precision: None }) } }], include_column_options: [] } }'
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: true, columns: [], wildcard_idx: None, constraints: [Unique { name: None, columns: [Ident { value: "event_id", quote_style: None }], is_primary: true }], source_name: ObjectName([Ident { value: "src", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "topic", quote_style: None }]), value: Value(SingleQuotedString("abc")) }, SqlOption { name: ObjectName([Ident { value: "kafka", quote_style: None }, Ident { value: "brokers", quote_style: None }]), value: Value(SingleQuotedString("localhost:1001")) }]), format_encode: V2(FormatEncodeOptions { format: Plain, row_encode: Protobuf, row_options: [SqlOption { name: ObjectName([Ident { value: "message", quote_style: None }]), value: Value(SingleQuotedString("Foo")) }, SqlOption { name: ObjectName([Ident { value: "schema", quote_style: None }, Ident { value: "registry", quote_style: None }]), value: Value(SingleQuotedString("http://")) }], key_encode: None }), source_watermarks: [SourceWatermark { column: Ident { value: "event_time", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "event_time", quote_style: None }), op: Minus, right: Value(Interval { value: "60", leading_field: Some(Second), leading_precision: None, last_field: None, fractional_seconds_precision: None }) } }], include_column_options: [] } }'
- input: CREATE SOURCE bid (auction INTEGER, bidder INTEGER, price INTEGER, WATERMARK FOR auction AS auction - 1, "date_time" TIMESTAMP) with (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0')
formatted_sql: CREATE SOURCE bid (auction INT, bidder INT, price INT, "date_time" TIMESTAMP, WATERMARK FOR auction AS auction - 1) WITH (connector = 'nexmark', nexmark.table.type = 'Bid', nexmark.split.num = '12', nexmark.min.event.gap.in.ns = '0') FORMAT NATIVE ENCODE NATIVE
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], wildcard_idx: None, constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: SingleQuotedString("nexmark") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: SingleQuotedString("Bid") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: SingleQuotedString("12") }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: SingleQuotedString("0") }]), format_encode: V2(FormatEncodeOptions { format: Native, row_encode: Native, row_options: [], key_encode: None }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }], include_column_options: [] } }'
formatted_ast: 'CreateSource { stmt: CreateSourceStatement { temporary: false, if_not_exists: false, columns: [ColumnDef { name: Ident { value: "auction", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "bidder", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "price", quote_style: None }, data_type: Some(Int), collation: None, options: [] }, ColumnDef { name: Ident { value: "date_time", quote_style: Some(''"'') }, data_type: Some(Timestamp(false)), collation: None, options: [] }], wildcard_idx: None, constraints: [], source_name: ObjectName([Ident { value: "bid", quote_style: None }]), with_properties: WithProperties([SqlOption { name: ObjectName([Ident { value: "connector", quote_style: None }]), value: Value(SingleQuotedString("nexmark")) }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "table", quote_style: None }, Ident { value: "type", quote_style: None }]), value: Value(SingleQuotedString("Bid")) }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "split", quote_style: None }, Ident { value: "num", quote_style: None }]), value: Value(SingleQuotedString("12")) }, SqlOption { name: ObjectName([Ident { value: "nexmark", quote_style: None }, Ident { value: "min", quote_style: None }, Ident { value: "event", quote_style: None }, Ident { value: "gap", quote_style: None }, Ident { value: "in", quote_style: None }, Ident { value: "ns", quote_style: None }]), value: Value(SingleQuotedString("0")) }]), format_encode: V2(FormatEncodeOptions { format: Native, row_encode: Native, row_options: [], key_encode: None }), source_watermarks: [SourceWatermark { column: Ident { value: "auction", quote_style: None }, expr: BinaryOp { left: Identifier(Ident { value: "auction", quote_style: None }), op: Minus, right: Value(Number("1")) } }], include_column_options: [] } }'
- input: |-
CREATE SOURCE s
(raw BYTEA)
Expand Down Expand Up @@ -138,6 +138,10 @@
formatted_sql: CREATE SINK snk FROM mv WITH (connector = 'kafka', properties.bootstrap.server = '127.0.0.1:9092', topic = 'test_topic') FORMAT PLAIN ENCODE JSON
- input: CREATE SINK snk FROM mv WITH (connector = 'kafka', properties.bootstrap.server = '127.0.0.1:9092', topic = 'test_topic') format upsert encode protobuf (schema.location = 'location', message = 'main_message');
formatted_sql: CREATE SINK snk FROM mv WITH (connector = 'kafka', properties.bootstrap.server = '127.0.0.1:9092', topic = 'test_topic') FORMAT UPSERT ENCODE PROTOBUF (schema.location = 'location', message = 'main_message')
- input: CREATE SINK snk FROM mv WITH (connector = 'kafka', connection = my_kafka_conn, topic = 'test_topic') format plain encode json;
formatted_sql: CREATE SINK snk FROM mv WITH (connector = 'kafka', connection = my_kafka_conn, topic = 'test_topic') FORMAT PLAIN ENCODE JSON
- input: CREATE SINK snk FROM mv WITH (connector = 'kafka', connection = connection my_kafka_conn, topic = 'test_topic') format plain encode json;
formatted_sql: CREATE SINK snk FROM mv WITH (connector = 'kafka', connection = my_kafka_conn, topic = 'test_topic') FORMAT PLAIN ENCODE JSON
- input: CREATE SINK snk into t FROM MV
formatted_sql: CREATE SINK snk INTO t FROM MV
- input: CREATE SINK snk into t AS SELECT * FROM t
Expand Down
Loading