risingwave_frontend/handler/
create_table.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
1997
1998
1999
2000
2001
2002
2003
2004
2005
2006
2007
2008
2009
2010
2011
2012
2013
2014
2015
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026
2027
2028
2029
2030
2031
2032
2033
2034
2035
2036
2037
2038
2039
2040
2041
2042
2043
2044
2045
2046
2047
2048
2049
2050
2051
2052
2053
2054
2055
2056
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093
2094
2095
2096
2097
2098
2099
2100
2101
2102
2103
2104
2105
2106
2107
2108
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183
2184
2185
2186
2187
2188
2189
2190
2191
2192
2193
2194
2195
2196
2197
2198
2199
2200
2201
2202
2203
2204
2205
2206
2207
2208
2209
2210
2211
2212
2213
2214
2215
2216
2217
2218
2219
2220
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278
2279
2280
2281
2282
2283
2284
2285
2286
2287
2288
2289
2290
2291
2292
2293
2294
2295
2296
2297
2298
2299
2300
2301
2302
2303
2304
2305
2306
2307
2308
2309
2310
2311
2312
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328
2329
2330
2331
2332
2333
2334
2335
2336
2337
2338
2339
2340
2341
2342
2343
2344
2345
2346
2347
2348
2349
2350
2351
2352
2353
2354
2355
2356
2357
2358
2359
2360
2361
2362
2363
2364
2365
2366
2367
2368
2369
2370
2371
2372
2373
2374
2375
2376
2377
2378
2379
2380
2381
2382
2383
2384
2385
2386
2387
2388
2389
2390
2391
2392
2393
2394
2395
2396
2397
2398
2399
2400
2401
2402
2403
2404
2405
2406
// Copyright 2025 RisingWave Labs
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
//     http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.

use std::collections::{BTreeMap, HashMap};
use std::rc::Rc;
use std::sync::Arc;

use anyhow::{anyhow, Context};
use clap::ValueEnum;
use either::Either;
use fixedbitset::FixedBitSet;
use itertools::Itertools;
use pgwire::pg_response::{PgResponse, StatementType};
use prost::Message as _;
use risingwave_common::catalog::{
    CdcTableDesc, ColumnCatalog, ColumnDesc, ConflictBehavior, Engine, FieldLike, TableId,
    TableVersionId, DEFAULT_SCHEMA_NAME, INITIAL_TABLE_VERSION_ID, RISINGWAVE_ICEBERG_ROW_ID,
    ROW_ID_COLUMN_NAME,
};
use risingwave_common::config::MetaBackend;
use risingwave_common::license::Feature;
use risingwave_common::session_config::sink_decouple::SinkDecouple;
use risingwave_common::system_param::reader::SystemParamsRead;
use risingwave_common::types::DataType;
use risingwave_common::util::sort_util::{ColumnOrder, OrderType};
use risingwave_common::util::value_encoding::DatumToProtoExt;
use risingwave_common::{bail, bail_not_implemented};
use risingwave_connector::jvm_runtime::JVM;
use risingwave_connector::sink::decouple_checkpoint_log_sink::COMMIT_CHECKPOINT_INTERVAL;
use risingwave_connector::source::cdc::build_cdc_table_id;
use risingwave_connector::source::cdc::external::{
    ExternalTableConfig, ExternalTableImpl, DATABASE_NAME_KEY, SCHEMA_NAME_KEY, TABLE_NAME_KEY,
};
use risingwave_connector::{source, WithOptionsSecResolved, WithPropertiesExt};
use risingwave_pb::catalog::connection::Info as ConnectionInfo;
use risingwave_pb::catalog::connection_params::ConnectionType;
use risingwave_pb::catalog::source::OptionalAssociatedTableId;
use risingwave_pb::catalog::{PbSource, PbTable, PbWebhookSourceInfo, Table, WatermarkDesc};
use risingwave_pb::ddl_service::TableJobType;
use risingwave_pb::plan_common::column_desc::GeneratedOrDefaultColumn;
use risingwave_pb::plan_common::{
    AdditionalColumn, ColumnDescVersion, DefaultColumnDesc, GeneratedColumnDesc,
};
use risingwave_pb::secret::secret_ref::PbRefAsType;
use risingwave_pb::secret::PbSecretRef;
use risingwave_pb::stream_plan::StreamFragmentGraph;
use risingwave_sqlparser::ast::{
    CdcTableInfo, ColumnDef, ColumnOption, CompatibleFormatEncode, ConnectionRefValue, CreateSink,
    CreateSinkStatement, CreateSourceStatement, DataType as AstDataType, ExplainOptions, Format,
    FormatEncodeOptions, Ident, ObjectName, OnConflict, SecretRefAsType, SourceWatermark,
    Statement, TableConstraint, WebhookSourceInfo, WithProperties,
};
use risingwave_sqlparser::parser::{IncludeOption, Parser};
use thiserror_ext::AsReport;

use super::create_source::{bind_columns_from_source, CreateSourceType, SqlColumnStrategy};
use super::{create_sink, create_source, RwPgResponse};
use crate::binder::{bind_data_type, Clause, SecureCompareContext};
use crate::catalog::root_catalog::SchemaPath;
use crate::catalog::source_catalog::SourceCatalog;
use crate::catalog::table_catalog::{TableVersion, ICEBERG_SINK_PREFIX, ICEBERG_SOURCE_PREFIX};
use crate::catalog::{check_column_name_not_reserved, ColumnId, DatabaseId, SchemaId};
use crate::error::{bail_bind_error, ErrorCode, Result, RwError};
use crate::expr::{Expr, ExprImpl, ExprRewriter};
use crate::handler::create_source::{
    bind_connector_props, bind_create_source_or_table_with_connector, bind_source_watermark,
    handle_addition_columns, UPSTREAM_SOURCE_KEY,
};
use crate::handler::util::SourceSchemaCompatExt;
use crate::handler::HandlerArgs;
use crate::optimizer::plan_node::generic::{CdcScanOptions, SourceNodeKind};
use crate::optimizer::plan_node::{LogicalCdcScan, LogicalSource};
use crate::optimizer::property::{Order, RequiredDist};
use crate::optimizer::{OptimizerContext, OptimizerContextRef, PlanRef, PlanRoot};
use crate::session::SessionImpl;
use crate::stream_fragmenter::build_graph;
use crate::utils::OverwriteOptions;
use crate::{Binder, TableCatalog, WithOptions};

/// Column ID generator for a new table or a new version of an existing table to alter.
#[derive(Debug)]
pub struct ColumnIdGenerator {
    /// Existing column names and their IDs and data types.
    ///
    /// This is used for aligning column IDs between versions (`ALTER`s). If a column already
    /// exists and the data type matches, its ID is reused. Otherwise, a new ID is generated.
    ///
    /// For a new table, this is empty.
    pub existing: HashMap<String, (ColumnId, DataType)>,

    /// The next column ID to generate, used for new columns that do not exist in `existing`.
    pub next_column_id: ColumnId,

    /// The version ID of the table to be created or altered.
    ///
    /// For a new table, this is 0. For altering an existing table, this is the **next** version ID
    /// of the `version_id` field in the original table catalog.
    pub version_id: TableVersionId,
}

impl ColumnIdGenerator {
    /// Creates a new [`ColumnIdGenerator`] for altering an existing table.
    pub fn new_alter(original: &TableCatalog) -> Self {
        let existing = original
            .columns()
            .iter()
            .map(|col| {
                (
                    col.name().to_owned(),
                    (col.column_id(), col.data_type().clone()),
                )
            })
            .collect();

        let version = original.version().expect("version field not set");

        Self {
            existing,
            next_column_id: version.next_column_id,
            version_id: version.version_id + 1,
        }
    }

    /// Creates a new [`ColumnIdGenerator`] for a new table.
    pub fn new_initial() -> Self {
        Self {
            existing: HashMap::new(),
            next_column_id: ColumnId::first_user_column(),
            version_id: INITIAL_TABLE_VERSION_ID,
        }
    }

    /// Generates a new [`ColumnId`] for a column with the given field.
    ///
    /// Returns an error if the data type of the column has been changed.
    pub fn generate(&mut self, field: impl FieldLike) -> Result<ColumnId> {
        if let Some((id, original_type)) = self.existing.get(field.name()) {
            // Intentionally not using `datatype_equals` here because we want nested types to be
            // exactly the same, **NOT** ignoring field names as they may be referenced in expressions
            // of generated columns or downstream jobs.
            // TODO: support compatible changes on types, typically for `STRUCT` types.
            //       https://github.com/risingwavelabs/risingwave/issues/19755
            if original_type == field.data_type() {
                Ok(*id)
            } else {
                bail_not_implemented!(
                    "The data type of column \"{}\" has been changed from \"{}\" to \"{}\". \
                     This is currently not supported, even if it could be a compatible change in external systems.",
                    field.name(),
                    original_type,
                    field.data_type()
                );
            }
        } else {
            let id = self.next_column_id;
            self.next_column_id = self.next_column_id.next();
            Ok(id)
        }
    }

    /// Consume this generator and return a [`TableVersion`] for the table to be created or altered.
    pub fn into_version(self) -> TableVersion {
        TableVersion {
            version_id: self.version_id,
            next_column_id: self.next_column_id,
        }
    }
}

fn ensure_column_options_supported(c: &ColumnDef) -> Result<()> {
    for option_def in &c.options {
        match option_def.option {
            ColumnOption::GeneratedColumns(_) => {}
            ColumnOption::DefaultValue(_) => {}
            ColumnOption::DefaultValueInternal { .. } => {}
            ColumnOption::Unique { is_primary: true } => {}
            _ => bail_not_implemented!("column constraints \"{}\"", option_def),
        }
    }
    Ok(())
}

