risingwave_frontend/handler/
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
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154
1155
1156
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174
1175
1176
1177
1178
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198
1199
1200
1201
1202
1203
1204
1205
1206
1207
1208
1209
1210
1211
1212
1213
1214
1215
1216
1217
1218
1219
1220
1221
1222
1223
1224
1225
1226
1227
1228
1229
1230
1231
1232
1233
1234
1235
1236
1237
1238
1239
1240
1241
1242
1243
1244
1245
1246
1247
1248
1249
1250
1251
1252
1253
1254
1255
1256
1257
1258
1259
1260
1261
1262
1263
1264
1265
1266
1267
1268
1269
1270
1271
1272
1273
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290
1291
1292
1293
1294
1295
1296
1297
1298
1299
1300
1301
1302
1303
1304
1305
1306
1307
1308
1309
1310
1311
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321
1322
1323
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337
1338
1339
1340
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372
1373
1374
1375
1376
1377
1378
1379
1380
1381
1382
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394
1395
1396
1397
1398
1399
1400
1401
1402
1403
1404
1405
1406
1407
1408
1409
1410
1411
1412
1413
1414
1415
1416
1417
1418
1419
1420
1421
1422
1423
1424
1425
1426
1427
1428
1429
1430
1431
1432
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469
1470
1471
1472
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483
1484
1485
1486
1487
1488
1489
1490
1491
1492
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511
1512
1513
1514
1515
1516
1517
1518
1519
1520
1521
1522
1523
1524
1525
1526
1527
1528
1529
1530
1531
1532
1533
1534
1535
1536
1537
1538
1539
1540
1541
1542
1543
1544
1545
1546
1547
1548
1549
1550
1551
1552
1553
1554
1555
1556
1557
1558
1559
1560
1561
1562
1563
1564
1565
1566
1567
1568
1569
1570
1571
1572
1573
1574
1575
1576
1577
1578
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588
1589
1590
1591
1592
1593
1594
1595
1596
1597
1598
1599
1600
1601
1602
1603
1604
1605
1606
1607
1608
1609
1610
1611
1612
1613
1614
1615
1616
1617
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627
1628
1629
1630
1631
1632
1633
1634
1635
1636
1637
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647
1648
1649
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663
1664
1665
1666
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678
1679
1680
1681
1682
1683
1684
1685
1686
1687
1688
1689
1690
1691
1692
1693
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704
1705
1706
1707
1708
1709
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721
1722
1723
1724
1725
1726
1727
1728
1729
1730
1731
1732
1733
1734
1735
1736
1737
1738
1739
1740
1741
1742
1743
1744
1745
1746
1747
1748
1749
1750
1751
1752
1753
1754
1755
1756
1757
1758
1759
1760
1761
1762
1763
1764
1765
1766
1767
1768
1769
1770
1771
1772
1773
1774
1775
1776
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787
1788
1789
1790
1791
1792
1793
1794
1795
1796
1797
1798
1799
1800
1801
1802
1803
1804
1805
1806
1807
1808
1809
1810
1811
1812
1813
1814
1815
1816
1817
1818
1819
1820
1821
1822
1823
1824
1825
1826
1827
1828
1829
1830
1831
1832
1833
1834
1835
1836
1837
1838
1839
1840
1841
1842
1843
1844
1845
1846
1847
1848
1849
1850
1851
1852
1853
1854
1855
1856
1857
1858
1859
1860
1861
1862
1863
1864
1865
1866
1867
1868
1869
1870
1871
1872
1873
1874
1875
1876
1877
1878
1879
1880
1881
1882
1883
1884
1885
1886
1887
1888
1889
1890
1891
1892
1893
1894
1895
1896
1897
1898
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916
1917
1918
1919
1920
1921
1922
1923
1924
1925
1926
1927
1928
1929
1930
1931
1932
1933
1934
1935
1936
1937
1938
1939
1940
1941
1942
1943
1944
1945
1946
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956
1957
1958
1959
1960
1961
1962
1963
1964
1965
1966
1967
1968
1969
1970
1971
1972
1973
1974
1975
1976
1977
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996
// Copyright 2024 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::rc::Rc;
use std::sync::LazyLock;

use anyhow::{anyhow, Context};
use either::Either;
use itertools::Itertools;
use maplit::{convert_args, hashmap};
use pgwire::pg_response::{PgResponse, StatementType};
use rand::Rng;
use risingwave_common::array::arrow::{arrow_schema_iceberg, IcebergArrowConvert};
use risingwave_common::bail_not_implemented;
use risingwave_common::catalog::{
    debug_assert_column_ids_distinct, ColumnCatalog, ColumnDesc, ColumnId, Schema, TableId,
    INITIAL_SOURCE_VERSION_ID, KAFKA_TIMESTAMP_COLUMN_NAME,
};
use risingwave_common::license::Feature;
use risingwave_common::secret::LocalSecretManager;
use risingwave_common::types::DataType;
use risingwave_connector::parser::additional_columns::{
    build_additional_column_desc, get_supported_additional_columns,
};
use risingwave_connector::parser::{
    fetch_json_schema_and_map_to_columns, AvroParserConfig, DebeziumAvroParserConfig,
    ProtobufParserConfig, SchemaLocation, SpecificParserConfig, TimestamptzHandling,
    DEBEZIUM_IGNORE_KEY,
};
use risingwave_connector::schema::schema_registry::{
    name_strategy_from_str, SchemaRegistryAuth, SCHEMA_REGISTRY_PASSWORD, SCHEMA_REGISTRY_USERNAME,
};
use risingwave_connector::schema::AWS_GLUE_SCHEMA_ARN_KEY;
use risingwave_connector::source::cdc::{
    CDC_AUTO_SCHEMA_CHANGE_KEY, CDC_SHARING_MODE_KEY, CDC_SNAPSHOT_BACKFILL, CDC_SNAPSHOT_MODE_KEY,
    CDC_TRANSACTIONAL_KEY, CDC_WAIT_FOR_STREAMING_START_TIMEOUT, CITUS_CDC_CONNECTOR,
    MONGODB_CDC_CONNECTOR, MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, SQL_SERVER_CDC_CONNECTOR,
};
use risingwave_connector::source::datagen::DATAGEN_CONNECTOR;
use risingwave_connector::source::iceberg::ICEBERG_CONNECTOR;
use risingwave_connector::source::nexmark::source::{get_event_data_types_with_names, EventType};
use risingwave_connector::source::test_source::TEST_CONNECTOR;
pub use risingwave_connector::source::UPSTREAM_SOURCE_KEY;
use risingwave_connector::source::{
    ConnectorProperties, AZBLOB_CONNECTOR, GCS_CONNECTOR, GOOGLE_PUBSUB_CONNECTOR, KAFKA_CONNECTOR,
    KINESIS_CONNECTOR, MQTT_CONNECTOR, NATS_CONNECTOR, NEXMARK_CONNECTOR, OPENDAL_S3_CONNECTOR,
    POSIX_FS_CONNECTOR, PULSAR_CONNECTOR, S3_CONNECTOR,
};
use risingwave_connector::WithPropertiesExt;
use risingwave_pb::catalog::{PbSchemaRegistryNameStrategy, StreamSourceInfo, WatermarkDesc};
use risingwave_pb::plan_common::additional_column::ColumnType as AdditionalColumnType;
use risingwave_pb::plan_common::{EncodeType, FormatType};
use risingwave_sqlparser::ast::{
    get_delimiter, AstString, ColumnDef, CreateSourceStatement, Encode, Format,
    FormatEncodeOptions, ObjectName, ProtobufSchema, SourceWatermark, TableConstraint,
};
use risingwave_sqlparser::parser::{IncludeOption, IncludeOptionItem};
use thiserror_ext::AsReport;

