risingwave_frontend/handler/create_source/
external_schema.rs

1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
// Copyright 2025 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

//! bind columns from external schema

use super::*;

mod json;
use json::*;
mod avro;
use avro::extract_avro_table_schema;
pub mod debezium;
pub mod iceberg;
use iceberg::extract_iceberg_columns;
mod protobuf;
use protobuf::extract_protobuf_table_schema;
pub mod nexmark;

/// Resolves the schema of the source from external schema file.
/// See <https://www.risingwave.dev/docs/current/sql-create-source> for more information.
///
/// Note: the returned schema strictly corresponds to the schema.
/// Other special columns like additional columns (`INCLUDE`), and `row_id` column are not included.
pub async fn bind_columns_from_source(
    session: &SessionImpl,
    format_encode: &FormatEncodeOptions,
    with_properties: Either<&WithOptions, &WithOptionsSecResolved>,
    create_source_type: CreateSourceType,
) -> Result<(Option<Vec<ColumnCatalog>>, StreamSourceInfo)> {
    let (columns_from_resolve_source, mut source_info) =
        if create_source_type == CreateSourceType::SharedCdc {
            bind_columns_from_source_for_cdc(session, format_encode)?
        } else {
            bind_columns_from_source_for_non_cdc(session, format_encode, with_properties).await?
        };
    if create_source_type.is_shared() {
        // Note: this field should be called is_shared. Check field doc for more details.
        source_info.cdc_source_job = true;
        source_info.is_distributed = create_source_type == CreateSourceType::SharedNonCdc;
    }
    Ok((columns_from_resolve_source, source_info))
}

