risingwave_connector/parser/
config.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
// 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.

use std::collections::HashSet;

use risingwave_common::bail;
use risingwave_common::secret::LocalSecretManager;
use risingwave_connector_codec::decoder::avro::MapHandling;
use risingwave_pb::catalog::{PbSchemaRegistryNameStrategy, StreamSourceInfo};

use super::utils::get_kafka_topic;
use super::{DebeziumProps, TimestamptzHandling};
use crate::connector_common::AwsAuthProps;
use crate::error::ConnectorResult;
use crate::parser::PROTOBUF_MESSAGES_AS_JSONB;
use crate::schema::schema_registry::SchemaRegistryAuth;
use crate::schema::AWS_GLUE_SCHEMA_ARN_KEY;
use crate::source::{extract_source_struct, SourceColumnDesc, SourceEncode, SourceFormat};
use crate::WithOptionsSecResolved;

/// Note: this is created in `SourceReader::build_stream`
#[derive(Debug, Clone, Default)]
pub struct ParserConfig {
    pub common: CommonParserConfig,
    pub specific: SpecificParserConfig,
}

impl ParserConfig {
    pub fn get_config(self) -> (Vec<SourceColumnDesc>, SpecificParserConfig) {
        (self.common.rw_columns, self.specific)
    }
}

#[derive(Debug, Clone, Default)]
pub struct CommonParserConfig {
    /// Note: this is created by `SourceDescBuilder::builder`
    pub rw_columns: Vec<SourceColumnDesc>,
}

#[derive(Debug, Clone, Default)]
pub struct SpecificParserConfig {
    pub encoding_config: EncodingProperties,
    pub protocol_config: ProtocolProperties,
}

#[derive(Debug, Default, Clone)]
pub enum EncodingProperties {
    Avro(AvroProperties),
    Protobuf(ProtobufProperties),
    Csv(CsvProperties),
    Json(JsonProperties),
    MongoJson,
    Bytes(BytesProperties),
    Parquet,
    Native,
    /// Encoding can't be specified because the source will determines it. Now only used in Iceberg.
    None,
    #[default]
    Unspecified,
}

#[derive(Debug, Default, Clone)]
pub enum ProtocolProperties {
    Debezium(DebeziumProps),
    DebeziumMongo,
    Maxwell,
    Canal,
    Plain,
    Upsert,
    Native,
    /// Protocol can't be specified because the source will determines it. Now only used in Iceberg.
    None,
    #[default]
    Unspecified,
}

impl SpecificParserConfig {
    // for test only
    pub const DEFAULT_PLAIN_JSON: SpecificParserConfig = SpecificParserConfig {
        encoding_config: EncodingProperties::Json(JsonProperties {
            use_schema_registry: false,
            timestamptz_handling: None,
        }),
        protocol_config: ProtocolProperties::Plain,
    };