/// Binds the column schemas declared in CREATE statement into `ColumnDesc`.
/// If a column is marked as `primary key`, its `ColumnId` is also returned.
/// This primary key is not combined with table constraints yet.
pub fn bind_sql_columns(
    column_defs: &[ColumnDef],
    is_for_drop_table_connector: bool,
) -> Result<Vec<ColumnCatalog>> {
    let mut columns = Vec::with_capacity(column_defs.len());

    for column in column_defs {
        ensure_column_options_supported(column)?;
        // Destruct to make sure all fields are properly handled rather than ignored.
        // Do NOT use `..` to ignore fields you do not want to deal with.
        // Reject them with a clear NotImplemented error.
        let ColumnDef {
            name,
            data_type,
            collation,
            ..
        } = column;

        let data_type = data_type
            .clone()
            .ok_or_else(|| ErrorCode::InvalidInputSyntax("data type is not specified".into()))?;
        if let Some(collation) = collation {
            // PostgreSQL will limit the datatypes that collate can work on.
            // https://www.postgresql.org/docs/16/collation.html#COLLATION-CONCEPTS
            //   > The built-in collatable data types are `text`, `varchar`, and `char`.
            //
            // But we don't support real collation, we simply ignore it here.
            if !["C", "POSIX"].contains(&collation.real_value().as_str()) {
                bail_not_implemented!(
                    "Collate collation other than `C` or `POSIX` is not implemented"
                );
            }

            match data_type {
                AstDataType::Text | AstDataType::Varchar | AstDataType::Char(_) => {}
                _ => {
                    return Err(ErrorCode::NotSupported(
                        format!("{} is not a collatable data type", data_type),
                        "The only built-in collatable data types are `varchar`, please check your type".into(),
                    ).into());
                }
            }
        }

        if !is_for_drop_table_connector {
            // additional column name may have prefix _rw
            // When converting dropping the connector from table, the additional columns are converted to normal columns and keep the original name.
            // Under this case, we loosen the check for _rw prefix.
            check_column_name_not_reserved(&name.real_value())?;
        }

        columns.push(ColumnCatalog {
            column_desc: ColumnDesc {
                data_type: bind_data_type(&data_type)?,
                column_id: ColumnId::placeholder(),
                name: name.real_value(),
                generated_or_default_column: None,
                description: None,
                additional_column: AdditionalColumn { column_type: None },
                version: ColumnDescVersion::LATEST,
                system_column: None,
            },
            is_hidden: false,
        });
    }

    Ok(columns)
}

fn check_generated_column_constraints(
    column_name: &String,
    column_id: ColumnId,
    expr: &ExprImpl,
    column_catalogs: &[ColumnCatalog],
    generated_column_names: &[String],
    pk_column_ids: &[ColumnId],
) -> Result<()> {
    let input_refs = expr.collect_input_refs(column_catalogs.len());
    for idx in input_refs.ones() {
        let referred_generated_column = &column_catalogs[idx].column_desc.name;
        if generated_column_names
            .iter()
            .any(|c| c == referred_generated_column)
        {
            return Err(ErrorCode::BindError(format!(
                "Generated can not reference another generated column. \
                But here generated column \"{}\" referenced another generated column \"{}\"",
                column_name, referred_generated_column
            ))
            .into());
        }
    }

    if pk_column_ids.contains(&column_id) && expr.is_impure() {
        return Err(ErrorCode::BindError(format!(
            "Generated columns with impure expressions should not be part of the primary key. \
            Here column \"{}\" is defined as part of the primary key.",
            column_name
        ))
        .into());
    }

    Ok(())
}

/// Binds constraints that can be only specified in column definitions,
/// currently generated columns and default columns.
pub fn bind_sql_column_constraints(
    session: &SessionImpl,
    table_name: String,
    column_catalogs: &mut [ColumnCatalog],
    columns: Vec<ColumnDef>,
    pk_column_ids: &[ColumnId],
) -> Result<()> {
    let generated_column_names = {
        let mut names = vec![];
        for column in &columns {
            for option_def in &column.options {
                if let ColumnOption::GeneratedColumns(_) = option_def.option {
                    names.push(column.name.real_value());
                    break;
                }
            }
        }
        names
    };

    let mut binder = Binder::new_for_ddl(session);
    binder.bind_columns_to_context(table_name.clone(), column_catalogs)?;

    for column in columns {
        let Some(idx) = column_catalogs
            .iter()
            .position(|c| c.name() == column.name.real_value())
        else {
            // It's possible that we don't follow the user defined columns in SQL but take the
            // ones resolved from the source, thus missing some columns. Simply ignore them.
            continue;
        };

        for option_def in column.options {
            match option_def.option {
                ColumnOption::GeneratedColumns(expr) => {
                    binder.set_clause(Some(Clause::GeneratedColumn));

                    let expr_impl = binder.bind_expr(expr).with_context(|| {
                        format!(
                            "fail to bind expression in generated column \"{}\"",
                            column.name.real_value()
                        )
                    })?;

                    check_generated_column_constraints(
                        &column.name.real_value(),
                        column_catalogs[idx].column_id(),
                        &expr_impl,
                        column_catalogs,
                        &generated_column_names,
                        pk_column_ids,
                    )?;

                    column_catalogs[idx].column_desc.generated_or_default_column = Some(
                        GeneratedOrDefaultColumn::GeneratedColumn(GeneratedColumnDesc {
                            expr: Some(expr_impl.to_expr_proto()),
                        }),
                    );
                    binder.set_clause(None);
                }
                ColumnOption::DefaultValue(expr) => {
                    let expr_impl = binder
                        .bind_expr(expr)?
                        .cast_assign(column_catalogs[idx].data_type().clone())?;

                    // Rewrite expressions to evaluate a snapshot value, used for missing values in the case of
                    // schema change.
                    //
                    // TODO: Currently we don't support impure expressions other than `now()` (like `random()`),
                    // so the rewritten expression should almost always be pure and we directly call `fold_const`
                    // here. Actually we do not require purity of the expression here since we're only to get a
                    // snapshot value.
                    let rewritten_expr_impl = session
                        .pinned_snapshot()
                        .inline_now_proc_time()
                        .rewrite_expr(expr_impl.clone());

                    if let Some(snapshot_value) = rewritten_expr_impl.try_fold_const() {
                        let snapshot_value = snapshot_value?;

                        column_catalogs[idx].column_desc.generated_or_default_column =
                            Some(GeneratedOrDefaultColumn::DefaultColumn(DefaultColumnDesc {
                                snapshot_value: Some(snapshot_value.to_protobuf()),
                                expr: Some(expr_impl.to_expr_proto()),
                                // persist the original expression
                            }));
                    } else {
                        return Err(ErrorCode::BindError(format!(
                            "Default expression used in column `{}` cannot be evaluated. \
                            Use generated columns instead if you mean to reference other columns.",
                            column.name
                        ))
                        .into());
                    }
                }
                ColumnOption::DefaultValueInternal { persisted, expr: _ } => {
                    // When a `DEFAULT INTERNAL` is used internally for schema change, the persisted value
                    // should already be set during purifcation. So if we encounter an empty value here, it
                    // means the user has specified it explicitly in the SQL statement, typically by
                    // directly copying the result of `SHOW CREATE TABLE` and executing it.
                    if persisted.is_empty() {
                        bail_bind_error!(
                            "DEFAULT INTERNAL is only used for internal purposes, \
                             please specify a concrete default value"
                        );
                    }

                    let desc = DefaultColumnDesc::decode(&*persisted)
                        .expect("failed to decode persisted `DefaultColumnDesc`");

                    column_catalogs[idx].column_desc.generated_or_default_column =
                        Some(GeneratedOrDefaultColumn::DefaultColumn(desc));
                }
                _ => {}
            }
        }
    }
    Ok(())
}

/// Currently we only support Primary key table constraint, so just return pk names if it exists
pub fn bind_table_constraints(table_constraints: &[TableConstraint]) -> Result<Vec<String>> {
    let mut pk_column_names = vec![];

    for constraint in table_constraints {
        match constraint {
            TableConstraint::Unique {
                name: _,
                columns,
                is_primary: true,
            } => {
                if !pk_column_names.is_empty() {
                    return Err(multiple_pk_definition_err());
                }
                pk_column_names = columns.iter().map(|c| c.real_value()).collect_vec();
            }
            _ => bail_not_implemented!("table constraint \"{}\"", constraint),
        }
    }
    Ok(pk_column_names)
}

pub fn bind_sql_pk_names(
    columns_defs: &[ColumnDef],
    pk_names_from_table_constraints: Vec<String>,
) -> Result<Vec<String>> {
    let mut pk_column_names = pk_names_from_table_constraints;

    for column in columns_defs {
        for option_def in &column.options {
            if let ColumnOption::Unique { is_primary: true } = option_def.option {
                if !pk_column_names.is_empty() {
                    return Err(multiple_pk_definition_err());
                }
                pk_column_names.push(column.name.real_value());
            };
        }
    }

    Ok(pk_column_names)
}

fn multiple_pk_definition_err() -> RwError {
    ErrorCode::BindError("multiple primary keys are not allowed".into()).into()
}

/// Binds primary keys defined in SQL.
///
/// It returns the columns together with `pk_column_ids`, and an optional row id column index if
/// added.
pub fn bind_pk_and_row_id_on_relation(
    mut columns: Vec<ColumnCatalog>,
    pk_names: Vec<String>,
    must_need_pk: bool,
) -> Result<(Vec<ColumnCatalog>, Vec<ColumnId>, Option<usize>)> {
    for c in &columns {
        assert!(c.column_id() != ColumnId::placeholder());
    }

    // Mapping from column name to column id.
    let name_to_id = columns
        .iter()
        .map(|c| (c.name(), c.column_id()))
        .collect::<HashMap<_, _>>();

    let mut pk_column_ids: Vec<_> = pk_names
        .iter()
        .map(|name| {
            name_to_id.get(name.as_str()).copied().ok_or_else(|| {
                ErrorCode::BindError(format!("column \"{name}\" named in key does not exist"))
            })
        })
        .try_collect()?;

    // Add `_row_id` column if `pk_column_ids` is empty and must_need_pk
    let need_row_id = pk_column_ids.is_empty() && must_need_pk;

    let row_id_index = need_row_id.then(|| {
        let column = ColumnCatalog::row_id_column();
        let index = columns.len();
        pk_column_ids = vec![column.column_id()];
        columns.push(column);
        index
    });

    if let Some(col) = columns.iter().map(|c| c.name()).duplicates().next() {
        Err(ErrorCode::InvalidInputSyntax(format!(
            "column \"{col}\" specified more than once"
        )))?;
    }

    Ok((columns, pk_column_ids, row_id_index))
}