async fn bind_columns_from_source_for_non_cdc(
    session: &SessionImpl,
    format_encode: &FormatEncodeOptions,
    with_properties: Either<&WithOptions, &WithOptionsSecResolved>,
) -> Result<(Option<Vec<ColumnCatalog>>, StreamSourceInfo)> {
    const MESSAGE_NAME_KEY: &str = "message";
    const KEY_MESSAGE_NAME_KEY: &str = "key.message";
    const NAME_STRATEGY_KEY: &str = "schema.registry.name.strategy";

    let options_with_secret = match with_properties {
        Either::Left(options) => {
            let (sec_resolve_props, connection_type, _) = resolve_connection_ref_and_secret_ref(
                options.clone(),
                session,
                TelemetryDatabaseObject::Source,
            )?;
            if !ALLOWED_CONNECTION_CONNECTOR.contains(&connection_type) {
                return Err(RwError::from(ProtocolError(format!(
                    "connection type {:?} is not allowed, allowed types: {:?}",
                    connection_type, ALLOWED_CONNECTION_CONNECTOR
                ))));
            }

            sec_resolve_props
        }
        Either::Right(options_with_secret) => options_with_secret.clone(),
    };

    let is_kafka: bool = options_with_secret.is_kafka_connector();

    // todo: need to resolve connection ref for schema registry
    let (sec_resolve_props, connection_type, schema_registry_conn_ref) =
        resolve_connection_ref_and_secret_ref(
            WithOptions::try_from(format_encode.row_options())?,
            session,
            TelemetryDatabaseObject::Source,
        )?;
    ensure_connection_type_allowed(connection_type, &ALLOWED_CONNECTION_SCHEMA_REGISTRY)?;

    let (format_encode_options, format_encode_secret_refs) = sec_resolve_props.into_parts();
    // Need real secret to access the schema registry
    let mut format_encode_options_to_consume = LocalSecretManager::global().fill_secrets(
        format_encode_options.clone(),
        format_encode_secret_refs.clone(),
    )?;

    fn get_key_message_name(options: &mut BTreeMap<String, String>) -> Option<String> {
        consume_string_from_options(options, KEY_MESSAGE_NAME_KEY)
            .map(|ele| Some(ele.0))
            .unwrap_or(None)
    }
    fn get_sr_name_strategy_check(
        options: &mut BTreeMap<String, String>,
        use_sr: bool,
    ) -> Result<Option<i32>> {
        let name_strategy = get_name_strategy_or_default(try_consume_string_from_options(
            options,
            NAME_STRATEGY_KEY,
        ))?;
        if !use_sr && name_strategy.is_some() {
            return Err(RwError::from(ProtocolError(
                "schema registry name strategy only works with schema registry enabled".to_owned(),
            )));
        }
        Ok(name_strategy)
    }

    let mut stream_source_info = StreamSourceInfo {
        format: format_to_prost(&format_encode.format) as i32,
        row_encode: row_encode_to_prost(&format_encode.row_encode) as i32,
        format_encode_options,
        format_encode_secret_refs,
        connection_id: schema_registry_conn_ref,
        ..Default::default()
    };

    if format_encode.format == Format::Debezium {
        try_consume_string_from_options(&mut format_encode_options_to_consume, DEBEZIUM_IGNORE_KEY);
    }

    let columns = match (&format_encode.format, &format_encode.row_encode) {
        (Format::Native, Encode::Native)
        | (Format::Plain, Encode::Bytes)
        | (Format::DebeziumMongo, Encode::Json) => None,
        (Format::Plain, Encode::Protobuf) | (Format::Upsert, Encode::Protobuf) => {
            let (row_schema_location, use_schema_registry) =
                get_schema_location(&mut format_encode_options_to_consume)?;
            let protobuf_schema = ProtobufSchema {
                message_name: consume_string_from_options(
                    &mut format_encode_options_to_consume,
                    MESSAGE_NAME_KEY,
                )?,
                row_schema_location,
                use_schema_registry,
            };
            let name_strategy = get_sr_name_strategy_check(
                &mut format_encode_options_to_consume,
                protobuf_schema.use_schema_registry,
            )?;

            stream_source_info.use_schema_registry = protobuf_schema.use_schema_registry;
            stream_source_info
                .row_schema_location
                .clone_from(&protobuf_schema.row_schema_location.0);
            stream_source_info
                .proto_message_name
                .clone_from(&protobuf_schema.message_name.0);
            stream_source_info.key_message_name =
                get_key_message_name(&mut format_encode_options_to_consume);
            stream_source_info.name_strategy =
                name_strategy.unwrap_or(PbSchemaRegistryNameStrategy::Unspecified as i32);

            Some(
                extract_protobuf_table_schema(
                    &protobuf_schema,
                    &options_with_secret,
                    &mut format_encode_options_to_consume,
                )
                .await?,
            )
        }
        (format @ (Format::Plain | Format::Upsert | Format::Debezium), Encode::Avro) => {
            if format_encode_options_to_consume
                .remove(AWS_GLUE_SCHEMA_ARN_KEY)
                .is_none()
            {
                // Legacy logic that assumes either `schema.location` or confluent `schema.registry`.
                // The handling of newly added aws glue is centralized in `connector::parser`.
                // TODO(xiangjinwu): move these option parsing to `connector::parser` as well.

                let (row_schema_location, use_schema_registry) =
                    get_schema_location(&mut format_encode_options_to_consume)?;

                if matches!(format, Format::Debezium) && !use_schema_registry {
                    return Err(RwError::from(ProtocolError(
                        "schema location for DEBEZIUM_AVRO row format is not supported".to_owned(),
                    )));
                }

                let message_name = try_consume_string_from_options(
                    &mut format_encode_options_to_consume,
                    MESSAGE_NAME_KEY,
                );
                let name_strategy = get_sr_name_strategy_check(
                    &mut format_encode_options_to_consume,
                    use_schema_registry,
                )?;

                stream_source_info.use_schema_registry = use_schema_registry;
                stream_source_info
                    .row_schema_location
                    .clone_from(&row_schema_location.0);
                stream_source_info.proto_message_name =
                    message_name.unwrap_or(AstString("".into())).0;
                stream_source_info.key_message_name =
                    get_key_message_name(&mut format_encode_options_to_consume);
                stream_source_info.name_strategy =
                    name_strategy.unwrap_or(PbSchemaRegistryNameStrategy::Unspecified as i32);
            }

            Some(
                extract_avro_table_schema(
                    &stream_source_info,
                    &options_with_secret,
                    &mut format_encode_options_to_consume,
                    matches!(format, Format::Debezium),
                )
                .await?,
            )
        }
        (Format::Plain, Encode::Csv) => {
            let chars =
                consume_string_from_options(&mut format_encode_options_to_consume, "delimiter")?.0;
            let delimiter = get_delimiter(chars.as_str()).context("failed to parse delimiter")?;
            let has_header = try_consume_string_from_options(
                &mut format_encode_options_to_consume,
                "without_header",
            )
            .map(|s| s.0 == "false")
            .unwrap_or(true);

            if is_kafka && has_header {
                return Err(RwError::from(ProtocolError(
                    "CSV HEADER is not supported when creating table with Kafka connector"
                        .to_owned(),
                )));
            }

            stream_source_info.csv_delimiter = delimiter as i32;
            stream_source_info.csv_has_header = has_header;

            None
        }
        // For parquet format, this step is implemented in parquet parser.
        (Format::Plain, Encode::Parquet) => None,
        (
            Format::Plain | Format::Upsert | Format::Maxwell | Format::Canal | Format::Debezium,
            Encode::Json,
        ) => {
            if matches!(
                format_encode.format,
                Format::Plain | Format::Upsert | Format::Debezium
            ) {
                // Parse the value but throw it away.
                // It would be too late to report error in `SpecificParserConfig::new`,
                // which leads to recovery loop.
                // TODO: rely on SpecificParserConfig::new to validate, like Avro
                TimestamptzHandling::from_options(&format_encode_options_to_consume)
                    .map_err(|err| InvalidInputSyntax(err.message))?;
                try_consume_string_from_options(
                    &mut format_encode_options_to_consume,
                    TimestamptzHandling::OPTION_KEY,
                );
            }

            let schema_config = get_json_schema_location(&mut format_encode_options_to_consume)?;
            stream_source_info.use_schema_registry =
                json_schema_infer_use_schema_registry(&schema_config);

            extract_json_table_schema(
                &schema_config,
                &options_with_secret,
                &mut format_encode_options_to_consume,
            )
            .await?
        }
        (Format::None, Encode::None) => {
            if options_with_secret.is_iceberg_connector() {
                Some(
                    extract_iceberg_columns(&options_with_secret)
                        .await
                        .map_err(|err| ProtocolError(err.to_report_string()))?,
                )
            } else {
                None
            }
        }
        (format, encoding) => {
            return Err(RwError::from(ProtocolError(format!(
                "Unknown combination {:?} {:?}",
                format, encoding
            ))));
        }
    };

    if !format_encode_options_to_consume.is_empty() {
        let err_string = format!(
            "Get unknown format_encode_options for {:?} {:?}: {}",
            format_encode.format,
            format_encode.row_encode,
            format_encode_options_to_consume
                .keys()
                .map(|k| k.to_string())
                .collect::<Vec<String>>()
                .join(","),
        );
        session.notice_to_user(err_string);
    }
    Ok((columns, stream_source_info))
}