    // The validity of (format, encode) is ensured by `extract_format_encode`
    pub fn new(
        info: &StreamSourceInfo,
        with_properties: &WithOptionsSecResolved,
    ) -> ConnectorResult<Self> {
        let info = info.clone();
        let source_struct = extract_source_struct(&info)?;
        let format_encode_options_with_secret = LocalSecretManager::global()
            .fill_secrets(info.format_encode_options, info.format_encode_secret_refs)?;
        let (options, secret_refs) = with_properties.clone().into_parts();
        // Make sure `with_properties` is no longer used by accident.
        // All reads shall go to `options_with_secret` instead.
        #[expect(unused_variables)]
        let with_properties = ();
        let options_with_secret =
            LocalSecretManager::global().fill_secrets(options, secret_refs)?;
        let format = source_struct.format;
        let encode = source_struct.encode;
        // this transformation is needed since there may be config for the protocol
        // in the future
        let protocol_config = match format {
            SourceFormat::Native => ProtocolProperties::Native,
            SourceFormat::None => ProtocolProperties::None,
            SourceFormat::Debezium => {
                let debezium_props = DebeziumProps::from(&format_encode_options_with_secret);
                ProtocolProperties::Debezium(debezium_props)
            }
            SourceFormat::DebeziumMongo => ProtocolProperties::DebeziumMongo,
            SourceFormat::Maxwell => ProtocolProperties::Maxwell,
            SourceFormat::Canal => ProtocolProperties::Canal,
            SourceFormat::Upsert => ProtocolProperties::Upsert,
            SourceFormat::Plain => ProtocolProperties::Plain,
            _ => unreachable!(),
        };

        let encoding_config = match (format, encode) {
            (SourceFormat::Plain, SourceEncode::Csv) => EncodingProperties::Csv(CsvProperties {
                delimiter: info.csv_delimiter as u8,
                has_header: info.csv_has_header,
            }),
            (SourceFormat::Plain, SourceEncode::Parquet) => EncodingProperties::Parquet,
            (SourceFormat::Plain, SourceEncode::Avro)
            | (SourceFormat::Upsert, SourceEncode::Avro) => {
                let mut config = AvroProperties {
                    record_name: if info.proto_message_name.is_empty() {
                        None
                    } else {
                        Some(info.proto_message_name.clone())
                    },
                    key_record_name: info.key_message_name.clone(),
                    map_handling: MapHandling::from_options(&format_encode_options_with_secret)?,
                    ..Default::default()
                };
                config.schema_location = if let Some(schema_arn) =
                    format_encode_options_with_secret.get(AWS_GLUE_SCHEMA_ARN_KEY)
                {
                    risingwave_common::license::Feature::GlueSchemaRegistry
                        .check_available()
                        .map_err(anyhow::Error::from)?;
                    SchemaLocation::Glue {
                        schema_arn: schema_arn.clone(),
                        aws_auth_props: serde_json::from_value::<AwsAuthProps>(
                            serde_json::to_value(format_encode_options_with_secret.clone())
                                .unwrap(),
                        )
                        .map_err(|e| anyhow::anyhow!(e))?,
                        // The option `mock_config` is not public and we can break compatibility.
                        mock_config: format_encode_options_with_secret
                            .get("aws.glue.mock_config")
                            .cloned(),
                    }
                } else if info.use_schema_registry {
                    SchemaLocation::Confluent {
                        urls: info.row_schema_location.clone(),
                        client_config: SchemaRegistryAuth::from(&format_encode_options_with_secret),
                        name_strategy: PbSchemaRegistryNameStrategy::try_from(info.name_strategy)
                            .unwrap(),
                        topic: get_kafka_topic(&options_with_secret)?.clone(),
                    }
                } else {
                    SchemaLocation::File {
                        url: info.row_schema_location.clone(),
                        aws_auth_props: Some(
                            serde_json::from_value::<AwsAuthProps>(
                                serde_json::to_value(format_encode_options_with_secret.clone())
                                    .unwrap(),
                            )
                            .map_err(|e| anyhow::anyhow!(e))?,
                        ),
                    }
                };
                EncodingProperties::Avro(config)
            }
            (SourceFormat::Plain, SourceEncode::Protobuf)
            | (SourceFormat::Upsert, SourceEncode::Protobuf) => {
                if info.row_schema_location.is_empty() {
                    bail!("protobuf file location not provided");
                }
                let mut messages_as_jsonb = if let Some(messages_as_jsonb) =
                    format_encode_options_with_secret.get(PROTOBUF_MESSAGES_AS_JSONB)
                {
                    messages_as_jsonb.split(',').map(|s| s.to_owned()).collect()
                } else {
                    HashSet::new()
                };
                messages_as_jsonb.insert("google.protobuf.Any".to_owned());

                let mut config = ProtobufProperties {
                    message_name: info.proto_message_name.clone(),
                    use_schema_registry: info.use_schema_registry,
                    row_schema_location: info.row_schema_location.clone(),
                    name_strategy: PbSchemaRegistryNameStrategy::try_from(info.name_strategy)
                        .unwrap(),
                    key_message_name: info.key_message_name.clone(),
                    messages_as_jsonb,
                    ..Default::default()
                };
                if format == SourceFormat::Upsert {
                    config.enable_upsert = true;
                }
                if info.use_schema_registry {
                    config
                        .topic
                        .clone_from(get_kafka_topic(&options_with_secret)?);
                    config.client_config =
                        SchemaRegistryAuth::from(&format_encode_options_with_secret);
                } else {
                    config.aws_auth_props = Some(
                        serde_json::from_value::<AwsAuthProps>(
                            serde_json::to_value(format_encode_options_with_secret.clone())
                                .unwrap(),
                        )
                        .map_err(|e| anyhow::anyhow!(e))?,
                    );
                }
                EncodingProperties::Protobuf(config)
            }
            (SourceFormat::Debezium, SourceEncode::Avro) => {
                EncodingProperties::Avro(AvroProperties {
                    record_name: if info.proto_message_name.is_empty() {
                        None
                    } else {
                        Some(info.proto_message_name.clone())
                    },
                    key_record_name: info.key_message_name.clone(),
                    schema_location: SchemaLocation::Confluent {
                        urls: info.row_schema_location.clone(),
                        client_config: SchemaRegistryAuth::from(&format_encode_options_with_secret),
                        name_strategy: PbSchemaRegistryNameStrategy::try_from(info.name_strategy)
                            .unwrap(),
                        topic: get_kafka_topic(&options_with_secret).unwrap().clone(),
                    },
                    ..Default::default()
                })
            }
            (
                SourceFormat::Plain
                | SourceFormat::Debezium
                | SourceFormat::Maxwell
                | SourceFormat::Canal
                | SourceFormat::Upsert,
                SourceEncode::Json,
            ) => EncodingProperties::Json(JsonProperties {
                use_schema_registry: info.use_schema_registry,
                timestamptz_handling: TimestamptzHandling::from_options(
                    &format_encode_options_with_secret,
                )?,
            }),
            (SourceFormat::DebeziumMongo, SourceEncode::Json) => {
                EncodingProperties::Json(JsonProperties {
                    use_schema_registry: false,
                    timestamptz_handling: None,
                })
            }
            (SourceFormat::Plain, SourceEncode::Bytes) => {
                EncodingProperties::Bytes(BytesProperties { column_name: None })
            }
            (SourceFormat::Native, SourceEncode::Native) => EncodingProperties::Native,
            (SourceFormat::None, SourceEncode::None) => EncodingProperties::None,
            (format, encode) => {
                bail!("Unsupported format {:?} encode {:?}", format, encode);
            }
        };
        Ok(Self {
            encoding_config,
            protocol_config,
        })
    }
}