/// `gen_create_table_plan_with_source` generates the plan for creating a table with an external
/// stream source.
#[allow(clippy::too_many_arguments)]
pub(crate) async fn gen_create_table_plan_with_source(
    mut handler_args: HandlerArgs,
    explain_options: ExplainOptions,
    table_name: ObjectName,
    column_defs: Vec<ColumnDef>,
    wildcard_idx: Option<usize>,
    constraints: Vec<TableConstraint>,
    format_encode: FormatEncodeOptions,
    source_watermarks: Vec<SourceWatermark>,
    mut col_id_gen: ColumnIdGenerator,
    include_column_options: IncludeOption,
    props: CreateTableProps,
    sql_column_strategy: SqlColumnStrategy,
) -> Result<(PlanRef, Option<PbSource>, PbTable)> {
    if props.append_only
        && format_encode.format != Format::Plain
        && format_encode.format != Format::Native
    {
        return Err(ErrorCode::BindError(format!(
            "Append only table does not support format {}.",
            format_encode.format
        ))
        .into());
    }

    let session = &handler_args.session;
    let with_properties = bind_connector_props(&handler_args, &format_encode, false)?;
    if with_properties.is_shareable_cdc_connector() {
        generated_columns_check_for_cdc_table(&column_defs)?;
    }

    let db_name: &str = &session.database();
    let (schema_name, _) = Binder::resolve_schema_qualified_name(db_name, table_name.clone())?;

    // TODO: omit this step if `sql_column_strategy` is `Follow`.
    let (columns_from_resolve_source, source_info) = bind_columns_from_source(
        session,
        &format_encode,
        Either::Left(&with_properties),
        CreateSourceType::Table,
    )
    .await?;

    let overwrite_options = OverwriteOptions::new(&mut handler_args);
    let rate_limit = overwrite_options.source_rate_limit;
    let source_catalog = bind_create_source_or_table_with_connector(
        handler_args.clone(),
        table_name,
        format_encode,
        with_properties,
        &column_defs,
        constraints,
        wildcard_idx,
        source_watermarks,
        columns_from_resolve_source,
        source_info,
        include_column_options,
        &mut col_id_gen,
        CreateSourceType::Table,
        rate_limit,
        sql_column_strategy,
    )
    .await?;

    let pb_source = source_catalog.to_prost();

    let context = OptimizerContext::new(handler_args, explain_options);

    let (plan, table) = gen_table_plan_with_source(
        context.into(),
        schema_name,
        source_catalog,
        col_id_gen.into_version(),
        props,
    )?;

    Ok((plan, Some(pb_source), table))
}

/// `gen_create_table_plan` generates the plan for creating a table without an external stream
/// source.
#[allow(clippy::too_many_arguments)]
pub(crate) fn gen_create_table_plan(
    context: OptimizerContext,
    table_name: ObjectName,
    column_defs: Vec<ColumnDef>,
    constraints: Vec<TableConstraint>,
    mut col_id_gen: ColumnIdGenerator,
    source_watermarks: Vec<SourceWatermark>,
    props: CreateTableProps,
    is_for_replace_plan: bool,
) -> Result<(PlanRef, PbTable)> {
    let mut columns = bind_sql_columns(&column_defs, is_for_replace_plan)?;
    for c in &mut columns {
        c.column_desc.column_id = col_id_gen.generate(&*c)?;
    }

    let (_, secret_refs, connection_refs) = context.with_options().clone().into_parts();
    if !secret_refs.is_empty() || !connection_refs.is_empty() {
        return Err(crate::error::ErrorCode::InvalidParameterValue("Secret reference and Connection reference are not allowed in options when creating table without external source".to_owned()).into());
    }

    gen_create_table_plan_without_source(
        context,
        table_name,
        columns,
        column_defs,
        constraints,
        source_watermarks,
        col_id_gen.into_version(),
        props,
    )
}

#[allow(clippy::too_many_arguments)]
pub(crate) fn gen_create_table_plan_without_source(
    context: OptimizerContext,
    table_name: ObjectName,
    columns: Vec<ColumnCatalog>,
    column_defs: Vec<ColumnDef>,
    constraints: Vec<TableConstraint>,
    source_watermarks: Vec<SourceWatermark>,
    version: TableVersion,
    props: CreateTableProps,
) -> Result<(PlanRef, PbTable)> {
    // XXX: Why not bind outside?
    let pk_names = bind_sql_pk_names(&column_defs, bind_table_constraints(&constraints)?)?;
    let (mut columns, pk_column_ids, row_id_index) =
        bind_pk_and_row_id_on_relation(columns, pk_names, true)?;

    let watermark_descs: Vec<WatermarkDesc> = bind_source_watermark(
        context.session_ctx(),
        table_name.real_value(),
        source_watermarks,
        &columns,
    )?;

    bind_sql_column_constraints(
        context.session_ctx(),
        table_name.real_value(),
        &mut columns,
        column_defs,
        &pk_column_ids,
    )?;
    let session = context.session_ctx().clone();

    let db_name = &session.database();
    let (schema_name, table_name) = Binder::resolve_schema_qualified_name(db_name, table_name)?;

    let info = CreateTableInfo {
        columns,
        pk_column_ids,
        row_id_index,
        watermark_descs,
        source_catalog: None,
        version,
    };

    gen_table_plan_inner(context.into(), schema_name, table_name, info, props)
}

fn gen_table_plan_with_source(
    context: OptimizerContextRef,
    schema_name: Option<String>,
    source_catalog: SourceCatalog,
    version: TableVersion,
    props: CreateTableProps,
) -> Result<(PlanRef, PbTable)> {
    let table_name = source_catalog.name.clone();

    let info = CreateTableInfo {
        columns: source_catalog.columns.clone(),
        pk_column_ids: source_catalog.pk_col_ids.clone(),
        row_id_index: source_catalog.row_id_index,
        watermark_descs: source_catalog.watermark_descs.clone(),
        source_catalog: Some(source_catalog),
        version,
    };

    gen_table_plan_inner(context, schema_name, table_name, info, props)
}

/// On-conflict behavior either from user input or existing table catalog.
#[derive(Clone, Copy)]
pub enum EitherOnConflict {
    Ast(Option<OnConflict>),
    Resolved(ConflictBehavior),
}

impl From<Option<OnConflict>> for EitherOnConflict {
    fn from(v: Option<OnConflict>) -> Self {
        Self::Ast(v)
    }
}

impl From<ConflictBehavior> for EitherOnConflict {
    fn from(v: ConflictBehavior) -> Self {
        Self::Resolved(v)
    }
}

impl EitherOnConflict {
    /// Resolves the conflict behavior based on the given information.
    pub fn to_behavior(self, append_only: bool, row_id_as_pk: bool) -> Result<ConflictBehavior> {
        let conflict_behavior = match self {
            EitherOnConflict::Ast(on_conflict) => {
                if append_only {
                    if row_id_as_pk {
                        // Primary key will be generated, no conflict check needed.
                        ConflictBehavior::NoCheck
                    } else {
                        // User defined PK on append-only table, enforce `DO NOTHING`.
                        if let Some(on_conflict) = on_conflict
                            && on_conflict != OnConflict::Nothing
                        {
                            return Err(ErrorCode::InvalidInputSyntax(
                                "When PRIMARY KEY constraint applied to an APPEND ONLY table, \
                                     the ON CONFLICT behavior must be DO NOTHING."
                                    .to_owned(),
                            )
                            .into());
                        }
                        ConflictBehavior::IgnoreConflict
                    }
                } else {
                    // Default to `UPDATE FULL` for non-append-only tables.
                    match on_conflict.unwrap_or(OnConflict::UpdateFull) {
                        OnConflict::UpdateFull => ConflictBehavior::Overwrite,
                        OnConflict::Nothing => ConflictBehavior::IgnoreConflict,
                        OnConflict::UpdateIfNotNull => ConflictBehavior::DoUpdateIfNotNull,
                    }
                }
            }
            EitherOnConflict::Resolved(b) => b,
        };

        Ok(conflict_behavior)
    }
}

/// Arguments of the functions that generate a table plan, part 1.
///
/// Compared to [`CreateTableProps`], this struct contains fields that need some work of binding
/// or resolving based on the user input.
pub struct CreateTableInfo {
    pub columns: Vec<ColumnCatalog>,
    pub pk_column_ids: Vec<ColumnId>,
    pub row_id_index: Option<usize>,
    pub watermark_descs: Vec<WatermarkDesc>,
    pub source_catalog: Option<SourceCatalog>,
    pub version: TableVersion,
}

/// Arguments of the functions that generate a table plan, part 2.
///
/// Compared to [`CreateTableInfo`], this struct contains fields that can be (relatively) simply
/// obtained from the input or the context.
pub struct CreateTableProps {
    pub definition: String,
    pub append_only: bool,
    pub on_conflict: EitherOnConflict,
    pub with_version_column: Option<String>,
    pub webhook_info: Option<PbWebhookSourceInfo>,
    pub engine: Engine,
}

