-
Notifications
You must be signed in to change notification settings - Fork 540
/
create_source.rs
1220 lines (1132 loc) · 48.7 KB
/
create_source.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
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
// Copyright 2023 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::{BTreeMap, HashMap};
use std::sync::LazyLock;
use anyhow::anyhow;
use itertools::Itertools;
use maplit::{convert_args, hashmap};
use pgwire::pg_response::{PgResponse, StatementType};
use risingwave_common::catalog::{
is_column_ids_dedup, ColumnCatalog, ColumnDesc, TableId, DEFAULT_KEY_COLUMN_NAME,
INITIAL_SOURCE_VERSION_ID, KAFKA_TIMESTAMP_COLUMN_NAME,
};
use risingwave_common::error::ErrorCode::{self, InvalidInputSyntax, ProtocolError};
use risingwave_common::error::{Result, RwError};
use risingwave_common::types::DataType;
use risingwave_connector::parser::{
name_strategy_from_str, schema_to_columns, AvroParserConfig, DebeziumAvroParserConfig,
ProtobufParserConfig, SpecificParserConfig,
};
use risingwave_connector::source::cdc::{
CITUS_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR,
};
use risingwave_connector::source::datagen::DATAGEN_CONNECTOR;
use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType};
use risingwave_connector::source::{
SourceEncode, SourceFormat, SourceStruct, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR,
KINESIS_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR,
S3_V2_CONNECTOR,
};
use risingwave_pb::catalog::{
PbSchemaRegistryNameStrategy, PbSource, StreamSourceInfo, WatermarkDesc,
};
use risingwave_pb::plan_common::{EncodeType, FormatType};
use risingwave_sqlparser::ast::{
self, get_delimiter, AstString, AvroSchema, ColumnDef, ColumnOption, CreateSourceStatement,
DebeziumAvroSchema, Encode, Format, ProtobufSchema, SourceSchemaV2, SourceWatermark,
};
use super::RwPgResponse;
use crate::binder::Binder;
use crate::catalog::ColumnId;
use crate::expr::Expr;
use crate::handler::create_table::{
bind_pk_names, bind_pk_on_relation, bind_sql_column_constraints, bind_sql_columns,
ensure_table_constraints_supported, ColumnIdGenerator,
};
use crate::handler::util::{get_connector, is_kafka_connector};
use crate::handler::HandlerArgs;
use crate::session::SessionImpl;
use crate::utils::resolve_privatelink_in_with_option;
use crate::{bind_data_type, WithOptions};
pub(crate) const UPSTREAM_SOURCE_KEY: &str = "connector";
pub(crate) const CONNECTION_NAME_KEY: &str = "connection.name";
/// Map a JSON schema to a relational schema
async fn extract_json_table_schema(
schema_config: &Option<(AstString, bool)>,
with_properties: &HashMap<String, String>,
) -> Result<Option<Vec<ColumnCatalog>>> {
match schema_config {
None => Ok(None),
Some((schema_location, use_schema_registry)) => Ok(Some(
schema_to_columns(&schema_location.0, *use_schema_registry, with_properties)
.await?
.into_iter()
.map(|col| ColumnCatalog {
column_desc: col.into(),
is_hidden: false,
})
.collect_vec(),
)),
}
}
fn json_schema_infer_use_schema_registry(schema_config: &Option<(AstString, bool)>) -> bool {
match schema_config {
None => false,
Some((_, use_registry)) => *use_registry,
}
}
/// Map an Avro schema to a relational schema.
async fn extract_avro_table_schema(
info: &StreamSourceInfo,
with_properties: &HashMap<String, String>,
) -> Result<Vec<ColumnCatalog>> {
let parser_config = SpecificParserConfig::new(
SourceStruct::new(SourceFormat::Plain, SourceEncode::Avro),
info,
with_properties,
)?;
let conf = AvroParserConfig::new(parser_config.encoding_config).await?;
let vec_column_desc = conf.map_to_columns()?;
Ok(vec_column_desc
.into_iter()
.map(|col| ColumnCatalog {
column_desc: col.into(),
is_hidden: false,
})
.collect_vec())
}
/// Map an Avro schema to a relational schema. And extract primary key columns.
async fn extract_upsert_avro_table_schema(
info: &StreamSourceInfo,
with_properties: &HashMap<String, String>,
) -> Result<(Vec<ColumnCatalog>, Vec<String>)> {
let parser_config = SpecificParserConfig::new(
SourceStruct::new(SourceFormat::Upsert, SourceEncode::Avro),
info,
with_properties,
)?;
let conf = AvroParserConfig::new(parser_config.encoding_config).await?;
let vec_column_desc = conf.map_to_columns()?;
let mut vec_column_catalog = vec_column_desc
.clone()
.into_iter()
.map(|col| ColumnCatalog {
column_desc: col.into(),
is_hidden: false,
})
.collect_vec();
// For upsert avro, if we can't extract pk from schema, use message key as primary key
let pks = if let Ok(pk_desc) = conf.extract_pks() {
pk_desc
.into_iter()
.map(|desc| {
vec_column_desc
.iter()
.find(|x| x.name == desc.name)
.ok_or_else(|| {
RwError::from(ErrorCode::InternalError(format!(
"Can not found primary key column {} in value schema",
desc.name
)))
})
})
.map_ok(|desc| desc.name.clone())
.collect::<Result<Vec<_>>>()?
} else {
add_upsert_default_key_column(&mut vec_column_catalog);
vec![DEFAULT_KEY_COLUMN_NAME.into()]
};
Ok((vec_column_catalog, pks))
}
async fn extract_debezium_avro_table_pk_columns(
info: &StreamSourceInfo,
with_properties: &HashMap<String, String>,
) -> Result<Vec<String>> {
let parser_config = SpecificParserConfig::new(
SourceStruct::new(SourceFormat::Debezium, SourceEncode::Avro),
info,
with_properties,
)?;
let conf = DebeziumAvroParserConfig::new(parser_config.encoding_config).await?;
Ok(conf.extract_pks()?.drain(..).map(|c| c.name).collect())
}
// Map an Avro schema to a relational schema and return the pk_column_ids.
async fn extract_debezium_avro_table_schema(
info: &StreamSourceInfo,
with_properties: &HashMap<String, String>,
) -> Result<Vec<ColumnCatalog>> {
let parser_config = SpecificParserConfig::new(
SourceStruct::new(SourceFormat::Debezium, SourceEncode::Avro),
info,
with_properties,
)?;
let conf = DebeziumAvroParserConfig::new(parser_config.encoding_config).await?;
let vec_column_desc = conf.map_to_columns()?;
let column_catalog = vec_column_desc
.into_iter()
.map(|col| ColumnCatalog {
column_desc: col.into(),
is_hidden: false,
})
.collect_vec();
Ok(column_catalog)
}
/// Map a protobuf schema to a relational schema.
async fn extract_protobuf_table_schema(
schema: &ProtobufSchema,
with_properties: HashMap<String, String>,
) -> Result<Vec<ColumnCatalog>> {
let info = StreamSourceInfo {
proto_message_name: schema.message_name.0.clone(),
row_schema_location: schema.row_schema_location.0.clone(),
use_schema_registry: schema.use_schema_registry,
..Default::default()
};
let parser_config = SpecificParserConfig::new(
SourceStruct::new(SourceFormat::Plain, SourceEncode::Protobuf),
&info,
&with_properties,
)?;
let conf = ProtobufParserConfig::new(parser_config.encoding_config).await?;
let column_descs = conf.map_to_columns()?;
Ok(column_descs
.into_iter()
.map(|col| ColumnCatalog {
column_desc: col.into(),
is_hidden: false,
})
.collect_vec())
}
fn non_generated_sql_columns(columns: &[ColumnDef]) -> Vec<ColumnDef> {
columns
.iter()
.filter(|c| {
c.options
.iter()
.all(|option| !matches!(option.option, ColumnOption::GeneratedColumns(_)))
})
.cloned()
.collect()
}
fn try_consume_string_from_options(
row_options: &mut BTreeMap<String, String>,
key: &str,
) -> Option<AstString> {
row_options.remove(key).map(AstString)
}
fn consume_string_from_options(
row_options: &mut BTreeMap<String, String>,
key: &str,
) -> Result<AstString> {
try_consume_string_from_options(row_options, key).ok_or(RwError::from(ProtocolError(format!(
"missing field {} in options",
key
))))
}
pub fn get_json_schema_location(
row_options: &mut BTreeMap<String, String>,
) -> Result<Option<(AstString, bool)>> {
let schema_location = try_consume_string_from_options(row_options, "schema.location");
let schema_registry = try_consume_string_from_options(row_options, "schema.registry");
match (schema_location, schema_registry) {
(None, None) => Ok(None),
(None, Some(schema_registry)) => Ok(Some((schema_registry, true))),
(Some(schema_location), None) => Ok(Some((schema_location, false))),
(Some(_), Some(_)) => Err(RwError::from(ProtocolError(
"only need either the schema location or the schema registry".to_string(),
))),
}
}
fn get_schema_location(row_options: &mut BTreeMap<String, String>) -> Result<(AstString, bool)> {
let schema_location = try_consume_string_from_options(row_options, "schema.location");
let schema_registry = try_consume_string_from_options(row_options, "schema.registry");
match (schema_location, schema_registry) {
(None, None) => Err(RwError::from(ProtocolError(
"missing either a schema location or a schema registry".to_string(),
))),
(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_string(),
))),
}
}
#[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)),
}
}
/// resolve the schema of the source from external schema file, return the relation's columns. see <https://www.risingwave.dev/docs/current/sql-create-source> for more information.
/// return `(columns, pk_names, source info)`
pub(crate) async fn try_bind_columns_from_source(
source_schema: &SourceSchemaV2,
sql_defined_pk_names: Vec<String>,
sql_defined_columns: &[ColumnDef],
with_properties: &HashMap<String, String>,
) -> Result<(Option<Vec<ColumnCatalog>>, Vec<String>, 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 sql_defined_pk = !sql_defined_pk_names.is_empty();
let sql_defined_schema = !sql_defined_columns.is_empty();
let is_kafka: bool = is_kafka_connector(with_properties);
let mut options = source_schema.gen_options().map_err(|e| anyhow!(e))?;
let 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)
};
let 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_string(),
)));
}
Ok(name_strategy)
};
let res = match (&source_schema.format, &source_schema.row_encode) {
(Format::Native, Encode::Native) => (
None,
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Native as i32,
row_encode: EncodeType::Native as i32,
..Default::default()
},
),
(Format::Plain, Encode::Protobuf) => {
if sql_defined_schema {
return Err(RwError::from(ProtocolError(
"User-defined schema is not allowed with FORMAT PLAIN ENCODE PROTOBUF. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#protobuf for more information.".to_string())));
};
let (row_schema_location, use_schema_registry) = get_schema_location(&mut options)?;
let protobuf_schema = ProtobufSchema {
message_name: consume_string_from_options(&mut options, MESSAGE_NAME_KEY)?,
row_schema_location,
use_schema_registry,
};
let name_strategy =
get_sr_name_strategy_check(&mut options, protobuf_schema.use_schema_registry)?;
(
Some(
extract_protobuf_table_schema(&protobuf_schema, with_properties.clone())
.await?,
),
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Plain as i32,
row_encode: EncodeType::Protobuf as i32,
row_schema_location: protobuf_schema.row_schema_location.0.clone(),
use_schema_registry: protobuf_schema.use_schema_registry,
proto_message_name: protobuf_schema.message_name.0.clone(),
key_message_name: get_key_message_name(&mut options),
name_strategy: name_strategy.unwrap_or(
PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32,
),
..Default::default()
},
)
}
(Format::Plain, Encode::Json) => {
let schema_config = get_json_schema_location(&mut options)?;
if schema_config.is_some() && sql_defined_schema {
return Err(RwError::from(ProtocolError(
"User-defined schema is not allowed with schema registry.".to_string(),
)));
}
if schema_config.is_none() && sql_defined_columns.is_empty() {
return Err(RwError::from(InvalidInputSyntax(
"schema definition is required for ENCODE JSON".to_owned(),
)));
}
(
extract_json_table_schema(&schema_config, with_properties).await?,
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Plain as i32,
row_encode: EncodeType::Json as i32,
use_schema_registry: json_schema_infer_use_schema_registry(&schema_config),
..Default::default()
},
)
}
(Format::Plain, Encode::Avro) => {
let (row_schema_location, use_schema_registry) = get_schema_location(&mut options)?;
let avro_schema = AvroSchema {
row_schema_location,
use_schema_registry,
};
if sql_defined_schema {
return Err(RwError::from(ProtocolError(
"User-defined schema is not allowed with FORMAT PLAIN ENCODE AVRO. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#avro for more information.".to_string())));
}
let key_message_name = get_key_message_name(&mut options);
let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY);
let name_strategy =
get_sr_name_strategy_check(&mut options, avro_schema.use_schema_registry)?;
let stream_source_info = StreamSourceInfo {
format: FormatType::Plain as i32,
row_encode: EncodeType::Avro as i32,
row_schema_location: avro_schema.row_schema_location.0.clone(),
use_schema_registry: avro_schema.use_schema_registry,
proto_message_name: message_name.unwrap_or(AstString("".into())).0,
key_message_name,
name_strategy: name_strategy
.unwrap_or(PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32),
..Default::default()
};
(
Some(extract_avro_table_schema(&stream_source_info, with_properties).await?),
sql_defined_pk_names,
stream_source_info,
)
}
(Format::Plain, Encode::Csv) => {
let chars = consume_string_from_options(&mut options, "delimiter")?.0;
let delimiter =
get_delimiter(chars.as_str()).map_err(|e| RwError::from(e.to_string()))?;
let has_header = try_consume_string_from_options(&mut options, "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(),
)));
}
(
None,
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Plain as i32,
row_encode: EncodeType::Csv as i32,
csv_delimiter: delimiter as i32,
csv_has_header: has_header,
..Default::default()
},
)
}
(Format::Plain, Encode::Bytes) => {
if !sql_defined_schema || sql_defined_columns.len() != 1 {
return Err(RwError::from(ProtocolError(
"BYTES format only accepts one column".to_string(),
)));
}
match sql_defined_columns[0].data_type {
Some(ast::DataType::Bytea) => {}
_ => {
return Err(RwError::from(ProtocolError(
"BYTES format only accepts BYTEA type".to_string(),
)))
}
}
(
None,
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Plain as i32,
row_encode: EncodeType::Bytes as i32,
..Default::default()
},
)
}
(Format::Upsert, Encode::Json) => {
let schema_config = get_json_schema_location(&mut options)?;
let columns = extract_json_table_schema(&schema_config, with_properties).await?;
let (columns, pk_names) = if !sql_defined_pk {
let mut columns = match columns {
None => bind_sql_columns(sql_defined_columns)?,
Some(columns) => columns,
};
add_upsert_default_key_column(&mut columns);
(Some(columns), vec![DEFAULT_KEY_COLUMN_NAME.into()])
} else {
(columns, sql_defined_pk_names)
};
(
columns,
pk_names,
StreamSourceInfo {
format: FormatType::Upsert as i32,
row_encode: EncodeType::Json as i32,
use_schema_registry: json_schema_infer_use_schema_registry(&schema_config),
..Default::default()
},
)
}
(Format::Upsert, Encode::Avro) => {
let (row_schema_location, use_schema_registry) = get_schema_location(&mut options)?;
let avro_schema = AvroSchema {
row_schema_location,
use_schema_registry,
};
if sql_defined_schema {
return Err(RwError::from(ProtocolError(
"User-defined schema is not allowed with row format upsert avro. Please refer to https://www.risingwave.dev/docs/current/sql-create-source/#avro for more information.".to_string())));
}
let name_strategy =
get_sr_name_strategy_check(&mut options, avro_schema.use_schema_registry)?
.unwrap_or(PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32);
let key_message_name = get_key_message_name(&mut options);
let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY);
if sql_defined_pk {
if sql_defined_pk_names.len() != 1 {
return Err(RwError::from(ProtocolError(
"upsert avro supports only one primary key column.".to_string(),
)));
}
let upsert_avro_primary_key = sql_defined_pk_names[0].clone();
let stream_source_info = StreamSourceInfo {
key_message_name,
format: FormatType::Upsert as i32,
row_encode: EncodeType::Avro as i32,
row_schema_location: avro_schema.row_schema_location.0.clone(),
use_schema_registry: avro_schema.use_schema_registry,
proto_message_name: message_name.unwrap_or(AstString("".into())).0,
upsert_avro_primary_key,
name_strategy,
..Default::default()
};
let columns =
extract_avro_table_schema(&stream_source_info, with_properties).await?;
(Some(columns), sql_defined_pk_names, stream_source_info)
} else {
let stream_source_info = StreamSourceInfo {
format: FormatType::Upsert as i32,
row_encode: EncodeType::Avro as i32,
row_schema_location: avro_schema.row_schema_location.0.clone(),
use_schema_registry: avro_schema.use_schema_registry,
proto_message_name: message_name.unwrap_or(AstString("".into())).0,
name_strategy,
key_message_name,
..Default::default()
};
let (columns, pk_from_avro) =
extract_upsert_avro_table_schema(&stream_source_info, with_properties).await?;
(Some(columns), pk_from_avro, stream_source_info)
}
}
(Format::Debezium, Encode::Json) => {
if !sql_defined_pk {
return Err(RwError::from(ProtocolError(
"Primary key must be specified when creating source with format debezium."
.to_string(),
)));
}
let schema_config = get_json_schema_location(&mut options)?;
(
extract_json_table_schema(&schema_config, with_properties).await?,
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Debezium as i32,
row_encode: EncodeType::Json as i32,
use_schema_registry: json_schema_infer_use_schema_registry(&schema_config),
..Default::default()
},
)
}
(Format::Debezium, Encode::Avro) => {
let (row_schema_location, use_schema_registry) = get_schema_location(&mut options)?;
if !use_schema_registry {
return Err(RwError::from(ProtocolError(
"schema location for DEBEZIUM_AVRO row format is not supported".to_string(),
)));
}
let avro_schema = DebeziumAvroSchema {
row_schema_location,
};
if sql_defined_schema {
return Err(RwError::from(ProtocolError(
"User-defined schema is not allowed with row format debezium avro.".to_string(),
)));
}
// no need to check whether works schema registry because debezium avro always work with
// schema registry
let name_strategy = get_sr_name_strategy_check(&mut options, true)?;
let message_name = try_consume_string_from_options(&mut options, MESSAGE_NAME_KEY);
let key_message_name = get_key_message_name(&mut options);
let stream_source_info = StreamSourceInfo {
use_schema_registry,
proto_message_name: message_name.unwrap_or(AstString("".into())).0,
name_strategy: name_strategy
.unwrap_or(PbSchemaRegistryNameStrategy::TopicNameStrategyUnspecified as i32),
format: FormatType::Debezium as i32,
row_encode: EncodeType::Avro as i32,
row_schema_location: avro_schema.row_schema_location.0.clone(),
key_message_name,
..Default::default()
};
let full_columns =
extract_debezium_avro_table_schema(&stream_source_info, with_properties).await?;
let pk_names = if sql_defined_pk {
sql_defined_pk_names
} else {
let pk_names =
extract_debezium_avro_table_pk_columns(&stream_source_info, with_properties)
.await?;
// extract pk(s) from schema registry
for pk_name in &pk_names {
full_columns
.iter()
.find(|c: &&ColumnCatalog| c.name().eq(pk_name))
.ok_or_else(|| {
RwError::from(ProtocolError(format!(
"avro's key column {} not exists in avro's row schema",
pk_name
)))
})?;
}
pk_names
};
(Some(full_columns), pk_names, stream_source_info)
}
(Format::DebeziumMongo, Encode::Json) => {
let mut columns = vec![
ColumnCatalog {
column_desc: ColumnDesc {
data_type: DataType::Varchar,
column_id: 0.into(),
name: "_id".to_string(),
field_descs: vec![],
type_name: "".to_string(),
generated_or_default_column: None,
},
is_hidden: false,
},
ColumnCatalog {
column_desc: ColumnDesc {
data_type: DataType::Jsonb,
column_id: 0.into(),
name: "payload".to_string(),
field_descs: vec![],
type_name: "".to_string(),
generated_or_default_column: None,
},
is_hidden: false,
},
];
if sql_defined_schema {
let non_generated_sql_defined_columns =
non_generated_sql_columns(sql_defined_columns);
if non_generated_sql_defined_columns.len() != 2
&& non_generated_sql_defined_columns[0].name.real_value() != columns[0].name()
&& non_generated_sql_defined_columns[1].name.real_value() != columns[1].name()
{
return Err(RwError::from(ProtocolError(
"the not generated columns of the source with row format DebeziumMongoJson
must be (_id [Jsonb | Varchar | Int32 | Int64], payload jsonb)."
.to_string(),
)));
}
if let Some(key_data_type) = &non_generated_sql_defined_columns[0].data_type {
let key_data_type = bind_data_type(key_data_type)?;
match key_data_type {
DataType::Jsonb | DataType::Varchar | DataType::Int32 | DataType::Int64 => {
columns[0].column_desc.data_type = key_data_type;
}
_ => {
return Err(RwError::from(ProtocolError(
"the `_id` column of the source with row format DebeziumMongoJson
must be [Jsonb | Varchar | Int32 | Int64]"
.to_string(),
)));
}
}
}
if let Some(value_data_type) = &non_generated_sql_defined_columns[1].data_type {
if !matches!(bind_data_type(value_data_type)?, DataType::Jsonb) {
return Err(RwError::from(ProtocolError(
"the `payload` column of the source with row format DebeziumMongoJson
must be Jsonb datatype"
.to_string(),
)));
}
}
}
let pk_names = if sql_defined_pk {
sql_defined_pk_names
} else {
vec!["_id".to_string()]
};
(
Some(columns),
pk_names,
StreamSourceInfo {
format: FormatType::DebeziumMongo as i32,
row_encode: EncodeType::Json as i32,
..Default::default()
},
)
}
(Format::Maxwell, Encode::Json) => {
if !sql_defined_pk {
return Err(RwError::from(ProtocolError(
"Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON."
.to_string(),
)));
}
let schema_config = get_json_schema_location(&mut options)?;
(
extract_json_table_schema(&schema_config, with_properties).await?,
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Maxwell as i32,
row_encode: EncodeType::Json as i32,
use_schema_registry: json_schema_infer_use_schema_registry(&schema_config),
..Default::default()
},
)
}
(Format::Canal, Encode::Json) => {
if !sql_defined_pk {
return Err(RwError::from(ProtocolError(
"Primary key must be specified when creating source with row format cannal_json."
.to_string(),
)));
}
let schema_config = get_json_schema_location(&mut options)?;
(
extract_json_table_schema(&schema_config, with_properties).await?,
sql_defined_pk_names,
StreamSourceInfo {
format: FormatType::Canal as i32,
row_encode: EncodeType::Json as i32,
use_schema_registry: json_schema_infer_use_schema_registry(&schema_config),
..Default::default()
},
)
}
(format, encoding) => {
return Err(RwError::from(ProtocolError(format!(
"Unknown combination {:?} {:?}",
format, encoding
))));
}
};
if !options.is_empty() {
return Err(RwError::from(ProtocolError(format!(
"Unknown options for {:?} {:?}: {}",
source_schema.format,
source_schema.row_encode,
options
.iter()
.map(|(k, v)| format!("{}:{}", k, v))
.collect::<Vec<String>>()
.join(","),
))));
}
Ok(res)
}
// Add a hidden column `_rw_kafka_timestamp` to each message from Kafka source.
fn check_and_add_timestamp_column(
with_properties: &HashMap<String, String>,
columns: &mut Vec<ColumnCatalog>,
) {
if is_kafka_connector(with_properties) {
let kafka_timestamp_column = ColumnCatalog {
column_desc: ColumnDesc {
data_type: DataType::Timestamptz,
column_id: ColumnId::placeholder(),
name: KAFKA_TIMESTAMP_COLUMN_NAME.to_string(),
field_descs: vec![],
type_name: "".to_string(),
generated_or_default_column: None,
},
is_hidden: true,
};
columns.push(kafka_timestamp_column);
}
}
fn add_upsert_default_key_column(columns: &mut Vec<ColumnCatalog>) {
let column = ColumnCatalog {
column_desc: ColumnDesc {
data_type: DataType::Bytea,
column_id: ColumnId::new(columns.len() as i32),
name: DEFAULT_KEY_COLUMN_NAME.to_string(),
field_descs: vec![],
type_name: "".to_string(),
generated_or_default_column: None,
},
is_hidden: true,
};
columns.push(column);
}
pub(super) fn bind_source_watermark(
session: &SessionImpl,
name: String,
source_watermarks: Vec<SourceWatermark>,
column_catalogs: &[ColumnCatalog],
) -> Result<Vec<WatermarkDesc>> {
let mut binder = Binder::new_for_ddl(session);
binder.bind_columns_to_context(name.clone(), column_catalogs.to_vec())?;
let watermark_descs = source_watermarks
.into_iter()
.map(|source_watermark| {
let col_name = source_watermark.column.real_value();
let watermark_idx = binder.get_column_binding_index(name.clone(), &col_name)?;
let expr = binder.bind_expr(source_watermark.expr)?;
let watermark_col_type = column_catalogs[watermark_idx].data_type();
let watermark_expr_type = &expr.return_type();
if watermark_col_type != watermark_expr_type {
Err(RwError::from(ErrorCode::BindError(
format!("The return value type of the watermark expression must be identical to the watermark column data type. Current data type of watermark return value: `{}`, column `{}`",watermark_expr_type, watermark_col_type),
)))
} else {
let expr_proto = expr.to_expr_proto();
Ok::<_, RwError>(WatermarkDesc {
watermark_idx: watermark_idx as u32,
expr: Some(expr_proto),
})
}
})
.try_collect()?;
Ok(watermark_descs)
}
// TODO: Better design if we want to support ENCODE KEY where we will have 4 dimensional array
static CONNECTORS_COMPATIBLE_FORMATS: LazyLock<HashMap<String, HashMap<Format, Vec<Encode>>>> =
LazyLock::new(|| {
convert_args!(hashmap!(
KAFKA_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Json, Encode::Protobuf, Encode::Avro, Encode::Bytes, Encode::Csv],
Format::Upsert => vec![Encode::Json, Encode::Avro],
Format::Debezium => vec![Encode::Json, Encode::Avro],
Format::Maxwell => vec![Encode::Json],
Format::Canal => vec![Encode::Json],
Format::DebeziumMongo => vec![Encode::Json],
),
PULSAR_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Json, Encode::Protobuf, Encode::Avro, Encode::Bytes],
Format::Upsert => vec![Encode::Json, Encode::Avro],
Format::Debezium => vec![Encode::Json],
Format::Maxwell => vec![Encode::Json],
Format::Canal => vec![Encode::Json],
),
KINESIS_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Json, Encode::Protobuf, Encode::Avro, Encode::Bytes],
Format::Upsert => vec![Encode::Json, Encode::Avro],
Format::Debezium => vec![Encode::Json],
Format::Maxwell => vec![Encode::Json],
Format::Canal => vec![Encode::Json],
),
GOOGLE_PUBSUB_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Json, Encode::Protobuf, Encode::Avro, Encode::Bytes],
Format::Debezium => vec![Encode::Json],
Format::Maxwell => vec![Encode::Json],
Format::Canal => vec![Encode::Json],
),
NEXMARK_CONNECTOR => hashmap!(
Format::Native => vec![Encode::Native],
Format::Plain => vec![Encode::Bytes],
),
DATAGEN_CONNECTOR => hashmap!(
Format::Native => vec![Encode::Native],
Format::Plain => vec![Encode::Bytes, Encode::Json],
),
S3_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Csv, Encode::Json],
),
S3_V2_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Csv, Encode::Json],
),
MYSQL_CDC_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Bytes],
Format::Debezium => vec![Encode::Json],
),
POSTGRES_CDC_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Bytes],
Format::Debezium => vec![Encode::Json],
),
CITUS_CDC_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Bytes],
Format::Debezium => vec![Encode::Json],
),
NATS_CONNECTOR => hashmap!(
Format::Plain => vec![Encode::Json],
),
))
});
pub fn validate_compatibility(
source_schema: &SourceSchemaV2,
props: &mut HashMap<String, String>,
) -> Result<()> {
let connector = get_connector(props)
.ok_or_else(|| RwError::from(ProtocolError("missing field 'connector'".to_string())))?;
let compatible_formats = CONNECTORS_COMPATIBLE_FORMATS
.get(&connector)
.ok_or_else(|| {
RwError::from(ProtocolError(format!(
"connector {} is not supported",
connector
)))
})?;
if connector != KAFKA_CONNECTOR {
let res = match (&source_schema.format, &source_schema.row_encode) {
(Format::Plain, Encode::Protobuf) | (Format::Plain, Encode::Avro) => {
let mut options = source_schema.gen_options().map_err(|e| anyhow!(e))?;
let (_, use_schema_registry) = get_schema_location(&mut options)?;
use_schema_registry
}
(Format::Debezium, Encode::Avro) => true,
(_, _) => false,
};
if res {
return Err(RwError::from(ProtocolError(format!(
"The {} must be kafka when schema registry is used",
UPSTREAM_SOURCE_KEY
))));
}
}
let compatible_encodes = compatible_formats
.get(&source_schema.format)
.ok_or_else(|| {
RwError::from(ProtocolError(format!(
"connector {} does not support format {:?}",
connector, source_schema.format
)))
})?;
if !compatible_encodes.contains(&source_schema.row_encode) {
return Err(RwError::from(ProtocolError(format!(
"connector {} does not support format {:?} with encode {:?}",
connector, source_schema.format, source_schema.row_encode
))));
}
if connector == POSTGRES_CDC_CONNECTOR || connector == CITUS_CDC_CONNECTOR {
if !props.contains_key("slot.name") {
// Build a random slot name with UUID
// e.g. "rw_cdc_f9a3567e6dd54bf5900444c8b1c03815"
let uuid = uuid::Uuid::new_v4().to_string().replace('-', "");
props.insert("slot.name".into(), format!("rw_cdc_{}", uuid));
}
if !props.contains_key("schema.name") {
// Default schema name is "public"
props.insert("schema.name".into(), "public".into());
}
if !props.contains_key("publication.name") {
// Default publication name is "rw_publication"
props.insert("publication.name".into(), "rw_publication".into());
}
if !props.contains_key("publication.create.enable") {
// Default auto create publication if doesn't exist
props.insert("publication.create.enable".into(), "true".into());
}
}
Ok(())
}
/// Performs early stage checking in frontend to see if the schema of the given `columns` is
/// compatible with the connector extracted from the properties. Currently this only works for
/// `nexmark` connector since it's in chunk format.
///
/// One should only call this function after all properties of all columns are resolved, like
/// generated column descriptors.
pub(super) fn check_source_schema(
props: &HashMap<String, String>,
row_id_index: Option<usize>,
columns: &[ColumnCatalog],
) -> Result<()> {
let Some(connector) = get_connector(props) else {
return Ok(());
};