fn bind_columns_from_source_for_cdc(
    session: &SessionImpl,
    format_encode: &FormatEncodeOptions,
) -> Result<(Option<Vec<ColumnCatalog>>, StreamSourceInfo)> {
    let with_options = WithOptions::try_from(format_encode.row_options())?;
    if !with_options.connection_ref().is_empty() {
        return Err(RwError::from(NotSupported(
            "CDC connector does not support connection ref yet".to_owned(),
            "Explicitly specify the connection in WITH clause".to_owned(),
        )));
    }
    let (format_encode_options, format_encode_secret_refs) =
        resolve_secret_ref_in_with_options(with_options, session)?.into_parts();

    // Need real secret to access the schema registry
    let mut format_encode_options_to_consume = LocalSecretManager::global().fill_secrets(
        format_encode_options.clone(),
        format_encode_secret_refs.clone(),
    )?;

    match (&format_encode.format, &format_encode.row_encode) {
        (Format::Plain, Encode::Json) => (),
        (format, encoding) => {
            // Note: parser will also check this. Just be extra safe here
            return Err(RwError::from(ProtocolError(format!(
                "Row format for CDC connectors should be either omitted or set to `FORMAT PLAIN ENCODE JSON`, got: {:?} {:?}",
                format, encoding
            ))));
        }
    };

    let columns = ColumnCatalog::debezium_cdc_source_cols().to_vec();
    let schema_config = get_json_schema_location(&mut format_encode_options_to_consume)?;

    let stream_source_info = StreamSourceInfo {
        format: format_to_prost(&format_encode.format) as i32,
        row_encode: row_encode_to_prost(&format_encode.row_encode) as i32,
        format_encode_options,
        use_schema_registry: json_schema_infer_use_schema_registry(&schema_config),
        cdc_source_job: true,
        is_distributed: false,
        format_encode_secret_refs,
        ..Default::default()
    };
    if !format_encode_options_to_consume.is_empty() {
        let err_string = format!(
            "Get unknown format_encode_options for {:?} {:?}: {}",
            format_encode.format,
            format_encode.row_encode,
            format_encode_options_to_consume
                .keys()
                .map(|k| k.to_string())
                .collect::<Vec<String>>()
                .join(","),
        );
        session.notice_to_user(err_string);
    }
    Ok((Some(columns), stream_source_info))
}