#[allow(clippy::too_many_arguments)]
fn gen_table_plan_inner(
    context: OptimizerContextRef,
    schema_name: Option<String>,
    table_name: String,
    info: CreateTableInfo,
    props: CreateTableProps,
) -> Result<(PlanRef, PbTable)> {
    let CreateTableInfo {
        ref columns,
        row_id_index,
        ref watermark_descs,
        ref source_catalog,
        ..
    } = info;
    let CreateTableProps { append_only, .. } = props;

    let (database_id, schema_id) = context
        .session_ctx()
        .get_database_and_schema_id_for_create(schema_name)?;

    let session = context.session_ctx().clone();
    let retention_seconds = context.with_options().retention_seconds();

    let source_node: PlanRef = LogicalSource::new(
        source_catalog.clone().map(Rc::new),
        columns.clone(),
        row_id_index,
        SourceNodeKind::CreateTable,
        context.clone(),
        None,
    )?
    .into();

    let required_cols = FixedBitSet::with_capacity(columns.len());
    let plan_root = PlanRoot::new_with_logical_plan(
        source_node,
        RequiredDist::Any,
        Order::any(),
        required_cols,
        vec![],
    );

    if !append_only && !watermark_descs.is_empty() {
        return Err(ErrorCode::NotSupported(
            "Defining watermarks on table requires the table to be append only.".to_owned(),
            "Use the key words `APPEND ONLY`".to_owned(),
        )
        .into());
    }

    if !append_only && retention_seconds.is_some() {
        return Err(ErrorCode::NotSupported(
            "Defining retention seconds on table requires the table to be append only.".to_owned(),
            "Use the key words `APPEND ONLY`".to_owned(),
        )
        .into());
    }

    let materialize =
        plan_root.gen_table_plan(context, table_name, database_id, schema_id, info, props)?;

    let mut table = materialize.table().to_prost();

    table.owner = session.user_id();
    Ok((materialize.into(), table))
}

/// Generate stream plan for cdc table based on shared source.
/// In replace workflow, the `table_id` is the id of the table to be replaced
/// in create table workflow, the `table_id` is a placeholder will be filled in the Meta
#[allow(clippy::too_many_arguments)]
pub(crate) fn gen_create_table_plan_for_cdc_table(
    context: OptimizerContextRef,
    source: Arc<SourceCatalog>,
    external_table_name: String,
    column_defs: Vec<ColumnDef>,
    mut columns: Vec<ColumnCatalog>,
    pk_names: Vec<String>,
    cdc_with_options: WithOptionsSecResolved,
    mut col_id_gen: ColumnIdGenerator,
    on_conflict: Option<OnConflict>,
    with_version_column: Option<String>,
    include_column_options: IncludeOption,
    table_name: ObjectName,
    resolved_table_name: String, // table name without schema prefix
    database_id: DatabaseId,
    schema_id: SchemaId,
    table_id: TableId,
    engine: Engine,
) -> Result<(PlanRef, PbTable)> {
    let session = context.session_ctx().clone();

    // append additional columns to the end
    handle_addition_columns(
        None,
        &cdc_with_options,
        include_column_options,
        &mut columns,
        true,
    )?;

    for c in &mut columns {
        c.column_desc.column_id = col_id_gen.generate(&*c)?;
    }

    let (mut columns, pk_column_ids, _row_id_index) =
        bind_pk_and_row_id_on_relation(columns, pk_names, true)?;

    // NOTES: In auto schema change, default value is not provided in column definition.
    bind_sql_column_constraints(
        context.session_ctx(),
        table_name.real_value(),
        &mut columns,
        column_defs,
        &pk_column_ids,
    )?;

    let definition = context.normalized_sql().to_owned();

    let pk_column_indices = {
        let mut id_to_idx = HashMap::new();
        columns.iter().enumerate().for_each(|(idx, c)| {
            id_to_idx.insert(c.column_id(), idx);
        });
        // pk column id must exist in table columns.
        pk_column_ids
            .iter()
            .map(|c| id_to_idx.get(c).copied().unwrap())
            .collect_vec()
    };
    let table_pk = pk_column_indices
        .iter()
        .map(|idx| ColumnOrder::new(*idx, OrderType::ascending()))
        .collect();

    let (options, secret_refs) = cdc_with_options.into_parts();

    let non_generated_column_descs = columns
        .iter()
        .filter(|&c| (!c.is_generated()))
        .map(|c| c.column_desc.clone())
        .collect_vec();
    let non_generated_column_num = non_generated_column_descs.len();

    let cdc_table_desc = CdcTableDesc {
        table_id,
        source_id: source.id.into(), // id of cdc source streaming job
        external_table_name: external_table_name.clone(),
        pk: table_pk,
        columns: non_generated_column_descs,
        stream_key: pk_column_indices,
        connect_properties: options,
        secret_refs,
    };

    tracing::debug!(?cdc_table_desc, "create cdc table");

    let options = CdcScanOptions::from_with_options(context.with_options())?;

    let logical_scan = LogicalCdcScan::create(
        external_table_name.clone(),
        Rc::new(cdc_table_desc),
        context.clone(),
        options,
    );

    let scan_node: PlanRef = logical_scan.into();
    let required_cols = FixedBitSet::with_capacity(non_generated_column_num);
    let plan_root = PlanRoot::new_with_logical_plan(
        scan_node,
        RequiredDist::Any,
        Order::any(),
        required_cols,
        vec![],
    );

    let cdc_table_id = build_cdc_table_id(source.id, &external_table_name);
    let materialize = plan_root.gen_table_plan(
        context,
        resolved_table_name,
        database_id,
        schema_id,
        CreateTableInfo {
            columns,
            pk_column_ids,
            row_id_index: None,
            watermark_descs: vec![],
            source_catalog: Some((*source).clone()),
            version: col_id_gen.into_version(),
        },
        CreateTableProps {
            definition,
            append_only: false,
            on_conflict: on_conflict.into(),
            with_version_column,
            webhook_info: None,
            engine,
        },
    )?;

    let mut table = materialize.table().to_prost();
    table.owner = session.user_id();
    table.cdc_table_id = Some(cdc_table_id);
    table.dependent_relations = vec![source.id];

    Ok((materialize.into(), table))
}

fn derive_with_options_for_cdc_table(
    source_with_properties: &WithOptionsSecResolved,
    external_table_name: String,
) -> Result<WithOptionsSecResolved> {
    use source::cdc::{MYSQL_CDC_CONNECTOR, POSTGRES_CDC_CONNECTOR, SQL_SERVER_CDC_CONNECTOR};
    // we should remove the prefix from `full_table_name`
    let mut with_options = source_with_properties.clone();
    if let Some(connector) = source_with_properties.get(UPSTREAM_SOURCE_KEY) {
        match connector.as_str() {
            MYSQL_CDC_CONNECTOR => {
                // MySQL doesn't allow '.' in database name and table name, so we can split the
                // external table name by '.' to get the table name
                let (db_name, table_name) = external_table_name.split_once('.').ok_or_else(|| {
                    anyhow!("The upstream table name must contain database name prefix, e.g. 'database.table'")
                })?;
                with_options.insert(DATABASE_NAME_KEY.into(), db_name.into());
                with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
            }
            POSTGRES_CDC_CONNECTOR => {
                let (schema_name, table_name) = external_table_name
                    .split_once('.')
                    .ok_or_else(|| anyhow!("The upstream table name must contain schema name prefix, e.g. 'public.table'"))?;

                // insert 'schema.name' into connect properties
                with_options.insert(SCHEMA_NAME_KEY.into(), schema_name.into());
                with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
            }
            SQL_SERVER_CDC_CONNECTOR => {
                // SQL Server external table name is in 'databaseName.schemaName.tableName' pattern,
                // we remove the database name prefix and split the schema name and table name
                let schema_table_name = external_table_name
                    .split_once('.')
                    .ok_or_else(|| {
                        anyhow!("The upstream table name must be in 'database.schema.table' format")
                    })?
                    .1;

                let (schema_name, table_name) =
                    schema_table_name.split_once('.').ok_or_else(|| {
                        anyhow!("The table name must contain schema name prefix, e.g. 'dbo.table'")
                    })?;

                // insert 'schema.name' into connect properties
                with_options.insert(SCHEMA_NAME_KEY.into(), schema_name.into());
                with_options.insert(TABLE_NAME_KEY.into(), table_name.into());
            }
            _ => {
                return Err(RwError::from(anyhow!(
                    "connector {} is not supported for cdc table",
                    connector
                )));
            }
        };
    }
    Ok(with_options)
}