use super::RwPgResponse;
use crate::binder::Binder;
use crate::catalog::source_catalog::SourceCatalog;
use crate::catalog::{CatalogError, DatabaseId, SchemaId};
use crate::error::ErrorCode::{self, Deprecated, InvalidInputSyntax, NotSupported, ProtocolError};
use crate::error::{Result, RwError};
use crate::expr::Expr;
use crate::handler::create_table::{
    bind_pk_and_row_id_on_relation, bind_sql_column_constraints, bind_sql_columns,
    bind_sql_pk_names, bind_table_constraints, ColumnIdGenerator,
};
use crate::handler::util::SourceSchemaCompatExt;
use crate::handler::HandlerArgs;
use crate::optimizer::plan_node::generic::SourceNodeKind;
use crate::optimizer::plan_node::{LogicalSource, ToStream, ToStreamContext};
use crate::session::SessionImpl;
use crate::utils::{
    resolve_privatelink_in_with_option, resolve_secret_ref_in_with_options, OverwriteOptions,
};
use crate::{bind_data_type, build_graph, OptimizerContext, WithOptions, WithOptionsSecResolved};

/// Map a JSON schema to a relational schema
async fn extract_json_table_schema(
    schema_config: &Option<(AstString, bool)>,
    with_properties: &BTreeMap<String, String>,
    format_encode_options: &mut BTreeMap<String, String>,
) -> Result<Option<Vec<ColumnCatalog>>> {
    match schema_config {
        None => Ok(None),
        Some((schema_location, use_schema_registry)) => {
            let schema_registry_auth = use_schema_registry.then(|| {
                let auth = SchemaRegistryAuth::from(&*format_encode_options);
                try_consume_string_from_options(format_encode_options, SCHEMA_REGISTRY_USERNAME);
                try_consume_string_from_options(format_encode_options, SCHEMA_REGISTRY_PASSWORD);
                auth
            });
            Ok(Some(
                fetch_json_schema_and_map_to_columns(
                    &schema_location.0,
                    schema_registry_auth,
                    with_properties,
                )
                .await?
                .into_iter()
                .map(|col| ColumnCatalog {
                    column_desc: col.into(),
                    is_hidden: false,
                })
                .collect_vec(),
            ))
        }
    }
}

/// Note: these columns are added in `SourceStreamChunkRowWriter::do_action`.
/// May also look for the usage of `SourceColumnType`.
pub fn debezium_cdc_source_schema() -> Vec<ColumnCatalog> {
    let columns = vec![
        ColumnCatalog {
            column_desc: ColumnDesc::named("payload", ColumnId::placeholder(), DataType::Jsonb),
            is_hidden: false,
        },
        ColumnCatalog::offset_column(),
        ColumnCatalog::cdc_table_name_column(),
    ];
    columns
}

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: &WithOptionsSecResolved,
    format_encode_options: &mut BTreeMap<String, String>,
    is_debezium: bool,
) -> Result<Vec<ColumnCatalog>> {
    let parser_config = SpecificParserConfig::new(info, with_properties)?;
    try_consume_string_from_options(format_encode_options, SCHEMA_REGISTRY_USERNAME);
    try_consume_string_from_options(format_encode_options, SCHEMA_REGISTRY_PASSWORD);
    consume_aws_config_from_options(format_encode_options);

    let vec_column_desc = if is_debezium {
        let conf = DebeziumAvroParserConfig::new(parser_config.encoding_config).await?;
        conf.map_to_columns()?
    } else {
        if let risingwave_connector::parser::EncodingProperties::Avro(avro_props) =
            &parser_config.encoding_config
            && matches!(avro_props.schema_location, SchemaLocation::File { .. })
            && format_encode_options
                .get("with_deprecated_file_header")
                .is_none_or(|v| v != "true")
        {
            bail_not_implemented!(issue = 12871, "avro without schema registry");
        }
        let conf = AvroParserConfig::new(parser_config.encoding_config).await?;
        conf.map_to_columns()?
    };
    Ok(vec_column_desc
        .into_iter()
        .map(|col| ColumnCatalog {
            column_desc: col.into(),
            is_hidden: false,
        })
        .collect_vec())
}

async fn extract_debezium_avro_table_pk_columns(
    info: &StreamSourceInfo,
    with_properties: &WithOptionsSecResolved,
) -> Result<Vec<String>> {
    let parser_config = SpecificParserConfig::new(info, with_properties)?;
    let conf = DebeziumAvroParserConfig::new(parser_config.encoding_config).await?;
    Ok(conf.extract_pks()?.drain(..).map(|c| c.name).collect())
}

/// Map a protobuf schema to a relational schema.
async fn extract_protobuf_table_schema(
    schema: &ProtobufSchema,
    with_properties: &WithOptionsSecResolved,
    format_encode_options: &mut BTreeMap<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,
        format: FormatType::Plain.into(),
        row_encode: EncodeType::Protobuf.into(),
        format_encode_options: format_encode_options.clone(),
        ..Default::default()
    };
    let parser_config = SpecificParserConfig::new(&info, with_properties)?;
    try_consume_string_from_options(format_encode_options, SCHEMA_REGISTRY_USERNAME);
    try_consume_string_from_options(format_encode_options, SCHEMA_REGISTRY_PASSWORD);
    consume_aws_config_from_options(format_encode_options);

    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.is_generated())
        .cloned()
        .collect()
}

fn try_consume_string_from_options(
    format_encode_options: &mut BTreeMap<String, String>,
    key: &str,
) -> Option<AstString> {
    format_encode_options.remove(key).map(AstString)
}

fn consume_string_from_options(
    format_encode_options: &mut BTreeMap<String, String>,
    key: &str,
) -> Result<AstString> {
    try_consume_string_from_options(format_encode_options, key).ok_or(RwError::from(ProtocolError(
        format!("missing field {} in options", key),
    )))
}

fn consume_aws_config_from_options(format_encode_options: &mut BTreeMap<String, String>) {
    format_encode_options.retain(|key, _| !key.starts_with("aws."))
}

pub fn get_json_schema_location(
    format_encode_options: &mut BTreeMap<String, String>,
) -> Result<Option<(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) => 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(
    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_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)),
    }
}

/// 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(crate) async fn bind_columns_from_source(
    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) => resolve_secret_ref_in_with_options(options.clone(), session)?,
        Either::Right(options_with_secret) => options_with_secret.clone(),
    };

    let is_kafka: bool = options_with_secret.is_kafka_connector();
    let (format_encode_options, format_encode_secret_refs) = resolve_secret_ref_in_with_options(
        WithOptions::try_from(format_encode.row_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(),
    )?;

    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_string(),
            )));
        }
        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,
        ..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_string(),
                    )));
                }

                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 (format_encode_options, format_encode_secret_refs) = resolve_secret_ref_in_with_options(
        WithOptions::try_from(format_encode.row_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 = debezium_cdc_source_schema();
    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))
}