fn format_to_prost(format: &Format) -> FormatType {
    match format {
        Format::Native => FormatType::Native,
        Format::Plain => FormatType::Plain,
        Format::Upsert => FormatType::Upsert,
        Format::Debezium => FormatType::Debezium,
        Format::DebeziumMongo => FormatType::DebeziumMongo,
        Format::Maxwell => FormatType::Maxwell,
        Format::Canal => FormatType::Canal,
        Format::None => FormatType::None,
    }
}
fn row_encode_to_prost(row_encode: &Encode) -> EncodeType {
    match row_encode {
        Encode::Native => EncodeType::Native,
        Encode::Json => EncodeType::Json,
        Encode::Avro => EncodeType::Avro,
        Encode::Protobuf => EncodeType::Protobuf,
        Encode::Csv => EncodeType::Csv,
        Encode::Bytes => EncodeType::Bytes,
        Encode::Template => EncodeType::Template,
        Encode::Parquet => EncodeType::Parquet,
        Encode::None => EncodeType::None,
        Encode::Text => EncodeType::Text,
    }
}

pub fn get_schema_location(
    format_encode_options: &mut BTreeMap<String, String>,
) -> Result<(AstString, bool)> {
    let schema_location = try_consume_string_from_options(format_encode_options, "schema.location");
    let schema_registry = try_consume_string_from_options(format_encode_options, "schema.registry");
    match (schema_location, schema_registry) {
        (None, None) => Err(RwError::from(ProtocolError(
            "missing either a schema location or a schema registry".to_owned(),
        ))),
        (None, Some(schema_registry)) => Ok((schema_registry, true)),
        (Some(schema_location), None) => Ok((schema_location, false)),
        (Some(_), Some(_)) => Err(RwError::from(ProtocolError(
            "only need either the schema location or the schema registry".to_owned(),
        ))),
    }
}

pub fn schema_has_schema_registry(schema: &FormatEncodeOptions) -> bool {
    match schema.row_encode {
        Encode::Avro | Encode::Protobuf => true,
        Encode::Json => {
            let mut options = WithOptions::try_from(schema.row_options()).unwrap();
            matches!(get_json_schema_location(options.inner_mut()), Ok(Some(_)))
        }
        _ => false,
    }
}

#[inline]
fn get_name_strategy_or_default(name_strategy: Option<AstString>) -> Result<Option<i32>> {
    match name_strategy {
        None => Ok(None),
        Some(name) => Ok(Some(name_strategy_from_str(name.0.as_str())
            .ok_or_else(|| RwError::from(ProtocolError(format!("\
            expect strategy name in topic_name_strategy, record_name_strategy and topic_record_name_strategy, but got {}", name))))? as i32)),
    }
}