#[allow(clippy::too_many_arguments)]
pub(super) async fn handle_create_table_plan(
    handler_args: HandlerArgs,
    explain_options: ExplainOptions,
    format_encode: Option<FormatEncodeOptions>,
    cdc_table_info: Option<CdcTableInfo>,
    table_name: ObjectName,
    column_defs: Vec<ColumnDef>,
    wildcard_idx: Option<usize>,
    constraints: Vec<TableConstraint>,
    source_watermarks: Vec<SourceWatermark>,
    append_only: bool,
    on_conflict: Option<OnConflict>,
    with_version_column: Option<String>,
    include_column_options: IncludeOption,
    webhook_info: Option<WebhookSourceInfo>,
    engine: Engine,
) -> Result<(PlanRef, Option<PbSource>, PbTable, TableJobType)> {
    let col_id_gen = ColumnIdGenerator::new_initial();
    let format_encode = check_create_table_with_source(
        &handler_args.with_options,
        format_encode,
        &include_column_options,
        &cdc_table_info,
    )?;
    let webhook_info = webhook_info
        .map(|info| bind_webhook_info(&handler_args.session, &column_defs, info))
        .transpose()?;

    let props = CreateTableProps {
        definition: handler_args.normalized_sql.clone(),
        append_only,
        on_conflict: on_conflict.into(),
        with_version_column: with_version_column.clone(),
        webhook_info,
        engine,
    };

    let ((plan, source, table), job_type) = match (format_encode, cdc_table_info.as_ref()) {
        (Some(format_encode), None) => (
            gen_create_table_plan_with_source(
                handler_args,
                explain_options,
                table_name.clone(),
                column_defs,
                wildcard_idx,
                constraints,
                format_encode,
                source_watermarks,
                col_id_gen,
                include_column_options,
                props,
                SqlColumnStrategy::FollowChecked,
            )
            .await?,
            TableJobType::General,
        ),
        (None, None) => {
            let context = OptimizerContext::new(handler_args, explain_options);
            let (plan, table) = gen_create_table_plan(
                context,
                table_name.clone(),
                column_defs,
                constraints,
                col_id_gen,
                source_watermarks,
                props,
                false,
            )?;

            ((plan, None, table), TableJobType::General)
        }

        (None, Some(cdc_table)) => {
            sanity_check_for_cdc_table(
                append_only,
                &column_defs,
                &wildcard_idx,
                &constraints,
                &source_watermarks,
            )?;

            let session = &handler_args.session;
            let db_name = &session.database();
            let user_name = &session.user_name();
            let search_path = session.config().search_path();
            let (schema_name, resolved_table_name) =
                Binder::resolve_schema_qualified_name(db_name, table_name.clone())?;
            let (database_id, schema_id) =
                session.get_database_and_schema_id_for_create(schema_name.clone())?;

            // cdc table cannot be append-only
            let (format_encode, source_name) =
                Binder::resolve_schema_qualified_name(db_name, cdc_table.source_name.clone())?;

            generated_columns_check_for_cdc_table(&column_defs)?;

            let source = {
                let catalog_reader = session.env().catalog_reader().read_guard();
                let schema_path =
                    SchemaPath::new(format_encode.as_deref(), &search_path, user_name);

                let (source, _) = catalog_reader.get_source_by_name(
                    db_name,
                    schema_path,
                    source_name.as_str(),
                )?;
                source.clone()
            };
            let cdc_with_options: WithOptionsSecResolved = derive_with_options_for_cdc_table(
                &source.with_properties,
                cdc_table.external_table_name.clone(),
            )?;

            let (columns, pk_names) = match wildcard_idx {
                Some(_) => bind_cdc_table_schema_externally(cdc_with_options.clone()).await?,
                None => {
                    for column_def in &column_defs {
                        for option_def in &column_def.options {
                            if let ColumnOption::DefaultValue(_)
                            | ColumnOption::DefaultValueInternal { .. } = option_def.option
                            {
                                return Err(ErrorCode::NotSupported(
                                            "Default value for columns defined on the table created from a CDC source".into(),
                                            "Remove the default value expression in the column definitions".into(),
                                        )
                                            .into());
                            }
                        }
                    }

                    let (mut columns, pk_names) =
                        bind_cdc_table_schema(&column_defs, &constraints, false)?;
                    // read default value definition from external db
                    let (options, secret_refs) = cdc_with_options.clone().into_parts();
                    let config = ExternalTableConfig::try_from_btreemap(options, secret_refs)
                        .context("failed to extract external table config")?;

                    let table: ExternalTableImpl = ExternalTableImpl::connect(config)
                        .await
                        .context("failed to auto derive table schema")?;
                    let external_columns: HashMap<&str, &ColumnDesc> = table
                        .column_descs()
                        .iter()
                        .map(|column_desc| (column_desc.name.as_str(), column_desc))
                        .collect();

                    for col in &mut columns {
                        // Keep the default column aligned with external table.
                        // Note the generated columns have not been initialized yet. If a column is not found here, it should be a generated column.
                        if let Some(external_column_desc) = external_columns.get(col.name()) {
                            col.column_desc.generated_or_default_column =
                                external_column_desc.generated_or_default_column.clone();
                        }
                    }
                    (columns, pk_names)
                }
            };

            let context: OptimizerContextRef =
                OptimizerContext::new(handler_args, explain_options).into();
            let (plan, table) = gen_create_table_plan_for_cdc_table(
                context,
                source,
                cdc_table.external_table_name.clone(),
                column_defs,
                columns,
                pk_names,
                cdc_with_options,
                col_id_gen,
                on_conflict,
                with_version_column,
                include_column_options,
                table_name,
                resolved_table_name,
                database_id,
                schema_id,
                TableId::placeholder(),
                engine,
            )?;

            ((plan, None, table), TableJobType::SharedCdcSource)
        }
        (Some(_), Some(_)) => {
            return Err(ErrorCode::NotSupported(
                "Data format and encoding format doesn't apply to table created from a CDC source"
                    .into(),
                "Remove the FORMAT and ENCODE specification".into(),
            )
            .into())
        }
    };
    Ok((plan, source, table, job_type))
}

fn generated_columns_check_for_cdc_table(columns: &Vec<ColumnDef>) -> Result<()> {
    let mut found_generated_column = false;
    for column in columns {
        let mut is_generated = false;

        for option_def in &column.options {
            if let ColumnOption::GeneratedColumns(_) = option_def.option {
                is_generated = true;
                break;
            }
        }

        if is_generated {
            found_generated_column = true;
        } else if found_generated_column {
            return Err(ErrorCode::NotSupported(
                "Non-generated column found after a generated column.".into(),
                "Ensure that all generated columns appear at the end of the cdc table definition."
                    .into(),
            )
            .into());
        }
    }
    Ok(())
}

fn sanity_check_for_cdc_table(
    append_only: bool,
    column_defs: &Vec<ColumnDef>,
    wildcard_idx: &Option<usize>,
    constraints: &Vec<TableConstraint>,
    source_watermarks: &Vec<SourceWatermark>,
) -> Result<()> {
    // wildcard cannot be used with column definitions
    if wildcard_idx.is_some() && !column_defs.is_empty() {
        return Err(ErrorCode::NotSupported(
            "wildcard(*) and column definitions cannot be used together".to_owned(),
            "Remove the wildcard or column definitions".to_owned(),
        )
        .into());
    }

    // cdc table must have primary key constraint or primary key column
    if !wildcard_idx.is_some()
        && !constraints.iter().any(|c| {
            matches!(
                c,
                TableConstraint::Unique {
                    is_primary: true,
                    ..
                }
            )
        })
        && !column_defs.iter().any(|col| {
            col.options
                .iter()
                .any(|opt| matches!(opt.option, ColumnOption::Unique { is_primary: true }))
        })
    {
        return Err(ErrorCode::NotSupported(
            "CDC table without primary key constraint is not supported".to_owned(),
            "Please define a primary key".to_owned(),
        )
        .into());
    }
    if append_only {
        return Err(ErrorCode::NotSupported(
            "append only modifier on the table created from a CDC source".into(),
            "Remove the APPEND ONLY clause".into(),
        )
        .into());
    }

    if !source_watermarks.is_empty() {
        return Err(ErrorCode::NotSupported(
            "watermark defined on the table created from a CDC source".into(),
            "Remove the Watermark definitions".into(),
        )
        .into());
    }

    Ok(())
}

/// Derive schema for cdc table when create a new Table or alter an existing Table
async fn bind_cdc_table_schema_externally(
    cdc_with_options: WithOptionsSecResolved,
) -> Result<(Vec<ColumnCatalog>, Vec<String>)> {
    // read cdc table schema from external db or parsing the schema from SQL definitions
    Feature::CdcTableSchemaMap.check_available().map_err(
        |err: risingwave_common::license::FeatureNotAvailable| {
            ErrorCode::NotSupported(
                err.to_report_string(),
                "Please define the schema manually".to_owned(),
            )
        },
    )?;
    let (options, secret_refs) = cdc_with_options.into_parts();
    let config = ExternalTableConfig::try_from_btreemap(options, secret_refs)
        .context("failed to extract external table config")?;

    let table = ExternalTableImpl::connect(config)
        .await
        .context("failed to auto derive table schema")?;
    Ok((
        table
            .column_descs()
            .iter()
            .cloned()
            .map(|column_desc| ColumnCatalog {
                column_desc,
                is_hidden: false,
            })
            .collect(),
        table.pk_names().clone(),
    ))
}

/// Derive schema for cdc table when create a new Table or alter an existing Table
fn bind_cdc_table_schema(
    column_defs: &Vec<ColumnDef>,
    constraints: &Vec<TableConstraint>,
    is_for_replace_plan: bool,
) -> Result<(Vec<ColumnCatalog>, Vec<String>)> {
    let columns = bind_sql_columns(column_defs, is_for_replace_plan)?;

    let pk_names = bind_sql_pk_names(column_defs, bind_table_constraints(constraints)?)?;
    Ok((columns, pk_names))
}