// check the additional column compatibility with the format and encode
fn check_additional_column_compatibility(
    column_def: &IncludeOptionItem,
    format_encode: Option<&FormatEncodeOptions>,
) -> Result<()> {
    // only allow header column have inner field
    if column_def.inner_field.is_some()
        && !column_def
            .column_type
            .real_value()
            .eq_ignore_ascii_case("header")
    {
        return Err(RwError::from(ProtocolError(format!(
            "Only header column can have inner field, but got {:?}",
            column_def.column_type.real_value(),
        ))));
    }

    // Payload column only allowed when encode is JSON
    if let Some(schema) = format_encode
        && column_def
            .column_type
            .real_value()
            .eq_ignore_ascii_case("payload")
        && !matches!(schema.row_encode, Encode::Json)
    {
        return Err(RwError::from(ProtocolError(format!(
            "INCLUDE payload is only allowed when using ENCODE JSON, but got ENCODE {:?}",
            schema.row_encode
        ))));
    }
    Ok(())
}

/// add connector-spec columns to the end of column catalog
pub fn handle_addition_columns(
    format_encode: Option<&FormatEncodeOptions>,
    with_properties: &BTreeMap<String, String>,
    mut additional_columns: IncludeOption,
    columns: &mut Vec<ColumnCatalog>,
    is_cdc_backfill_table: bool,
) -> Result<()> {
    let connector_name = with_properties.get_connector().unwrap(); // there must be a connector in source

    if get_supported_additional_columns(connector_name.as_str(), is_cdc_backfill_table).is_none()
        && !additional_columns.is_empty()
    {
        return Err(RwError::from(ProtocolError(format!(
            "Connector {} accepts no additional column but got {:?}",
            connector_name, additional_columns
        ))));
    }

    while let Some(item) = additional_columns.pop() {
        check_additional_column_compatibility(&item, format_encode)?;

        let data_type = item
            .header_inner_expect_type
            .map(|dt| bind_data_type(&dt))
            .transpose()?;
        if let Some(dt) = &data_type
            && !matches!(dt, DataType::Bytea | DataType::Varchar)
        {
            return Err(
                ErrorCode::BindError(format!("invalid additional column data type: {dt}")).into(),
            );
        }
        let col = build_additional_column_desc(
            ColumnId::placeholder(),
            connector_name.as_str(),
            item.column_type.real_value().as_str(),
            item.column_alias.map(|alias| alias.real_value()),
            item.inner_field.as_deref(),
            data_type.as_ref(),
            true,
            is_cdc_backfill_table,
        )?;
        columns.push(ColumnCatalog::visible(col));
    }

    Ok(())
}

/// Bind columns from both source and sql defined.
pub(crate) fn bind_all_columns(
    format_encode: &FormatEncodeOptions,
    cols_from_source: Option<Vec<ColumnCatalog>>,
    cols_from_sql: Vec<ColumnCatalog>,
    col_defs_from_sql: &[ColumnDef],
    wildcard_idx: Option<usize>,
) -> Result<Vec<ColumnCatalog>> {
    if let Some(cols_from_source) = cols_from_source {
        if cols_from_sql.is_empty() {
            Ok(cols_from_source)
        } else if let Some(wildcard_idx) = wildcard_idx {
            if col_defs_from_sql.iter().any(|c| !c.is_generated()) {
                Err(RwError::from(NotSupported(
                    "Only generated columns are allowed in user-defined schema from SQL"
                        .to_string(),
                    "Remove the non-generated columns".to_string(),
                )))
            } else {
                // Replace `*` with `cols_from_source`
                let mut cols_from_sql = cols_from_sql;
                let mut cols_from_source = cols_from_source;
                let mut cols_from_sql_r = cols_from_sql.split_off(wildcard_idx);
                cols_from_sql.append(&mut cols_from_source);
                cols_from_sql.append(&mut cols_from_sql_r);
                Ok(cols_from_sql)
            }
        } else {
            // TODO(yuhao): https://github.com/risingwavelabs/risingwave/issues/12209
            Err(RwError::from(ProtocolError(
                    format!("User-defined schema from SQL is not allowed with FORMAT {} ENCODE {}. \
                    Please refer to https://www.risingwave.dev/docs/current/sql-create-source/ for more information.", format_encode.format, format_encode.row_encode))))
        }
    } else {
        if wildcard_idx.is_some() {
            return Err(RwError::from(NotSupported(
                "Wildcard in user-defined schema is only allowed when there exists columns from external schema".to_string(),
                "Remove the wildcard or use a source with external schema".to_string(),
            )));
        }
        let non_generated_sql_defined_columns = non_generated_sql_columns(col_defs_from_sql);
        match (&format_encode.format, &format_encode.row_encode) {
            (Format::DebeziumMongo, Encode::Json) => {
                let mut columns = vec![
                    ColumnCatalog {
                        column_desc: ColumnDesc::named("_id", 0.into(), DataType::Varchar),
                        is_hidden: false,
                    },
                    ColumnCatalog {
                        column_desc: ColumnDesc::named("payload", 0.into(), DataType::Jsonb),
                        is_hidden: false,
                    },
                ];
                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(),
                    )));
                }
                // ok to unwrap since it was checked at `bind_sql_columns`
                let key_data_type = bind_data_type(
                    non_generated_sql_defined_columns[0]
                        .data_type
                        .as_ref()
                        .unwrap(),
                )?;
                match key_data_type {
                    DataType::Jsonb | DataType::Varchar | DataType::Int32 | DataType::Int64 => {
                        columns[0].column_desc.data_type = key_data_type.clone();
                    }
                    _ => {
                        return Err(RwError::from(ProtocolError(
                            "the `_id` column of the source with row format DebeziumMongoJson
        must be [Jsonb | Varchar | Int32 | Int64]"
                                .to_string(),
                        )));
                    }
                }

                // ok to unwrap since it was checked at `bind_sql_columns`
                let value_data_type = bind_data_type(
                    non_generated_sql_defined_columns[1]
                        .data_type
                        .as_ref()
                        .unwrap(),
                )?;
                if !matches!(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(),
                    )));
                }
                Ok(columns)
            }
            (Format::Plain, Encode::Bytes) => {
                let err = Err(RwError::from(ProtocolError(
                    "ENCODE BYTES only accepts one BYTEA type column".to_string(),
                )));
                if non_generated_sql_defined_columns.len() == 1 {
                    // ok to unwrap `data_type`` since it was checked at `bind_sql_columns`
                    let col_data_type = bind_data_type(
                        non_generated_sql_defined_columns[0]
                            .data_type
                            .as_ref()
                            .unwrap(),
                    )?;
                    if col_data_type == DataType::Bytea {
                        Ok(cols_from_sql)
                    } else {
                        err
                    }
                } else {
                    err
                }
            }
            (_, _) => Ok(cols_from_sql),
        }
    }
}