#[derive(Debug, Default, Clone)]
pub struct AvroProperties {
    pub schema_location: SchemaLocation,
    pub record_name: Option<String>,
    pub key_record_name: Option<String>,
    pub map_handling: Option<MapHandling>,
}

/// WIP: may cover protobuf and json schema later.
#[derive(Debug, Clone)]
pub enum SchemaLocation {
    /// Avsc from `https://`, `s3://` or `file://`.
    File {
        url: String,
        aws_auth_props: Option<AwsAuthProps>, // for s3
    },
    /// <https://docs.confluent.io/platform/current/schema-registry/index.html>
    Confluent {
        urls: String,
        client_config: SchemaRegistryAuth,
        name_strategy: PbSchemaRegistryNameStrategy,
        topic: String,
    },
    /// <https://docs.aws.amazon.com/glue/latest/dg/schema-registry.html>
    Glue {
        schema_arn: String,
        aws_auth_props: AwsAuthProps,
        // When `Some(_)`, ignore AWS and load schemas from provided config
        mock_config: Option<String>,
    },
}

// TODO: `SpecificParserConfig` shall not `impl`/`derive` a `Default`
impl Default for SchemaLocation {
    fn default() -> Self {
        // backward compatible but undesired
        Self::File {
            url: Default::default(),
            aws_auth_props: None,
        }
    }
}

#[derive(Debug, Default, Clone)]
pub struct ProtobufProperties {
    pub message_name: String,
    pub use_schema_registry: bool,
    pub row_schema_location: String,
    pub aws_auth_props: Option<AwsAuthProps>,
    pub client_config: SchemaRegistryAuth,
    pub enable_upsert: bool,
    pub topic: String,
    pub key_message_name: Option<String>,
    pub name_strategy: PbSchemaRegistryNameStrategy,
    pub messages_as_jsonb: HashSet<String>,
}

#[derive(Debug, Default, Clone, Copy)]
pub struct CsvProperties {
    pub delimiter: u8,
    pub has_header: bool,
}

#[derive(Debug, Default, Clone)]
pub struct JsonProperties {
    pub use_schema_registry: bool,
    pub timestamptz_handling: Option<TimestamptzHandling>,
}

#[derive(Debug, Default, Clone)]
pub struct BytesProperties {
    pub column_name: Option<String>,
}