#[allow(clippy::too_many_arguments)]
pub async fn handle_create_table(
    handler_args: HandlerArgs,
    table_name: ObjectName,
    column_defs: Vec<ColumnDef>,
    wildcard_idx: Option<usize>,
    constraints: Vec<TableConstraint>,
    if_not_exists: bool,
    format_encode: Option<FormatEncodeOptions>,
    source_watermarks: Vec<SourceWatermark>,
    append_only: bool,
    on_conflict: Option<OnConflict>,
    with_version_column: Option<String>,
    cdc_table_info: Option<CdcTableInfo>,
    include_column_options: IncludeOption,
    webhook_info: Option<WebhookSourceInfo>,
    ast_engine: risingwave_sqlparser::ast::Engine,
) -> Result<RwPgResponse> {
    let session = handler_args.session.clone();

    if append_only {
        session.notice_to_user("APPEND ONLY TABLE is currently an experimental feature.");
    }

    session.check_cluster_limits().await?;

    let engine = match ast_engine {
        risingwave_sqlparser::ast::Engine::Hummock => Engine::Hummock,
        risingwave_sqlparser::ast::Engine::Iceberg => Engine::Iceberg,
    };

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

    let (graph, source, table, job_type) = {
        let (plan, source, table, job_type) = handle_create_table_plan(
            handler_args.clone(),
            ExplainOptions::default(),
            format_encode,
            cdc_table_info,
            table_name.clone(),
            column_defs.clone(),
            wildcard_idx,
            constraints.clone(),
            source_watermarks,
            append_only,
            on_conflict,
            with_version_column,
            include_column_options,
            webhook_info,
            engine,
        )
        .await?;

        let graph = build_graph(plan)?;

        (graph, source, table, job_type)
    };

    tracing::trace!(
        "name={}, graph=\n{}",
        table_name,
        serde_json::to_string_pretty(&graph).unwrap()
    );

    // Handle engine
    match engine {
        Engine::Hummock => {
            let catalog_writer = session.catalog_writer()?;
            catalog_writer
                .create_table(source, table, graph, job_type)
                .await?;
        }
        Engine::Iceberg => {
            create_iceberg_engine_table(
                session,
                handler_args,
                source,
                table,
                graph,
                job_type,
                column_defs,
                constraints,
                table_name,
            )
            .await?;
        }
    }

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

#[allow(clippy::too_many_arguments)]
pub async fn create_iceberg_engine_table(
    session: Arc<SessionImpl>,
    handler_args: HandlerArgs,
    mut source: Option<PbSource>,
    table: PbTable,
    graph: StreamFragmentGraph,
    job_type: TableJobType,
    column_defs: Vec<ColumnDef>,
    constraints: Vec<TableConstraint>,
    table_name: ObjectName,
) -> Result<()> {
    risingwave_common::license::Feature::IcebergEngine
        .check_available()
        .map_err(|e| anyhow::anyhow!(e))?;
    // 1. fetch iceberg engine options from the meta node. Or use iceberg engine connection provided by users.
    // 2. create a hummock table
    // 3. create an iceberg sink
    // 4. create an iceberg source

    let meta_client = session.env().meta_client();
    let system_params = meta_client.get_system_params().await?;
    let state_store_endpoint = system_params.state_store().to_owned();
    let data_directory = system_params.data_directory().to_owned();
    let meta_store_endpoint = meta_client.get_meta_store_endpoint().await?;

    let meta_store_endpoint = url::Url::parse(&meta_store_endpoint).map_err(|_| {
        ErrorCode::InternalError("failed to parse the meta store endpoint".to_owned())
    })?;
    let meta_store_backend = meta_store_endpoint.scheme().to_owned();
    let meta_store_user = meta_store_endpoint.username().to_owned();
    let meta_store_password = meta_store_endpoint
        .password()
        .ok_or_else(|| {
            ErrorCode::InternalError("failed to parse password from meta store endpoint".to_owned())
        })?
        .to_owned();
    let meta_store_host = meta_store_endpoint
        .host_str()
        .ok_or_else(|| {
            ErrorCode::InternalError("failed to parse host from meta store endpoint".to_owned())
        })?
        .to_owned();
    let meta_store_port = meta_store_endpoint.port().ok_or_else(|| {
        ErrorCode::InternalError("failed to parse port from meta store endpoint".to_owned())
    })?;
    let meta_store_database = meta_store_endpoint
        .path()
        .trim_start_matches('/')
        .to_owned();

    let Ok(meta_backend) = MetaBackend::from_str(&meta_store_backend, true) else {
        bail!("failed to parse meta backend: {}", meta_store_backend);
    };

    let catalog_uri = match meta_backend {
        MetaBackend::Postgres => {
            format!(
                "jdbc:postgresql://{}:{}/{}",
                meta_store_host.clone(),
                meta_store_port.clone(),
                meta_store_database.clone()
            )
        }
        MetaBackend::Mysql => {
            format!(
                "jdbc:mysql://{}:{}/{}",
                meta_store_host.clone(),
                meta_store_port.clone(),
                meta_store_database.clone()
            )
        }
        MetaBackend::Sqlite | MetaBackend::Sql | MetaBackend::Mem => {
            bail!(
                "Unsupported meta backend for iceberg engine table: {}",
                meta_store_backend
            );
        }
    };

    let rw_db_name = session
        .env()
        .catalog_reader()
        .read_guard()
        .get_database_by_id(&table.database_id)?
        .name()
        .to_owned();
    let rw_schema_name = session
        .env()
        .catalog_reader()
        .read_guard()
        .get_schema_by_id(&table.database_id, &table.schema_id)?
        .name()
        .clone();
    let iceberg_catalog_name = rw_db_name.clone();
    let iceberg_database_name = rw_schema_name.clone();
    let iceberg_table_name = table_name.0.last().unwrap().real_value();

    let iceberg_engine_connection: String = session.config().iceberg_engine_connection();

    let mut connection_ref = BTreeMap::new();
    let with_common = if iceberg_engine_connection.is_empty() {
        #[allow(dead_code)]
        {
            let (s3_region, s3_endpoint, s3_ak, s3_sk, warehouse_path) = match state_store_endpoint
            {
                s3 if s3.starts_with("hummock+s3://") => {
                    let s3_region = if let Ok(s3_region) = std::env::var("AWS_REGION") {
                        s3_region
                    } else {
                        bail!("To create an iceberg engine table with s3 backend, AWS_REGION needed to be set");
                    };
                    let s3_bucket = s3.strip_prefix("hummock+s3://").unwrap().to_owned();
                    (
                        Some(s3_region),
                        None,
                        None,
                        None,
                        Some(format!(
                            "s3://{}/{}/iceberg/{}",
                            s3_bucket, data_directory, iceberg_catalog_name
                        )),
                    )
                }
                minio if minio.starts_with("hummock+minio://") => {
                    let server = minio.strip_prefix("hummock+minio://").unwrap();
                    let (access_key_id, rest) = server.split_once(':').unwrap();
                    let (secret_access_key, mut rest) = rest.split_once('@').unwrap();
                    let endpoint_prefix = if let Some(rest_stripped) = rest.strip_prefix("https://")
                    {
                        rest = rest_stripped;
                        "https://"
                    } else if let Some(rest_stripped) = rest.strip_prefix("http://") {
                        rest = rest_stripped;
                        "http://"
                    } else {
                        "http://"
                    };
                    let (address, s3_bucket) = rest.split_once('/').unwrap();
                    (
                        Some("us-east-1".to_owned()),
                        Some(format!("{}{}", endpoint_prefix, address)),
                        Some(access_key_id.to_owned()),
                        Some(secret_access_key.to_owned()),
                        Some(format!(
                            "s3://{}/{}/iceberg/{}",
                            s3_bucket, data_directory, iceberg_catalog_name
                        )),
                    )
                }
                _ => {
                    bail!(
                        "iceberg engine can't operate with this state store endpoint: {}",
                        state_store_endpoint
                    );
                }
            };

            let mut with_common = BTreeMap::new();
            with_common.insert("enable_config_load".to_owned(), "true".to_owned());
            with_common.insert("connector".to_owned(), "iceberg".to_owned());
            with_common.insert("catalog.type".to_owned(), "jdbc".to_owned());
            if let Some(warehouse_path) = warehouse_path.clone() {
                with_common.insert("warehouse.path".to_owned(), warehouse_path.to_owned());
            }
            if let Some(s3_endpoint) = s3_endpoint.clone() {
                with_common.insert("s3.endpoint".to_owned(), s3_endpoint);
            }
            if let Some(s3_ak) = s3_ak.clone() {
                with_common.insert("s3.access.key".to_owned(), s3_ak.to_owned());
            }
            if let Some(s3_sk) = s3_sk.clone() {
                with_common.insert("s3.secret.key".to_owned(), s3_sk.to_owned());
            }
            if let Some(s3_region) = s3_region.clone() {
                with_common.insert("s3.region".to_owned(), s3_region.to_owned());
            }
            with_common.insert("catalog.uri".to_owned(), catalog_uri.to_owned());
            with_common.insert("catalog.jdbc.user".to_owned(), meta_store_user.to_owned());
            with_common.insert(
                "catalog.jdbc.password".to_owned(),
                meta_store_password.clone(),
            );
            with_common.insert("catalog.name".to_owned(), iceberg_catalog_name.to_owned());
            with_common.insert("database.name".to_owned(), iceberg_database_name.to_owned());
            with_common.insert("table.name".to_owned(), iceberg_table_name.to_owned());
        }
        bail!("to use iceberg engine table, the variable `iceberg_engine_connection` must be set.");
    } else {
        let parts: Vec<&str> = iceberg_engine_connection.split('.').collect();
        assert_eq!(parts.len(), 2);
        let connection_catalog =
            session.get_connection_by_name(Some(parts[0].to_owned()), parts[1])?;
        if let ConnectionInfo::ConnectionParams(params) = &connection_catalog.info {
            if params.connection_type == ConnectionType::Iceberg as i32 {
                // With iceberg engine connection:
                connection_ref.insert(
                    "connection".to_owned(),
                    ConnectionRefValue {
                        connection_name: ObjectName::from(vec![
                            Ident::from(parts[0]),
                            Ident::from(parts[1]),
                        ]),
                    },
                );

                let mut with_common = BTreeMap::new();
                with_common.insert("connector".to_owned(), "iceberg".to_owned());
                with_common.insert("database.name".to_owned(), iceberg_database_name.to_owned());
                with_common.insert("table.name".to_owned(), iceberg_table_name.to_owned());
                with_common
            } else {
                return Err(RwError::from(ErrorCode::InvalidParameterValue(
                    "Only iceberg connection could be used in iceberg engine".to_owned(),
                )));
            }
        } else {
            return Err(RwError::from(ErrorCode::InvalidParameterValue(
                "Private Link Service has been deprecated. Please create a new connection instead."
                    .to_owned(),
            )));
        }
    };

    // Iceberg sinks require a primary key, if none is provided, we will use the _row_id column
    // Fetch primary key from columns
    let mut pks = column_defs
        .into_iter()
        .filter(|c| {
            c.options
                .iter()
                .any(|o| matches!(o.option, ColumnOption::Unique { is_primary: true }))
        })
        .map(|c| c.name.to_string())
        .collect::<Vec<String>>();

    // Fetch primary key from constraints
    if pks.is_empty() {
        pks = constraints
            .into_iter()
            .filter(|c| {
                matches!(
                    c,
                    TableConstraint::Unique {
                        is_primary: true,
                        ..
                    }
                )
            })
            .flat_map(|c| match c {
                TableConstraint::Unique { columns, .. } => columns
                    .into_iter()
                    .map(|c| c.to_string())
                    .collect::<Vec<String>>(),
                _ => vec![],
            })
            .collect::<Vec<String>>();
    }

    // For the table without primary key. We will use `_row_id` as primary key
    let sink_from = if pks.is_empty() {
        pks = vec![RISINGWAVE_ICEBERG_ROW_ID.to_owned()];
        let [stmt]: [_; 1] = Parser::parse_sql(&format!(
            "select {} as {}, * from {}",
            ROW_ID_COLUMN_NAME, RISINGWAVE_ICEBERG_ROW_ID, table_name
        ))
        .context("unable to parse query")?
        .try_into()
        .unwrap();

        let Statement::Query(query) = &stmt else {
            panic!("unexpected statement: {:?}", stmt);
        };
        CreateSink::AsQuery(query.clone())
    } else {
        CreateSink::From(table_name.clone())
    };

    let mut sink_name = table_name.clone();
    *sink_name.0.last_mut().unwrap() = Ident::from(
        (ICEBERG_SINK_PREFIX.to_owned() + &sink_name.0.last().unwrap().real_value()).as_str(),
    );
    let create_sink_stmt = CreateSinkStatement {
        if_not_exists: false,
        sink_name,
        with_properties: WithProperties(vec![]),
        sink_from,
        columns: vec![],
        emit_mode: None,
        sink_schema: None,
        into_table_name: None,
    };

    let mut sink_handler_args = handler_args.clone();

    let mut sink_with = with_common.clone();

    sink_with.insert("primary_key".to_owned(), pks.join(","));
    sink_with.insert("type".to_owned(), "upsert".to_owned());

    let commit_checkpoint_interval = handler_args
        .with_options
        .get(COMMIT_CHECKPOINT_INTERVAL)
        .map(|v| v.to_owned())
        .unwrap_or_else(|| "60".to_owned());
    let commit_checkpoint_interval = commit_checkpoint_interval.parse::<u32>().map_err(|_| {
        ErrorCode::InvalidInputSyntax(format!(
            "commit_checkpoint_interval must be a positive integer: {}",
            commit_checkpoint_interval
        ))
    })?;

    if commit_checkpoint_interval == 0 {
        bail!("commit_checkpoint_interval must be a positive integer: 0");
    }

    // remove commit_checkpoint_interval from source options, otherwise it will be considered as an unknown field.
    source
        .as_mut()
        .map(|x| x.with_properties.remove(COMMIT_CHECKPOINT_INTERVAL));

    let sink_decouple = session.config().sink_decouple();
    if matches!(sink_decouple, SinkDecouple::Disable) && commit_checkpoint_interval > 1 {
        bail!("config conflict: `commit_checkpoint_interval` larger than 1 means that sink decouple must be enabled, but session config sink_decouple is disabled")
    }

    sink_with.insert(
        COMMIT_CHECKPOINT_INTERVAL.to_owned(),
        commit_checkpoint_interval.to_string(),
    );
    sink_with.insert("create_table_if_not_exists".to_owned(), "true".to_owned());

    sink_handler_args.with_options =
        WithOptions::new(sink_with, Default::default(), connection_ref.clone());

    let mut source_name = table_name.clone();
    *source_name.0.last_mut().unwrap() = Ident::from(
        (ICEBERG_SOURCE_PREFIX.to_owned() + &source_name.0.last().unwrap().real_value()).as_str(),
    );
    let create_source_stmt = CreateSourceStatement {
        temporary: false,
        if_not_exists: false,
        columns: vec![],
        source_name,
        wildcard_idx: Some(0),
        constraints: vec![],
        with_properties: WithProperties(vec![]),
        format_encode: CompatibleFormatEncode::V2(FormatEncodeOptions::none()),
        source_watermarks: vec![],
        include_column_options: vec![],
    };

    let mut source_handler_args = handler_args.clone();
    let source_with = with_common;
    source_handler_args.with_options =
        WithOptions::new(source_with, Default::default(), connection_ref);

    // before we create the table, ensure the JVM is initialized as we use jdbc catalog right now.
    // If JVM isn't initialized successfully, current not atomic ddl will result in a partially created iceberg engine table.
    let _ = JVM.get_or_init()?;

    let catalog_writer = session.catalog_writer()?;
    // TODO(iceberg): make iceberg engine table creation ddl atomic
    catalog_writer
        .create_table(source, table, graph, job_type)
        .await?;
    create_sink::handle_create_sink(sink_handler_args, create_sink_stmt).await?;
    create_source::handle_create_source(source_handler_args, create_source_stmt).await?;

    Ok(())
}

pub fn check_create_table_with_source(
    with_options: &WithOptions,
    format_encode: Option<FormatEncodeOptions>,
    include_column_options: &IncludeOption,
    cdc_table_info: &Option<CdcTableInfo>,
) -> Result<Option<FormatEncodeOptions>> {
    // skip check for cdc table
    if cdc_table_info.is_some() {
        return Ok(format_encode);
    }
    let defined_source = with_options.is_source_connector();

    if !include_column_options.is_empty() && !defined_source {
        return Err(ErrorCode::InvalidInputSyntax(
            "INCLUDE should be used with a connector".to_owned(),
        )
        .into());
    }
    if defined_source {
        format_encode.as_ref().ok_or_else(|| {
            ErrorCode::InvalidInputSyntax("Please specify a source schema using FORMAT".to_owned())
        })?;
    }
    Ok(format_encode)
}

#[allow(clippy::too_many_arguments)]
pub async fn generate_stream_graph_for_replace_table(
    _session: &Arc<SessionImpl>,
    table_name: ObjectName,
    original_catalog: &Arc<TableCatalog>,
    handler_args: HandlerArgs,
    statement: Statement,
    col_id_gen: ColumnIdGenerator,
    sql_column_strategy: SqlColumnStrategy,
) -> Result<(StreamFragmentGraph, Table, Option<PbSource>, TableJobType)> {
    use risingwave_pb::catalog::table::OptionalAssociatedSourceId;

    let Statement::CreateTable {
        columns,
        constraints,
        source_watermarks,
        append_only,
        on_conflict,
        with_version_column,
        wildcard_idx,
        cdc_table_info,
        format_encode,
        include_column_options,
        engine,
        with_options,
        ..
    } = statement
    else {
        panic!("unexpected statement type: {:?}", statement);
    };

    let format_encode = format_encode
        .clone()
        .map(|format_encode| format_encode.into_v2_with_warning());

    let engine = match engine {
        risingwave_sqlparser::ast::Engine::Hummock => Engine::Hummock,
        risingwave_sqlparser::ast::Engine::Iceberg => Engine::Iceberg,
    };

    let is_drop_connector =
        original_catalog.associated_source_id().is_some() && format_encode.is_none();
    if is_drop_connector {
        debug_assert!(
            source_watermarks.is_empty()
                && include_column_options.is_empty()
                && with_options
                    .iter()
                    .all(|opt| opt.name.real_value().to_lowercase() != "connector")
        );
    }

    let props = CreateTableProps {
        definition: handler_args.normalized_sql.clone(),
        append_only,
        on_conflict: on_conflict.into(),
        with_version_column: with_version_column.clone(),
        webhook_info: original_catalog.webhook_info.clone(),
        engine,
    };

    let ((plan, mut source, table), job_type) = match (format_encode, cdc_table_info.as_ref()) {
        (Some(format_encode), None) => (
            gen_create_table_plan_with_source(
                handler_args,
                ExplainOptions::default(),
                table_name,
                columns,
                wildcard_idx,
                constraints,
                format_encode,
                source_watermarks,
                col_id_gen,
                include_column_options,
                props,
                sql_column_strategy,
            )
            .await?,
            TableJobType::General,
        ),
        (None, None) => {
            let context = OptimizerContext::from_handler_args(handler_args);
            let (plan, table) = gen_create_table_plan(
                context,
                table_name,
                columns,
                constraints,
                col_id_gen,
                source_watermarks,
                props,
                true,
            )?;
            ((plan, None, table), TableJobType::General)
        }
        (None, Some(cdc_table)) => {
            let session = &handler_args.session;
            let (source, resolved_table_name, database_id, schema_id) =
                get_source_and_resolved_table_name(session, cdc_table.clone(), table_name.clone())?;

            let cdc_with_options = derive_with_options_for_cdc_table(
                &source.with_properties,
                cdc_table.external_table_name.clone(),
            )?;

            let (column_catalogs, pk_names) = bind_cdc_table_schema(&columns, &constraints, true)?;

            let context: OptimizerContextRef =
                OptimizerContext::new(handler_args, ExplainOptions::default()).into();
            let (plan, table) = gen_create_table_plan_for_cdc_table(
                context,
                source,
                cdc_table.external_table_name.clone(),
                columns,
                column_catalogs,
                pk_names,
                cdc_with_options,
                col_id_gen,
                on_conflict,
                with_version_column,
                IncludeOption::default(),
                table_name,
                resolved_table_name,
                database_id,
                schema_id,
                original_catalog.id(),
                engine,
            )?;

            ((plan, None, table), TableJobType::SharedCdcSource)
        }
        (Some(_), Some(_)) => {
            return Err(ErrorCode::NotSupported(
                "Data format and encoding format doesn't apply to table created from a CDC source"
                    .into(),
                "Remove the FORMAT and ENCODE specification".into(),
            )
            .into());
        }
    };

    // TODO: avoid this backward conversion.
    if TableCatalog::from(&table).pk_column_ids() != original_catalog.pk_column_ids() {
        Err(ErrorCode::InvalidInputSyntax(
            "alter primary key of table is not supported".to_owned(),
        ))?
    }

    let graph = build_graph(plan)?;

    // Fill the original table ID.
    let mut table = Table {
        id: original_catalog.id().table_id(),
        ..table
    };
    if !is_drop_connector && let Some(source_id) = original_catalog.associated_source_id() {
        table.optional_associated_source_id = Some(OptionalAssociatedSourceId::AssociatedSourceId(
            source_id.table_id,
        ));
        source.as_mut().unwrap().id = source_id.table_id;
        source.as_mut().unwrap().optional_associated_table_id =
            Some(OptionalAssociatedTableId::AssociatedTableId(table.id))
    }

    Ok((graph, table, source, job_type))
}

fn get_source_and_resolved_table_name(
    session: &Arc<SessionImpl>,
    cdc_table: CdcTableInfo,
    table_name: ObjectName,
) -> Result<(Arc<SourceCatalog>, String, DatabaseId, SchemaId)> {
    let db_name = &session.database();
    let (schema_name, resolved_table_name) =
        Binder::resolve_schema_qualified_name(db_name, table_name)?;
    let (database_id, schema_id) =
        session.get_database_and_schema_id_for_create(schema_name.clone())?;

    let (format_encode, source_name) =
        Binder::resolve_schema_qualified_name(db_name, cdc_table.source_name.clone())?;

    let source = {
        let catalog_reader = session.env().catalog_reader().read_guard();
        let schema_name = format_encode.unwrap_or(DEFAULT_SCHEMA_NAME.to_owned());
        let (source, _) = catalog_reader.get_source_by_name(
            db_name,
            SchemaPath::Name(schema_name.as_str()),
            source_name.as_str(),
        )?;
        source.clone()
    };

    Ok((source, resolved_table_name, database_id, schema_id))
}

// validate the webhook_info and also bind the webhook_info to protobuf
fn bind_webhook_info(
    session: &Arc<SessionImpl>,
    columns_defs: &[ColumnDef],
    webhook_info: WebhookSourceInfo,
) -> Result<PbWebhookSourceInfo> {
    // validate columns
    if columns_defs.len() != 1 || columns_defs[0].data_type.as_ref().unwrap() != &AstDataType::Jsonb
    {
        return Err(ErrorCode::InvalidInputSyntax(
            "Table with webhook source should have exactly one JSONB column".to_owned(),
        )
        .into());
    }

    let WebhookSourceInfo {
        secret_ref,
        signature_expr,
        wait_for_persistence,
    } = webhook_info;

    // validate secret_ref
    let (pb_secret_ref, secret_name) = if let Some(secret_ref) = secret_ref {
        let db_name = &session.database();
        let (schema_name, secret_name) =
            Binder::resolve_schema_qualified_name(db_name, secret_ref.secret_name.clone())?;
        let secret_catalog = session.get_secret_by_name(schema_name, &secret_name)?;
        (
            Some(PbSecretRef {
                secret_id: secret_catalog.id.secret_id(),
                ref_as: match secret_ref.ref_as {
                    SecretRefAsType::Text => PbRefAsType::Text,
                    SecretRefAsType::File => PbRefAsType::File,
                }
                .into(),
            }),
            Some(secret_name),
        )
    } else {
        (None, None)
    };

    let secure_compare_context = SecureCompareContext {
        column_name: columns_defs[0].name.real_value(),
        secret_name,
    };
    let mut binder = Binder::new_for_ddl_with_secure_compare(session, secure_compare_context);
    let expr = binder.bind_expr(signature_expr.clone())?;

    // validate expr, ensuring it is SECURE_COMPARE()
    if expr.as_function_call().is_none()
        || expr.as_function_call().unwrap().func_type()
            != crate::optimizer::plan_node::generic::ExprType::SecureCompare
    {
        return Err(ErrorCode::InvalidInputSyntax(
            "The signature verification function must be SECURE_COMPARE()".to_owned(),
        )
        .into());
    }

    let pb_webhook_info = PbWebhookSourceInfo {
        secret_ref: pb_secret_ref,
        signature_expr: Some(expr.to_expr_proto()),
        wait_for_persistence,
    };

    Ok(pb_webhook_info)
}

#[cfg(test)]
mod tests {
    use risingwave_common::catalog::{
        Field, DEFAULT_DATABASE_NAME, ROW_ID_COLUMN_NAME, RW_TIMESTAMP_COLUMN_NAME,
    };
    use risingwave_common::types::{DataType, StructType};

    use super::*;
    use crate::test_utils::{create_proto_file, LocalFrontend, PROTO_FILE_DATA};

    struct BrandNewColumn(&'static str);
    use BrandNewColumn as B;

    impl FieldLike for BrandNewColumn {
        fn name(&self) -> &str {
            self.0
        }

        fn data_type(&self) -> &DataType {
            unreachable!("for brand new columns, data type will not be accessed")
        }
    }

    #[test]
    fn test_col_id_gen_initial() {
        let mut gen = ColumnIdGenerator::new_initial();
        assert_eq!(gen.generate(B("v1")).unwrap(), ColumnId::new(1));
        assert_eq!(gen.generate(B("v2")).unwrap(), ColumnId::new(2));
    }

    #[test]
    fn test_col_id_gen_alter() {
        let mut gen = ColumnIdGenerator::new_alter(&TableCatalog {
            columns: vec![
                ColumnCatalog {
                    column_desc: ColumnDesc::from_field_with_column_id(
                        &Field::with_name(DataType::Float32, "f32"),
                        1,
                    ),
                    is_hidden: false,
                },
                ColumnCatalog {
                    column_desc: ColumnDesc::from_field_with_column_id(
                        &Field::with_name(DataType::Float64, "f64"),
                        2,
                    ),
                    is_hidden: false,
                },
                ColumnCatalog {
                    column_desc: ColumnDesc::from_field_with_column_id(
                        &Field::with_name(
                            StructType::new([("f1", DataType::Int32)]).into(),
                            "nested",
                        ),
                        3,
                    ),
                    is_hidden: false,
                },
            ],
            version: Some(TableVersion::new_initial_for_test(ColumnId::new(3))),
            ..Default::default()
        });

        assert_eq!(gen.generate(B("v1")).unwrap(), ColumnId::new(4));
        assert_eq!(gen.generate(B("v2")).unwrap(), ColumnId::new(5));
        assert_eq!(
            gen.generate(Field::new("f32", DataType::Float32)).unwrap(),
            ColumnId::new(1)
        );

        // mismatched data type
        gen.generate(Field::new("f64", DataType::Float32))
            .unwrap_err();

        // mismatched data type
        // we require the nested data type to be exactly the same
        gen.generate(Field::new(
            "nested",
            StructType::new([("f1", DataType::Int32), ("f2", DataType::Int64)]).into(),
        ))
        .unwrap_err();

        assert_eq!(gen.generate(B("v3")).unwrap(), ColumnId::new(6));
    }

    #[tokio::test]
    async fn test_create_table_handler() {
        let sql =
            "create table t (v1 smallint, v2 struct<v3 bigint, v4 float, v5 double>) append only;";
        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 table exists.
        let (table, _) = catalog_reader
            .get_created_table_by_name(DEFAULT_DATABASE_NAME, schema_path, "t")
            .unwrap();
        assert_eq!(table.name(), "t");

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

        let expected_columns = maplit::hashmap! {
            ROW_ID_COLUMN_NAME => DataType::Serial,
            "v1" => DataType::Int16,
            "v2" => StructType::new(
                vec![("v3", DataType::Int64),("v4", DataType::Float64),("v5", DataType::Float64)],
            ).into(),
            RW_TIMESTAMP_COLUMN_NAME => DataType::Timestamptz,
        };

        assert_eq!(columns, expected_columns);
    }

    #[test]
    fn test_bind_primary_key() {
        // Note: Column ID 0 is reserved for row ID column.

        for (sql, expected) in [
            ("create table t (v1 int, v2 int)", Ok(&[0] as &[_])),
            ("create table t (v1 int primary key, v2 int)", Ok(&[1])),
            ("create table t (v1 int, v2 int primary key)", Ok(&[2])),
            (
                "create table t (v1 int primary key, v2 int primary key)",
                Err("multiple primary keys are not allowed"),
            ),
            (
                "create table t (v1 int primary key primary key, v2 int)",
                Err("multiple primary keys are not allowed"),
            ),
            (
                "create table t (v1 int, v2 int, primary key (v1))",
                Ok(&[1]),
            ),
            (
                "create table t (v1 int, primary key (v2), v2 int)",
                Ok(&[2]),
            ),
            (
                "create table t (primary key (v2, v1), v1 int, v2 int)",
                Ok(&[2, 1]),
            ),
            (
                "create table t (v1 int, primary key (v1), v2 int, primary key (v1))",
                Err("multiple primary keys are not allowed"),
            ),
            (
                "create table t (v1 int primary key, primary key (v1), v2 int)",
                Err("multiple primary keys are not allowed"),
            ),
            (
                "create table t (v1 int, primary key (V3), v2 int)",
                Err("column \"v3\" named in key does not exist"),
            ),
        ] {
            let mut ast = risingwave_sqlparser::parser::Parser::parse_sql(sql).unwrap();
            let risingwave_sqlparser::ast::Statement::CreateTable {
                columns: column_defs,
                constraints,
                ..
            } = ast.remove(0)
            else {
                panic!("test case should be create table")
            };
            let actual: Result<_> = (|| {
                let mut columns = bind_sql_columns(&column_defs, false)?;
                let mut col_id_gen = ColumnIdGenerator::new_initial();
                for c in &mut columns {
                    c.column_desc.column_id = col_id_gen.generate(&*c).unwrap();
                }

                let pk_names =
                    bind_sql_pk_names(&column_defs, bind_table_constraints(&constraints)?)?;
                let (_, pk_column_ids, _) =
                    bind_pk_and_row_id_on_relation(columns, pk_names, true)?;
                Ok(pk_column_ids)
            })();
            match (expected, actual) {
                (Ok(expected), Ok(actual)) => assert_eq!(
                    expected.iter().copied().map(ColumnId::new).collect_vec(),
                    actual,
                    "sql: {sql}"
                ),
                (Ok(_), Err(actual)) => panic!("sql: {sql}\nunexpected error: {actual:?}"),
                (Err(_), Ok(actual)) => panic!("sql: {sql}\nexpects error but got: {actual:?}"),
                (Err(expected), Err(actual)) => assert!(
                    actual.to_string().contains(expected),
                    "sql: {sql}\nexpected: {expected:?}\nactual: {actual:?}"
                ),
            }
        }
    }

    #[tokio::test]
    async fn test_duplicate_props_options() {
        let proto_file = create_proto_file(PROTO_FILE_DATA);
        let sql = format!(
            r#"CREATE TABLE 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"));
    }
}