/// TODO: perhaps put the hint in notice is better. The error message format might be not that reliable.
fn hint_upsert(encode: &Encode) -> String {
    format!(
        r#"Hint: For FORMAT UPSERT ENCODE {encode:}, INCLUDE KEY must be specified and the key column must be used as primary key.
example:
    CREATE TABLE <table_name> ( PRIMARY KEY ([rw_key | <key_name>]) )
    INCLUDE KEY [AS <key_name>]
    WITH (...)
    FORMAT UPSERT ENCODE {encode:} (...)
"#
    )
}

/// Bind column from source. Add key column to table columns if necessary.
/// Return `pk_names`.
pub(crate) async fn bind_source_pk(
    format_encode: &FormatEncodeOptions,
    source_info: &StreamSourceInfo,
    columns: &mut [ColumnCatalog],
    sql_defined_pk_names: Vec<String>,
    with_properties: &WithOptionsSecResolved,
) -> Result<Vec<String>> {
    let sql_defined_pk = !sql_defined_pk_names.is_empty();
    let include_key_column_name: Option<String> = {
        // iter columns to check if contains additional columns from key part
        // return the key column names if exists
        columns.iter().find_map(|catalog| {
            if matches!(
                catalog.column_desc.additional_column.column_type,
                Some(AdditionalColumnType::Key(_))
            ) {
                Some(catalog.name().to_string())
            } else {
                None
            }
        })
    };
    let additional_column_names = columns
        .iter()
        .filter_map(|col| {
            if col.column_desc.additional_column.column_type.is_some() {
                Some(col.name().to_string())
            } else {
                None
            }
        })
        .collect_vec();

    let res = match (&format_encode.format, &format_encode.row_encode) {
        (Format::Native, Encode::Native) | (Format::None, Encode::None) | (Format::Plain, _) => {
            sql_defined_pk_names
        }

        // For all Upsert formats, we only accept one and only key column as primary key.
        // Additional KEY columns must be set in this case and must be primary key.
        (
            Format::Upsert,
            encode @ Encode::Json | encode @ Encode::Avro | encode @ Encode::Protobuf,
        ) => {
            if let Some(ref key_column_name) = include_key_column_name
                && sql_defined_pk
            {
                // pk is set. check if it's valid

                // the column name have been converted to real value in `handle_addition_columns`
                // so we don't ignore ascii case here
                if sql_defined_pk_names.len() != 1
                    || !key_column_name.eq(sql_defined_pk_names[0].as_str())
                {
                    return Err(RwError::from(ProtocolError(format!(
                        "Only \"{}\" can be used as primary key\n\n{}",
                        key_column_name,
                        hint_upsert(encode)
                    ))));
                }
                sql_defined_pk_names
            } else {
                // pk not set, or even key not included
                return if let Some(include_key_column_name) = include_key_column_name {
                    Err(RwError::from(ProtocolError(format!(
                        "Primary key must be specified to {}\n\n{}",
                        include_key_column_name,
                        hint_upsert(encode)
                    ))))
                } else {
                    Err(RwError::from(ProtocolError(format!(
                        "INCLUDE KEY clause not set\n\n{}",
                        hint_upsert(encode)
                    ))))
                };
            }
        }

        (Format::Debezium, Encode::Json) => {
            if !additional_column_names.is_empty() {
                return Err(RwError::from(ProtocolError(format!(
                    "FORMAT DEBEZIUM forbids additional columns, but got {:?}",
                    additional_column_names
                ))));
            }
            if !sql_defined_pk {
                return Err(RwError::from(ProtocolError(
                    "Primary key must be specified when creating source with FORMAT DEBEZIUM."
                        .to_string(),
                )));
            }
            sql_defined_pk_names
        }
        (Format::Debezium, Encode::Avro) => {
            if !additional_column_names.is_empty() {
                return Err(RwError::from(ProtocolError(format!(
                    "FORMAT DEBEZIUM forbids additional columns, but got {:?}",
                    additional_column_names
                ))));
            }
            if sql_defined_pk {
                sql_defined_pk_names
            } else {
                let pk_names =
                    extract_debezium_avro_table_pk_columns(source_info, with_properties).await?;
                // extract pk(s) from schema registry
                for pk_name in &pk_names {
                    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
            }
        }
        (Format::DebeziumMongo, Encode::Json) => {
            if sql_defined_pk {
                sql_defined_pk_names
            } else {
                vec!["_id".to_string()]
            }
        }

        (Format::Maxwell, Encode::Json) => {
            if !additional_column_names.is_empty() {
                return Err(RwError::from(ProtocolError(format!(
                    "FORMAT MAXWELL forbids additional columns, but got {:?}",
                    additional_column_names
                ))));
            }
            if !sql_defined_pk {
                return Err(RwError::from(ProtocolError(
    "Primary key must be specified when creating source with FORMAT MAXWELL ENCODE JSON."
    .to_string(),
    )));
            }
            sql_defined_pk_names
        }

        (Format::Canal, Encode::Json) => {
            if !additional_column_names.is_empty() {
                return Err(RwError::from(ProtocolError(format!(
                    "FORMAT CANAL forbids additional columns, but got {:?}",
                    additional_column_names
                ))));
            }
            if !sql_defined_pk {
                return Err(RwError::from(ProtocolError(
    "Primary key must be specified when creating source with FORMAT CANAL ENCODE JSON."
    .to_string(),
    )));
            }
            sql_defined_pk_names
        }
        (format, encoding) => {
            return Err(RwError::from(ProtocolError(format!(
                "Unknown combination {:?} {:?}",
                format, encoding
            ))));
        }
    };
    Ok(res)
}

// Add a hidden column `_rw_kafka_timestamp` to each message from Kafka source.
fn check_and_add_timestamp_column(with_properties: &WithOptions, columns: &mut Vec<ColumnCatalog>) {
    if with_properties.is_kafka_connector() {
        if columns.iter().any(|col| {
            matches!(
                col.column_desc.additional_column.column_type,
                Some(AdditionalColumnType::Timestamp(_))
            )
        }) {
            // already has timestamp column, no need to add a new one
            return;
        }

        // add a hidden column `_rw_kafka_timestamp` to each message from Kafka source
        let col = build_additional_column_desc(
            ColumnId::placeholder(),
            KAFKA_CONNECTOR,
            "timestamp",
            Some(KAFKA_TIMESTAMP_COLUMN_NAME.to_string()),
            None,
            None,
            true,
            false,
        )
        .unwrap();
        columns.push(ColumnCatalog::hidden(col));
    }
}

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)?;

    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, Encode::Protobuf],
                    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],
                ),
                OPENDAL_S3_CONNECTOR => hashmap!(
                    Format::Plain => vec![Encode::Csv, Encode::Json, Encode::Parquet],
                ),
                GCS_CONNECTOR => hashmap!(
                    Format::Plain => vec![Encode::Csv, Encode::Json, Encode::Parquet],
                ),
                AZBLOB_CONNECTOR => hashmap!(
                    Format::Plain => vec![Encode::Csv, Encode::Json, Encode::Parquet],
                ),
                POSIX_FS_CONNECTOR => hashmap!(
                    Format::Plain => vec![Encode::Csv],
                ),
                MYSQL_CDC_CONNECTOR => hashmap!(
                    Format::Debezium => vec![Encode::Json],
                    // support source stream job
                    Format::Plain => vec![Encode::Json],
                ),
                POSTGRES_CDC_CONNECTOR => hashmap!(
                    Format::Debezium => vec![Encode::Json],
                    // support source stream job
                    Format::Plain => vec![Encode::Json],
                ),
                CITUS_CDC_CONNECTOR => hashmap!(
                    Format::Debezium => vec![Encode::Json],
                ),
                MONGODB_CDC_CONNECTOR => hashmap!(
                    Format::DebeziumMongo => vec![Encode::Json],
                ),
                NATS_CONNECTOR => hashmap!(
                    Format::Plain => vec![Encode::Json, Encode::Protobuf, Encode::Bytes],
                ),
                MQTT_CONNECTOR => hashmap!(
                    Format::Plain => vec![Encode::Json, Encode::Bytes],
                ),
                TEST_CONNECTOR => hashmap!(
                    Format::Plain => vec![Encode::Json],
                ),
                ICEBERG_CONNECTOR => hashmap!(
                    Format::None => vec![Encode::None],
                ),
                SQL_SERVER_CDC_CONNECTOR => hashmap!(
                    Format::Debezium => vec![Encode::Json],
                    // support source stream job
                    Format::Plain => vec![Encode::Json],
                ),
        ))
    });

pub fn validate_license(connector: &str) -> Result<()> {
    if connector == SQL_SERVER_CDC_CONNECTOR {
        Feature::SqlServerCdcSource
            .check_available()
            .map_err(|e| anyhow::anyhow!(e))?;
    }
    Ok(())
}

pub fn validate_compatibility(
    format_encode: &FormatEncodeOptions,
    props: &mut BTreeMap<String, String>,
) -> Result<()> {
    let mut connector = props
        .get_connector()
        .ok_or_else(|| RwError::from(ProtocolError("missing field 'connector'".to_string())))?;

    if connector == OPENDAL_S3_CONNECTOR {
        // reject s3_v2 creation
        return Err(RwError::from(Deprecated(
            OPENDAL_S3_CONNECTOR.to_string(),
            S3_CONNECTOR.to_string(),
        )));
    }
    if connector == S3_CONNECTOR {
        // S3 connector is deprecated, use OPENDAL_S3_CONNECTOR instead
        // do s3 -> s3_v2 migration
        let entry = props.get_mut(UPSTREAM_SOURCE_KEY).unwrap();
        *entry = OPENDAL_S3_CONNECTOR.to_string();
        connector = OPENDAL_S3_CONNECTOR.to_string();
    }

    let compatible_formats = CONNECTORS_COMPATIBLE_FORMATS
        .get(&connector)
        .ok_or_else(|| {
            RwError::from(ProtocolError(format!(
                "connector {:?} is not supported, accept {:?}",
                connector,
                CONNECTORS_COMPATIBLE_FORMATS.keys()
            )))
        })?;

    validate_license(&connector)?;
    if connector != KAFKA_CONNECTOR {
        let res = match (&format_encode.format, &format_encode.row_encode) {
            (Format::Plain, Encode::Protobuf) | (Format::Plain, Encode::Avro) => {
                let mut options = WithOptions::try_from(format_encode.row_options())?;
                let (_, use_schema_registry) = get_schema_location(options.inner_mut())?;
                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(&format_encode.format)
        .ok_or_else(|| {
            RwError::from(ProtocolError(format!(
                "connector {} does not support format {:?}",
                connector, format_encode.format
            )))
        })?;
    if !compatible_encodes.contains(&format_encode.row_encode) {
        return Err(RwError::from(ProtocolError(format!(
            "connector {} does not support format {:?} with encode {:?}",
            connector, format_encode.format, format_encode.row_encode
        ))));
    }

    if connector == POSTGRES_CDC_CONNECTOR || connector == CITUS_CDC_CONNECTOR {
        match props.get("slot.name") {
            None => {
                // Build a random slot name with UUID
                // e.g. "rw_cdc_f9a3567e6dd54bf5900444c8b1c03815"
                let uuid = uuid::Uuid::new_v4();
                props.insert("slot.name".into(), format!("rw_cdc_{}", uuid.simple()));
            }
            Some(slot_name) => {
                // please refer to
                // - https://github.com/debezium/debezium/blob/97956ce25b7612e3413d363658661896b7d2e0a2/debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresConnectorConfig.java#L1179
                // - https://doxygen.postgresql.org/slot_8c.html#afac399f07320b9adfd2c599cf822aaa3
                if !slot_name
                    .chars()
                    .all(|c| c.is_ascii_lowercase() || c.is_ascii_digit() || c == '_')
                    || slot_name.len() > 63
                {
                    return Err(RwError::from(ProtocolError(format!(
                        "Invalid replication slot name: {:?}. Valid replication slot name must contain only digits, lowercase characters and underscores with length <= 63",
                        slot_name
                    ))));
                }
            }
        }

        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());
        }
    }

    if connector == SQL_SERVER_CDC_CONNECTOR && !props.contains_key("schema.name") {
        // Default schema name is "dbo"
        props.insert("schema.name".into(), "dbo".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.
///
/// One should only call this function after all properties of all columns are resolved, like
/// generated column descriptors.
pub(super) async fn check_format_encode(
    props: &WithOptionsSecResolved,
    row_id_index: Option<usize>,
    columns: &[ColumnCatalog],
) -> Result<()> {
    let Some(connector) = props.get_connector() else {
        return Ok(());
    };

    if connector == NEXMARK_CONNECTOR {
        check_nexmark_schema(props, row_id_index, columns)
    } else if connector == ICEBERG_CONNECTOR {
        Ok(check_iceberg_source(props, columns)
            .await
            .map_err(|err| ProtocolError(err.to_report_string()))?)
    } else {
        Ok(())
    }
}

pub(super) fn check_nexmark_schema(
    props: &WithOptionsSecResolved,
    row_id_index: Option<usize>,
    columns: &[ColumnCatalog],
) -> Result<()> {
    let table_type = props
        .get("nexmark.table.type")
        .map(|t| t.to_ascii_lowercase());

    let event_type = match table_type.as_deref() {
        None => None,
        Some("bid") => Some(EventType::Bid),
        Some("auction") => Some(EventType::Auction),
        Some("person") => Some(EventType::Person),
        Some(t) => {
            return Err(RwError::from(ProtocolError(format!(
                "unsupported table type for nexmark source: {}",
                t
            ))))
        }
    };

    // Ignore the generated columns and map the index of row_id column.
    let user_defined_columns = columns.iter().filter(|c| !c.is_generated());
    let row_id_index = if let Some(index) = row_id_index {
        let col_id = columns[index].column_id();
        user_defined_columns
            .clone()
            .position(|c| c.column_id() == col_id)
            .unwrap()
            .into()
    } else {
        None
    };

    let expected = get_event_data_types_with_names(event_type, row_id_index);
    let user_defined = user_defined_columns
        .map(|c| {
            (
                c.column_desc.name.to_ascii_lowercase(),
                c.column_desc.data_type.to_owned(),
            )
        })
        .collect_vec();

    if expected != user_defined {
        let cmp = pretty_assertions::Comparison::new(&expected, &user_defined);
        return Err(RwError::from(ProtocolError(format!(
            "The schema of the nexmark source must specify all columns in order:\n{cmp}",
        ))));
    }
    Ok(())
}

pub async fn extract_iceberg_columns(
    with_properties: &WithOptionsSecResolved,
) -> anyhow::Result<Vec<ColumnCatalog>> {
    let props = ConnectorProperties::extract(with_properties.clone(), true)?;
    if let ConnectorProperties::Iceberg(properties) = props {
        let table = properties.load_table_v2().await?;
        let iceberg_schema: arrow_schema_iceberg::Schema =
            iceberg::arrow::schema_to_arrow_schema(table.metadata().current_schema())?;

        let columns = iceberg_schema
            .fields()
            .iter()
            .enumerate()
            .map(|(i, field)| {
                let column_desc = ColumnDesc::named(
                    field.name(),
                    ColumnId::new((i + 1).try_into().unwrap()),
                    IcebergArrowConvert.type_from_field(field).unwrap(),
                );
                ColumnCatalog {
                    column_desc,
                    is_hidden: false,
                }
            })
            .collect();

        Ok(columns)
    } else {
        Err(anyhow!(format!(
            "Invalid properties for iceberg source: {:?}",
            props
        )))
    }
}

pub async fn check_iceberg_source(
    props: &WithOptionsSecResolved,
    columns: &[ColumnCatalog],
) -> anyhow::Result<()> {
    let props = ConnectorProperties::extract(props.clone(), true)?;
    let ConnectorProperties::Iceberg(properties) = props else {
        return Err(anyhow!(format!(
            "Invalid properties for iceberg source: {:?}",
            props
        )));
    };

    let schema = Schema {
        fields: columns
            .iter()
            .cloned()
            .map(|c| c.column_desc.into())
            .collect(),
    };

    let table = properties.load_table_v2().await?;

    let iceberg_schema = iceberg::arrow::schema_to_arrow_schema(table.metadata().current_schema())?;

    for f1 in schema.fields() {
        if !iceberg_schema.fields.iter().any(|f2| f2.name() == &f1.name) {
            return Err(anyhow::anyhow!(format!(
                "Column {} not found in iceberg table",
                f1.name
            )));
        }
    }

    let new_iceberg_field = iceberg_schema
        .fields
        .iter()
        .filter(|f1| schema.fields.iter().any(|f2| f1.name() == &f2.name))
        .cloned()
        .collect::<Vec<_>>();
    let new_iceberg_schema = arrow_schema_iceberg::Schema::new(new_iceberg_field);

    risingwave_connector::sink::iceberg::try_matches_arrow_schema(&schema, &new_iceberg_schema)?;

    Ok(())
}

pub fn bind_connector_props(
    handler_args: &HandlerArgs,
    format_encode: &FormatEncodeOptions,
    is_create_source: bool,
) -> Result<WithOptions> {
    let mut with_properties = handler_args.with_options.clone().into_connector_props();
    validate_compatibility(format_encode, &mut with_properties)?;
    let create_cdc_source_job = with_properties.is_shareable_cdc_connector();

    if !is_create_source && with_properties.is_shareable_only_cdc_connector() {
        return Err(RwError::from(ProtocolError(format!(
            "connector {} does not support `CREATE TABLE`, please use `CREATE SOURCE` instead",
            with_properties.get_connector().unwrap(),
        ))));
    }
    if is_create_source && create_cdc_source_job {
        if let Some(value) = with_properties.get(CDC_AUTO_SCHEMA_CHANGE_KEY)
            && value
                .parse::<bool>()
                .map_err(|_| anyhow!("invalid value of '{}' option", CDC_AUTO_SCHEMA_CHANGE_KEY))?
        {
            Feature::CdcAutoSchemaChange
                .check_available()
                .map_err(|e| anyhow::anyhow!(e))?;
        }

        // set connector to backfill mode
        with_properties.insert(CDC_SNAPSHOT_MODE_KEY.into(), CDC_SNAPSHOT_BACKFILL.into());
        // enable cdc sharing mode, which will capture all tables in the given `database.name`
        with_properties.insert(CDC_SHARING_MODE_KEY.into(), "true".into());
        // enable transactional cdc
        if with_properties.enable_transaction_metadata() {
            with_properties.insert(CDC_TRANSACTIONAL_KEY.into(), "true".into());
        }
        with_properties.insert(
            CDC_WAIT_FOR_STREAMING_START_TIMEOUT.into(),
            handler_args
                .session
                .config()
                .cdc_source_wait_streaming_start_timeout()
                .to_string(),
        );
    }
    if with_properties.is_mysql_cdc_connector() {
        // Generate a random server id for mysql cdc source if needed
        // `server.id` (in the range from 1 to 2^32 - 1). This value MUST be unique across whole replication
        // group (that is, different from any other server id being used by any master or slave)
        with_properties
            .entry("server.id".to_string())
            .or_insert(rand::thread_rng().gen_range(1..u32::MAX).to_string());
    }
    Ok(with_properties)
}

#[allow(clippy::too_many_arguments)]
pub async fn bind_create_source_or_table_with_connector(
    handler_args: HandlerArgs,
    full_name: ObjectName,
    format_encode: FormatEncodeOptions,
    with_properties: WithOptions,
    sql_columns_defs: &[ColumnDef],
    constraints: Vec<TableConstraint>,
    wildcard_idx: Option<usize>,
    source_watermarks: Vec<SourceWatermark>,
    columns_from_resolve_source: Option<Vec<ColumnCatalog>>,
    source_info: StreamSourceInfo,
    include_column_options: IncludeOption,
    col_id_gen: &mut ColumnIdGenerator,
    // `true` for "create source", `false` for "create table with connector"
    is_create_source: bool,
    source_rate_limit: Option<u32>,
) -> Result<(SourceCatalog, DatabaseId, SchemaId)> {
    let session = &handler_args.session;
    let db_name: &str = session.database();
    let (schema_name, source_name) = Binder::resolve_schema_qualified_name(db_name, full_name)?;
    let (database_id, schema_id) =
        session.get_database_and_schema_id_for_create(schema_name.clone())?;

    if !is_create_source && with_properties.is_iceberg_connector() {
        return Err(ErrorCode::BindError(
            "can't CREATE TABLE with iceberg connector\n\nHint: use CREATE SOURCE instead"
                .to_string(),
        )
        .into());
    }
    if is_create_source {
        match format_encode.format {
            Format::Upsert => {
                return Err(ErrorCode::BindError(format!(
                    "can't CREATE SOURCE with FORMAT UPSERT\n\nHint: use CREATE TABLE instead\n\n{}",
                    hint_upsert(&format_encode.row_encode)
                ))
                .into());
            }
            _ => {
                // TODO: enhance error message for other formats
            }
        }
    }

    let sql_pk_names = bind_sql_pk_names(sql_columns_defs, bind_table_constraints(&constraints)?)?;

    let columns_from_sql = bind_sql_columns(sql_columns_defs)?;

    let mut columns = bind_all_columns(
        &format_encode,
        columns_from_resolve_source,
        columns_from_sql,
        sql_columns_defs,
        wildcard_idx,
    )?;

    // add additional columns before bind pk, because `format upsert` requires the key column
    handle_addition_columns(
        Some(&format_encode),
        &with_properties,
        include_column_options,
        &mut columns,
        false,
    )?;

    if columns.is_empty() {
        return Err(RwError::from(ProtocolError(
            "Schema definition is required, either from SQL or schema registry.".to_string(),
        )));
    }

    // compatible with the behavior that add a hidden column `_rw_kafka_timestamp` to each message from Kafka source
    if is_create_source {
        // must behind `handle_addition_columns`
        check_and_add_timestamp_column(&with_properties, &mut columns);
    }

    // resolve privatelink connection for Kafka
    let mut with_properties = with_properties;
    resolve_privatelink_in_with_option(&mut with_properties)?;

    let with_properties = resolve_secret_ref_in_with_options(with_properties, session)?;

    let pk_names = bind_source_pk(
        &format_encode,
        &source_info,
        &mut columns,
        sql_pk_names,
        &with_properties,
    )
    .await?;

    if is_create_source && !pk_names.is_empty() {
        return Err(ErrorCode::InvalidInputSyntax(
            "Source does not support PRIMARY KEY constraint, please use \"CREATE TABLE\" instead"
                .to_owned(),
        )
        .into());
    }

    // XXX: why do we use col_id_gen here? It doesn't seem to be very necessary.
    // XXX: should we also chenge the col id for struct fields?
    for c in &mut columns {
        c.column_desc.column_id = col_id_gen.generate(c.name())
    }
    debug_assert_column_ids_distinct(&columns);

    let must_need_pk = if is_create_source {
        with_properties.connector_need_pk()
    } else {
        // For those connectors that do not need generate a `row_id`` column in the source schema such as iceberg.
        // But in such case, we can not create mv or table on the source because there is not a pk.
        assert!(with_properties.connector_need_pk());

        true
    };

    let (mut columns, pk_col_ids, row_id_index) =
        bind_pk_and_row_id_on_relation(columns, pk_names, must_need_pk)?;

    let watermark_descs =
        bind_source_watermark(session, source_name.clone(), source_watermarks, &columns)?;
    // TODO(yuhao): allow multiple watermark on source.
    assert!(watermark_descs.len() <= 1);

    bind_sql_column_constraints(
        session,
        source_name.clone(),
        &mut columns,
        // TODO(st1page): pass the ref
        sql_columns_defs.to_vec(),
        &pk_col_ids,
    )?;
    check_format_encode(&with_properties, row_id_index, &columns).await?;

    let definition = handler_args.normalized_sql.clone();

    let associated_table_id = if is_create_source {
        None
    } else {
        Some(TableId::placeholder())
    };
    let source = SourceCatalog {
        id: TableId::placeholder().table_id,
        name: source_name,
        columns,
        pk_col_ids,
        append_only: row_id_index.is_some(),
        owner: session.user_id(),
        info: source_info,
        row_id_index,
        with_properties,
        watermark_descs,
        associated_table_id,
        definition,
        connection_id: None, // deprecated: private link connection id
        created_at_epoch: None,
        initialized_at_epoch: None,
        version: INITIAL_SOURCE_VERSION_ID,
        created_at_cluster_version: None,
        initialized_at_cluster_version: None,
        rate_limit: source_rate_limit,
    };
    Ok((source, database_id, schema_id))
}

pub async fn handle_create_source(
    mut handler_args: HandlerArgs,
    stmt: CreateSourceStatement,
) -> Result<RwPgResponse> {
    let session = handler_args.session.clone();
    let overwrite_options = OverwriteOptions::new(&mut handler_args);

    if let Either::Right(resp) = session.check_relation_name_duplicated(
        stmt.source_name.clone(),
        StatementType::CREATE_SOURCE,
        stmt.if_not_exists,
    )? {
        return Ok(resp);
    }

    if handler_args.with_options.is_empty() {
        return Err(RwError::from(InvalidInputSyntax(
            "missing WITH clause".to_string(),
        )));
    }

    let format_encode = stmt.format_encode.into_v2_with_warning();
    let with_properties = bind_connector_props(&handler_args, &format_encode, true)?;

    let create_cdc_source_job = with_properties.is_shareable_cdc_connector();
    let is_shared = create_cdc_source_job
        || (with_properties.is_shareable_non_cdc_connector()
            && session
                .env()
                .streaming_config()
                .developer
                .enable_shared_source
            && session.config().streaming_use_shared_source());

    let (columns_from_resolve_source, mut source_info) = if create_cdc_source_job {
        bind_columns_from_source_for_cdc(&session, &format_encode)?
    } else {
        bind_columns_from_source(&session, &format_encode, Either::Left(&with_properties)).await?
    };
    if 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_cdc_source_job;
    }
    let mut col_id_gen = ColumnIdGenerator::new_initial();

    let (source_catalog, database_id, schema_id) = bind_create_source_or_table_with_connector(
        handler_args.clone(),
        stmt.source_name,
        format_encode,
        with_properties,
        &stmt.columns,
        stmt.constraints,
        stmt.wildcard_idx,
        stmt.source_watermarks,
        columns_from_resolve_source,
        source_info,
        stmt.include_column_options,
        &mut col_id_gen,
        true,
        overwrite_options.source_rate_limit,
    )
    .await?;

    // If it is a temporary source, put it into SessionImpl.
    if stmt.temporary {
        if session.get_temporary_source(&source_catalog.name).is_some() {
            return Err(CatalogError::Duplicated("source", source_catalog.name.clone()).into());
        }
        session.create_temporary_source(source_catalog);
        return Ok(PgResponse::empty_result(StatementType::CREATE_SOURCE));
    }

    let source = source_catalog.to_prost(schema_id, database_id);

    let catalog_writer = session.catalog_writer()?;

    if is_shared {
        let graph = {
            let context = OptimizerContext::from_handler_args(handler_args);
            let source_node = LogicalSource::with_catalog(
                Rc::new(source_catalog),
                SourceNodeKind::CreateSharedSource,
                context.into(),
                None,
            )?;

            let stream_plan = source_node.to_stream(&mut ToStreamContext::new(false))?;
            build_graph(stream_plan)?
        };
        catalog_writer.create_source(source, Some(graph)).await?;
    } else {
        // For other sources we don't create a streaming job
        catalog_writer.create_source(source, None).await?;
    }

    Ok(PgResponse::empty_result(StatementType::CREATE_SOURCE))
}

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,
    }
}

#[cfg(test)]
pub mod tests {
    use std::collections::HashMap;
    use std::sync::Arc;

    use risingwave_common::catalog::{
        CDC_SOURCE_COLUMN_NUM, DEFAULT_DATABASE_NAME, DEFAULT_SCHEMA_NAME, OFFSET_COLUMN_NAME,
        ROWID_PREFIX, TABLE_NAME_COLUMN_NAME,
    };
    use risingwave_common::types::DataType;

    use crate::catalog::root_catalog::SchemaPath;
    use crate::catalog::source_catalog::SourceCatalog;
    use crate::handler::create_source::debezium_cdc_source_schema;
    use crate::test_utils::{create_proto_file, LocalFrontend, PROTO_FILE_DATA};

    const GET_COLUMN_FROM_CATALOG: fn(&Arc<SourceCatalog>) -> HashMap<&str, DataType> =
        |catalog: &Arc<SourceCatalog>| -> HashMap<&str, DataType> {
            catalog
                .columns
                .iter()
                .map(|col| (col.name(), col.data_type().clone()))
                .collect::<HashMap<&str, DataType>>()
        };

    #[tokio::test]
    async fn test_create_source_handler() {
        let proto_file = create_proto_file(PROTO_FILE_DATA);
        let sql = format!(
            r#"CREATE SOURCE t
    WITH (connector = 'kinesis')
    FORMAT PLAIN ENCODE PROTOBUF (message = '.test.TestRecord', schema.location = 'file://{}')"#,
            proto_file.path().to_str().unwrap()
        );
        let frontend = LocalFrontend::new(Default::default()).await;
        frontend.run_sql(sql).await.unwrap();

        let session = frontend.session_ref();
        let catalog_reader = session.env().catalog_reader().read_guard();
        let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);

        // Check source exists.
        let (source, _) = catalog_reader
            .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
            .unwrap();
        assert_eq!(source.name, "t");

        let columns = GET_COLUMN_FROM_CATALOG(source);

        let city_type = DataType::new_struct(
            vec![DataType::Varchar, DataType::Varchar],
            vec!["address".to_string(), "zipcode".to_string()],
        );
        let expected_columns = maplit::hashmap! {
            ROWID_PREFIX => DataType::Serial,
            "id" => DataType::Int32,
            "zipcode" => DataType::Int64,
            "rate" => DataType::Float32,
            "country" => DataType::new_struct(
                vec![DataType::Varchar,city_type,DataType::Varchar],
                vec!["address".to_string(), "city".to_string(), "zipcode".to_string()],
            ),
        };
        assert_eq!(columns, expected_columns);
    }

    #[tokio::test]
    async fn test_duplicate_props_options() {
        let proto_file = create_proto_file(PROTO_FILE_DATA);
        let sql = format!(
            r#"CREATE SOURCE t
    WITH (
        connector = 'kinesis',
        aws.region='user_test_topic',
        endpoint='172.10.1.1:9090,172.10.1.2:9090',
        aws.credentials.access_key_id = 'your_access_key_1',
        aws.credentials.secret_access_key = 'your_secret_key_1'
    )
    FORMAT PLAIN ENCODE PROTOBUF (
        message = '.test.TestRecord',
        aws.credentials.access_key_id = 'your_access_key_2',
        aws.credentials.secret_access_key = 'your_secret_key_2',
        schema.location = 'file://{}',
    )"#,
            proto_file.path().to_str().unwrap()
        );
        let frontend = LocalFrontend::new(Default::default()).await;
        frontend.run_sql(sql).await.unwrap();

        let session = frontend.session_ref();
        let catalog_reader = session.env().catalog_reader().read_guard();
        let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);

        // Check source exists.
        let (source, _) = catalog_reader
            .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
            .unwrap();
        assert_eq!(source.name, "t");

        // AwsAuth params exist in options.
        assert_eq!(
            source
                .info
                .format_encode_options
                .get("aws.credentials.access_key_id")
                .unwrap(),
            "your_access_key_2"
        );
        assert_eq!(
            source
                .info
                .format_encode_options
                .get("aws.credentials.secret_access_key")
                .unwrap(),
            "your_secret_key_2"
        );

        // AwsAuth params exist in props.
        assert_eq!(
            source
                .with_properties
                .get("aws.credentials.access_key_id")
                .unwrap(),
            "your_access_key_1"
        );
        assert_eq!(
            source
                .with_properties
                .get("aws.credentials.secret_access_key")
                .unwrap(),
            "your_secret_key_1"
        );

        // Options are not merged into props.
        assert!(!source.with_properties.contains_key("schema.location"));
    }

    #[tokio::test]
    async fn test_multi_table_cdc_create_source_handler() {
        let sql =
            "CREATE SOURCE t2 WITH (connector = 'mysql-cdc') FORMAT PLAIN ENCODE JSON".to_string();
        let frontend = LocalFrontend::new(Default::default()).await;
        let session = frontend.session_ref();

        frontend
            .run_sql_with_session(session.clone(), sql)
            .await
            .unwrap();
        let catalog_reader = session.env().catalog_reader().read_guard();
        let schema_path = SchemaPath::Name(DEFAULT_SCHEMA_NAME);

        // Check source exists.
        let (source, _) = catalog_reader
            .get_source_by_name(DEFAULT_DATABASE_NAME, schema_path, "t2")
            .unwrap();
        assert_eq!(source.name, "t2");

        let columns = source
            .columns
            .iter()
            .map(|col| (col.name(), col.data_type().clone()))
            .collect::<HashMap<&str, DataType>>();

        let expected_columns = maplit::hashmap! {
            ROWID_PREFIX => DataType::Serial,
            "payload" => DataType::Jsonb,
            OFFSET_COLUMN_NAME => DataType::Varchar,
            TABLE_NAME_COLUMN_NAME => DataType::Varchar,
        };
        assert_eq!(columns, expected_columns);
    }

    #[tokio::test]
    async fn test_cdc_source_job_schema() {
        let columns = debezium_cdc_source_schema();
        // make sure it doesn't broken by future PRs
        assert_eq!(CDC_SOURCE_COLUMN_NUM, columns.len() as u32);
    }

    #[tokio::test]
    async fn test_source_addition_columns() {
        // test derive include column for format plain
        let sql =
            "CREATE SOURCE s (v1 int) include key as _rw_kafka_key with (connector = 'kafka') format plain encode json"
                .to_string();
        let frontend = LocalFrontend::new(Default::default()).await;
        frontend.run_sql(sql).await.unwrap();
        let session = frontend.session_ref();
        let catalog_reader = session.env().catalog_reader().read_guard();
        let (source, _) = catalog_reader
            .get_source_by_name(
                DEFAULT_DATABASE_NAME,
                SchemaPath::Name(DEFAULT_SCHEMA_NAME),
                "s",
            )
            .unwrap();
        assert_eq!(source.name, "s");

        let columns = GET_COLUMN_FROM_CATALOG(source);
        let expect_columns = maplit::hashmap! {
            ROWID_PREFIX => DataType::Serial,
            "v1" => DataType::Int32,
            "_rw_kafka_key" => DataType::Bytea,
            // todo: kafka connector will automatically derive the column
            // will change to a required field in the include clause
            "_rw_kafka_timestamp" => DataType::Timestamptz,
        };
        assert_eq!(columns, expect_columns);

        let sql =
            "CREATE SOURCE s3 (v1 int) include timestamp 'header1' as header_col with (connector = 'kafka') format plain encode json"
                .to_string();
        match frontend.run_sql(sql).await {
            Err(e) => {
                assert_eq!(
                    e.to_string(),
                    "Protocol error: Only header column can have inner field, but got \"timestamp\""
                )
            }
            _ => unreachable!(),
        }
    }
}